From 1ae884e25b21c89ed3cf527102e7e3db75a8999d Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Fri, 3 Feb 2023 15:29:47 -0800 Subject: [PATCH 01/29] Upgrade to Azure Storage v12 - in progress. Based on Netherite code. --- .../AzureStorageDevice/AzureStorageDevice.cs | 783 +++++---- .../AzureStorageNamedDeviceFactory.cs | 109 +- .../devices/AzureStorageDevice/BlobEntry.cs | 144 +- .../devices/AzureStorageDevice/BlobManager.cs | 1397 +++++++++++++++++ .../devices/AzureStorageDevice/BlobUtils.cs | 73 + .../AzureStorageDevice/BlobUtilsV12.cs | 244 +++ .../FASTER.devices.AzureStorageDevice.csproj | 2 +- .../FASTER.devices.AzureStorageDevice.nuspec | 8 +- .../IPartitionErrorHandler.cs | 57 + .../devices/AzureStorageDevice/LeaseTimer.cs | 158 ++ .../AzureStorageDevice/StorageOperations.cs | 214 +++ cs/src/devices/AzureStorageDevice/Utils.cs | 26 + 12 files changed, 2751 insertions(+), 464 deletions(-) create mode 100644 cs/src/devices/AzureStorageDevice/BlobManager.cs create mode 100644 cs/src/devices/AzureStorageDevice/BlobUtils.cs create mode 100644 cs/src/devices/AzureStorageDevice/BlobUtilsV12.cs create mode 100644 cs/src/devices/AzureStorageDevice/IPartitionErrorHandler.cs create mode 100644 cs/src/devices/AzureStorageDevice/LeaseTimer.cs create mode 100644 cs/src/devices/AzureStorageDevice/StorageOperations.cs create mode 100644 cs/src/devices/AzureStorageDevice/Utils.cs diff --git a/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs b/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs index 0a9ddf45a..59f6649ac 100644 --- a/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs +++ b/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs @@ -1,388 +1,383 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT license. - -using System; -using System.Collections.Concurrent; -using System.Diagnostics; -using System.IO; -using System.Threading; -using System.Threading.Tasks; -using Microsoft.Azure.Storage.Blob; -using FASTER.core; -using Microsoft.Azure.Storage; - -namespace FASTER.devices +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT License. + +namespace DurableTask.Netherite.Faster { + using System; + using System.Collections.Concurrent; + using System.Collections.Generic; + using System.Diagnostics; + using System.IO; + using System.Linq; + using System.Threading; + using System.Threading.Tasks; + using Azure.Storage.Blobs.Models; + using FASTER.core; + /// /// A IDevice Implementation that is backed byAzure Page Blob. /// This device is slower than a local SSD or HDD, but provides scalability and shared access in the cloud. /// - public class AzureStorageDevice : StorageDeviceBase + class AzureStorageDevice : StorageDeviceBase { - private readonly ConcurrentDictionary blobs; - private readonly CloudBlobDirectory blobDirectory; - private readonly string blobName; - private readonly bool underLease; + readonly ConcurrentDictionary blobs; + readonly BlobUtilsV12.BlobDirectory blockBlobDirectory; + readonly BlobUtilsV12.BlobDirectory pageBlobDirectory; + readonly string blobName; + readonly bool underLease; + readonly ConcurrentDictionary pendingReadWriteOperations; + readonly ConcurrentDictionary pendingRemoveOperations; + readonly Timer hangCheckTimer; + readonly SemaphoreSlim singleWriterSemaphore; + readonly TimeSpan limit; + + static long sequenceNumber; + + struct ReadWriteRequestInfo + { + public bool IsRead; + public DeviceIOCompletionCallback Callback; + public uint NumBytes; + public object Context; + public DateTime TimeStamp; + } - internal BlobRequestOptions BlobRequestOptionsWithoutRetry { get; private set; } - internal BlobRequestOptions BlobRequestOptionsWithRetry { get; private set; } + struct RemoveRequestInfo + { + public AsyncCallback Callback; + public IAsyncResult Result; + public DateTime TimeStamp; + } - // Azure Page Blobs have a fixed sector size of 512 bytes. - private const uint PAGE_BLOB_SECTOR_SIZE = 512; + public SemaphoreSlim SingleWriterSemaphore => this.singleWriterSemaphore; + internal IPartitionErrorHandler PartitionErrorHandler { get; private set; } + + // Azure Page Blobs have a fixed sector size of 512 bytes. + const uint PAGE_BLOB_SECTOR_SIZE = 512; // Max upload size must be at most 4MB // we use an even smaller value to improve retry/timeout behavior in highly contended situations - private const uint MAX_UPLOAD_SIZE = 1024 * 1024; + // Also, this allows us to use aggressive timeouts to kill stragglers + const uint MAX_UPLOAD_SIZE = 1024 * 1024; + const uint MAX_DOWNLOAD_SIZE = 1024 * 1024; - // Max Azure Page Blob size (used when segment size is not specified): we set this at 8 GB - private const long MAX_PAGEBLOB_SIZE = 8L * 1024 * 1024 * 1024; - - // Whether blob files are deleted on close - private readonly bool deleteOnClose; + const long MAX_PAGEBLOB_SIZE = 512L * 1024 * 1024 * 1024; // set this at 512 GB for now TODO consider implications /// /// Constructs a new AzureStorageDevice instance, backed by Azure Page Blobs /// - /// Cloud blob directory containing the blobs - /// A descriptive name that will be the prefix of all blobs created with this device - /// Blob manager instance - /// Whether we use leases - /// - /// True if the program should delete all blobs created on call to Close. False otherwise. - /// The container is not deleted even if it was created in this constructor - /// - /// The maximum number of bytes this storage device can accommondate, or CAPACITY_UNSPECIFIED if there is no such limit - public AzureStorageDevice(CloudBlobDirectory cloudBlobDirectory, string blobName, IBlobManager blobManager = null, bool underLease = false, bool deleteOnClose = false, long capacity = Devices.CAPACITY_UNSPECIFIED) - : base($"{cloudBlobDirectory}/{blobName}", PAGE_BLOB_SECTOR_SIZE, capacity) + /// A descriptive name that will be the prefix of all segments created + /// the directory containing the block blobs + /// the directory containing the page blobs + /// the blob manager handling the leases + /// whether this device needs to be protected by the lease + public AzureStorageDevice(string blobName, BlobUtilsV12.BlobDirectory blockBlobDirectory, BlobUtilsV12.BlobDirectory pageBlobDirectory, BlobManager blobManager, bool underLease) + : base($"{blockBlobDirectory}\\{blobName}", PAGE_BLOB_SECTOR_SIZE, Devices.CAPACITY_UNSPECIFIED) { - this.blobs = new(); - this.blobDirectory = cloudBlobDirectory; + this.blobs = new ConcurrentDictionary(); + this.pendingReadWriteOperations = new ConcurrentDictionary(); + this.pendingRemoveOperations = new ConcurrentDictionary(); + this.blockBlobDirectory = blockBlobDirectory; + this.pageBlobDirectory = pageBlobDirectory; this.blobName = blobName; + this.PartitionErrorHandler = blobManager.PartitionErrorHandler; + this.PartitionErrorHandler.Token.Register(this.CancelAllRequests); + this.BlobManager = blobManager; this.underLease = underLease; - this.deleteOnClose = deleteOnClose; - - this.BlobManager = blobManager ?? new DefaultBlobManager(this.underLease, this.blobDirectory); - this.BlobRequestOptionsWithoutRetry = BlobManager.GetBlobRequestOptionsWithoutRetry(); - this.BlobRequestOptionsWithRetry = BlobManager.GetBlobRequestOptionsWithRetry(); - - StartAsync().Wait(); + this.hangCheckTimer = new Timer(this.DetectHangs, null, 0, 20000); + this.singleWriterSemaphore = underLease ? new SemaphoreSlim(1) : null; + this.limit = TimeSpan.FromSeconds(90); } - /// - /// Constructs a new AzureStorageDevice instance, backed by Azure Page Blobs - /// - /// The connection string to use when estblishing connection to Azure Blobs - /// Name of the Azure Blob container to use. If there does not exist a container with the supplied name, one is created - /// Directory within blob container to use. - /// A descriptive name that will be the prefix of all blobs created with this device - /// Blob manager instance - /// Whether we use leases - /// - /// True if the program should delete all blobs created on call to Close. False otherwise. - /// The container is not deleted even if it was created in this constructor - /// - /// The maximum number of bytes this storage device can accommondate, or CAPACITY_UNSPECIFIED if there is no such limit - public AzureStorageDevice(string connectionString, string containerName, string directoryName, string blobName, IBlobManager blobManager = null, bool underLease = false, bool deleteOnClose = false, long capacity = Devices.CAPACITY_UNSPECIFIED) - : base($"{connectionString}/{containerName}/{directoryName}/{blobName}", PAGE_BLOB_SECTOR_SIZE, capacity) + /// + public override string ToString() { - var storageAccount = CloudStorageAccount.Parse(connectionString); - var client = storageAccount.CreateCloudBlobClient(); - var container = client.GetContainerReference(containerName); - container.CreateIfNotExists(); - - this.blobs = new(); - this.blobDirectory = container.GetDirectoryReference(directoryName); - this.blobName = blobName; - this.underLease = underLease; - this.deleteOnClose = deleteOnClose; - - this.BlobManager = blobManager ?? new DefaultBlobManager(this.underLease, this.blobDirectory); - this.BlobRequestOptionsWithoutRetry = BlobManager.GetBlobRequestOptionsWithoutRetry(); - this.BlobRequestOptionsWithRetry = BlobManager.GetBlobRequestOptionsWithRetry(); - - StartAsync().Wait(); + return $"AzureStorageDevice {this.blockBlobDirectory}{this.blobName}"; } - private async Task StartAsync() + public async Task StartAsync() { - // list all the blobs representing the segments + try + { + this.BlobManager?.StorageTracer?.FasterStorageProgress($"StorageOpCalled AzureStorageDevice.StartAsync target={this.pageBlobDirectory}{this.blobName}"); - int prevSegmentId = -1; - var prefix = $"{blobDirectory.Prefix}{blobName}."; + // list all the blobs representing the segments + var prefix = $"{this.blockBlobDirectory}{this.blobName}."; - BlobContinuationToken continuationToken = null; - do - { - if (this.underLease) - { - await this.BlobManager.ConfirmLeaseAsync().ConfigureAwait(false); - } - var response = await this.blobDirectory.ListBlobsSegmentedAsync(useFlatBlobListing: false, blobListingDetails: BlobListingDetails.None, maxResults: 1000, - currentToken: continuationToken, options: this.BlobRequestOptionsWithRetry, operationContext: null) - .ConfigureAwait(BlobManager.ConfigureAwaitForStorage); + string continuationToken = null; + IReadOnlyList pageResults = null; - foreach (IListBlobItem item in response.Results) + do { - if (item is CloudPageBlob pageBlob) + await this.BlobManager.PerformWithRetriesAsync( + BlobManager.AsynchronousStorageReadMaxConcurrency, + this.underLease, + "BlobContainerClient.GetBlobsAsync", + "RecoverDevice", + $"continuationToken={continuationToken}", + this.pageBlobDirectory.ToString(), + 2000, + true, + async (numAttempts) => + { + var client = this.pageBlobDirectory.Client.WithRetries; + + var page = await client.GetBlobsAsync( + prefix: prefix, + cancellationToken: this.PartitionErrorHandler.Token) + .AsPages(continuationToken, 100) + .FirstAsync(); + + pageResults = page.Values; + continuationToken = page.ContinuationToken; + return page.Values.Count; // not accurate, in terms of bytes, but still useful for tracing purposes + }); + + foreach (var item in pageResults) { - if (Int32.TryParse(pageBlob.Name.Replace(prefix, ""), out int segmentId)) + if (Int32.TryParse(item.Name.Replace(prefix, ""), out int segmentId)) { - if (segmentId != prevSegmentId + 1) - { - startSegment = segmentId; - } - else + this.BlobManager?.StorageTracer?.FasterStorageProgress($"AzureStorageDevice.StartAsync found segment={item.Name}"); + + bool ret = this.blobs.TryAdd(segmentId, new BlobEntry(BlobUtilsV12.GetPageBlobClients(this.pageBlobDirectory.Client, item.Name), item.Properties.ETag.Value, this)); + + if (!ret) { - endSegment = segmentId; + throw new InvalidOperationException("Recovery of blobs is single-threaded and should not yield any failure due to concurrency"); } - prevSegmentId = segmentId; } } } - continuationToken = response.ContinuationToken; - } - while (continuationToken != null); + while (!string.IsNullOrEmpty(continuationToken)); - for (int i = startSegment; i <= endSegment; i++) - { - bool ret = this.blobs.TryAdd(i, new BlobEntry(this.blobDirectory.GetPageBlobReference(GetSegmentBlobName(i)), this.BlobManager)); + // make sure we did not lose the lease while iterating to find the blobs + await this.BlobManager.ConfirmLeaseIsGoodForAWhileAsync(); + this.PartitionErrorHandler.Token.ThrowIfCancellationRequested(); - if (!ret) + + // find longest contiguous sequence at end + var keys = this.blobs.Keys.ToList(); + if (keys.Count == 0) { - throw new InvalidOperationException("Recovery of blobs is single-threaded and should not yield any failure due to concurrency"); + // nothing has been written to this device so far. + this.startSegment = 0; + this.endSegment = -1; + } + else + { + keys.Sort(); + this.endSegment = this.startSegment = keys[keys.Count - 1]; + for (int i = keys.Count - 2; i >= 0; i--) + { + if (keys[i] == keys[i + 1] - 1) + { + this.startSegment = i; + } + } } + + this.BlobManager?.StorageTracer?.FasterStorageProgress($"StorageOpReturned AzureStorageDevice.StartAsync, determined segment range for {this.pageBlobDirectory.Prefix}{this.blobName}: start={this.startSegment} end={this.endSegment}"); + } + catch + { + this.BlobManager?.StorageTracer?.FasterStorageProgress($"StorageOpReturned AzureStorageDevice.StartAsync failed"); + throw; } } + /// /// Is called on exceptions, if non-null; can be set by application /// - private IBlobManager BlobManager { get; set; } + internal BlobManager BlobManager { get; set; } - private string GetSegmentBlobName(int segmentId) + string GetSegmentBlobName(int segmentId) { - return $"{blobName}.{segmentId}"; + return $"{this.blobName}.{segmentId}"; } - /// - public override void Dispose() + internal void DetectHangs(object _) { - // Unlike in LocalStorageDevice, we explicitly remove all page blobs if the deleteOnClose flag is set, instead of relying on the operating system - // to delete files after the end of our process. This leads to potential problems if multiple instances are sharing the same underlying page blobs. - // Since this flag is only used for testing, it is probably fine. - if (deleteOnClose) - PurgeAll(); - } + DateTime threshold = DateTime.UtcNow - (Debugger.IsAttached ? TimeSpan.FromMinutes(30) : this.limit); - /// - /// Purge all blobs related to this device. Do not use if - /// multiple instances are sharing the same underlying page blobs. - /// - public void PurgeAll() - { - foreach (var entry in blobs) + foreach (var kvp in this.pendingReadWriteOperations) { - entry.Value.PageBlob?.Delete(); - } - } - - /// - /// - /// - /// - /// - /// - public override void RemoveSegmentAsync(int segment, AsyncCallback callback, IAsyncResult result) - { - CloudPageBlob pageBlob; - if (blobs.TryRemove(segment, out BlobEntry blob)) - pageBlob = blob.PageBlob; - else - pageBlob = blobDirectory.GetPageBlobReference(GetSegmentBlobName(segment)); - if (!pageBlob.Exists()) return; - - if (underLease) - { - BlobManager.ConfirmLeaseAsync().AsTask() - .GetAwaiter() - .GetResult(); + if (kvp.Value.TimeStamp < threshold) + { + this.BlobManager.PartitionErrorHandler.HandleError("DetectHangs", $"storage operation id={kvp.Key} has exceeded the time limit {this.limit}", null, true, false); + return; + } } - - if (!BlobManager.CancellationToken.IsCancellationRequested) + foreach (var kvp in this.pendingRemoveOperations) { - var t = pageBlob.DeleteAsync(cancellationToken: BlobManager.CancellationToken); - t.GetAwaiter().OnCompleted(() => + if (kvp.Value.TimeStamp < threshold) { - if (t.IsFaulted) - { - BlobManager?.HandleBlobError(nameof(RemoveSegmentAsync), "could not remove page blob for segment", pageBlob?.Name, t.Exception, false); - } - callback(result); - }); + this.BlobManager.PartitionErrorHandler.HandleError("DetectHangs", $"storage operation id={kvp.Key} has exceeded the time limit {this.limit}", null, true, false); + return; + } } } - /// - public override long GetFileSize(int segmentId) - { - // We didn't find segment in blob cache - if (!blobs.TryGetValue(segmentId, out _)) - return 0; - return segmentSize == -1 ? MAX_PAGEBLOB_SIZE : segmentSize; - } - - //---- The actual read and write accesses to the page blobs - - private unsafe Task WritePortionToBlobUnsafeAsync(CloudPageBlob blob, IntPtr sourceAddress, long destinationAddress, long offset, uint length) - { - return this.WritePortionToBlobAsync(new UnmanagedMemoryStream((byte*)sourceAddress + offset, length), blob, sourceAddress, destinationAddress, offset, length); - } - - private async Task WritePortionToBlobAsync(UnmanagedMemoryStream stream, CloudPageBlob blob, IntPtr sourceAddress, long destinationAddress, long offset, uint length) + void CancelAllRequests() { - try + foreach (var id in this.pendingReadWriteOperations.Keys.ToList()) { - await BlobManager.AsyncStorageWriteMaxConcurrency.WaitAsync().ConfigureAwait(false); - - int numAttempts = 0; - long streamPosition = stream.Position; - - while (true) // retry loop + if (this.pendingReadWriteOperations.TryRemove(id, out var request)) { - numAttempts++; - try + if (request.IsRead) { - if (this.underLease) - { - await this.BlobManager.ConfirmLeaseAsync().ConfigureAwait(false); - } - - if (length > 0) - { - await blob.WritePagesAsync(stream, destinationAddress + offset, - contentChecksum: null, accessCondition: null, options: this.BlobRequestOptionsWithoutRetry, operationContext: null, cancellationToken: this.BlobManager.CancellationToken) - .ConfigureAwait(BlobManager.ConfigureAwaitForStorage); - } - break; + this.BlobManager?.StorageTracer?.FasterStorageProgress($"StorageOpReturned AzureStorageDevice.ReadAsync id={id} (Canceled)"); } - catch (StorageException e) when (this.underLease && IsTransientStorageError(e) && numAttempts < BlobManager.MaxRetries) + else { - TimeSpan nextRetryIn = TimeSpan.FromSeconds(1 + Math.Pow(2, (numAttempts - 1))); - this.BlobManager?.HandleBlobError(nameof(WritePortionToBlobAsync), $"could not write to page blob, will retry in {nextRetryIn}s", blob?.Name, e, false); - await Task.Delay(nextRetryIn).ConfigureAwait(false); - stream.Seek(streamPosition, SeekOrigin.Begin); // must go back to original position before retrying - continue; + this.BlobManager?.StorageTracer?.FasterStorageProgress($"StorageOpReturned AzureStorageDevice.WriteAsync id={id} (Canceled)"); } - catch (Exception exception) when (!IsFatal(exception)) - { - this.BlobManager?.HandleBlobError(nameof(WritePortionToBlobAsync), "could not write to page blob", blob?.Name, exception, false); - throw; - } - }; + request.Callback(uint.MaxValue, request.NumBytes, request.Context); + } } - finally + foreach (var id in this.pendingRemoveOperations.Keys.ToList()) { - BlobManager.AsyncStorageWriteMaxConcurrency.Release(); - stream.Dispose(); + if (this.pendingRemoveOperations.TryRemove(id, out var request)) + { + this.BlobManager?.StorageTracer?.FasterStorageProgress($"StorageOpReturned AzureStorageDevice.RemoveSegmentAsync id={id} (Canceled)"); + request.Callback(request.Result); + } } } - private unsafe Task ReadFromBlobUnsafeAsync(CloudPageBlob blob, long sourceAddress, long destinationAddress, uint readLength) + //---- the overridden methods represent the interface for a generic storage device + + /// + /// Inherited + /// + public override void Dispose() { - return this.ReadFromBlobAsync(new UnmanagedMemoryStream((byte*)destinationAddress, readLength, readLength, FileAccess.Write), blob, sourceAddress, destinationAddress, readLength); + this.hangCheckTimer.Dispose(); + this.singleWriterSemaphore?.Dispose(); } - private async Task ReadFromBlobAsync(UnmanagedMemoryStream stream, CloudPageBlob blob, long sourceAddress, long destinationAddress, uint readLength) + /// + /// + /// + /// + /// + /// + public override void RemoveSegmentAsync(int segment, AsyncCallback callback, IAsyncResult result) { - Debug.WriteLine($"AzureStorageDevice.ReadFromBlobAsync Called target={blob.Name}"); - - try - { - await BlobManager.AsyncStorageReadMaxConcurrency.WaitAsync().ConfigureAwait(false); + long id = Interlocked.Increment(ref AzureStorageDevice.sequenceNumber); - int numAttempts = 0; + this.BlobManager?.StorageTracer?.FasterStorageProgress($"StorageOpCalled AzureStorageDevice.RemoveSegmentAsync id={id} segment={segment}"); - while (true) // retry loop - { - numAttempts++; - try - { - if (this.underLease) - { - await this.BlobManager.ConfirmLeaseAsync().ConfigureAwait(false); - } - - Debug.WriteLine($"starting download target={blob.Name} readLength={readLength} sourceAddress={sourceAddress}"); - - if (readLength > 0) - { - await blob.DownloadRangeToStreamAsync(stream, sourceAddress, readLength, - accessCondition: null, options: this.BlobRequestOptionsWithoutRetry, operationContext: null, cancellationToken: this.BlobManager.CancellationToken) - .ConfigureAwait(BlobManager.ConfigureAwaitForStorage); - } + this.pendingRemoveOperations.TryAdd(id, new RemoveRequestInfo() + { + Callback = callback, + Result = result, + TimeStamp = DateTime.UtcNow + }); - Debug.WriteLine($"finished download target={blob.Name} readLength={readLength} sourceAddress={sourceAddress}"); + Task deletionTask = Task.CompletedTask; - if (stream.Position != readLength) - { - throw new InvalidDataException($"wrong amount of data received from page blob, expected={readLength}, actual={stream.Position}"); - } - break; - } - catch (StorageException e) when (this.underLease && IsTransientStorageError(e) && numAttempts < BlobManager.MaxRetries) - { - TimeSpan nextRetryIn = TimeSpan.FromSeconds(1 + Math.Pow(2, (numAttempts - 1))); - this.BlobManager?.HandleBlobError(nameof(ReadFromBlobAsync), $"could not write to page blob, will retry in {nextRetryIn}s", blob?.Name, e, false); - await Task.Delay(nextRetryIn).ConfigureAwait(false); - stream.Seek(0, SeekOrigin.Begin); // must go back to original position before retrying - continue; - } - catch (Exception exception) when (!IsFatal(exception)) + if (this.blobs.TryRemove(segment, out BlobEntry entry)) + { + deletionTask = this.BlobManager.PerformWithRetriesAsync( + null, + this.underLease, + "BlobBaseClient.DeleteAsync", + "DeleteDeviceSegment", + "", + entry.PageBlob.Default.Name, + 5000, + true, + async (numAttempts) => { - this.BlobManager?.HandleBlobError(nameof(ReadFromBlobAsync), "could not read from page blob", blob?.Name, exception, false); - throw; - } - } + var client = (numAttempts > 1) ? entry.PageBlob.Default : entry.PageBlob.Aggressive; + await client.DeleteAsync(cancellationToken: this.PartitionErrorHandler.Token); + return 1; + }); } - finally + + deletionTask.ContinueWith((Task t) => { - BlobManager.AsyncStorageReadMaxConcurrency.Release(); - stream.Dispose(); - } + if (this.pendingRemoveOperations.TryRemove(id, out var request)) + { + this.BlobManager?.StorageTracer?.FasterStorageProgress($"StorageOpReturned AzureStorageDevice.RemoveSegmentAsync id={id}"); + request.Callback(request.Result); + } + }, TaskContinuationOptions.ExecuteSynchronously); } - //---- the overridden methods represent the interface for a generic storage device + /// + /// Delete the device blobs in storage. + /// + /// + Task DeleteAsync() + { + Task Delete(BlobEntry entry) + { + return this.BlobManager.PerformWithRetriesAsync( + BlobManager.AsynchronousStorageWriteMaxConcurrency, + this.underLease, + "BlobBaseClient.DeleteAsync", + "DeleteDevice", + "", + entry.PageBlob.Default.Name, + 5000, + false, + async (numAttempts) => + { + var client = (numAttempts > 1) ? entry.PageBlob.Default : entry.PageBlob.Aggressive; + await client.DeleteAsync(cancellationToken: this.PartitionErrorHandler.Token); + return 1; + }); + } + + return Task.WhenAll(this.blobs.Values.Select(Delete).ToList()); + } /// /// Inherited /// public override unsafe void ReadAsync(int segmentId, ulong sourceAddress, IntPtr destinationAddress, uint readLength, DeviceIOCompletionCallback callback, object context) { - Debug.WriteLine($"AzureStorageDevice.ReadAsync Called segmentId={segmentId} sourceAddress={sourceAddress} readLength={readLength}"); + long id = Interlocked.Increment(ref AzureStorageDevice.sequenceNumber); + + this.BlobManager?.StorageTracer?.FasterStorageProgress($"StorageOpCalled AzureStorageDevice.ReadAsync id={id} segmentId={segmentId} sourceAddress={sourceAddress} readLength={readLength}"); + + this.pendingReadWriteOperations.TryAdd(id, new ReadWriteRequestInfo() + { + IsRead = true, + Callback = callback, + NumBytes = readLength, + Context = context, + TimeStamp = DateTime.UtcNow + }); // It is up to the allocator to make sure no reads are issued to segments before they are written - if (!blobs.TryGetValue(segmentId, out BlobEntry blobEntry)) + if (!this.blobs.TryGetValue(segmentId, out BlobEntry blobEntry)) { - var nonLoadedBlob = this.blobDirectory.GetPageBlobReference(GetSegmentBlobName(segmentId)); + var nonLoadedBlob = this.pageBlobDirectory.GetPageBlobClient(this.GetSegmentBlobName(segmentId)); var exception = new InvalidOperationException("Attempt to read a non-loaded segment"); - this.BlobManager?.HandleBlobError(nameof(ReadAsync), exception.Message, nonLoadedBlob?.Name, exception, false); - throw new FasterException(exception.Message, exception); + this.BlobManager?.HandleStorageError(nameof(ReadAsync), exception.Message, nonLoadedBlob.Default?.Name, exception, true, false); + throw exception; } - var t = this.ReadFromBlobUnsafeAsync(blobEntry.PageBlob, (long)sourceAddress, (long)destinationAddress, readLength); - t.GetAwaiter().OnCompleted(() => // REVIEW: this method cannot avoid GetAwaiter - { - if (t.IsFaulted) - { - Debug.WriteLine("AzureStorageDevice.ReadAsync Returned (Failure)"); - callback(uint.MaxValue, readLength, context); - } - else + this.ReadFromBlobUnsafeAsync(blobEntry.PageBlob, (long)sourceAddress, (long)destinationAddress, readLength, id) + .ContinueWith((Task t) => + { + if (this.pendingReadWriteOperations.TryRemove(id, out ReadWriteRequestInfo request)) { - Debug.WriteLine("AzureStorageDevice.ReadAsync Returned"); - callback(0, readLength, context); + if (t.IsFaulted) + { + this.BlobManager?.StorageTracer?.FasterStorageProgress($"StorageOpReturned AzureStorageDevice.ReadAsync id={id} (Failure)"); + request.Callback(uint.MaxValue, request.NumBytes, request.Context); + } + else + { + this.BlobManager?.StorageTracer?.FasterStorageProgress($"StorageOpReturned AzureStorageDevice.ReadAsync id={id}"); + request.Callback(0, request.NumBytes, request.Context); + } } - }); + }, TaskContinuationOptions.ExecuteSynchronously); } /// @@ -390,91 +385,205 @@ public override unsafe void ReadAsync(int segmentId, ulong sourceAddress, IntPtr /// public override void WriteAsync(IntPtr sourceAddress, int segmentId, ulong destinationAddress, uint numBytesToWrite, DeviceIOCompletionCallback callback, object context) { - Debug.WriteLine($"AzureStorageDevice.WriteAsync Called segmentId={segmentId} destinationAddress={destinationAddress} numBytesToWrite={numBytesToWrite}"); + long id = Interlocked.Increment(ref AzureStorageDevice.sequenceNumber); + + this.BlobManager?.StorageTracer?.FasterStorageProgress($"StorageOpCalled AzureStorageDevice.WriteAsync id={id} segmentId={segmentId} destinationAddress={destinationAddress} numBytesToWrite={numBytesToWrite}"); + + this.pendingReadWriteOperations.TryAdd(id, new ReadWriteRequestInfo() + { + IsRead = false, + Callback = callback, + NumBytes = numBytesToWrite, + Context = context, + TimeStamp = DateTime.UtcNow + }); - if (!blobs.TryGetValue(segmentId, out BlobEntry blobEntry)) + if (!this.blobs.TryGetValue(segmentId, out BlobEntry blobEntry)) { - BlobEntry entry = new(this.BlobManager); - if (blobs.TryAdd(segmentId, entry)) + BlobEntry entry = new BlobEntry(this); + if (this.blobs.TryAdd(segmentId, entry)) { - CloudPageBlob pageBlob = this.blobDirectory.GetPageBlobReference(GetSegmentBlobName(segmentId)); + var pageBlob = this.pageBlobDirectory.GetPageBlobClient(this.GetSegmentBlobName(segmentId)); - // If segment size is -1, which denotes absence, we request the largest possible blob. This is okay because - // page blobs are not backed by real pages on creation, and the given size is only a the physical limit of - // how large it can grow to. - var size = segmentSize == -1 ? MAX_PAGEBLOB_SIZE : segmentSize; + // If segment size is -1 we use a default + var size = this.segmentSize == -1 ? AzureStorageDevice.MAX_PAGEBLOB_SIZE : this.segmentSize; // If no blob exists for the segment, we must first create the segment asynchronouly. (Create call takes ~70 ms by measurement) // After creation is done, we can call write. - entry.CreateAsync(size, pageBlob).GetAwaiter().GetResult(); // REVIEW: this method cannot avoid GetAwaiter + _ = entry.CreateAsync(size, pageBlob); } // Otherwise, some other thread beat us to it. Okay to use their blobs. - blobEntry = blobs[segmentId]; + blobEntry = this.blobs[segmentId]; } - this.TryWriteAsync(blobEntry, sourceAddress, destinationAddress, numBytesToWrite, callback, context); + this.TryWriteAsync(blobEntry, sourceAddress, destinationAddress, numBytesToWrite, id); } - private void TryWriteAsync(BlobEntry blobEntry, IntPtr sourceAddress, ulong destinationAddress, uint numBytesToWrite, DeviceIOCompletionCallback callback, object context) + //---- The actual read and write accesses to the page blobs + + unsafe Task WritePortionToBlobUnsafeAsync(BlobEntry blobEntry, IntPtr sourceAddress, long destinationAddress, long offset, uint length, long id) { - // If pageBlob is null, it is being created. Attempt to queue the write for the creator to complete after it is done - if (blobEntry.PageBlob == null - && blobEntry.TryQueueAction(p => this.WriteToBlobAsync(p, sourceAddress, destinationAddress, numBytesToWrite, callback, context))) + return this.WritePortionToBlobAsync(new UnmanagedMemoryStream((byte*)sourceAddress + offset, length), blobEntry, sourceAddress, destinationAddress, offset, length, id); + } + + async Task WritePortionToBlobAsync(UnmanagedMemoryStream stream, BlobEntry blobEntry, IntPtr sourceAddress, long destinationAddress, long offset, uint length, long id) + { + using (stream) { - return; + long originalStreamPosition = stream.Position; + await this.BlobManager.PerformWithRetriesAsync( + BlobManager.AsynchronousStorageWriteMaxConcurrency, + true, + "PageBlobClient.UploadPagesAsync", + "WriteToDevice", + $"id={id} length={length} destinationAddress={destinationAddress + offset}", + blobEntry.PageBlob.Default.Name, + 1000 + (int)length / 1000, + true, + async (numAttempts) => + { + if (numAttempts > 0) + { + stream.Seek(originalStreamPosition, SeekOrigin.Begin); // must go back to original position before retry + } + + if (length > 0) + { + var client = numAttempts > 2 ? blobEntry.PageBlob.Default : blobEntry.PageBlob.Aggressive; + + var response = await client.UploadPagesAsync( + content: stream, + offset: destinationAddress + offset, + transactionalContentHash: null, + conditions: this.underLease ? new PageBlobRequestConditions() { IfMatch = blobEntry.ETag } : null, + progressHandler: null, + cancellationToken: this.PartitionErrorHandler.Token).ConfigureAwait(false); + + blobEntry.ETag = response.Value.ETag; + } + + return (long)length; + }, + async () => + { + var response = await blobEntry.PageBlob.Default.GetPropertiesAsync(); + blobEntry.ETag = response.Value.ETag; + + }).ConfigureAwait(false); } - // Otherwise, invoke directly. - this.WriteToBlobAsync(blobEntry.PageBlob, sourceAddress, destinationAddress, numBytesToWrite, callback, context); } - private unsafe void WriteToBlobAsync(CloudPageBlob blob, IntPtr sourceAddress, ulong destinationAddress, uint numBytesToWrite, DeviceIOCompletionCallback callback, object context) + unsafe Task ReadFromBlobUnsafeAsync(BlobUtilsV12.PageBlobClients blob, long sourceAddress, long destinationAddress, uint readLength, long id) { - Debug.WriteLine($"AzureStorageDevice.WriteToBlobAsync Called target={blob.Name}"); + return this.ReadFromBlobAsync(new UnmanagedMemoryStream((byte*)destinationAddress, readLength, readLength, FileAccess.Write), blob, sourceAddress, readLength, id); + } - var t = this.WriteToBlobAsync(blob, sourceAddress, (long)destinationAddress, numBytesToWrite); - t.GetAwaiter().OnCompleted(() => // REVIEW: this method cannot avoid GetAwaiter + async Task ReadFromBlobAsync(UnmanagedMemoryStream stream, BlobUtilsV12.PageBlobClients blob, long sourceAddress, uint readLength, long id) + { + using (stream) { - if (t.IsFaulted) - { - Debug.WriteLine("AzureStorageDevice.WriteAsync Returned (Failure)"); - callback(uint.MaxValue, numBytesToWrite, context); - } - else + long offset = 0; + while (readLength > 0) { - Debug.WriteLine("AzureStorageDevice.WriteAsync Returned"); - callback(0, numBytesToWrite, context); + var length = Math.Min(readLength, MAX_DOWNLOAD_SIZE); + + await this.BlobManager.PerformWithRetriesAsync( + BlobManager.AsynchronousStorageReadMaxConcurrency, + true, + "PageBlobClient.DownloadStreamingAsync", + "ReadFromDevice", + $"id={id} readLength={length} sourceAddress={sourceAddress + offset}", + blob.Default.Name, + 1000 + (int)length / 1000, + true, + async (numAttempts) => + { + if (numAttempts > 0) + { + stream.Seek(offset, SeekOrigin.Begin); // must go back to original position before retrying + } + + if (length > 0) + { + var client = (numAttempts > 1 || length == MAX_DOWNLOAD_SIZE) ? blob.Default : blob.Aggressive; + + var response = await client.DownloadStreamingAsync( + range: new Azure.HttpRange(sourceAddress + offset, length), + conditions: null, + rangeGetContentHash: false, + cancellationToken: this.PartitionErrorHandler.Token) + .ConfigureAwait(false); + + await response.Value.Content.CopyToAsync(stream).ConfigureAwait(false); + } + + if (stream.Position != offset + length) + { + throw new InvalidDataException($"wrong amount of data received from page blob, expected={length}, actual={stream.Position}"); + } + + return length; + }); + + readLength -= length; + offset += length; } - }); + } } - private async Task WriteToBlobAsync(CloudPageBlob blob, IntPtr sourceAddress, long destinationAddress, uint numBytesToWrite) + void TryWriteAsync(BlobEntry blobEntry, IntPtr sourceAddress, ulong destinationAddress, uint numBytesToWrite, long id) { - long offset = 0; - while (numBytesToWrite > 0) + // If pageBlob is null, it is being created. Attempt to queue the write for the creator to complete after it is done + if (blobEntry.PageBlob.Default == null + && blobEntry.TryQueueAction(() => this.WriteToBlobAsync(blobEntry, sourceAddress, destinationAddress, numBytesToWrite, id))) { - var length = Math.Min(numBytesToWrite, MAX_UPLOAD_SIZE); - await this.WritePortionToBlobUnsafeAsync(blob, sourceAddress, destinationAddress, offset, length).ConfigureAwait(false); - numBytesToWrite -= length; - offset += length; + return; } + // Otherwise, invoke directly. + this.WriteToBlobAsync(blobEntry, sourceAddress, destinationAddress, numBytesToWrite, id); } - private static bool IsTransientStorageError(StorageException e) + unsafe void WriteToBlobAsync(BlobEntry blobEntry, IntPtr sourceAddress, ulong destinationAddress, uint numBytesToWrite, long id) { - return (e.RequestInformation.HttpStatusCode == 408) //408 Request Timeout - || (e.RequestInformation.HttpStatusCode == 429) //429 Too Many Requests - || (e.RequestInformation.HttpStatusCode == 500) //500 Internal Server Error - || (e.RequestInformation.HttpStatusCode == 502) //502 Bad Gateway - || (e.RequestInformation.HttpStatusCode == 503) //503 Service Unavailable - || (e.RequestInformation.HttpStatusCode == 504); //504 Gateway Timeout + this.WriteToBlobAsync(blobEntry, sourceAddress, (long)destinationAddress, numBytesToWrite, id) + .ContinueWith((Task t) => + { + if (this.pendingReadWriteOperations.TryRemove(id, out ReadWriteRequestInfo request)) + { + if (t.IsFaulted) + { + this.BlobManager?.StorageTracer?.FasterStorageProgress($"StorageOpReturned AzureStorageDevice.WriteAsync id={id} (Failure)"); + request.Callback(uint.MaxValue, request.NumBytes, request.Context); + } + else + { + this.BlobManager?.StorageTracer?.FasterStorageProgress($"StorageOpReturned AzureStorageDevice.WriteAsync id={id}"); + request.Callback(0, request.NumBytes, request.Context); + } + } + + if (this.underLease) + { + this.SingleWriterSemaphore.Release(); + } + + }, TaskContinuationOptions.ExecuteSynchronously); } - private static bool IsFatal(Exception exception) + async Task WriteToBlobAsync(BlobEntry blobEntry, IntPtr sourceAddress, long destinationAddress, uint numBytesToWrite, long id) { - if (exception is OutOfMemoryException || exception is StackOverflowException) + if (this.underLease) { - return true; + await this.SingleWriterSemaphore.WaitAsync(); + } + + long offset = 0; + while (numBytesToWrite > 0) + { + var length = Math.Min(numBytesToWrite, MAX_UPLOAD_SIZE); + await this.WritePortionToBlobUnsafeAsync(blobEntry, sourceAddress, destinationAddress, offset, length, id).ConfigureAwait(false); + numBytesToWrite -= length; + offset += length; } - return false; } } } diff --git a/cs/src/devices/AzureStorageDevice/AzureStorageNamedDeviceFactory.cs b/cs/src/devices/AzureStorageDevice/AzureStorageNamedDeviceFactory.cs index aa4dd3b94..1041223d1 100644 --- a/cs/src/devices/AzureStorageDevice/AzureStorageNamedDeviceFactory.cs +++ b/cs/src/devices/AzureStorageDevice/AzureStorageNamedDeviceFactory.cs @@ -1,61 +1,70 @@ using FASTER.core; -using Microsoft.Azure.Storage; -using Microsoft.Azure.Storage.Blob; -using System; using System.Collections.Generic; -using System.Diagnostics; using System.Linq; -using System.Runtime.CompilerServices; -using System.Text; using System.Threading.Tasks; -namespace FASTER.devices +namespace DurableTask.Netherite.Faster { /// /// Device factory for Azure /// public class AzureStorageNamedDeviceFactory : INamedDeviceFactory { - private readonly CloudBlobClient client; - private CloudBlobDirectory baseRef; + readonly BlobUtilsV12.ServiceClients pageBlobAccount; + BlobUtilsV12.ContainerClients pageBlobContainer; + BlobUtilsV12.BlobDirectory pageBlobDirectory; /// /// Create instance of factory for Azure devices /// - /// - public AzureStorageNamedDeviceFactory(CloudBlobClient client) + /// + public AzureStorageNamedDeviceFactory(string connectionString) + : this(BlobUtilsV12.GetServiceClients(connectionString)) { - this.client = client; } /// /// Create instance of factory for Azure devices /// - /// - public AzureStorageNamedDeviceFactory(string connectionString) - : this(CloudStorageAccount.Parse(connectionString).CreateCloudBlobClient()) + /// + AzureStorageNamedDeviceFactory(BlobUtilsV12.ServiceClients pageBlobAccount) { + this.pageBlobAccount = pageBlobAccount; } + /// + public void Initialize(string baseName) + => InitializeAsync(baseName).GetAwaiter().GetResult(); + + + async Task InitializeAsync(string baseName) + { + var path = baseName.Split('/'); + var containerName = path[0]; + var dirName = string.Join("/", path.Skip(1)); + + this.pageBlobContainer = BlobUtilsV12.GetContainerClients(this.pageBlobAccount, containerName); + await this.pageBlobContainer.WithRetries.CreateIfNotExistsAsync(); + + pageBlobDirectory = new BlobUtilsV12.BlobDirectory(pageBlobContainer, dirName); + } + + /// public void Delete(FileDescriptor fileInfo) { + var dir = fileInfo.directoryName == "" ? pageBlobDirectory : pageBlobDirectory.GetSubDirectory(fileInfo.directoryName); + if (fileInfo.fileName != null) { - var dir = fileInfo.directoryName == "" ? baseRef : baseRef.GetDirectoryReference(fileInfo.directoryName); - // We only delete shard 0 - dir.GetBlobReference(fileInfo.fileName + ".0").DeleteIfExists(); + dir.GetPageBlobClient(fileInfo.fileName + ".0").Default.DeleteIfExists(); } else { - var dir = fileInfo.directoryName == "" ? baseRef : baseRef.GetDirectoryReference(fileInfo.directoryName); - foreach (IListBlobItem blob in dir.ListBlobs(true)) + foreach (var blob in dir.Client.WithRetries.GetBlobs()) { - if (blob.GetType() == typeof(CloudBlob) || blob.GetType().BaseType == typeof(CloudBlob)) - { - ((CloudBlob)blob).DeleteIfExists(); - } + dir.GetPageBlobClient(blob.Name).Default.DeleteIfExists(); } } } @@ -63,48 +72,34 @@ public void Delete(FileDescriptor fileInfo) /// public IDevice Get(FileDescriptor fileInfo) { - return new AzureStorageDevice(baseRef.GetDirectoryReference(fileInfo.directoryName), fileInfo.fileName); - } - - /// - public void Initialize(string baseName) - { - var path = baseName.Split('/'); - var containerName = path[0]; - var dirName = string.Join("/", path.Skip(1)); - - var containerRef = client.GetContainerReference(containerName); - containerRef.CreateIfNotExists(); - baseRef = containerRef.GetDirectoryReference(dirName); + return new AzureStorageDevice(fileInfo.fileName, default, pageBlobDirectory.GetSubDirectory(fileInfo.directoryName), null, false); } /// public IEnumerable ListContents(string path) { - foreach (var entry in baseRef.GetDirectoryReference(path).ListBlobs().Where(b => b as CloudBlobDirectory != null) - .OrderByDescending(f => GetLastModified((CloudBlobDirectory)f))) + var dir = pageBlobDirectory.GetSubDirectory(path); + var client = dir.Client.WithRetries; + foreach (var item in client.GetBlobsByHierarchy() + .OrderByDescending(f => dir.GetPageBlobClient(f.Blob.Name).Default.GetProperties().Value.LastModified)) { - yield return new FileDescriptor + if (item.IsPrefix) { - directoryName = entry.Uri.LocalPath, - fileName = "" - }; - } - - foreach (var entry in baseRef.GetDirectoryReference(path).ListBlobs().Where(b => b as CloudPageBlob != null) - .OrderByDescending(f => ((CloudPageBlob)f).Properties.LastModified)) - { - yield return new FileDescriptor + yield return new FileDescriptor + { + directoryName = item.Prefix, + fileName = "" + }; + } + else { - directoryName = "", - fileName = ((CloudPageBlob)entry).Name - }; + yield return new FileDescriptor + { + directoryName = item.Blob.Name, + fileName = "" + }; + } } } - - private DateTimeOffset? GetLastModified(CloudBlobDirectory cloudBlobDirectory) - { - return cloudBlobDirectory.ListBlobs().Select(e => ((CloudPageBlob)e).Properties.LastModified).OrderByDescending(e => e).First(); - } } } diff --git a/cs/src/devices/AzureStorageDevice/BlobEntry.cs b/cs/src/devices/AzureStorageDevice/BlobEntry.cs index 9e0080414..75b94124a 100644 --- a/cs/src/devices/AzureStorageDevice/BlobEntry.cs +++ b/cs/src/devices/AzureStorageDevice/BlobEntry.cs @@ -1,16 +1,13 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT license. - -using System; -using System.Collections.Concurrent; -using System.Diagnostics; -using System.Globalization; -using System.Threading; -using System.Threading.Tasks; -using Microsoft.Azure.Storage.Blob; - -namespace FASTER.devices +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT License. + +namespace DurableTask.Netherite.Faster { + using System; + using System.Collections.Concurrent; + using System.Threading; + using System.Threading.Tasks; + // This class bundles a page blob object with a queue and a counter to ensure // 1) BeginCreate is not called more than once // 2) No writes are issued before EndCreate @@ -19,35 +16,35 @@ namespace FASTER.devices // In-progress creation is denoted by a null value on the underlying page blob class BlobEntry { - public CloudPageBlob PageBlob { get; private set; } + public BlobUtilsV12.PageBlobClients PageBlob { get; private set; } + public Azure.ETag ETag { get; set; } + + ConcurrentQueue pendingWrites; + readonly AzureStorageDevice azureStorageDevice; + int waitingCount; + - private ConcurrentQueue> pendingWrites; - private readonly IBlobManager blobManager; - private int waitingCount; /// - /// Creates a new BlobEntry to hold the given pageBlob. The pageBlob must already be created. + /// Creates a new BlobEntry to represent a page blob that already exists in storage. /// /// - /// - public BlobEntry(CloudPageBlob pageBlob, IBlobManager blobManager) + /// + public BlobEntry(BlobUtilsV12.PageBlobClients pageBlob, Azure.ETag eTag, AzureStorageDevice azureStorageDevice) { this.PageBlob = pageBlob; - this.blobManager = blobManager; - if (pageBlob == null) - { - // Only need to allocate a queue when we potentially need to asynchronously create a blob - pendingWrites = new ConcurrentQueue>(); - waitingCount = 0; - } + this.azureStorageDevice = azureStorageDevice; + this.ETag = eTag; } /// - /// Creates a new BlobEntry, does not initialize a page blob. Use - /// for actual creation. + /// Creates a new BlobEntry to represent a page blob that will be created by . /// - public BlobEntry(IBlobManager blobManager) : this(null, blobManager) + public BlobEntry(AzureStorageDevice azureStorageDevice) { + this.azureStorageDevice = azureStorageDevice; + this.pendingWrites = new ConcurrentQueue(); + this.waitingCount = 0; } /// @@ -55,43 +52,60 @@ public BlobEntry(IBlobManager blobManager) : this(null, blobManager) /// /// maximum size of the blob /// The page blob to create - public async Task CreateAsync(long size, CloudPageBlob pageBlob) + public async Task CreateAsync(long size, BlobUtilsV12.PageBlobClients pageBlob) { - try + if (this.waitingCount != 0) { - if (this.waitingCount != 0) + this.azureStorageDevice.BlobManager?.HandleStorageError(nameof(CreateAsync), "expect to be called on blobs that don't already exist and exactly once", pageBlob.Default?.Name, null, false, false); + } + + await this.azureStorageDevice.BlobManager.PerformWithRetriesAsync( + BlobManager.AsynchronousStorageReadMaxConcurrency, + true, + "PageBlobClient.CreateAsync", + "CreateDevice", + "", + pageBlob.Default.Name, + 3000, + true, + async (numAttempts) => { - this.blobManager.HandleBlobError(nameof(CreateAsync), "expect to be called on blobs that don't already exist and exactly once", pageBlob?.Name, null, false); - } - - await pageBlob.CreateAsync(size, - accessCondition: null, options: this.blobManager.GetBlobRequestOptionsWithRetry(), operationContext: null, this.blobManager.CancellationToken).ConfigureAwait(false); - - // At this point the blob is fully created. After this line all consequent writers will write immediately. We just - // need to clear the queue of pending writers. - this.PageBlob = pageBlob; - - // Take a snapshot of the current waiting count. Exactly this many actions will be cleared. - // Swapping in -1 will inform any stragglers that we are not taking their actions and prompt them to retry (and call write directly) - int waitingCountSnapshot = Interlocked.Exchange(ref waitingCount, -1); - Action action; - // Clear actions - for (int i = 0; i < waitingCountSnapshot; i++) + var client = (numAttempts > 1) ? pageBlob.Default : pageBlob.Aggressive; + + var response = await client.CreateAsync( + size: size, + conditions: new Azure.Storage.Blobs.Models.PageBlobRequestConditions() { IfNoneMatch = Azure.ETag.All }, + cancellationToken: this.azureStorageDevice.PartitionErrorHandler.Token); + + this.ETag = response.Value.ETag; + return 1; + }, + async () => { - // inserts into the queue may lag behind the creation thread. We have to wait until that happens. - // This is so rare, that we are probably okay with a busy wait. - while (!pendingWrites.TryDequeue(out action)) { } - action(pageBlob); - } - - // Mark for deallocation for the GC - pendingWrites = null; - } - catch (Exception e) + var response = await pageBlob.Default.GetPropertiesAsync(); + this.ETag = response.Value.ETag; + }); + + // At this point the blob is fully created. After this line all consequent writers will write immediately. We just + // need to clear the queue of pending writers. + this.PageBlob = pageBlob; + + // Take a snapshot of the current waiting count. Exactly this many actions will be cleared. + // Swapping in -1 will inform any stragglers that we are not taking their actions and prompt them to retry (and call write directly) + int waitingCountSnapshot = Interlocked.Exchange(ref this.waitingCount, -1); + Action action; + + // Clear actions + for (int i = 0; i < waitingCountSnapshot; i++) { - this.blobManager.HandleBlobError(nameof(CreateAsync), "could not create page blob", pageBlob?.Name, e, true); - throw; + // inserts into the queue may lag behind the creation thread. We have to wait until that happens. + // This is so rare, that we are probably okay with a busy wait. + while (!this.pendingWrites.TryDequeue(out action)) { } + action(); } + + // Mark for deallocation for the GC + this.pendingWrites = null; } /// @@ -101,22 +115,22 @@ await pageBlob.CreateAsync(size, /// /// The write action to perform /// Whether the action was successfully enqueued - public bool TryQueueAction(Action writeAction) + public bool TryQueueAction(Action writeAction) { int currentCount; do { - currentCount = waitingCount; + currentCount = this.waitingCount; // If current count became -1, creation is complete. New queue entries will not be processed and we must call the action ourselves. if (currentCount == -1) return false; - } while (Interlocked.CompareExchange(ref waitingCount, currentCount + 1, currentCount) != currentCount); + } while (Interlocked.CompareExchange(ref this.waitingCount, currentCount + 1, currentCount) != currentCount); // Enqueue last. The creation thread is obliged to wait until it has processed waitingCount many actions. // It is extremely unlikely that we will get scheduled out here anyways. - pendingWrites.Enqueue(writeAction); + this.pendingWrites.Enqueue(writeAction); return true; } } -} \ No newline at end of file +} diff --git a/cs/src/devices/AzureStorageDevice/BlobManager.cs b/cs/src/devices/AzureStorageDevice/BlobManager.cs new file mode 100644 index 000000000..832342327 --- /dev/null +++ b/cs/src/devices/AzureStorageDevice/BlobManager.cs @@ -0,0 +1,1397 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT License. + +namespace DurableTask.Netherite.Faster +{ + using DurableTask.Core.Common; + using FASTER.core; + using Azure.Storage.Blobs; + using Microsoft.Extensions.Logging; + using Newtonsoft.Json; + using Newtonsoft.Json.Linq; + using System; + using System.Collections.Generic; + using System.Diagnostics; + using System.IO; + using System.Linq; + using System.Threading; + using System.Threading.Tasks; + using Azure.Storage.Blobs.Specialized; + using Azure.Storage.Blobs.Models; + using System.Net; + using System.Text; + + /// + /// Provides management of blobs and blob names associated with a partition, and logic for partition lease maintenance and termination. + /// + partial class BlobManager : ICheckpointManager, ILogCommitManager + { + readonly NetheriteOrchestrationServiceSettings settings; + readonly uint partitionId; + readonly CancellationTokenSource shutDownOrTermination; + readonly string taskHubPrefix; + + BlobUtilsV12.ServiceClients blockBlobAccount; + BlobUtilsV12.ServiceClients pageBlobAccount; + + BlobUtilsV12.ContainerClients blockBlobContainer; + BlobUtilsV12.ContainerClients pageBlobContainer; + + BlobUtilsV12.BlockBlobClients eventLogCommitBlob; + BlobLeaseClient leaseClient; + + BlobUtilsV12.BlobDirectory pageBlobPartitionDirectory; + BlobUtilsV12.BlobDirectory blockBlobPartitionDirectory; + + readonly TimeSpan LeaseDuration = TimeSpan.FromSeconds(45); // max time the lease stays after unclean shutdown + readonly TimeSpan LeaseRenewal = TimeSpan.FromSeconds(30); // how often we renew the lease + readonly TimeSpan LeaseSafetyBuffer = TimeSpan.FromSeconds(10); // how much time we want left on the lease before issuing a protected access + + internal CheckpointInfo CheckpointInfo { get; } + Azure.ETag? CheckpointInfoETag { get; set; } + + internal FasterTraceHelper TraceHelper { get; private set; } + internal FasterTraceHelper StorageTracer => this.TraceHelper.IsTracingAtMostDetailedLevel ? this.TraceHelper : null; + + public IDevice EventLogDevice { get; private set; } + public IDevice HybridLogDevice { get; private set; } + public IDevice ObjectLogDevice { get; private set; } + + public DateTime IncarnationTimestamp { get; private set; } + + public string ContainerName { get; } + + internal BlobUtilsV12.ContainerClients BlockBlobContainer => this.blockBlobContainer; + internal BlobUtilsV12.ContainerClients PageBlobContainer => this.pageBlobContainer; + + public int PartitionId => (int)this.partitionId; + + public IPartitionErrorHandler PartitionErrorHandler { get; private set; } + + internal static SemaphoreSlim AsynchronousStorageReadMaxConcurrency = new SemaphoreSlim(Math.Min(100, Environment.ProcessorCount * 10)); + internal static SemaphoreSlim AsynchronousStorageWriteMaxConcurrency = new SemaphoreSlim(Math.Min(50, Environment.ProcessorCount * 7)); + + internal volatile int LeaseUsers; + + volatile System.Diagnostics.Stopwatch leaseTimer; + + internal const long HashTableSize = 1L << 14; // 16 k buckets, 1 MB + internal const long HashTableSizeBytes = HashTableSize * 64; + + public class FasterTuningParameters + { + public int? EventLogPageSizeBits; + public int? EventLogSegmentSizeBits; + public int? EventLogMemorySizeBits; + public int? StoreLogPageSizeBits; + public int? StoreLogSegmentSizeBits; + public int? StoreLogMemorySizeBits; + public double? StoreLogMutableFraction; + public int? EstimatedAverageObjectSize; + public int? NumPagesToPreload; + } + + public FasterLogSettings GetDefaultEventLogSettings(bool useSeparatePageBlobStorage, FasterTuningParameters tuningParameters) => new FasterLogSettings + { + LogDevice = this.EventLogDevice, + LogCommitManager = this.UseLocalFiles + ? null // TODO: fix this: new LocalLogCommitManager($"{this.LocalDirectoryPath}\\{this.PartitionFolderName}\\{CommitBlobName}") + : (ILogCommitManager)this, + PageSizeBits = tuningParameters?.EventLogPageSizeBits ?? 21, // 2MB + SegmentSizeBits = tuningParameters?.EventLogSegmentSizeBits ?? + (useSeparatePageBlobStorage ? 35 // 32 GB + : 26), // 64 MB + MemorySizeBits = tuningParameters?.EventLogMemorySizeBits ?? 22, // 2MB + }; + + public LogSettings GetDefaultStoreLogSettings( + bool useSeparatePageBlobStorage, + long upperBoundOnAvailable, + FasterTuningParameters tuningParameters) + { + int pageSizeBits = tuningParameters?.StoreLogPageSizeBits ?? 10; // default page size is 1k + + // compute a reasonable memory size for the log considering maximally available memory, and expansion factor + int memorybits = 0; + if (tuningParameters?.StoreLogMemorySizeBits != null) + { + memorybits = tuningParameters.StoreLogMemorySizeBits.Value; + } + else + { + double expansionFactor = (24 + ((double)(tuningParameters?.EstimatedAverageObjectSize ?? 216))) / 24; + long estimate = (long)(upperBoundOnAvailable / expansionFactor); + + while (estimate > 0) + { + memorybits++; + estimate >>= 1; + } + memorybits = Math.Max(pageSizeBits + 2, memorybits); // never use less than 4 pages + } + + return new LogSettings + { + LogDevice = this.HybridLogDevice, + ObjectLogDevice = this.ObjectLogDevice, + PageSizeBits = pageSizeBits, + MutableFraction = tuningParameters?.StoreLogMutableFraction ?? 0.9, + SegmentSizeBits = tuningParameters?.StoreLogSegmentSizeBits ?? + (useSeparatePageBlobStorage ? 35 // 32 GB + : 32), // 4 GB + PreallocateLog = false, + ReadFlags = ReadFlags.None, + ReadCacheSettings = null, // no read cache + MemorySizeBits = memorybits, + }; + } + + + static readonly int[] StorageFormatVersion = new int[] { + 1, //initial version + 2, //0.7.0-beta changed singleton storage, and adds dequeue count + 3, //changed organization of files + 4, //use Faster v2, reduced page size + 5, //support EventHub recovery + }; + + public static string GetStorageFormat(NetheriteOrchestrationServiceSettings settings) + { + return JsonConvert.SerializeObject(new StorageFormatSettings() + { + UseAlternateObjectStore = settings.UseAlternateObjectStore, + FormatVersion = StorageFormatVersion.Last(), + }, + serializerSettings); + } + + [JsonObject] + class StorageFormatSettings + { + // this must stay the same + + [JsonProperty("FormatVersion")] + public int FormatVersion { get; set; } + + // the following can be changed between versions + + [JsonProperty("UseAlternateObjectStore", DefaultValueHandling = DefaultValueHandling.Ignore)] + public bool? UseAlternateObjectStore { get; set; } + } + + static readonly JsonSerializerSettings serializerSettings = new JsonSerializerSettings() + { + TypeNameHandling = TypeNameHandling.None, + MissingMemberHandling = MissingMemberHandling.Ignore, + CheckAdditionalContent = false, + Formatting = Formatting.None, + }; + + public static void CheckStorageFormat(string format, NetheriteOrchestrationServiceSettings settings) + { + try + { + var taskhubFormat = JsonConvert.DeserializeObject(format, serializerSettings); + + if (taskhubFormat.UseAlternateObjectStore != settings.UseAlternateObjectStore) + { + throw new NetheriteConfigurationException("The Netherite configuration setting 'UseAlternateObjectStore' is incompatible with the existing taskhub."); + } + if (taskhubFormat.FormatVersion != StorageFormatVersion.Last()) + { + throw new NetheriteConfigurationException($"The current storage format version (={StorageFormatVersion.Last()}) is incompatible with the existing taskhub (={taskhubFormat.FormatVersion})."); + } + } + catch (Exception e) + { + throw new NetheriteConfigurationException("The taskhub has an incompatible storage format", e); + } + } + + public void Dispose() + { + // we do not need to dispose any resources for the commit manager, because any such resources are deleted together with the taskhub + } + + public void Purge(Guid token) + { + throw new NotImplementedException("Purges are handled directly on recovery, not via FASTER"); + } + + public void PurgeAll() + { + throw new NotImplementedException("Purges are handled directly on recovery, not via FASTER"); + } + + public void OnRecovery(Guid indexToken, Guid logToken) + { + // we handle cleanup of old checkpoints somewhere else + } + + public CheckpointSettings StoreCheckpointSettings => new CheckpointSettings + { + CheckpointManager = this.UseLocalFiles ? (ICheckpointManager)this.LocalCheckpointManager : (ICheckpointManager)this, + }; + + public const int MaxRetries = 10; + + public static TimeSpan GetDelayBetweenRetries(int numAttempts) + => TimeSpan.FromSeconds(Math.Pow(2, (numAttempts - 1))); + + /// + /// Create a blob manager. + /// + /// The cloud storage account, or null if using local file paths + /// The storage account to use for page blobs + /// The local file path, or null if using cloud storage + /// The name of the taskhub + /// A logger for logging + /// A limit on log event level emitted + /// The partition id + /// A handler for errors encountered in this partition + public BlobManager( + NetheriteOrchestrationServiceSettings settings, + string taskHubName, + string taskHubPrefix, + FaultInjector faultInjector, + ILogger logger, + ILogger performanceLogger, + Microsoft.Extensions.Logging.LogLevel logLevelLimit, + uint partitionId, + IPartitionErrorHandler errorHandler) + { + this.settings = settings; + this.ContainerName = GetContainerName(taskHubName); + this.taskHubPrefix = taskHubPrefix; + this.FaultInjector = faultInjector; + this.partitionId = partitionId; + this.CheckpointInfo = new CheckpointInfo(); + this.CheckpointInfoETag = default; + + if (!string.IsNullOrEmpty(settings.UseLocalDirectoryForPartitionStorage)) + { + this.UseLocalFiles = true; + this.LocalFileDirectoryForTestingAndDebugging = settings.UseLocalDirectoryForPartitionStorage; + this.LocalCheckpointManager = new LocalFileCheckpointManager( + this.CheckpointInfo, + this.LocalCheckpointDirectoryPath, + this.GetCheckpointCompletedBlobName()); + } + else + { + this.UseLocalFiles = false; + } + this.TraceHelper = new FasterTraceHelper(logger, logLevelLimit, performanceLogger, this.partitionId, this.UseLocalFiles ? "none" : this.settings.StorageAccountName, taskHubName); + this.PartitionErrorHandler = errorHandler; + this.shutDownOrTermination = CancellationTokenSource.CreateLinkedTokenSource(errorHandler.Token); + } + + string PartitionFolderName => $"{this.taskHubPrefix}p{this.partitionId:D2}"; + + // For testing and debugging with local files + bool UseLocalFiles { get; } + LocalFileCheckpointManager LocalCheckpointManager { get; set; } + string LocalFileDirectoryForTestingAndDebugging { get; } + string LocalDirectoryPath => $"{this.LocalFileDirectoryForTestingAndDebugging}\\{this.ContainerName}"; + string LocalCheckpointDirectoryPath => $"{this.LocalDirectoryPath}\\chkpts{this.partitionId:D2}"; + + const string EventLogBlobName = "commit-log"; + const string CommitBlobName = "commit-lease"; + const string HybridLogBlobName = "store"; + const string ObjectLogBlobName = "store.obj"; + + Task LeaseMaintenanceLoopTask = Task.CompletedTask; + volatile Task NextLeaseRenewalTask = Task.CompletedTask; + + public static string GetContainerName(string taskHubName) => taskHubName.ToLowerInvariant() + "-storage"; + + public async Task StartAsync() + { + if (this.UseLocalFiles) + { + this.LocalCheckpointManager = new LocalFileCheckpointManager( + this.CheckpointInfo, + this.LocalCheckpointDirectoryPath, + this.GetCheckpointCompletedBlobName()); + + Directory.CreateDirectory($"{this.LocalDirectoryPath}\\{this.PartitionFolderName}"); + + this.EventLogDevice = Devices.CreateLogDevice($"{this.LocalDirectoryPath}\\{this.PartitionFolderName}\\{EventLogBlobName}"); + this.HybridLogDevice = Devices.CreateLogDevice($"{this.LocalDirectoryPath}\\{this.PartitionFolderName}\\{HybridLogBlobName}"); + this.ObjectLogDevice = Devices.CreateLogDevice($"{this.LocalDirectoryPath}\\{this.PartitionFolderName}\\{ObjectLogBlobName}"); + + // This does not acquire any blob ownership, but is needed for the lease maintenance loop which calls PartitionErrorHandler.TerminateNormally() when done. + await this.AcquireOwnership(); + } + else + { + this.blockBlobAccount = BlobUtilsV12.GetServiceClients(this.settings.BlobStorageConnection); + this.blockBlobContainer = BlobUtilsV12.GetContainerClients(this.blockBlobAccount, this.ContainerName); + await this.blockBlobContainer.WithRetries.CreateIfNotExistsAsync(); + this.blockBlobPartitionDirectory = new BlobUtilsV12.BlobDirectory(this.blockBlobContainer, this.PartitionFolderName); + + if (this.settings.PageBlobStorageConnection != null) + { + this.pageBlobAccount = BlobUtilsV12.GetServiceClients(this.settings.PageBlobStorageConnection); + this.pageBlobContainer = BlobUtilsV12.GetContainerClients(this.pageBlobAccount, this.ContainerName); + await this.pageBlobContainer.WithRetries.CreateIfNotExistsAsync(); + this.pageBlobPartitionDirectory = new BlobUtilsV12.BlobDirectory(this.pageBlobContainer, this.PartitionFolderName); + } + else + { + this.pageBlobAccount = this.blockBlobAccount; + this.pageBlobContainer = this.BlockBlobContainer; + this.pageBlobPartitionDirectory = this.blockBlobPartitionDirectory; + } + + this.eventLogCommitBlob = this.blockBlobPartitionDirectory.GetBlockBlobClient(CommitBlobName); + this.leaseClient = this.eventLogCommitBlob.WithRetries.GetBlobLeaseClient(); + + AzureStorageDevice createDevice(string name) => + new AzureStorageDevice(name, this.blockBlobPartitionDirectory.GetSubDirectory(name), this.pageBlobPartitionDirectory.GetSubDirectory(name), this, true); + + var eventLogDevice = createDevice(EventLogBlobName); + var hybridLogDevice = createDevice(HybridLogBlobName); + var objectLogDevice = createDevice(ObjectLogBlobName); + + await this.AcquireOwnership(); + + this.TraceHelper.FasterProgress("Starting Faster Devices"); + var startTasks = new List + { + eventLogDevice.StartAsync(), + hybridLogDevice.StartAsync(), + objectLogDevice.StartAsync() + }; + await Task.WhenAll(startTasks); + this.TraceHelper.FasterProgress("Started Faster Devices"); + + this.EventLogDevice = eventLogDevice; + this.HybridLogDevice = hybridLogDevice; + this.ObjectLogDevice = objectLogDevice; + } + } + + internal void DisposeDevices() + { + Dispose(this.HybridLogDevice); + Dispose(this.ObjectLogDevice); + + void Dispose(IDevice device) + { + this.TraceHelper.FasterStorageProgress($"Disposing Device {device.FileName}"); + device.Dispose(); + } + } + + public void HandleStorageError(string where, string message, string blobName, Exception e, bool isFatal, bool isWarning) + { + if (blobName == null) + { + this.PartitionErrorHandler.HandleError(where, message, e, isFatal, isWarning); + } + else + { + this.PartitionErrorHandler.HandleError(where, $"{message} blob={blobName}", e, isFatal, isWarning); + } + } + + // clean shutdown, wait for everything, then terminate + public async Task StopAsync() + { + this.shutDownOrTermination.Cancel(); // has no effect if already cancelled + + await this.LeaseMaintenanceLoopTask; // wait for loop to terminate cleanly + } + + public static async Task DeleteTaskhubStorageAsync(NetheriteOrchestrationServiceSettings settings, string pathPrefix) + { + var containerName = GetContainerName(settings.HubName); + + if (!string.IsNullOrEmpty(settings.UseLocalDirectoryForPartitionStorage)) + { + DirectoryInfo di = new DirectoryInfo($"{settings.UseLocalDirectoryForPartitionStorage}\\{containerName}"); //TODO fine-grained deletion + if (di.Exists) + { + di.Delete(true); + } + } + else + { + var blockBlobAccount = BlobUtilsV12.GetServiceClients(settings.BlobStorageConnection); + await DeleteContainerContents(blockBlobAccount.WithRetries); + + if (settings.PageBlobStorageConnection != null) + { + var pageBlobAccount = BlobUtilsV12.GetServiceClients(settings.PageBlobStorageConnection); + await DeleteContainerContents(pageBlobAccount.Default); + } + + async Task DeleteContainerContents(BlobServiceClient account) + { + var container = account.GetBlobContainerClient(containerName); + var deletionTasks = new List(); + try + { + await foreach (BlobItem blob in container.GetBlobsAsync(BlobTraits.None, BlobStates.None, string.Empty)) + { + deletionTasks.Add(BlobUtilsV12.ForceDeleteAsync(container, blob.Name)); + } + } + catch (Azure.RequestFailedException e) when (e.Status == (int)HttpStatusCode.NotFound) + { + } + await Task.WhenAll(deletionTasks); + } + + // We are not deleting the container itself because it creates problems when trying to recreate + // the same container soon afterwards. So we prefer to leave an empty container behind. + } + } + + public ValueTask ConfirmLeaseIsGoodForAWhileAsync() + { + if (this.leaseTimer?.Elapsed < this.LeaseDuration - this.LeaseSafetyBuffer && !this.shutDownOrTermination.IsCancellationRequested) + { + return default; + } + this.TraceHelper.LeaseProgress("Access is waiting for fresh lease"); + return new ValueTask(this.NextLeaseRenewalTask); + } + + public void ConfirmLeaseIsGoodForAWhile() + { + if (this.leaseTimer?.Elapsed < this.LeaseDuration - this.LeaseSafetyBuffer && !this.shutDownOrTermination.IsCancellationRequested) + { + return; + } + this.TraceHelper.LeaseProgress("Access is waiting for fresh lease"); + this.NextLeaseRenewalTask.Wait(); + } + + async Task AcquireOwnership() + { + var newLeaseTimer = new System.Diagnostics.Stopwatch(); + int numAttempts = 0; + + while (true) + { + this.PartitionErrorHandler.Token.ThrowIfCancellationRequested(); + numAttempts++; + + try + { + newLeaseTimer.Restart(); + + if (!this.UseLocalFiles) + { + this.FaultInjector?.StorageAccess(this, "AcquireLeaseAsync", "AcquireOwnership", this.eventLogCommitBlob.Name); + await this.leaseClient.AcquireAsync( + this.LeaseDuration, + null, + this.PartitionErrorHandler.Token) + .ConfigureAwait(false); + this.TraceHelper.LeaseAcquired(); + } + + this.IncarnationTimestamp = DateTime.UtcNow; + this.leaseTimer = newLeaseTimer; + this.LeaseMaintenanceLoopTask = Task.Run(() => this.MaintenanceLoopAsync()); + return; + } + catch (Azure.RequestFailedException ex) when (BlobUtilsV12.LeaseConflictOrExpired(ex)) + { + this.TraceHelper.LeaseProgress("Waiting for lease"); + + this.FaultInjector?.BreakLease(this.eventLogCommitBlob); // during fault injection tests, we don't want to wait + + // the previous owner has not released the lease yet, + // try again until it becomes available, should be relatively soon + // as the transport layer is supposed to shut down the previous owner when starting this + await Task.Delay(TimeSpan.FromSeconds(1), this.PartitionErrorHandler.Token); + + continue; + } + catch (Azure.RequestFailedException ex) when (BlobUtilsV12.BlobDoesNotExist(ex)) + { + // Create blob with empty content, then try again + await this.PerformWithRetriesAsync( + null, + false, + "CloudBlockBlob.UploadFromByteArrayAsync", + "CreateCommitLog", + "", + this.eventLogCommitBlob.Default.Name, + 2000, + true, + async (numAttempts) => + { + try + { + var client = numAttempts > 2 ? this.eventLogCommitBlob.Default : this.eventLogCommitBlob.Aggressive; + await client.UploadAsync(new MemoryStream()); + } + catch (Azure.RequestFailedException ex2) when (BlobUtilsV12.LeaseConflictOrExpired(ex2)) + { + // creation race, try from top + this.TraceHelper.LeaseProgress("Creation race observed, retrying"); + } + + return 1; + }); + + continue; + } + catch (OperationCanceledException) when (this.PartitionErrorHandler.IsTerminated) + { + throw; // o.k. during termination or shutdown + } + catch (Exception e) when (this.PartitionErrorHandler.IsTerminated) + { + string message = $"Lease acquisition was canceled"; + this.TraceHelper.LeaseProgress(message); + throw new OperationCanceledException(message, e); + } + catch (Exception ex) when (numAttempts < BlobManager.MaxRetries + && !this.PartitionErrorHandler.IsTerminated && BlobUtils.IsTransientStorageError(ex)) + { + if (BlobUtils.IsTimeout(ex)) + { + this.TraceHelper.FasterPerfWarning($"Lease acquisition timed out, retrying now"); + } + else + { + TimeSpan nextRetryIn = BlobManager.GetDelayBetweenRetries(numAttempts); + this.TraceHelper.FasterPerfWarning($"Lease acquisition failed transiently, retrying in {nextRetryIn}"); + await Task.Delay(nextRetryIn); + } + continue; + } + catch (Exception e) when (!Utils.IsFatal(e)) + { + this.PartitionErrorHandler.HandleError(nameof(AcquireOwnership), "Could not acquire partition lease", e, true, false); + throw; + } + } + } + + public async Task RenewLeaseTask() + { + try + { + this.shutDownOrTermination.Token.ThrowIfCancellationRequested(); + + var nextLeaseTimer = new System.Diagnostics.Stopwatch(); + nextLeaseTimer.Start(); + + if (!this.UseLocalFiles) + { + this.TraceHelper.LeaseProgress($"Renewing lease at {this.leaseTimer.Elapsed.TotalSeconds - this.LeaseDuration.TotalSeconds}s"); + this.FaultInjector?.StorageAccess(this, "RenewLeaseAsync", "RenewLease", this.eventLogCommitBlob.Name); + await this.leaseClient.RenewAsync(null, this.PartitionErrorHandler.Token).ConfigureAwait(false); + this.TraceHelper.LeaseRenewed(this.leaseTimer.Elapsed.TotalSeconds, this.leaseTimer.Elapsed.TotalSeconds - this.LeaseDuration.TotalSeconds); + + if (nextLeaseTimer.ElapsedMilliseconds > 2000) + { + this.TraceHelper.FasterPerfWarning($"RenewLeaseAsync took {nextLeaseTimer.Elapsed.TotalSeconds:F1}s, which is excessive; {this.leaseTimer.Elapsed.TotalSeconds - this.LeaseDuration.TotalSeconds}s past expiry"); + } + } + + this.leaseTimer = nextLeaseTimer; + } + catch (OperationCanceledException) when (this.PartitionErrorHandler.IsTerminated) + { + throw; // o.k. during termination or shutdown + } + catch (Exception e) when (!Utils.IsFatal(e)) + { + this.TraceHelper.LeaseLost(this.leaseTimer.Elapsed.TotalSeconds, nameof(RenewLeaseTask)); + throw; + } + } + + public async Task MaintenanceLoopAsync() + { + this.TraceHelper.LeaseProgress("Started lease maintenance loop"); + try + { + while (true) + { + int timeLeft = (int)(this.LeaseRenewal - this.leaseTimer.Elapsed).TotalMilliseconds; + + if (timeLeft <= 0) + { + this.NextLeaseRenewalTask = this.RenewLeaseTask(); + } + else + { + this.NextLeaseRenewalTask = LeaseTimer.Instance.Schedule(timeLeft, this.RenewLeaseTask, this.shutDownOrTermination.Token); + } + + // wait for successful renewal, or exit the loop as this throws + await this.NextLeaseRenewalTask; + } + } + catch (OperationCanceledException) + { + // it's o.k. to cancel while waiting + this.TraceHelper.LeaseProgress("Lease renewal loop cleanly canceled"); + } + catch (Azure.RequestFailedException e) when (e.InnerException != null && e.InnerException is OperationCanceledException) + { + // it's o.k. to cancel a lease renewal + this.TraceHelper.LeaseProgress("Lease renewal storage operation canceled"); + } + catch (Azure.RequestFailedException ex) when (BlobUtilsV12.LeaseConflict(ex)) + { + // We lost the lease to someone else. Terminate ownership immediately. + this.PartitionErrorHandler.HandleError(nameof(MaintenanceLoopAsync), "Lost partition lease", ex, true, true); + } + catch (Exception e) when (!Utils.IsFatal(e)) + { + this.PartitionErrorHandler.HandleError(nameof(MaintenanceLoopAsync), "Could not maintain partition lease", e, true, false); + } + + this.TraceHelper.LeaseProgress("Exited lease maintenance loop"); + + while (this.LeaseUsers > 0 + && !this.PartitionErrorHandler.IsTerminated + && (this.leaseTimer?.Elapsed < this.LeaseDuration)) + { + await Task.Delay(20); // give storage accesses that are in progress and require the lease a chance to complete + } + + this.TraceHelper.LeaseProgress("Waited for lease users to complete"); + + // release the lease + if (!this.UseLocalFiles) + { + try + { + this.TraceHelper.LeaseProgress("Releasing lease"); + + this.FaultInjector?.StorageAccess(this, "ReleaseLeaseAsync", "ReleaseLease", this.eventLogCommitBlob.Name); + await this.leaseClient.ReleaseAsync(null, this.PartitionErrorHandler.Token).ConfigureAwait(false); + this.TraceHelper.LeaseReleased(this.leaseTimer.Elapsed.TotalSeconds); + } + catch (OperationCanceledException) + { + // it's o.k. if termination is triggered while waiting + } + catch (Azure.RequestFailedException e) when (e.InnerException != null && e.InnerException is OperationCanceledException) + { + // it's o.k. if termination is triggered while we are releasing the lease + } + catch (Exception e) + { + // we swallow, but still report exceptions when releasing a lease + this.PartitionErrorHandler.HandleError(nameof(MaintenanceLoopAsync), "Could not release partition lease during shutdown", e, false, true); + } + } + + this.PartitionErrorHandler.TerminateNormally(); + + this.TraceHelper.LeaseProgress("Blob manager stopped"); + } + + public async Task RemoveObsoleteCheckpoints() + { + if (this.UseLocalFiles) + { + //TODO + return; + } + else + { + string token1 = this.CheckpointInfo.LogToken.ToString(); + string token2 = this.CheckpointInfo.IndexToken.ToString(); + + this.TraceHelper.FasterProgress($"Removing obsolete checkpoints, keeping only {token1} and {token2}"); + + var tasks = new List>(); + + tasks.Add(RemoveObsoleteCheckpoints(this.blockBlobPartitionDirectory.GetSubDirectory(cprCheckpointPrefix))); + tasks.Add(RemoveObsoleteCheckpoints(this.blockBlobPartitionDirectory.GetSubDirectory(indexCheckpointPrefix))); + + if (this.settings.PageBlobStorageConnection != null) + { + tasks.Add(RemoveObsoleteCheckpoints(this.pageBlobPartitionDirectory.GetSubDirectory(cprCheckpointPrefix))); + tasks.Add(RemoveObsoleteCheckpoints(this.pageBlobPartitionDirectory.GetSubDirectory(indexCheckpointPrefix))); + } + + await Task.WhenAll(tasks); + + this.TraceHelper.FasterProgress($"Removed {tasks.Select(t => t.Result.Item1).Sum()} checkpoint directories containing {tasks.Select(t => t.Result.Item2).Sum()} blobs"); + + async Task<(int, int)> RemoveObsoleteCheckpoints(BlobUtilsV12.BlobDirectory directory) + { + List results = null; + + await this.PerformWithRetriesAsync( + BlobManager.AsynchronousStorageWriteMaxConcurrency, + true, + "BlobContainerClient.GetBlobsAsync", + "RemoveObsoleteCheckpoints", + "", + directory.Prefix, + 1000, + false, + async (numAttempts) => + { + results = await directory.GetBlobsAsync(this.shutDownOrTermination.Token); + return results.Count(); + }); + + + var checkpointFoldersToDelete = results + .GroupBy((s) => s.Split('/')[3]) + .Where(g => g.Key != token1 && g.Key != token2) + .ToList(); + + var deletionTasks = new List(); + + foreach (var folder in checkpointFoldersToDelete) + { + deletionTasks.Add(DeleteCheckpointDirectory(folder)); + } + + await Task.WhenAll(deletionTasks); + return (checkpointFoldersToDelete.Count, results.Count); + + async Task DeleteCheckpointDirectory(IEnumerable blobsToDelete) + { + var deletionTasks = new List(); + foreach (var blobName in blobsToDelete) + { + deletionTasks.Add( + this.PerformWithRetriesAsync( + BlobManager.AsynchronousStorageWriteMaxConcurrency, + false, + "BlobUtils.ForceDeleteAsync", + "DeleteCheckpointDirectory", + "", + blobName, + 1000, + false, + async (numAttempts) => (await BlobUtilsV12.ForceDeleteAsync(directory.Client.Default, blobName) ? 1 : 0))); + } + await Task.WhenAll(deletionTasks); + } + } + } + } + + #region Blob Name Management + + string GetCheckpointCompletedBlobName() => "last-checkpoint.json"; + + const string indexCheckpointPrefix = "index-checkpoints/"; + + const string cprCheckpointPrefix = "cpr-checkpoints/"; + + string GetIndexCheckpointMetaBlobName(Guid token) => $"{indexCheckpointPrefix}{token}/info.dat"; + + (string, string) GetPrimaryHashTableBlobName(Guid token) => ($"{indexCheckpointPrefix}{token}", "ht.dat"); + + string GetHybridLogCheckpointMetaBlobName(Guid token) => $"{cprCheckpointPrefix}{token}/info.dat"; + + (string, string) GetLogSnapshotBlobName(Guid token) => ($"{cprCheckpointPrefix}{token}", "snapshot.dat"); + + (string, string) GetObjectLogSnapshotBlobName(Guid token) => ($"{cprCheckpointPrefix}{token}", "snapshot.obj.dat"); + + (string, string) GetDeltaLogSnapshotBlobName(Guid token) => ($"{cprCheckpointPrefix}{token}", "snapshot.delta.dat"); + + string GetSingletonsSnapshotBlobName(Guid token) => $"{cprCheckpointPrefix}{token}/singletons.dat"; + + #endregion + + #region ILogCommitManager + + void ILogCommitManager.Commit(long beginAddress, long untilAddress, byte[] commitMetadata, long commitNum) + { + try + { + this.StorageTracer?.FasterStorageProgress($"StorageOpCalled ILogCommitManager.Commit beginAddress={beginAddress} untilAddress={untilAddress}"); + + this.PerformWithRetries( + false, + "BlockBlobClient.Upload", + "WriteCommitLogMetadata", + "", + this.eventLogCommitBlob.Default.Name, + 1000, + true, + (int numAttempts) => + { + try + { + var client = numAttempts > 2 ? this.eventLogCommitBlob.Default : this.eventLogCommitBlob.Aggressive; + + client.Upload( + content: new MemoryStream(commitMetadata), + options: new BlobUploadOptions() { Conditions = new BlobRequestConditions() { LeaseId = this.leaseClient.LeaseId } }, + cancellationToken: this.PartitionErrorHandler.Token); + + return (commitMetadata.Length, true); + } + catch (Azure.RequestFailedException ex) when (BlobUtilsV12.LeaseConflict(ex)) + { + // We lost the lease to someone else. Terminate ownership immediately. + this.TraceHelper.LeaseLost(this.leaseTimer.Elapsed.TotalSeconds, nameof(ILogCommitManager.Commit)); + this.HandleStorageError(nameof(ILogCommitManager.Commit), "could not commit because of lost lease", this.eventLogCommitBlob.Name, ex, true, this.PartitionErrorHandler.IsTerminated); + throw; + } + catch (Azure.RequestFailedException ex) when (BlobUtilsV12.LeaseExpired(ex) && numAttempts < BlobManager.MaxRetries) + { + // if we get here, the lease renewal task did not complete in time + // give it another chance to complete + this.TraceHelper.LeaseProgress("ILogCommitManager.Commit: wait for next renewal"); + this.NextLeaseRenewalTask.Wait(); + this.TraceHelper.LeaseProgress("ILogCommitManager.Commit: renewal complete"); + return (commitMetadata.Length, false); + } + }); + + this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ILogCommitManager.Commit"); + } + catch + { + this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ILogCommitManager.Commit failed"); + throw; + } + } + + + IEnumerable ILogCommitManager.ListCommits() + { + // we only use a single commit file in this implementation + yield return 0; + } + + void ILogCommitManager.OnRecovery(long commitNum) + { + // TODO: make sure our use of single commit is safe + } + + void ILogCommitManager.RemoveAllCommits() + { + // TODO: make sure our use of single commit is safe + } + + void ILogCommitManager.RemoveCommit(long commitNum) + { + // TODO: make sure our use of single commit is safe + } + + byte[] ILogCommitManager.GetCommitMetadata(long commitNum) + { + try + { + this.StorageTracer?.FasterStorageProgress($"StorageOpCalled ILogCommitManager.GetCommitMetadata (thread={Thread.CurrentThread.ManagedThreadId})"); + + using var stream = new MemoryStream(); + + this.PerformWithRetries( + false, + "BlobClient.DownloadTo", + "ReadCommitLogMetadata", + "", + this.eventLogCommitBlob.Name, + 1000, + true, + (int numAttempts) => + { + if (numAttempts > 0) + { + stream.Seek(0, SeekOrigin.Begin); + } + + try + { + var client = numAttempts > 2 ? this.eventLogCommitBlob.Default : this.eventLogCommitBlob.Aggressive; + + client.DownloadTo( + destination: stream, + conditions: new BlobRequestConditions() { LeaseId = this.leaseClient.LeaseId }, + cancellationToken: this.PartitionErrorHandler.Token); + + return (stream.Position, true); + } + catch (Azure.RequestFailedException ex) when (BlobUtilsV12.LeaseConflict(ex)) + { + // We lost the lease to someone else. Terminate ownership immediately. + this.TraceHelper.LeaseLost(this.leaseTimer.Elapsed.TotalSeconds, nameof(ILogCommitManager.GetCommitMetadata)); + this.HandleStorageError(nameof(ILogCommitManager.Commit), "could not read latest commit due to lost lease", this.eventLogCommitBlob.Name, ex, true, this.PartitionErrorHandler.IsTerminated); + throw; + } + catch (Azure.RequestFailedException ex) when (BlobUtilsV12.LeaseExpired(ex) && numAttempts < BlobManager.MaxRetries) + { + // if we get here, the lease renewal task did not complete in time + // give it another chance to complete + this.TraceHelper.LeaseProgress("ILogCommitManager.Commit: wait for next renewal"); + this.NextLeaseRenewalTask.Wait(); + this.TraceHelper.LeaseProgress("ILogCommitManager.Commit: renewal complete"); + return (0, false); + } + }); + + var bytes = stream.ToArray(); + this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ILogCommitManager.GetCommitMetadata {bytes?.Length ?? null} bytes"); + return bytes.Length == 0 ? null : bytes; + } + catch (Exception e) + { + this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ILogCommitManager.GetCommitMetadata failed with {e.GetType().Name}: {e.Message}"); + throw; + } + } + + #endregion + + #region ICheckpointManager + + void ICheckpointManager.InitializeIndexCheckpoint(Guid indexToken) + { + // there is no need to create empty directories in a blob container + } + + void ICheckpointManager.InitializeLogCheckpoint(Guid logToken) + { + // there is no need to create empty directories in a blob container + } + + IEnumerable ICheckpointManager.GetIndexCheckpointTokens() + { + var indexToken = this.CheckpointInfo.IndexToken; + this.StorageTracer?.FasterStorageProgress($"StorageOp ICheckpointManager.GetIndexCheckpointTokens indexToken={indexToken}"); + yield return indexToken; + } + + IEnumerable ICheckpointManager.GetLogCheckpointTokens() + { + var logToken = this.CheckpointInfo.LogToken; + this.StorageTracer?.FasterStorageProgress($"StorageOp ICheckpointManager.GetLogCheckpointTokens logToken={logToken}"); + yield return logToken; + } + + internal async Task FindCheckpointsAsync(bool logIsEmpty) + { + BlobUtilsV12.BlockBlobClients checkpointCompletedBlob = default; + try + { + string jsonString = null; + + if (this.UseLocalFiles) + { + try + { + jsonString = this.LocalCheckpointManager.GetLatestCheckpointJson(); + } + catch (FileNotFoundException) when (logIsEmpty) + { + // ok to not have a checkpoint yet + } + } + else + { + var partDir = this.blockBlobPartitionDirectory; + checkpointCompletedBlob = partDir.GetBlockBlobClient(this.GetCheckpointCompletedBlobName()); + + await this.PerformWithRetriesAsync( + semaphore: null, + requireLease: true, + "BlockBlobClient.DownloadContentAsync", + "FindCheckpointsAsync", + "", + checkpointCompletedBlob.Name, + 1000, + true, + async (numAttempts) => + { + try + { + Azure.Response downloadResult = await checkpointCompletedBlob.WithRetries.DownloadContentAsync(); + jsonString = downloadResult.Value.Content.ToString(); + this.CheckpointInfoETag = downloadResult.Value.Details.ETag; + return 1; + } + catch (Azure.RequestFailedException e) when (BlobUtilsV12.BlobDoesNotExist(e) && logIsEmpty) + { + // ok to not have a checkpoint yet + return 0; + } + }); + } + + if (jsonString == null) + { + return false; + } + else + { + // read the fields from the json to update the checkpoint info + JsonConvert.PopulateObject(jsonString, this.CheckpointInfo); + return true; + } + } + catch (Exception e) + { + this.HandleStorageError(nameof(FindCheckpointsAsync), "could not find any checkpoint", checkpointCompletedBlob.Name, e, true, this.PartitionErrorHandler.IsTerminated); + throw; + } + } + + void ICheckpointManager.CommitIndexCheckpoint(Guid indexToken, byte[] commitMetadata) + { + try + { + this.StorageTracer?.FasterStorageProgress($"StorageOpCalled ICheckpointManager.CommitIndexCheckpoint, indexToken={indexToken}"); + var partDir = this.blockBlobPartitionDirectory; + var metaFileBlob = partDir.GetBlockBlobClient(this.GetIndexCheckpointMetaBlobName(indexToken)); + + this.PerformWithRetries( + false, + "BlockBlobClient.OpenWrite", + "WriteIndexCheckpointMetadata", + $"token={indexToken} size={commitMetadata.Length}", + metaFileBlob.Name, + 1000, + true, + (numAttempts) => + { + var client = metaFileBlob.WithRetries; + using var blobStream = client.OpenWrite(overwrite: true); + using var writer = new BinaryWriter(blobStream); + writer.Write(commitMetadata.Length); + writer.Write(commitMetadata); + writer.Flush(); + return (commitMetadata.Length, true); + }); + + this.CheckpointInfo.IndexToken = indexToken; + this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.CommitIndexCheckpoint, target={metaFileBlob.Name}"); + } + catch + { + this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.CommitIndexCheckpoint failed"); + throw; + } + } + + void ICheckpointManager.CommitLogCheckpoint(Guid logToken, byte[] commitMetadata) + { + try + { + this.StorageTracer?.FasterStorageProgress($"StorageOpCalled ICheckpointManager.CommitLogCheckpoint, logToken={logToken}"); + var partDir = this.blockBlobPartitionDirectory; + var metaFileBlob = partDir.GetBlockBlobClient(this.GetHybridLogCheckpointMetaBlobName(logToken)); + + this.PerformWithRetries( + false, + "BlockBlobClient.OpenWrite", + "WriteHybridLogCheckpointMetadata", + $"token={logToken}", + metaFileBlob.Name, + 1000, + true, + (numAttempts) => + { + var client = metaFileBlob.WithRetries; + using var blobStream = client.OpenWrite(overwrite: true); + using var writer = new BinaryWriter(blobStream); + writer.Write(commitMetadata.Length); + writer.Write(commitMetadata); + writer.Flush(); + return (commitMetadata.Length + 4, true); + }); + + this.CheckpointInfo.LogToken = logToken; + this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.CommitLogCheckpoint, target={metaFileBlob.Name}"); + } + catch + { + this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.CommitLogCheckpoint failed"); + throw; + } + } + + void ICheckpointManager.CommitLogIncrementalCheckpoint(Guid logToken, long version, byte[] commitMetadata, DeltaLog deltaLog) + { + throw new NotImplementedException("incremental checkpointing is not implemented"); + } + + byte[] ICheckpointManager.GetIndexCheckpointMetadata(Guid indexToken) + { + try + { + this.StorageTracer?.FasterStorageProgress($"StorageOpCalled ICheckpointManager.GetIndexCheckpointMetadata, indexToken={indexToken}"); + var partDir = this.blockBlobPartitionDirectory; + var metaFileBlob = partDir.GetBlockBlobClient(this.GetIndexCheckpointMetaBlobName(indexToken)); + byte[] result = null; + + this.PerformWithRetries( + false, + "BlockBlobClient.OpenRead", + "ReadIndexCheckpointMetadata", + "", + metaFileBlob.Name, + 1000, + true, + (numAttempts) => + { + var client = metaFileBlob.WithRetries; + using var blobstream = client.OpenRead(); + using var reader = new BinaryReader(blobstream); + var len = reader.ReadInt32(); + result = reader.ReadBytes(len); + return (len + 4, true); + }); + + this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.GetIndexCheckpointMetadata {result?.Length ?? null} bytes, target={metaFileBlob.Name}"); + return result; + } + catch + { + this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.GetIndexCheckpointMetadata failed"); + throw; + } + } + + byte[] ICheckpointManager.GetLogCheckpointMetadata(Guid logToken, DeltaLog deltaLog, bool scanDelta, long recoverTo) + { + try + { + this.StorageTracer?.FasterStorageProgress($"StorageOpCalled ICheckpointManager.GetLogCheckpointMetadata, logToken={logToken}"); + var partDir = this.blockBlobPartitionDirectory; + var metaFileBlob = partDir.GetBlockBlobClient(this.GetHybridLogCheckpointMetaBlobName(logToken)); + byte[] result = null; + + this.PerformWithRetries( + false, + "BlockBlobClient.OpenRead", + "ReadLogCheckpointMetadata", + "", + metaFileBlob.Name, + 1000, + true, + (numAttempts) => + { + var client = metaFileBlob.WithRetries; + using var blobstream = client.OpenRead(); + using var reader = new BinaryReader(blobstream); + var len = reader.ReadInt32(); + result = reader.ReadBytes(len); + return (len + 4, true); + }); + + this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.GetLogCheckpointMetadata {result?.Length ?? null} bytes, target={metaFileBlob.Name}"); + return result; + } + catch + { + this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.GetLogCheckpointMetadata failed"); + throw; + } + } + + void GetPartitionDirectories(string path, out BlobUtilsV12.BlobDirectory blockBlobDir, out BlobUtilsV12.BlobDirectory pageBlobDir) + { + var blockPartDir = this.blockBlobPartitionDirectory; + blockBlobDir = blockPartDir.GetSubDirectory(path); + var pagePartDir = this.pageBlobPartitionDirectory; + pageBlobDir = pagePartDir.GetSubDirectory(path); + } + + IDevice ICheckpointManager.GetIndexDevice(Guid indexToken) + { + try + { + this.StorageTracer?.FasterStorageProgress($"StorageOpCalled ICheckpointManager.GetIndexDevice, indexToken={indexToken}"); + var (path, blobName) = this.GetPrimaryHashTableBlobName(indexToken); + this.GetPartitionDirectories(path, out var blockBlobDir, out var pageBlobDir); + var device = new AzureStorageDevice(blobName, blockBlobDir, pageBlobDir, this, false); // we don't need a lease since the token provides isolation + device.StartAsync().Wait(); + this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.GetIndexDevice, target={blockBlobDir}{blobName}"); + return device; + } + catch + { + this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.GetIndexDevice failed"); + throw; + } + } + + IDevice ICheckpointManager.GetSnapshotLogDevice(Guid token) + { + try + { + this.StorageTracer?.FasterStorageProgress($"StorageOpCalled ICheckpointManager.GetSnapshotLogDevice, token={token}"); + var (path, blobName) = this.GetLogSnapshotBlobName(token); + this.GetPartitionDirectories(path, out var blockBlobDir, out var pageBlobDir); + var device = new AzureStorageDevice(blobName, blockBlobDir, pageBlobDir, this, false); // we don't need a lease since the token provides isolation + device.StartAsync().Wait(); + this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.GetSnapshotLogDevice, target={blockBlobDir}{blobName}"); + return device; + } + catch + { + this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.GetSnapshotLogDevice failed"); + throw; + } + } + + IDevice ICheckpointManager.GetSnapshotObjectLogDevice(Guid token) + { + try + { + this.StorageTracer?.FasterStorageProgress($"StorageOpCalled ICheckpointManager.GetSnapshotObjectLogDevice, token={token}"); + var (path, blobName) = this.GetObjectLogSnapshotBlobName(token); + this.GetPartitionDirectories(path, out var blockBlobDir, out var pageBlobDir); + var device = new AzureStorageDevice(blobName, blockBlobDir, pageBlobDir, this, false); // we don't need a lease since the token provides isolation + device.StartAsync().Wait(); + this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.GetSnapshotObjectLogDevice, target={blockBlobDir}{blobName}"); + return device; + } + catch + { + this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.GetSnapshotObjectLogDevice failed"); + throw; + } + } + + IDevice ICheckpointManager.GetDeltaLogDevice(Guid token) + { + try + { + this.StorageTracer?.FasterStorageProgress($"StorageOpCalled ICheckpointManager.GetDeltaLogDevice on, token={token}"); + var (path, blobName) = this.GetDeltaLogSnapshotBlobName(token); + this.GetPartitionDirectories(path, out var blockBlobDir, out var pageBlobDir); + var device = new AzureStorageDevice(blobName, blockBlobDir, pageBlobDir, this, false); // we don't need a lease since the token provides isolation + device.StartAsync().Wait(); + this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.GetDeltaLogDevice, target={blockBlobDir}{blobName}"); + return device; + } + catch + { + this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.GetDeltaLogDevice failed"); + throw; + } + } + + #endregion + + internal async Task PersistSingletonsAsync(byte[] singletons, Guid guid) + { + if (this.UseLocalFiles) + { + var path = Path.Combine(this.LocalCheckpointDirectoryPath, this.GetSingletonsSnapshotBlobName(guid)); + using var filestream = File.OpenWrite(path); + await filestream.WriteAsync(singletons, 0, singletons.Length); + await filestream.FlushAsync(); + } + else + { + var singletonsBlob = this.blockBlobPartitionDirectory.GetBlockBlobClient(this.GetSingletonsSnapshotBlobName(guid)); + await this.PerformWithRetriesAsync( + BlobManager.AsynchronousStorageWriteMaxConcurrency, + false, + "BlockBlobClient.UploadAsync", + "WriteSingletons", + "", + singletonsBlob.Name, + 1000 + singletons.Length / 5000, + false, + async (numAttempts) => + { + var client = singletonsBlob.WithRetries; + await client.UploadAsync( + new MemoryStream(singletons), + new BlobUploadOptions(), + this.PartitionErrorHandler.Token); + return singletons.Length; + }); + } + } + + internal async Task RecoverSingletonsAsync() + { + if (this.UseLocalFiles) + { + var path = Path.Combine(this.LocalCheckpointDirectoryPath, this.GetSingletonsSnapshotBlobName(this.CheckpointInfo.LogToken)); + var stream = File.OpenRead(path); + return stream; + } + else + { + var singletonsBlob = this.blockBlobPartitionDirectory.GetBlockBlobClient(this.GetSingletonsSnapshotBlobName(this.CheckpointInfo.LogToken)); + var stream = new MemoryStream(); + await this.PerformWithRetriesAsync( + BlobManager.AsynchronousStorageReadMaxConcurrency, + true, + "BlobBaseClient.DownloadToAsync", + "ReadSingletons", + "", + singletonsBlob.Name, + 20000, + true, + async (numAttempts) => + { + + var client = singletonsBlob.WithRetries; + var memoryStream = new MemoryStream(); + await client.DownloadToAsync(stream); + return stream.Position; + }); + + stream.Seek(0, SeekOrigin.Begin); + return stream; + } + } + + internal async Task FinalizeCheckpointCompletedAsync() + { + var jsonText = JsonConvert.SerializeObject(this.CheckpointInfo, Formatting.Indented); + if (this.UseLocalFiles) + { + File.WriteAllText(Path.Combine(this.LocalCheckpointDirectoryPath, this.GetCheckpointCompletedBlobName()), jsonText); + } + else + { + var checkpointCompletedBlob = this.blockBlobPartitionDirectory.GetBlockBlobClient(this.GetCheckpointCompletedBlobName()); + await this.PerformWithRetriesAsync( + BlobManager.AsynchronousStorageWriteMaxConcurrency, + true, + "BlockBlobClient.UploadAsync", + "WriteCheckpointMetadata", + "", + checkpointCompletedBlob.Name, + 1000, + true, + async (numAttempts) => + { + var client = numAttempts > 1 ? checkpointCompletedBlob.Default : checkpointCompletedBlob.Aggressive; + + var azureResponse = await client.UploadAsync( + new MemoryStream(Encoding.UTF8.GetBytes(jsonText)), + new BlobUploadOptions() + { + Conditions = this.CheckpointInfoETag.HasValue ? + new BlobRequestConditions() { IfMatch = this.CheckpointInfoETag.Value } + : new BlobRequestConditions() { IfNoneMatch = Azure.ETag.All }, + HttpHeaders = new BlobHttpHeaders() { ContentType = "application/json" }, + }, + this.PartitionErrorHandler.Token); + + this.CheckpointInfoETag = azureResponse.Value.ETag; + + return jsonText.Length; + }, + async () => + { + var response = await checkpointCompletedBlob.Default.GetPropertiesAsync(); + this.CheckpointInfoETag = response.Value.ETag; + }); + } + } + } +} diff --git a/cs/src/devices/AzureStorageDevice/BlobUtils.cs b/cs/src/devices/AzureStorageDevice/BlobUtils.cs new file mode 100644 index 000000000..fef2115d8 --- /dev/null +++ b/cs/src/devices/AzureStorageDevice/BlobUtils.cs @@ -0,0 +1,73 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT License. + +namespace DurableTask.Netherite +{ + using System; + using System.Threading.Tasks; + + static class BlobUtils + { + /// + /// Checks whether the given storage exception is transient, and + /// therefore meaningful to retry. + /// + /// The storage exception. + /// Whether this is a transient storage exception. + public static bool IsTransientStorageError(Exception exception) + { + // handle Azure V12 SDK exceptions + if (exception is Azure.RequestFailedException e1 && httpStatusIndicatesTransientError(e1.Status)) + { + return true; + } + + // Empirically observed: timeouts on synchronous calls + if (exception.InnerException is TimeoutException) + { + return true; + } + + // Empirically observed: transient cancellation exceptions that are not application initiated + if (exception is OperationCanceledException || exception.InnerException is OperationCanceledException) + { + return true; + } + + // Empirically observed: transient exception ('An existing connection was forcibly closed by the remote host') + if (exception.InnerException is System.Net.Http.HttpRequestException && exception.InnerException?.InnerException is System.IO.IOException) + { + return true; + } + + // Empirically observed: transient socket exceptions + if (exception is System.IO.IOException && exception.InnerException is System.Net.Sockets.SocketException) + { + return true; + } + + return false; + } + + /// + /// Checks whether the given exception is a timeout exception. + /// + /// The exception. + /// Whether this is a timeout storage exception. + public static bool IsTimeout(Exception exception) + { + return exception is System.TimeoutException + || (exception is Azure.RequestFailedException e1 && (e1.Status == 408 || e1.ErrorCode == "OperationTimedOut")) + || (exception is TaskCanceledException & exception.Message.StartsWith("The operation was cancelled because it exceeded the configured timeout")); + } + + // Transient http status codes as documented at https://docs.microsoft.com/en-us/azure/architecture/best-practices/retry-service-specific#azure-storage + static bool httpStatusIndicatesTransientError(int? statusCode) => + (statusCode == 408 //408 Request Timeout + || statusCode == 429 //429 Too Many Requests + || statusCode == 500 //500 Internal Server Error + || statusCode == 502 //502 Bad Gateway + || statusCode == 503 //503 Service Unavailable + || statusCode == 504); //504 Gateway Timeout + } +} diff --git a/cs/src/devices/AzureStorageDevice/BlobUtilsV12.cs b/cs/src/devices/AzureStorageDevice/BlobUtilsV12.cs new file mode 100644 index 000000000..f4530e30e --- /dev/null +++ b/cs/src/devices/AzureStorageDevice/BlobUtilsV12.cs @@ -0,0 +1,244 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT License. + +namespace DurableTask.Netherite +{ + using System; + using System.Collections.Generic; + using System.Threading; + using System.Threading.Tasks; + using Azure.Core; + using Azure.Core.Pipeline; + using Azure.Storage.Blobs; + using Azure.Storage.Blobs.Models; + using Azure.Storage.Blobs.Specialized; + + static class BlobUtilsV12 + { + public class ServerTimeoutPolicy : HttpPipelineSynchronousPolicy + { + readonly int timeout; + + public ServerTimeoutPolicy(int timeout) + { + this.timeout = timeout; + } + + public override void OnSendingRequest(HttpMessage message) + { + message.Request.Uri.AppendQuery("timeout", this.timeout.ToString()); + } + } + public struct ServiceClients + { + public BlobServiceClient Default; + public BlobServiceClient Aggressive; + public BlobServiceClient WithRetries; + } + + internal static ServiceClients GetServiceClients(string connectionString) + { + var aggressiveOptions = new BlobClientOptions(); + aggressiveOptions.Retry.MaxRetries = 0; + aggressiveOptions.Retry.NetworkTimeout = TimeSpan.FromSeconds(3); + aggressiveOptions.AddPolicy(new ServerTimeoutPolicy(2), HttpPipelinePosition.PerCall); + + var defaultOptions = new BlobClientOptions(); + defaultOptions.Retry.MaxRetries = 0; + defaultOptions.Retry.NetworkTimeout = TimeSpan.FromSeconds(16); + defaultOptions.AddPolicy(new ServerTimeoutPolicy(15), HttpPipelinePosition.PerCall); + + var withRetriesOptions = new BlobClientOptions(); + withRetriesOptions.Retry.MaxRetries = 10; + withRetriesOptions.Retry.Mode = RetryMode.Exponential; + withRetriesOptions.Retry.Delay = TimeSpan.FromSeconds(1); + withRetriesOptions.Retry.MaxDelay = TimeSpan.FromSeconds(30); + + return new ServiceClients() + { + Default = new BlobServiceClient(connectionString, defaultOptions), + Aggressive = new BlobServiceClient(connectionString, aggressiveOptions), + WithRetries = new BlobServiceClient(connectionString, withRetriesOptions), + }; + } + + public struct ContainerClients + { + public BlobContainerClient Default; + public BlobContainerClient Aggressive; + public BlobContainerClient WithRetries; + } + + internal static ContainerClients GetContainerClients(ServiceClients serviceClients, string blobContainerName) + { + return new ContainerClients() + { + Default = serviceClients.Default.GetBlobContainerClient(blobContainerName), + Aggressive = serviceClients.Aggressive.GetBlobContainerClient(blobContainerName), + WithRetries = serviceClients.WithRetries.GetBlobContainerClient(blobContainerName), + }; + + } + + public struct BlockBlobClients + { + public BlockBlobClient Default; + public BlockBlobClient Aggressive; + public BlockBlobClient WithRetries; + + public string Name => this.Default?.Name; + } + + internal static BlockBlobClients GetBlockBlobClients(ContainerClients containerClients, string blobName) + { + return new BlockBlobClients() + { + Default = containerClients.Default.GetBlockBlobClient(blobName), + Aggressive = containerClients.Aggressive.GetBlockBlobClient(blobName), + WithRetries = containerClients.WithRetries.GetBlockBlobClient(blobName), + }; + + } + + public struct PageBlobClients + { + public PageBlobClient Default; + public PageBlobClient Aggressive; + } + + internal static PageBlobClients GetPageBlobClients(ContainerClients containerClients, string blobName) + { + return new PageBlobClients() + { + Default = containerClients.Default.GetPageBlobClient(blobName), + Aggressive = containerClients.Aggressive.GetPageBlobClient(blobName), + }; + + } + + public struct BlobDirectory + { + readonly ContainerClients client; + readonly string prefix; + + public ContainerClients Client => this.client; + public string Prefix => this.prefix; + + public BlobDirectory(ContainerClients client, string prefix) + { + this.client = client; + this.prefix = string.Concat(prefix); + } + + public BlobDirectory GetSubDirectory(string path) + { + return new BlobDirectory(this.client, $"{this.prefix}/{path}"); + } + + public BlobUtilsV12.BlockBlobClients GetBlockBlobClient(string name) + { + return BlobUtilsV12.GetBlockBlobClients(this.client, $"{this.prefix}/{name}"); + } + + public BlobUtilsV12.PageBlobClients GetPageBlobClient(string name) + { + return BlobUtilsV12.GetPageBlobClients(this.client, $"{this.prefix}/{name}"); + } + + public async Task> GetBlobsAsync(CancellationToken cancellationToken) + { + var list = new List(); + await foreach (var blob in this.client.WithRetries.GetBlobsAsync(prefix: this.prefix, cancellationToken: cancellationToken)) + { + list.Add(blob.Name); + } + return list; + } + + public override string ToString() + { + return $"{this.prefix}/"; + } + } + + /// + /// Forcefully deletes a blob. + /// + /// The CloudBlob to delete. + /// A task that completes when the operation is finished. + public static async Task ForceDeleteAsync(BlobContainerClient containerClient, string blobName) + { + var blob = containerClient.GetBlobClient(blobName); + + try + { + await blob.DeleteAsync(); + return true; + } + catch (Azure.RequestFailedException e) when (BlobDoesNotExist(e)) + { + return false; + } + catch (Azure.RequestFailedException e) when (CannotDeleteBlobWithLease(e)) + { + try + { + var leaseClient = new BlobLeaseClient(blob); + await leaseClient.BreakAsync(TimeSpan.Zero); + } + catch + { + // we ignore exceptions in the lease breaking since there could be races + } + + // retry the delete + try + { + await blob.DeleteAsync(); + return true; + } + catch (Azure.RequestFailedException ex) when (BlobDoesNotExist(ex)) + { + return false; + } + } + } + + // Lease error codes are documented at https://docs.microsoft.com/en-us/rest/api/storageservices/lease-blob + + public static bool LeaseConflictOrExpired(Azure.RequestFailedException e) + { + return e.Status == 409 || e.Status == 412; + } + + public static bool LeaseConflict(Azure.RequestFailedException e) + { + return e.Status == 409; + } + + public static bool LeaseExpired(Azure.RequestFailedException e) + { + return e.Status == 412; + } + + public static bool CannotDeleteBlobWithLease(Azure.RequestFailedException e) + { + return e.Status == 412; + } + + public static bool PreconditionFailed(Azure.RequestFailedException e) + { + return e.Status == 409 || e.Status == 412; + } + + public static bool BlobDoesNotExist(Azure.RequestFailedException e) + { + return e.Status == 404 && e.ErrorCode == BlobErrorCode.BlobNotFound; + } + + public static bool BlobAlreadyExists(Azure.RequestFailedException e) + { + return e.Status == 409; + } + } +} diff --git a/cs/src/devices/AzureStorageDevice/FASTER.devices.AzureStorageDevice.csproj b/cs/src/devices/AzureStorageDevice/FASTER.devices.AzureStorageDevice.csproj index a2a5cfa41..da9ff812e 100644 --- a/cs/src/devices/AzureStorageDevice/FASTER.devices.AzureStorageDevice.csproj +++ b/cs/src/devices/AzureStorageDevice/FASTER.devices.AzureStorageDevice.csproj @@ -37,7 +37,7 @@ - + diff --git a/cs/src/devices/AzureStorageDevice/FASTER.devices.AzureStorageDevice.nuspec b/cs/src/devices/AzureStorageDevice/FASTER.devices.AzureStorageDevice.nuspec index fbd529226..f5d4092ba 100644 --- a/cs/src/devices/AzureStorageDevice/FASTER.devices.AzureStorageDevice.nuspec +++ b/cs/src/devices/AzureStorageDevice/FASTER.devices.AzureStorageDevice.nuspec @@ -17,19 +17,19 @@ - + - + - + - + diff --git a/cs/src/devices/AzureStorageDevice/IPartitionErrorHandler.cs b/cs/src/devices/AzureStorageDevice/IPartitionErrorHandler.cs new file mode 100644 index 000000000..beaed34f2 --- /dev/null +++ b/cs/src/devices/AzureStorageDevice/IPartitionErrorHandler.cs @@ -0,0 +1,57 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT License. + +namespace DurableTask.Netherite +{ + using System; + using System.Collections.Generic; + using System.Text; + using System.Threading; + using System.Threading.Tasks; + + /// + /// A handler for fatal or non-fatal errors encountered in a partition. + /// + public interface IPartitionErrorHandler + { + /// + /// A cancellation token that is cancelled when the partition is terminated. + /// + CancellationToken Token { get; } + + /// + /// A place to subscribe (potentially non-instantaneous) cleanup actions that execute on a dedicated thread. + /// + event Action OnShutdown; + + /// + /// A boolean indicating whether the partition is terminated. + /// + bool IsTerminated { get; } + + /// + /// A boolean indicating that normal termination has been initiated as part of a shutdown. + /// + bool NormalTermination { get; } + + /// + /// Wait for all termination operations to finish + /// + Task WaitForTermination(TimeSpan timeout); + + /// + /// Error handling for the partition. + /// + /// A brief description of the component that observed the error. + /// A message describing the circumstances. + /// The exception that was observed, or null. + /// whether this partition should be terminated (i.e. recycle and recover from storage). + /// whether this error should be reported with the severity of a warning. + void HandleError(string where, string message, Exception e, bool terminatePartition, bool reportAsWarning); + + /// + /// Terminates the partition normally, after shutdown. + /// + void TerminateNormally(); + } +} diff --git a/cs/src/devices/AzureStorageDevice/LeaseTimer.cs b/cs/src/devices/AzureStorageDevice/LeaseTimer.cs new file mode 100644 index 000000000..3e4b5dbd3 --- /dev/null +++ b/cs/src/devices/AzureStorageDevice/LeaseTimer.cs @@ -0,0 +1,158 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT License. + +namespace DurableTask.Netherite.Faster +{ + using System; + using System.Diagnostics; + using System.Threading; + using System.Threading.Tasks; + + /// + /// Lease timing requires better reliability that what we get from asynchronous Task.Delay, + /// so we implement a timer wheel. + /// + class LeaseTimer + { + const int MaxDelay = 60; + const int TicksPerSecond = 4; + + static readonly Lazy instance = new Lazy(() => new LeaseTimer()); + + readonly Timer timer; + readonly Entry[] schedule = new Entry[MaxDelay * TicksPerSecond]; + public readonly object reentrancyLock = new object(); + + readonly Stopwatch stopwatch = new Stopwatch(); + int performedSteps; + + int position = 0; + + public static LeaseTimer Instance => instance.Value; + + public Action DelayWarning { get; set; } + + class Entry + { + public TaskCompletionSource Tcs; + public CancellationTokenRegistration Registration; + public Func Callback; + public Entry Next; + + public async Task Run() + { + try + { + await this.Callback().ConfigureAwait(false); + this.Tcs.TrySetResult(true); + + } + catch (Exception exception) when (!Utils.IsFatal(exception)) + { + this.Tcs.TrySetException(exception); + } + this.Registration.Dispose(); + } + + public void Cancel() + { + this.Tcs.TrySetCanceled(); + this.Registration.Dispose(); + } + + public void RunAll() + { + var _ = this.Run(); + if (this.Next != null) + { + this.Next.RunAll(); + } + } + } + + LeaseTimer() + { + this.timer = new Timer(this.Run, null, 0, 1000 / TicksPerSecond); + this.stopwatch.Start(); + } + + public void Run(object _) + { + if (Monitor.TryEnter(this.reentrancyLock)) + { + try + { + var stepsToDo = (this.stopwatch.ElapsedMilliseconds * TicksPerSecond / 1000) - this.performedSteps; + + if (stepsToDo > 5 * TicksPerSecond) + { + this.DelayWarning?.Invoke((int)stepsToDo / TicksPerSecond); + } + + for (int i = 0; i < stepsToDo; i++) + { + this.AdvancePosition(); + } + } + finally + { + Monitor.Exit(this.reentrancyLock); + } + } + } + + void AdvancePosition() + { + int position = this.position; + this.position = (position + 1) % (MaxDelay * TicksPerSecond); + this.performedSteps++; + + Entry current; + while (true) + { + current = this.schedule[position]; + if (current == null || Interlocked.CompareExchange(ref this.schedule[position], null, current) == current) + { + break; + } + } + + current?.RunAll(); + } + + public Task Schedule(int delay, Func callback, CancellationToken token) + { + if ((delay / 1000) >= MaxDelay || delay < 0) + { + throw new ArgumentException(nameof(delay)); + } + + var entry = new Entry() + { + Tcs = new TaskCompletionSource(), + Callback = callback, + }; + + entry.Registration = token.Register(entry.Cancel); + + while (true) + { + int targetPosition = (this.position + (delay * TicksPerSecond) / 1000) % (MaxDelay * TicksPerSecond); + + if (targetPosition == this.position) + { + return callback(); + } + else + { + Entry current = this.schedule[targetPosition]; + entry.Next = current; + if (Interlocked.CompareExchange(ref this.schedule[targetPosition], entry, current) == current) + { + return entry.Tcs.Task; + } + } + } + } + } +} diff --git a/cs/src/devices/AzureStorageDevice/StorageOperations.cs b/cs/src/devices/AzureStorageDevice/StorageOperations.cs new file mode 100644 index 000000000..5e9b40c42 --- /dev/null +++ b/cs/src/devices/AzureStorageDevice/StorageOperations.cs @@ -0,0 +1,214 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT License. + +namespace DurableTask.Netherite.Faster +{ + using System; + using System.Diagnostics; + using System.Threading; + using System.Threading.Tasks; + + public partial class BlobManager + { + public async Task PerformWithRetriesAsync( + SemaphoreSlim semaphore, + bool requireLease, + string name, + string intent, + string data, + string target, + int expectedLatencyBound, + bool isCritical, + Func> operationAsync, + Func readETagAsync = null) + { + try + { + if (semaphore != null) + { + await semaphore.WaitAsync(); + } + + Stopwatch stopwatch = new Stopwatch(); + int numAttempts = 0; + bool mustReadETagFirst = false; + + while (true) // retry loop + { + numAttempts++; + try + { + if (requireLease) + { + // we can re-establish the e-tag here because we check the lease afterwards + if (mustReadETagFirst) + { + await readETagAsync().ConfigureAwait(false); + mustReadETagFirst = false; + } + + Interlocked.Increment(ref this.LeaseUsers); + await this.ConfirmLeaseIsGoodForAWhileAsync(); + } + + this.PartitionErrorHandler.Token.ThrowIfCancellationRequested(); + + this.StorageTracer?.FasterStorageProgress($"storage operation {name} ({intent}) started attempt {numAttempts}; target={target} {data}"); + + stopwatch.Restart(); + + long size = await operationAsync(numAttempts).ConfigureAwait(false); + + stopwatch.Stop(); + this.StorageTracer?.FasterStorageProgress($"storage operation {name} ({intent}) succeeded on attempt {numAttempts}; target={target} latencyMs={stopwatch.Elapsed.TotalMilliseconds:F1} {data}"); + + if (stopwatch.ElapsedMilliseconds > expectedLatencyBound) + { + this.TraceHelper.FasterPerfWarning($"storage operation {name} ({intent}) took {stopwatch.Elapsed.TotalSeconds:F1}s on attempt {numAttempts}, which is excessive; {data}"); + } + + this.TraceHelper.FasterAzureStorageAccessCompleted(intent, size, name, target, stopwatch.Elapsed.TotalMilliseconds, numAttempts); + + return; + } + catch (Exception e) when (this.PartitionErrorHandler.IsTerminated) + { + string message = $"storage operation {name} ({intent}) was canceled"; + this.StorageTracer?.FasterStorageProgress(message); + throw new OperationCanceledException(message, e); + } + catch (Exception e) when (BlobUtils.IsTransientStorageError(e) && numAttempts < BlobManager.MaxRetries) + { + stopwatch.Stop(); + + if (BlobUtils.IsTimeout(e)) + { + this.TraceHelper.FasterPerfWarning($"storage operation {name} ({intent}) timed out on attempt {numAttempts} after {stopwatch.Elapsed.TotalSeconds:F1}s, retrying now; target={target} {data}"); + } + else + { + TimeSpan nextRetryIn = BlobManager.GetDelayBetweenRetries(numAttempts); + this.HandleStorageError(name, $"storage operation {name} ({intent}) failed transiently on attempt {numAttempts}, retry in {nextRetryIn}s", target, e, false, true); + await Task.Delay(nextRetryIn); + } + continue; + } + catch (Azure.RequestFailedException ex) when (BlobUtilsV12.PreconditionFailed(ex) && readETagAsync != null) + { + this.StorageTracer?.FasterStorageProgress($"storage operation {name} ({intent}) failed precondition on attempt {numAttempts}; target={target} latencyMs={stopwatch.Elapsed.TotalMilliseconds:F1} {data}"); + mustReadETagFirst = true; + continue; + } + catch (Exception exception) + { + this.HandleStorageError(name, $"storage operation {name} ({intent}) failed on attempt {numAttempts}", target, exception, isCritical, this.PartitionErrorHandler.IsTerminated); + throw; + } + finally + { + if (requireLease) + { + Interlocked.Decrement(ref this.LeaseUsers); + } + } + } + } + finally + { + if (semaphore != null) + { + semaphore.Release(); + } + } + } + + public void PerformWithRetries( + bool requireLease, + string name, + string intent, + string data, + string target, + int expectedLatencyBound, + bool isCritical, + Func operation) + { + Stopwatch stopwatch = new Stopwatch(); + int numAttempts = 0; + + while (true) // retry loop + { + numAttempts++; + try + { + if (requireLease) + { + Interlocked.Increment(ref this.LeaseUsers); + this.ConfirmLeaseIsGoodForAWhile(); + } + + this.PartitionErrorHandler.Token.ThrowIfCancellationRequested(); + + this.StorageTracer?.FasterStorageProgress($"storage operation {name} ({intent}) started attempt {numAttempts}; target={target} {data}"); + stopwatch.Restart(); + + (long size, bool completed) = operation(numAttempts); + + if (!completed) + { + continue; + } + + stopwatch.Stop(); + this.StorageTracer?.FasterStorageProgress($"storage operation {name} ({intent}) succeeded on attempt {numAttempts}; target={target} latencyMs={stopwatch.Elapsed.TotalMilliseconds:F1} size={size} {data} "); + + this.TraceHelper.FasterAzureStorageAccessCompleted(intent, size, name, target, stopwatch.Elapsed.TotalMilliseconds, numAttempts); + + if (stopwatch.ElapsedMilliseconds > expectedLatencyBound) + { + this.TraceHelper.FasterPerfWarning($"storage operation {name} ({intent}) took {stopwatch.Elapsed.TotalSeconds:F1}s on attempt {numAttempts}, which is excessive; {data}"); + } + + return; + } + catch(Exception e) when (this.PartitionErrorHandler.IsTerminated) + { + string message = $"storage operation {name} ({intent}) was canceled"; + this.StorageTracer?.FasterStorageProgress(message); + throw new OperationCanceledException(message, e); + } + catch (Exception e) when (numAttempts < BlobManager.MaxRetries && BlobUtils.IsTransientStorageError(e)) + { + stopwatch.Stop(); + if (BlobUtils.IsTimeout(e)) + { + this.TraceHelper.FasterPerfWarning($"storage operation {name} ({intent}) timed out on attempt {numAttempts} after {stopwatch.Elapsed.TotalSeconds:F1}s, retrying now; target={target} {data}"); + } + else + { + TimeSpan nextRetryIn = BlobManager.GetDelayBetweenRetries(numAttempts); + this.HandleStorageError(name, $"storage operation {name} ({intent}) failed transiently on attempt {numAttempts}, retry in {nextRetryIn}s", target, e, false, true); + Thread.Sleep(nextRetryIn); + } + continue; + } + catch (Azure.RequestFailedException ex) when (BlobUtilsV12.PreconditionFailed(ex)) + { + // precondition failed, which indicates we are observing a rare partition race + this.HandleStorageError(name, $"storage operation {name} ({intent}) failed precondition on attempt {numAttempts}", target, ex, true, true); + } + catch (Exception exception) + { + this.HandleStorageError(name, $"storage operation {name} ({intent}) failed on attempt {numAttempts}", target, exception, isCritical, this.PartitionErrorHandler.IsTerminated); + throw; + } + finally + { + if (requireLease) + { + Interlocked.Decrement(ref this.LeaseUsers); + } + } + } + } + } +} diff --git a/cs/src/devices/AzureStorageDevice/Utils.cs b/cs/src/devices/AzureStorageDevice/Utils.cs new file mode 100644 index 000000000..77026bcf6 --- /dev/null +++ b/cs/src/devices/AzureStorageDevice/Utils.cs @@ -0,0 +1,26 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT License. + +namespace FASTER.devices +{ + using System; + + /// + /// Utility methods + /// + public static class Utils + { + /// + /// Returns true or false whether an exception is considered fatal + /// + public static bool IsFatal(Exception exception) + { + if (exception is OutOfMemoryException || exception is StackOverflowException) + { + return true; + } + + return false; + } + } +} From 92512885d320306b329c2f23a396c5f17a51678b Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Fri, 3 Feb 2023 15:33:54 -0800 Subject: [PATCH 02/29] update --- cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs | 2 +- .../AzureStorageDevice/AzureStorageNamedDeviceFactory.cs | 2 +- cs/src/devices/AzureStorageDevice/BlobEntry.cs | 2 +- cs/src/devices/AzureStorageDevice/BlobManager.cs | 2 +- cs/src/devices/AzureStorageDevice/BlobUtils.cs | 2 +- cs/src/devices/AzureStorageDevice/BlobUtilsV12.cs | 2 +- cs/src/devices/AzureStorageDevice/IPartitionErrorHandler.cs | 2 +- cs/src/devices/AzureStorageDevice/LeaseTimer.cs | 2 +- cs/src/devices/AzureStorageDevice/StorageOperations.cs | 2 +- 9 files changed, 9 insertions(+), 9 deletions(-) diff --git a/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs b/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs index 59f6649ac..689cd8ca4 100644 --- a/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs +++ b/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs @@ -1,7 +1,7 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT License. -namespace DurableTask.Netherite.Faster +namespace FASTER.devices { using System; using System.Collections.Concurrent; diff --git a/cs/src/devices/AzureStorageDevice/AzureStorageNamedDeviceFactory.cs b/cs/src/devices/AzureStorageDevice/AzureStorageNamedDeviceFactory.cs index 1041223d1..f3921e431 100644 --- a/cs/src/devices/AzureStorageDevice/AzureStorageNamedDeviceFactory.cs +++ b/cs/src/devices/AzureStorageDevice/AzureStorageNamedDeviceFactory.cs @@ -3,7 +3,7 @@ using System.Linq; using System.Threading.Tasks; -namespace DurableTask.Netherite.Faster +namespace FASTER.devices { /// /// Device factory for Azure diff --git a/cs/src/devices/AzureStorageDevice/BlobEntry.cs b/cs/src/devices/AzureStorageDevice/BlobEntry.cs index 75b94124a..bb703dbc2 100644 --- a/cs/src/devices/AzureStorageDevice/BlobEntry.cs +++ b/cs/src/devices/AzureStorageDevice/BlobEntry.cs @@ -1,7 +1,7 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT License. -namespace DurableTask.Netherite.Faster +namespace FASTER.devices { using System; using System.Collections.Concurrent; diff --git a/cs/src/devices/AzureStorageDevice/BlobManager.cs b/cs/src/devices/AzureStorageDevice/BlobManager.cs index 832342327..2d92980e1 100644 --- a/cs/src/devices/AzureStorageDevice/BlobManager.cs +++ b/cs/src/devices/AzureStorageDevice/BlobManager.cs @@ -1,7 +1,7 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT License. -namespace DurableTask.Netherite.Faster +namespace FASTER.devices { using DurableTask.Core.Common; using FASTER.core; diff --git a/cs/src/devices/AzureStorageDevice/BlobUtils.cs b/cs/src/devices/AzureStorageDevice/BlobUtils.cs index fef2115d8..248a9332c 100644 --- a/cs/src/devices/AzureStorageDevice/BlobUtils.cs +++ b/cs/src/devices/AzureStorageDevice/BlobUtils.cs @@ -1,7 +1,7 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT License. -namespace DurableTask.Netherite +namespace FASTER.devices { using System; using System.Threading.Tasks; diff --git a/cs/src/devices/AzureStorageDevice/BlobUtilsV12.cs b/cs/src/devices/AzureStorageDevice/BlobUtilsV12.cs index f4530e30e..f8dad1fee 100644 --- a/cs/src/devices/AzureStorageDevice/BlobUtilsV12.cs +++ b/cs/src/devices/AzureStorageDevice/BlobUtilsV12.cs @@ -1,7 +1,7 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT License. -namespace DurableTask.Netherite +namespace FASTER.devices { using System; using System.Collections.Generic; diff --git a/cs/src/devices/AzureStorageDevice/IPartitionErrorHandler.cs b/cs/src/devices/AzureStorageDevice/IPartitionErrorHandler.cs index beaed34f2..40560e167 100644 --- a/cs/src/devices/AzureStorageDevice/IPartitionErrorHandler.cs +++ b/cs/src/devices/AzureStorageDevice/IPartitionErrorHandler.cs @@ -1,7 +1,7 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT License. -namespace DurableTask.Netherite +namespace FASTER.devices { using System; using System.Collections.Generic; diff --git a/cs/src/devices/AzureStorageDevice/LeaseTimer.cs b/cs/src/devices/AzureStorageDevice/LeaseTimer.cs index 3e4b5dbd3..b92b6611d 100644 --- a/cs/src/devices/AzureStorageDevice/LeaseTimer.cs +++ b/cs/src/devices/AzureStorageDevice/LeaseTimer.cs @@ -1,7 +1,7 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT License. -namespace DurableTask.Netherite.Faster +namespace FASTER.devices { using System; using System.Diagnostics; diff --git a/cs/src/devices/AzureStorageDevice/StorageOperations.cs b/cs/src/devices/AzureStorageDevice/StorageOperations.cs index 5e9b40c42..8238e27a1 100644 --- a/cs/src/devices/AzureStorageDevice/StorageOperations.cs +++ b/cs/src/devices/AzureStorageDevice/StorageOperations.cs @@ -1,7 +1,7 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT License. -namespace DurableTask.Netherite.Faster +namespace FASTER.devices { using System; using System.Diagnostics; From a58732dad87babbc9a9ccba9acae4ae200cb3312 Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Fri, 3 Feb 2023 19:49:58 -0800 Subject: [PATCH 03/29] more cleanup --- .../AzureStorageDevice/AzureStorageDevice.cs | 14 +- .../AzureStorageNamedDeviceFactory.cs | 2 +- .../devices/AzureStorageDevice/BlobEntry.cs | 1 + .../devices/AzureStorageDevice/BlobManager.cs | 1079 +---------------- .../AzureStorageDevice/DefaultBlobManager.cs | 344 ------ .../AzureStorageDevice/FasterTraceHelper.cs | 184 +++ .../AzureStorageDevice/IBlobManager.cs | 69 -- 7 files changed, 213 insertions(+), 1480 deletions(-) delete mode 100644 cs/src/devices/AzureStorageDevice/DefaultBlobManager.cs create mode 100644 cs/src/devices/AzureStorageDevice/FasterTraceHelper.cs delete mode 100644 cs/src/devices/AzureStorageDevice/IBlobManager.cs diff --git a/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs b/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs index 689cd8ca4..dee3b8ddb 100644 --- a/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs +++ b/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs @@ -13,6 +13,7 @@ namespace FASTER.devices using System.Threading.Tasks; using Azure.Storage.Blobs.Models; using FASTER.core; + using Microsoft.Extensions.Logging; /// /// A IDevice Implementation that is backed byAzure Page Blob. @@ -21,7 +22,6 @@ namespace FASTER.devices class AzureStorageDevice : StorageDeviceBase { readonly ConcurrentDictionary blobs; - readonly BlobUtilsV12.BlobDirectory blockBlobDirectory; readonly BlobUtilsV12.BlobDirectory pageBlobDirectory; readonly string blobName; readonly bool underLease; @@ -67,22 +67,20 @@ struct RemoveRequestInfo /// Constructs a new AzureStorageDevice instance, backed by Azure Page Blobs /// /// A descriptive name that will be the prefix of all segments created - /// the directory containing the block blobs /// the directory containing the page blobs /// the blob manager handling the leases /// whether this device needs to be protected by the lease - public AzureStorageDevice(string blobName, BlobUtilsV12.BlobDirectory blockBlobDirectory, BlobUtilsV12.BlobDirectory pageBlobDirectory, BlobManager blobManager, bool underLease) - : base($"{blockBlobDirectory}\\{blobName}", PAGE_BLOB_SECTOR_SIZE, Devices.CAPACITY_UNSPECIFIED) + public AzureStorageDevice(string blobName, BlobUtilsV12.BlobDirectory pageBlobDirectory, BlobManager blobManager = null, bool underLease = false) + : base($"{pageBlobDirectory}/{blobName}", PAGE_BLOB_SECTOR_SIZE, Devices.CAPACITY_UNSPECIFIED) { this.blobs = new ConcurrentDictionary(); this.pendingReadWriteOperations = new ConcurrentDictionary(); this.pendingRemoveOperations = new ConcurrentDictionary(); - this.blockBlobDirectory = blockBlobDirectory; this.pageBlobDirectory = pageBlobDirectory; this.blobName = blobName; this.PartitionErrorHandler = blobManager.PartitionErrorHandler; this.PartitionErrorHandler.Token.Register(this.CancelAllRequests); - this.BlobManager = blobManager; + this.BlobManager = blobManager ?? new BlobManager(blobName, pageBlobDirectory, underLease, null, null, LogLevel.Information, null); this.underLease = underLease; this.hangCheckTimer = new Timer(this.DetectHangs, null, 0, 20000); this.singleWriterSemaphore = underLease ? new SemaphoreSlim(1) : null; @@ -92,7 +90,7 @@ public AzureStorageDevice(string blobName, BlobUtilsV12.BlobDirectory blockBlobD /// public override string ToString() { - return $"AzureStorageDevice {this.blockBlobDirectory}{this.blobName}"; + return $"AzureStorageDevice {this.pageBlobDirectory}{this.blobName}"; } public async Task StartAsync() @@ -102,7 +100,7 @@ public async Task StartAsync() this.BlobManager?.StorageTracer?.FasterStorageProgress($"StorageOpCalled AzureStorageDevice.StartAsync target={this.pageBlobDirectory}{this.blobName}"); // list all the blobs representing the segments - var prefix = $"{this.blockBlobDirectory}{this.blobName}."; + var prefix = $"{this.pageBlobDirectory}{this.blobName}."; string continuationToken = null; IReadOnlyList pageResults = null; diff --git a/cs/src/devices/AzureStorageDevice/AzureStorageNamedDeviceFactory.cs b/cs/src/devices/AzureStorageDevice/AzureStorageNamedDeviceFactory.cs index f3921e431..5d8107ec7 100644 --- a/cs/src/devices/AzureStorageDevice/AzureStorageNamedDeviceFactory.cs +++ b/cs/src/devices/AzureStorageDevice/AzureStorageNamedDeviceFactory.cs @@ -72,7 +72,7 @@ public void Delete(FileDescriptor fileInfo) /// public IDevice Get(FileDescriptor fileInfo) { - return new AzureStorageDevice(fileInfo.fileName, default, pageBlobDirectory.GetSubDirectory(fileInfo.directoryName), null, false); + return new AzureStorageDevice(fileInfo.fileName, pageBlobDirectory.GetSubDirectory(fileInfo.directoryName), null, false); } /// diff --git a/cs/src/devices/AzureStorageDevice/BlobEntry.cs b/cs/src/devices/AzureStorageDevice/BlobEntry.cs index bb703dbc2..8e404b658 100644 --- a/cs/src/devices/AzureStorageDevice/BlobEntry.cs +++ b/cs/src/devices/AzureStorageDevice/BlobEntry.cs @@ -29,6 +29,7 @@ class BlobEntry /// Creates a new BlobEntry to represent a page blob that already exists in storage. /// /// + /// /// public BlobEntry(BlobUtilsV12.PageBlobClients pageBlob, Azure.ETag eTag, AzureStorageDevice azureStorageDevice) { diff --git a/cs/src/devices/AzureStorageDevice/BlobManager.cs b/cs/src/devices/AzureStorageDevice/BlobManager.cs index 2d92980e1..f43218795 100644 --- a/cs/src/devices/AzureStorageDevice/BlobManager.cs +++ b/cs/src/devices/AzureStorageDevice/BlobManager.cs @@ -3,69 +3,43 @@ namespace FASTER.devices { - using DurableTask.Core.Common; using FASTER.core; - using Azure.Storage.Blobs; using Microsoft.Extensions.Logging; - using Newtonsoft.Json; - using Newtonsoft.Json.Linq; using System; - using System.Collections.Generic; using System.Diagnostics; using System.IO; - using System.Linq; using System.Threading; using System.Threading.Tasks; using Azure.Storage.Blobs.Specialized; - using Azure.Storage.Blobs.Models; - using System.Net; - using System.Text; /// /// Provides management of blobs and blob names associated with a partition, and logic for partition lease maintenance and termination. /// - partial class BlobManager : ICheckpointManager, ILogCommitManager + partial class BlobManager { - readonly NetheriteOrchestrationServiceSettings settings; readonly uint partitionId; readonly CancellationTokenSource shutDownOrTermination; readonly string taskHubPrefix; - BlobUtilsV12.ServiceClients blockBlobAccount; BlobUtilsV12.ServiceClients pageBlobAccount; - - BlobUtilsV12.ContainerClients blockBlobContainer; BlobUtilsV12.ContainerClients pageBlobContainer; - BlobUtilsV12.BlockBlobClients eventLogCommitBlob; BlobLeaseClient leaseClient; - BlobUtilsV12.BlobDirectory pageBlobPartitionDirectory; + BlobUtilsV12.BlobDirectory pageBlobDirectory; BlobUtilsV12.BlobDirectory blockBlobPartitionDirectory; readonly TimeSpan LeaseDuration = TimeSpan.FromSeconds(45); // max time the lease stays after unclean shutdown readonly TimeSpan LeaseRenewal = TimeSpan.FromSeconds(30); // how often we renew the lease readonly TimeSpan LeaseSafetyBuffer = TimeSpan.FromSeconds(10); // how much time we want left on the lease before issuing a protected access - internal CheckpointInfo CheckpointInfo { get; } - Azure.ETag? CheckpointInfoETag { get; set; } - internal FasterTraceHelper TraceHelper { get; private set; } internal FasterTraceHelper StorageTracer => this.TraceHelper.IsTracingAtMostDetailedLevel ? this.TraceHelper : null; - public IDevice EventLogDevice { get; private set; } - public IDevice HybridLogDevice { get; private set; } - public IDevice ObjectLogDevice { get; private set; } - public DateTime IncarnationTimestamp { get; private set; } - public string ContainerName { get; } - - internal BlobUtilsV12.ContainerClients BlockBlobContainer => this.blockBlobContainer; internal BlobUtilsV12.ContainerClients PageBlobContainer => this.pageBlobContainer; - public int PartitionId => (int)this.partitionId; - public IPartitionErrorHandler PartitionErrorHandler { get; private set; } internal static SemaphoreSlim AsynchronousStorageReadMaxConcurrency = new SemaphoreSlim(Math.Min(100, Environment.ProcessorCount * 10)); @@ -73,166 +47,11 @@ partial class BlobManager : ICheckpointManager, ILogCommitManager internal volatile int LeaseUsers; - volatile System.Diagnostics.Stopwatch leaseTimer; + volatile Stopwatch leaseTimer; internal const long HashTableSize = 1L << 14; // 16 k buckets, 1 MB internal const long HashTableSizeBytes = HashTableSize * 64; - public class FasterTuningParameters - { - public int? EventLogPageSizeBits; - public int? EventLogSegmentSizeBits; - public int? EventLogMemorySizeBits; - public int? StoreLogPageSizeBits; - public int? StoreLogSegmentSizeBits; - public int? StoreLogMemorySizeBits; - public double? StoreLogMutableFraction; - public int? EstimatedAverageObjectSize; - public int? NumPagesToPreload; - } - - public FasterLogSettings GetDefaultEventLogSettings(bool useSeparatePageBlobStorage, FasterTuningParameters tuningParameters) => new FasterLogSettings - { - LogDevice = this.EventLogDevice, - LogCommitManager = this.UseLocalFiles - ? null // TODO: fix this: new LocalLogCommitManager($"{this.LocalDirectoryPath}\\{this.PartitionFolderName}\\{CommitBlobName}") - : (ILogCommitManager)this, - PageSizeBits = tuningParameters?.EventLogPageSizeBits ?? 21, // 2MB - SegmentSizeBits = tuningParameters?.EventLogSegmentSizeBits ?? - (useSeparatePageBlobStorage ? 35 // 32 GB - : 26), // 64 MB - MemorySizeBits = tuningParameters?.EventLogMemorySizeBits ?? 22, // 2MB - }; - - public LogSettings GetDefaultStoreLogSettings( - bool useSeparatePageBlobStorage, - long upperBoundOnAvailable, - FasterTuningParameters tuningParameters) - { - int pageSizeBits = tuningParameters?.StoreLogPageSizeBits ?? 10; // default page size is 1k - - // compute a reasonable memory size for the log considering maximally available memory, and expansion factor - int memorybits = 0; - if (tuningParameters?.StoreLogMemorySizeBits != null) - { - memorybits = tuningParameters.StoreLogMemorySizeBits.Value; - } - else - { - double expansionFactor = (24 + ((double)(tuningParameters?.EstimatedAverageObjectSize ?? 216))) / 24; - long estimate = (long)(upperBoundOnAvailable / expansionFactor); - - while (estimate > 0) - { - memorybits++; - estimate >>= 1; - } - memorybits = Math.Max(pageSizeBits + 2, memorybits); // never use less than 4 pages - } - - return new LogSettings - { - LogDevice = this.HybridLogDevice, - ObjectLogDevice = this.ObjectLogDevice, - PageSizeBits = pageSizeBits, - MutableFraction = tuningParameters?.StoreLogMutableFraction ?? 0.9, - SegmentSizeBits = tuningParameters?.StoreLogSegmentSizeBits ?? - (useSeparatePageBlobStorage ? 35 // 32 GB - : 32), // 4 GB - PreallocateLog = false, - ReadFlags = ReadFlags.None, - ReadCacheSettings = null, // no read cache - MemorySizeBits = memorybits, - }; - } - - - static readonly int[] StorageFormatVersion = new int[] { - 1, //initial version - 2, //0.7.0-beta changed singleton storage, and adds dequeue count - 3, //changed organization of files - 4, //use Faster v2, reduced page size - 5, //support EventHub recovery - }; - - public static string GetStorageFormat(NetheriteOrchestrationServiceSettings settings) - { - return JsonConvert.SerializeObject(new StorageFormatSettings() - { - UseAlternateObjectStore = settings.UseAlternateObjectStore, - FormatVersion = StorageFormatVersion.Last(), - }, - serializerSettings); - } - - [JsonObject] - class StorageFormatSettings - { - // this must stay the same - - [JsonProperty("FormatVersion")] - public int FormatVersion { get; set; } - - // the following can be changed between versions - - [JsonProperty("UseAlternateObjectStore", DefaultValueHandling = DefaultValueHandling.Ignore)] - public bool? UseAlternateObjectStore { get; set; } - } - - static readonly JsonSerializerSettings serializerSettings = new JsonSerializerSettings() - { - TypeNameHandling = TypeNameHandling.None, - MissingMemberHandling = MissingMemberHandling.Ignore, - CheckAdditionalContent = false, - Formatting = Formatting.None, - }; - - public static void CheckStorageFormat(string format, NetheriteOrchestrationServiceSettings settings) - { - try - { - var taskhubFormat = JsonConvert.DeserializeObject(format, serializerSettings); - - if (taskhubFormat.UseAlternateObjectStore != settings.UseAlternateObjectStore) - { - throw new NetheriteConfigurationException("The Netherite configuration setting 'UseAlternateObjectStore' is incompatible with the existing taskhub."); - } - if (taskhubFormat.FormatVersion != StorageFormatVersion.Last()) - { - throw new NetheriteConfigurationException($"The current storage format version (={StorageFormatVersion.Last()}) is incompatible with the existing taskhub (={taskhubFormat.FormatVersion})."); - } - } - catch (Exception e) - { - throw new NetheriteConfigurationException("The taskhub has an incompatible storage format", e); - } - } - - public void Dispose() - { - // we do not need to dispose any resources for the commit manager, because any such resources are deleted together with the taskhub - } - - public void Purge(Guid token) - { - throw new NotImplementedException("Purges are handled directly on recovery, not via FASTER"); - } - - public void PurgeAll() - { - throw new NotImplementedException("Purges are handled directly on recovery, not via FASTER"); - } - - public void OnRecovery(Guid indexToken, Guid logToken) - { - // we handle cleanup of old checkpoints somewhere else - } - - public CheckpointSettings StoreCheckpointSettings => new CheckpointSettings - { - CheckpointManager = this.UseLocalFiles ? (ICheckpointManager)this.LocalCheckpointManager : (ICheckpointManager)this, - }; - public const int MaxRetries = 10; public static TimeSpan GetDelayBetweenRetries(int numAttempts) @@ -241,147 +60,41 @@ public static TimeSpan GetDelayBetweenRetries(int numAttempts) /// /// Create a blob manager. /// - /// The cloud storage account, or null if using local file paths - /// The storage account to use for page blobs - /// The local file path, or null if using cloud storage - /// The name of the taskhub + /// + /// + /// /// A logger for logging + /// /// A limit on log event level emitted - /// The partition id /// A handler for errors encountered in this partition - public BlobManager( - NetheriteOrchestrationServiceSettings settings, - string taskHubName, - string taskHubPrefix, - FaultInjector faultInjector, + internal BlobManager( + string leaseBlobName, + BlobUtilsV12.BlobDirectory pageBlobDirectory, + bool underLease, ILogger logger, ILogger performanceLogger, - Microsoft.Extensions.Logging.LogLevel logLevelLimit, - uint partitionId, + LogLevel logLevelLimit, IPartitionErrorHandler errorHandler) { - this.settings = settings; - this.ContainerName = GetContainerName(taskHubName); - this.taskHubPrefix = taskHubPrefix; - this.FaultInjector = faultInjector; - this.partitionId = partitionId; - this.CheckpointInfo = new CheckpointInfo(); - this.CheckpointInfoETag = default; - - if (!string.IsNullOrEmpty(settings.UseLocalDirectoryForPartitionStorage)) - { - this.UseLocalFiles = true; - this.LocalFileDirectoryForTestingAndDebugging = settings.UseLocalDirectoryForPartitionStorage; - this.LocalCheckpointManager = new LocalFileCheckpointManager( - this.CheckpointInfo, - this.LocalCheckpointDirectoryPath, - this.GetCheckpointCompletedBlobName()); - } - else - { - this.UseLocalFiles = false; - } - this.TraceHelper = new FasterTraceHelper(logger, logLevelLimit, performanceLogger, this.partitionId, this.UseLocalFiles ? "none" : this.settings.StorageAccountName, taskHubName); + if (leaseBlobName != null) LeaseBlobName = leaseBlobName; + this.pageBlobDirectory = pageBlobDirectory; + this.UseLocalFiles = false; + this.TraceHelper = new FasterTraceHelper(logger, logLevelLimit, performanceLogger); this.PartitionErrorHandler = errorHandler; this.shutDownOrTermination = CancellationTokenSource.CreateLinkedTokenSource(errorHandler.Token); } - string PartitionFolderName => $"{this.taskHubPrefix}p{this.partitionId:D2}"; - // For testing and debugging with local files bool UseLocalFiles { get; } - LocalFileCheckpointManager LocalCheckpointManager { get; set; } - string LocalFileDirectoryForTestingAndDebugging { get; } - string LocalDirectoryPath => $"{this.LocalFileDirectoryForTestingAndDebugging}\\{this.ContainerName}"; - string LocalCheckpointDirectoryPath => $"{this.LocalDirectoryPath}\\chkpts{this.partitionId:D2}"; - - const string EventLogBlobName = "commit-log"; - const string CommitBlobName = "commit-lease"; - const string HybridLogBlobName = "store"; - const string ObjectLogBlobName = "store.obj"; - + string LeaseBlobName = "commit-lease"; Task LeaseMaintenanceLoopTask = Task.CompletedTask; volatile Task NextLeaseRenewalTask = Task.CompletedTask; - public static string GetContainerName(string taskHubName) => taskHubName.ToLowerInvariant() + "-storage"; - public async Task StartAsync() { - if (this.UseLocalFiles) - { - this.LocalCheckpointManager = new LocalFileCheckpointManager( - this.CheckpointInfo, - this.LocalCheckpointDirectoryPath, - this.GetCheckpointCompletedBlobName()); - - Directory.CreateDirectory($"{this.LocalDirectoryPath}\\{this.PartitionFolderName}"); - - this.EventLogDevice = Devices.CreateLogDevice($"{this.LocalDirectoryPath}\\{this.PartitionFolderName}\\{EventLogBlobName}"); - this.HybridLogDevice = Devices.CreateLogDevice($"{this.LocalDirectoryPath}\\{this.PartitionFolderName}\\{HybridLogBlobName}"); - this.ObjectLogDevice = Devices.CreateLogDevice($"{this.LocalDirectoryPath}\\{this.PartitionFolderName}\\{ObjectLogBlobName}"); - - // This does not acquire any blob ownership, but is needed for the lease maintenance loop which calls PartitionErrorHandler.TerminateNormally() when done. - await this.AcquireOwnership(); - } - else - { - this.blockBlobAccount = BlobUtilsV12.GetServiceClients(this.settings.BlobStorageConnection); - this.blockBlobContainer = BlobUtilsV12.GetContainerClients(this.blockBlobAccount, this.ContainerName); - await this.blockBlobContainer.WithRetries.CreateIfNotExistsAsync(); - this.blockBlobPartitionDirectory = new BlobUtilsV12.BlobDirectory(this.blockBlobContainer, this.PartitionFolderName); - - if (this.settings.PageBlobStorageConnection != null) - { - this.pageBlobAccount = BlobUtilsV12.GetServiceClients(this.settings.PageBlobStorageConnection); - this.pageBlobContainer = BlobUtilsV12.GetContainerClients(this.pageBlobAccount, this.ContainerName); - await this.pageBlobContainer.WithRetries.CreateIfNotExistsAsync(); - this.pageBlobPartitionDirectory = new BlobUtilsV12.BlobDirectory(this.pageBlobContainer, this.PartitionFolderName); - } - else - { - this.pageBlobAccount = this.blockBlobAccount; - this.pageBlobContainer = this.BlockBlobContainer; - this.pageBlobPartitionDirectory = this.blockBlobPartitionDirectory; - } - - this.eventLogCommitBlob = this.blockBlobPartitionDirectory.GetBlockBlobClient(CommitBlobName); - this.leaseClient = this.eventLogCommitBlob.WithRetries.GetBlobLeaseClient(); - - AzureStorageDevice createDevice(string name) => - new AzureStorageDevice(name, this.blockBlobPartitionDirectory.GetSubDirectory(name), this.pageBlobPartitionDirectory.GetSubDirectory(name), this, true); - - var eventLogDevice = createDevice(EventLogBlobName); - var hybridLogDevice = createDevice(HybridLogBlobName); - var objectLogDevice = createDevice(ObjectLogBlobName); - - await this.AcquireOwnership(); - - this.TraceHelper.FasterProgress("Starting Faster Devices"); - var startTasks = new List - { - eventLogDevice.StartAsync(), - hybridLogDevice.StartAsync(), - objectLogDevice.StartAsync() - }; - await Task.WhenAll(startTasks); - this.TraceHelper.FasterProgress("Started Faster Devices"); - - this.EventLogDevice = eventLogDevice; - this.HybridLogDevice = hybridLogDevice; - this.ObjectLogDevice = objectLogDevice; - } - } - - internal void DisposeDevices() - { - Dispose(this.HybridLogDevice); - Dispose(this.ObjectLogDevice); - - void Dispose(IDevice device) - { - this.TraceHelper.FasterStorageProgress($"Disposing Device {device.FileName}"); - device.Dispose(); - } + this.eventLogCommitBlob = this.pageBlobDirectory.GetBlockBlobClient(LeaseBlobName); + this.leaseClient = this.eventLogCommitBlob.WithRetries.GetBlobLeaseClient(); + await this.AcquireOwnership(); } public void HandleStorageError(string where, string message, string blobName, Exception e, bool isFatal, bool isWarning) @@ -404,51 +117,6 @@ public async Task StopAsync() await this.LeaseMaintenanceLoopTask; // wait for loop to terminate cleanly } - public static async Task DeleteTaskhubStorageAsync(NetheriteOrchestrationServiceSettings settings, string pathPrefix) - { - var containerName = GetContainerName(settings.HubName); - - if (!string.IsNullOrEmpty(settings.UseLocalDirectoryForPartitionStorage)) - { - DirectoryInfo di = new DirectoryInfo($"{settings.UseLocalDirectoryForPartitionStorage}\\{containerName}"); //TODO fine-grained deletion - if (di.Exists) - { - di.Delete(true); - } - } - else - { - var blockBlobAccount = BlobUtilsV12.GetServiceClients(settings.BlobStorageConnection); - await DeleteContainerContents(blockBlobAccount.WithRetries); - - if (settings.PageBlobStorageConnection != null) - { - var pageBlobAccount = BlobUtilsV12.GetServiceClients(settings.PageBlobStorageConnection); - await DeleteContainerContents(pageBlobAccount.Default); - } - - async Task DeleteContainerContents(BlobServiceClient account) - { - var container = account.GetBlobContainerClient(containerName); - var deletionTasks = new List(); - try - { - await foreach (BlobItem blob in container.GetBlobsAsync(BlobTraits.None, BlobStates.None, string.Empty)) - { - deletionTasks.Add(BlobUtilsV12.ForceDeleteAsync(container, blob.Name)); - } - } - catch (Azure.RequestFailedException e) when (e.Status == (int)HttpStatusCode.NotFound) - { - } - await Task.WhenAll(deletionTasks); - } - - // We are not deleting the container itself because it creates problems when trying to recreate - // the same container soon afterwards. So we prefer to leave an empty container behind. - } - } - public ValueTask ConfirmLeaseIsGoodForAWhileAsync() { if (this.leaseTimer?.Elapsed < this.LeaseDuration - this.LeaseSafetyBuffer && !this.shutDownOrTermination.IsCancellationRequested) @@ -471,7 +139,7 @@ public void ConfirmLeaseIsGoodForAWhile() async Task AcquireOwnership() { - var newLeaseTimer = new System.Diagnostics.Stopwatch(); + var newLeaseTimer = new Stopwatch(); int numAttempts = 0; while (true) @@ -485,7 +153,6 @@ async Task AcquireOwnership() if (!this.UseLocalFiles) { - this.FaultInjector?.StorageAccess(this, "AcquireLeaseAsync", "AcquireOwnership", this.eventLogCommitBlob.Name); await this.leaseClient.AcquireAsync( this.LeaseDuration, null, @@ -503,8 +170,6 @@ await this.leaseClient.AcquireAsync( { this.TraceHelper.LeaseProgress("Waiting for lease"); - this.FaultInjector?.BreakLease(this.eventLogCommitBlob); // during fault injection tests, we don't want to wait - // the previous owner has not released the lease yet, // try again until it becomes available, should be relatively soon // as the transport layer is supposed to shut down the previous owner when starting this @@ -587,7 +252,6 @@ public async Task RenewLeaseTask() if (!this.UseLocalFiles) { this.TraceHelper.LeaseProgress($"Renewing lease at {this.leaseTimer.Elapsed.TotalSeconds - this.LeaseDuration.TotalSeconds}s"); - this.FaultInjector?.StorageAccess(this, "RenewLeaseAsync", "RenewLease", this.eventLogCommitBlob.Name); await this.leaseClient.RenewAsync(null, this.PartitionErrorHandler.Token).ConfigureAwait(false); this.TraceHelper.LeaseRenewed(this.leaseTimer.Elapsed.TotalSeconds, this.leaseTimer.Elapsed.TotalSeconds - this.LeaseDuration.TotalSeconds); @@ -670,7 +334,6 @@ public async Task MaintenanceLoopAsync() { this.TraceHelper.LeaseProgress("Releasing lease"); - this.FaultInjector?.StorageAccess(this, "ReleaseLeaseAsync", "ReleaseLease", this.eventLogCommitBlob.Name); await this.leaseClient.ReleaseAsync(null, this.PartitionErrorHandler.Token).ConfigureAwait(false); this.TraceHelper.LeaseReleased(this.leaseTimer.Elapsed.TotalSeconds); } @@ -693,705 +356,5 @@ public async Task MaintenanceLoopAsync() this.TraceHelper.LeaseProgress("Blob manager stopped"); } - - public async Task RemoveObsoleteCheckpoints() - { - if (this.UseLocalFiles) - { - //TODO - return; - } - else - { - string token1 = this.CheckpointInfo.LogToken.ToString(); - string token2 = this.CheckpointInfo.IndexToken.ToString(); - - this.TraceHelper.FasterProgress($"Removing obsolete checkpoints, keeping only {token1} and {token2}"); - - var tasks = new List>(); - - tasks.Add(RemoveObsoleteCheckpoints(this.blockBlobPartitionDirectory.GetSubDirectory(cprCheckpointPrefix))); - tasks.Add(RemoveObsoleteCheckpoints(this.blockBlobPartitionDirectory.GetSubDirectory(indexCheckpointPrefix))); - - if (this.settings.PageBlobStorageConnection != null) - { - tasks.Add(RemoveObsoleteCheckpoints(this.pageBlobPartitionDirectory.GetSubDirectory(cprCheckpointPrefix))); - tasks.Add(RemoveObsoleteCheckpoints(this.pageBlobPartitionDirectory.GetSubDirectory(indexCheckpointPrefix))); - } - - await Task.WhenAll(tasks); - - this.TraceHelper.FasterProgress($"Removed {tasks.Select(t => t.Result.Item1).Sum()} checkpoint directories containing {tasks.Select(t => t.Result.Item2).Sum()} blobs"); - - async Task<(int, int)> RemoveObsoleteCheckpoints(BlobUtilsV12.BlobDirectory directory) - { - List results = null; - - await this.PerformWithRetriesAsync( - BlobManager.AsynchronousStorageWriteMaxConcurrency, - true, - "BlobContainerClient.GetBlobsAsync", - "RemoveObsoleteCheckpoints", - "", - directory.Prefix, - 1000, - false, - async (numAttempts) => - { - results = await directory.GetBlobsAsync(this.shutDownOrTermination.Token); - return results.Count(); - }); - - - var checkpointFoldersToDelete = results - .GroupBy((s) => s.Split('/')[3]) - .Where(g => g.Key != token1 && g.Key != token2) - .ToList(); - - var deletionTasks = new List(); - - foreach (var folder in checkpointFoldersToDelete) - { - deletionTasks.Add(DeleteCheckpointDirectory(folder)); - } - - await Task.WhenAll(deletionTasks); - return (checkpointFoldersToDelete.Count, results.Count); - - async Task DeleteCheckpointDirectory(IEnumerable blobsToDelete) - { - var deletionTasks = new List(); - foreach (var blobName in blobsToDelete) - { - deletionTasks.Add( - this.PerformWithRetriesAsync( - BlobManager.AsynchronousStorageWriteMaxConcurrency, - false, - "BlobUtils.ForceDeleteAsync", - "DeleteCheckpointDirectory", - "", - blobName, - 1000, - false, - async (numAttempts) => (await BlobUtilsV12.ForceDeleteAsync(directory.Client.Default, blobName) ? 1 : 0))); - } - await Task.WhenAll(deletionTasks); - } - } - } - } - - #region Blob Name Management - - string GetCheckpointCompletedBlobName() => "last-checkpoint.json"; - - const string indexCheckpointPrefix = "index-checkpoints/"; - - const string cprCheckpointPrefix = "cpr-checkpoints/"; - - string GetIndexCheckpointMetaBlobName(Guid token) => $"{indexCheckpointPrefix}{token}/info.dat"; - - (string, string) GetPrimaryHashTableBlobName(Guid token) => ($"{indexCheckpointPrefix}{token}", "ht.dat"); - - string GetHybridLogCheckpointMetaBlobName(Guid token) => $"{cprCheckpointPrefix}{token}/info.dat"; - - (string, string) GetLogSnapshotBlobName(Guid token) => ($"{cprCheckpointPrefix}{token}", "snapshot.dat"); - - (string, string) GetObjectLogSnapshotBlobName(Guid token) => ($"{cprCheckpointPrefix}{token}", "snapshot.obj.dat"); - - (string, string) GetDeltaLogSnapshotBlobName(Guid token) => ($"{cprCheckpointPrefix}{token}", "snapshot.delta.dat"); - - string GetSingletonsSnapshotBlobName(Guid token) => $"{cprCheckpointPrefix}{token}/singletons.dat"; - - #endregion - - #region ILogCommitManager - - void ILogCommitManager.Commit(long beginAddress, long untilAddress, byte[] commitMetadata, long commitNum) - { - try - { - this.StorageTracer?.FasterStorageProgress($"StorageOpCalled ILogCommitManager.Commit beginAddress={beginAddress} untilAddress={untilAddress}"); - - this.PerformWithRetries( - false, - "BlockBlobClient.Upload", - "WriteCommitLogMetadata", - "", - this.eventLogCommitBlob.Default.Name, - 1000, - true, - (int numAttempts) => - { - try - { - var client = numAttempts > 2 ? this.eventLogCommitBlob.Default : this.eventLogCommitBlob.Aggressive; - - client.Upload( - content: new MemoryStream(commitMetadata), - options: new BlobUploadOptions() { Conditions = new BlobRequestConditions() { LeaseId = this.leaseClient.LeaseId } }, - cancellationToken: this.PartitionErrorHandler.Token); - - return (commitMetadata.Length, true); - } - catch (Azure.RequestFailedException ex) when (BlobUtilsV12.LeaseConflict(ex)) - { - // We lost the lease to someone else. Terminate ownership immediately. - this.TraceHelper.LeaseLost(this.leaseTimer.Elapsed.TotalSeconds, nameof(ILogCommitManager.Commit)); - this.HandleStorageError(nameof(ILogCommitManager.Commit), "could not commit because of lost lease", this.eventLogCommitBlob.Name, ex, true, this.PartitionErrorHandler.IsTerminated); - throw; - } - catch (Azure.RequestFailedException ex) when (BlobUtilsV12.LeaseExpired(ex) && numAttempts < BlobManager.MaxRetries) - { - // if we get here, the lease renewal task did not complete in time - // give it another chance to complete - this.TraceHelper.LeaseProgress("ILogCommitManager.Commit: wait for next renewal"); - this.NextLeaseRenewalTask.Wait(); - this.TraceHelper.LeaseProgress("ILogCommitManager.Commit: renewal complete"); - return (commitMetadata.Length, false); - } - }); - - this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ILogCommitManager.Commit"); - } - catch - { - this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ILogCommitManager.Commit failed"); - throw; - } - } - - - IEnumerable ILogCommitManager.ListCommits() - { - // we only use a single commit file in this implementation - yield return 0; - } - - void ILogCommitManager.OnRecovery(long commitNum) - { - // TODO: make sure our use of single commit is safe - } - - void ILogCommitManager.RemoveAllCommits() - { - // TODO: make sure our use of single commit is safe - } - - void ILogCommitManager.RemoveCommit(long commitNum) - { - // TODO: make sure our use of single commit is safe - } - - byte[] ILogCommitManager.GetCommitMetadata(long commitNum) - { - try - { - this.StorageTracer?.FasterStorageProgress($"StorageOpCalled ILogCommitManager.GetCommitMetadata (thread={Thread.CurrentThread.ManagedThreadId})"); - - using var stream = new MemoryStream(); - - this.PerformWithRetries( - false, - "BlobClient.DownloadTo", - "ReadCommitLogMetadata", - "", - this.eventLogCommitBlob.Name, - 1000, - true, - (int numAttempts) => - { - if (numAttempts > 0) - { - stream.Seek(0, SeekOrigin.Begin); - } - - try - { - var client = numAttempts > 2 ? this.eventLogCommitBlob.Default : this.eventLogCommitBlob.Aggressive; - - client.DownloadTo( - destination: stream, - conditions: new BlobRequestConditions() { LeaseId = this.leaseClient.LeaseId }, - cancellationToken: this.PartitionErrorHandler.Token); - - return (stream.Position, true); - } - catch (Azure.RequestFailedException ex) when (BlobUtilsV12.LeaseConflict(ex)) - { - // We lost the lease to someone else. Terminate ownership immediately. - this.TraceHelper.LeaseLost(this.leaseTimer.Elapsed.TotalSeconds, nameof(ILogCommitManager.GetCommitMetadata)); - this.HandleStorageError(nameof(ILogCommitManager.Commit), "could not read latest commit due to lost lease", this.eventLogCommitBlob.Name, ex, true, this.PartitionErrorHandler.IsTerminated); - throw; - } - catch (Azure.RequestFailedException ex) when (BlobUtilsV12.LeaseExpired(ex) && numAttempts < BlobManager.MaxRetries) - { - // if we get here, the lease renewal task did not complete in time - // give it another chance to complete - this.TraceHelper.LeaseProgress("ILogCommitManager.Commit: wait for next renewal"); - this.NextLeaseRenewalTask.Wait(); - this.TraceHelper.LeaseProgress("ILogCommitManager.Commit: renewal complete"); - return (0, false); - } - }); - - var bytes = stream.ToArray(); - this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ILogCommitManager.GetCommitMetadata {bytes?.Length ?? null} bytes"); - return bytes.Length == 0 ? null : bytes; - } - catch (Exception e) - { - this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ILogCommitManager.GetCommitMetadata failed with {e.GetType().Name}: {e.Message}"); - throw; - } - } - - #endregion - - #region ICheckpointManager - - void ICheckpointManager.InitializeIndexCheckpoint(Guid indexToken) - { - // there is no need to create empty directories in a blob container - } - - void ICheckpointManager.InitializeLogCheckpoint(Guid logToken) - { - // there is no need to create empty directories in a blob container - } - - IEnumerable ICheckpointManager.GetIndexCheckpointTokens() - { - var indexToken = this.CheckpointInfo.IndexToken; - this.StorageTracer?.FasterStorageProgress($"StorageOp ICheckpointManager.GetIndexCheckpointTokens indexToken={indexToken}"); - yield return indexToken; - } - - IEnumerable ICheckpointManager.GetLogCheckpointTokens() - { - var logToken = this.CheckpointInfo.LogToken; - this.StorageTracer?.FasterStorageProgress($"StorageOp ICheckpointManager.GetLogCheckpointTokens logToken={logToken}"); - yield return logToken; - } - - internal async Task FindCheckpointsAsync(bool logIsEmpty) - { - BlobUtilsV12.BlockBlobClients checkpointCompletedBlob = default; - try - { - string jsonString = null; - - if (this.UseLocalFiles) - { - try - { - jsonString = this.LocalCheckpointManager.GetLatestCheckpointJson(); - } - catch (FileNotFoundException) when (logIsEmpty) - { - // ok to not have a checkpoint yet - } - } - else - { - var partDir = this.blockBlobPartitionDirectory; - checkpointCompletedBlob = partDir.GetBlockBlobClient(this.GetCheckpointCompletedBlobName()); - - await this.PerformWithRetriesAsync( - semaphore: null, - requireLease: true, - "BlockBlobClient.DownloadContentAsync", - "FindCheckpointsAsync", - "", - checkpointCompletedBlob.Name, - 1000, - true, - async (numAttempts) => - { - try - { - Azure.Response downloadResult = await checkpointCompletedBlob.WithRetries.DownloadContentAsync(); - jsonString = downloadResult.Value.Content.ToString(); - this.CheckpointInfoETag = downloadResult.Value.Details.ETag; - return 1; - } - catch (Azure.RequestFailedException e) when (BlobUtilsV12.BlobDoesNotExist(e) && logIsEmpty) - { - // ok to not have a checkpoint yet - return 0; - } - }); - } - - if (jsonString == null) - { - return false; - } - else - { - // read the fields from the json to update the checkpoint info - JsonConvert.PopulateObject(jsonString, this.CheckpointInfo); - return true; - } - } - catch (Exception e) - { - this.HandleStorageError(nameof(FindCheckpointsAsync), "could not find any checkpoint", checkpointCompletedBlob.Name, e, true, this.PartitionErrorHandler.IsTerminated); - throw; - } - } - - void ICheckpointManager.CommitIndexCheckpoint(Guid indexToken, byte[] commitMetadata) - { - try - { - this.StorageTracer?.FasterStorageProgress($"StorageOpCalled ICheckpointManager.CommitIndexCheckpoint, indexToken={indexToken}"); - var partDir = this.blockBlobPartitionDirectory; - var metaFileBlob = partDir.GetBlockBlobClient(this.GetIndexCheckpointMetaBlobName(indexToken)); - - this.PerformWithRetries( - false, - "BlockBlobClient.OpenWrite", - "WriteIndexCheckpointMetadata", - $"token={indexToken} size={commitMetadata.Length}", - metaFileBlob.Name, - 1000, - true, - (numAttempts) => - { - var client = metaFileBlob.WithRetries; - using var blobStream = client.OpenWrite(overwrite: true); - using var writer = new BinaryWriter(blobStream); - writer.Write(commitMetadata.Length); - writer.Write(commitMetadata); - writer.Flush(); - return (commitMetadata.Length, true); - }); - - this.CheckpointInfo.IndexToken = indexToken; - this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.CommitIndexCheckpoint, target={metaFileBlob.Name}"); - } - catch - { - this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.CommitIndexCheckpoint failed"); - throw; - } - } - - void ICheckpointManager.CommitLogCheckpoint(Guid logToken, byte[] commitMetadata) - { - try - { - this.StorageTracer?.FasterStorageProgress($"StorageOpCalled ICheckpointManager.CommitLogCheckpoint, logToken={logToken}"); - var partDir = this.blockBlobPartitionDirectory; - var metaFileBlob = partDir.GetBlockBlobClient(this.GetHybridLogCheckpointMetaBlobName(logToken)); - - this.PerformWithRetries( - false, - "BlockBlobClient.OpenWrite", - "WriteHybridLogCheckpointMetadata", - $"token={logToken}", - metaFileBlob.Name, - 1000, - true, - (numAttempts) => - { - var client = metaFileBlob.WithRetries; - using var blobStream = client.OpenWrite(overwrite: true); - using var writer = new BinaryWriter(blobStream); - writer.Write(commitMetadata.Length); - writer.Write(commitMetadata); - writer.Flush(); - return (commitMetadata.Length + 4, true); - }); - - this.CheckpointInfo.LogToken = logToken; - this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.CommitLogCheckpoint, target={metaFileBlob.Name}"); - } - catch - { - this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.CommitLogCheckpoint failed"); - throw; - } - } - - void ICheckpointManager.CommitLogIncrementalCheckpoint(Guid logToken, long version, byte[] commitMetadata, DeltaLog deltaLog) - { - throw new NotImplementedException("incremental checkpointing is not implemented"); - } - - byte[] ICheckpointManager.GetIndexCheckpointMetadata(Guid indexToken) - { - try - { - this.StorageTracer?.FasterStorageProgress($"StorageOpCalled ICheckpointManager.GetIndexCheckpointMetadata, indexToken={indexToken}"); - var partDir = this.blockBlobPartitionDirectory; - var metaFileBlob = partDir.GetBlockBlobClient(this.GetIndexCheckpointMetaBlobName(indexToken)); - byte[] result = null; - - this.PerformWithRetries( - false, - "BlockBlobClient.OpenRead", - "ReadIndexCheckpointMetadata", - "", - metaFileBlob.Name, - 1000, - true, - (numAttempts) => - { - var client = metaFileBlob.WithRetries; - using var blobstream = client.OpenRead(); - using var reader = new BinaryReader(blobstream); - var len = reader.ReadInt32(); - result = reader.ReadBytes(len); - return (len + 4, true); - }); - - this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.GetIndexCheckpointMetadata {result?.Length ?? null} bytes, target={metaFileBlob.Name}"); - return result; - } - catch - { - this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.GetIndexCheckpointMetadata failed"); - throw; - } - } - - byte[] ICheckpointManager.GetLogCheckpointMetadata(Guid logToken, DeltaLog deltaLog, bool scanDelta, long recoverTo) - { - try - { - this.StorageTracer?.FasterStorageProgress($"StorageOpCalled ICheckpointManager.GetLogCheckpointMetadata, logToken={logToken}"); - var partDir = this.blockBlobPartitionDirectory; - var metaFileBlob = partDir.GetBlockBlobClient(this.GetHybridLogCheckpointMetaBlobName(logToken)); - byte[] result = null; - - this.PerformWithRetries( - false, - "BlockBlobClient.OpenRead", - "ReadLogCheckpointMetadata", - "", - metaFileBlob.Name, - 1000, - true, - (numAttempts) => - { - var client = metaFileBlob.WithRetries; - using var blobstream = client.OpenRead(); - using var reader = new BinaryReader(blobstream); - var len = reader.ReadInt32(); - result = reader.ReadBytes(len); - return (len + 4, true); - }); - - this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.GetLogCheckpointMetadata {result?.Length ?? null} bytes, target={metaFileBlob.Name}"); - return result; - } - catch - { - this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.GetLogCheckpointMetadata failed"); - throw; - } - } - - void GetPartitionDirectories(string path, out BlobUtilsV12.BlobDirectory blockBlobDir, out BlobUtilsV12.BlobDirectory pageBlobDir) - { - var blockPartDir = this.blockBlobPartitionDirectory; - blockBlobDir = blockPartDir.GetSubDirectory(path); - var pagePartDir = this.pageBlobPartitionDirectory; - pageBlobDir = pagePartDir.GetSubDirectory(path); - } - - IDevice ICheckpointManager.GetIndexDevice(Guid indexToken) - { - try - { - this.StorageTracer?.FasterStorageProgress($"StorageOpCalled ICheckpointManager.GetIndexDevice, indexToken={indexToken}"); - var (path, blobName) = this.GetPrimaryHashTableBlobName(indexToken); - this.GetPartitionDirectories(path, out var blockBlobDir, out var pageBlobDir); - var device = new AzureStorageDevice(blobName, blockBlobDir, pageBlobDir, this, false); // we don't need a lease since the token provides isolation - device.StartAsync().Wait(); - this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.GetIndexDevice, target={blockBlobDir}{blobName}"); - return device; - } - catch - { - this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.GetIndexDevice failed"); - throw; - } - } - - IDevice ICheckpointManager.GetSnapshotLogDevice(Guid token) - { - try - { - this.StorageTracer?.FasterStorageProgress($"StorageOpCalled ICheckpointManager.GetSnapshotLogDevice, token={token}"); - var (path, blobName) = this.GetLogSnapshotBlobName(token); - this.GetPartitionDirectories(path, out var blockBlobDir, out var pageBlobDir); - var device = new AzureStorageDevice(blobName, blockBlobDir, pageBlobDir, this, false); // we don't need a lease since the token provides isolation - device.StartAsync().Wait(); - this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.GetSnapshotLogDevice, target={blockBlobDir}{blobName}"); - return device; - } - catch - { - this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.GetSnapshotLogDevice failed"); - throw; - } - } - - IDevice ICheckpointManager.GetSnapshotObjectLogDevice(Guid token) - { - try - { - this.StorageTracer?.FasterStorageProgress($"StorageOpCalled ICheckpointManager.GetSnapshotObjectLogDevice, token={token}"); - var (path, blobName) = this.GetObjectLogSnapshotBlobName(token); - this.GetPartitionDirectories(path, out var blockBlobDir, out var pageBlobDir); - var device = new AzureStorageDevice(blobName, blockBlobDir, pageBlobDir, this, false); // we don't need a lease since the token provides isolation - device.StartAsync().Wait(); - this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.GetSnapshotObjectLogDevice, target={blockBlobDir}{blobName}"); - return device; - } - catch - { - this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.GetSnapshotObjectLogDevice failed"); - throw; - } - } - - IDevice ICheckpointManager.GetDeltaLogDevice(Guid token) - { - try - { - this.StorageTracer?.FasterStorageProgress($"StorageOpCalled ICheckpointManager.GetDeltaLogDevice on, token={token}"); - var (path, blobName) = this.GetDeltaLogSnapshotBlobName(token); - this.GetPartitionDirectories(path, out var blockBlobDir, out var pageBlobDir); - var device = new AzureStorageDevice(blobName, blockBlobDir, pageBlobDir, this, false); // we don't need a lease since the token provides isolation - device.StartAsync().Wait(); - this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.GetDeltaLogDevice, target={blockBlobDir}{blobName}"); - return device; - } - catch - { - this.StorageTracer?.FasterStorageProgress($"StorageOpReturned ICheckpointManager.GetDeltaLogDevice failed"); - throw; - } - } - - #endregion - - internal async Task PersistSingletonsAsync(byte[] singletons, Guid guid) - { - if (this.UseLocalFiles) - { - var path = Path.Combine(this.LocalCheckpointDirectoryPath, this.GetSingletonsSnapshotBlobName(guid)); - using var filestream = File.OpenWrite(path); - await filestream.WriteAsync(singletons, 0, singletons.Length); - await filestream.FlushAsync(); - } - else - { - var singletonsBlob = this.blockBlobPartitionDirectory.GetBlockBlobClient(this.GetSingletonsSnapshotBlobName(guid)); - await this.PerformWithRetriesAsync( - BlobManager.AsynchronousStorageWriteMaxConcurrency, - false, - "BlockBlobClient.UploadAsync", - "WriteSingletons", - "", - singletonsBlob.Name, - 1000 + singletons.Length / 5000, - false, - async (numAttempts) => - { - var client = singletonsBlob.WithRetries; - await client.UploadAsync( - new MemoryStream(singletons), - new BlobUploadOptions(), - this.PartitionErrorHandler.Token); - return singletons.Length; - }); - } - } - - internal async Task RecoverSingletonsAsync() - { - if (this.UseLocalFiles) - { - var path = Path.Combine(this.LocalCheckpointDirectoryPath, this.GetSingletonsSnapshotBlobName(this.CheckpointInfo.LogToken)); - var stream = File.OpenRead(path); - return stream; - } - else - { - var singletonsBlob = this.blockBlobPartitionDirectory.GetBlockBlobClient(this.GetSingletonsSnapshotBlobName(this.CheckpointInfo.LogToken)); - var stream = new MemoryStream(); - await this.PerformWithRetriesAsync( - BlobManager.AsynchronousStorageReadMaxConcurrency, - true, - "BlobBaseClient.DownloadToAsync", - "ReadSingletons", - "", - singletonsBlob.Name, - 20000, - true, - async (numAttempts) => - { - - var client = singletonsBlob.WithRetries; - var memoryStream = new MemoryStream(); - await client.DownloadToAsync(stream); - return stream.Position; - }); - - stream.Seek(0, SeekOrigin.Begin); - return stream; - } - } - - internal async Task FinalizeCheckpointCompletedAsync() - { - var jsonText = JsonConvert.SerializeObject(this.CheckpointInfo, Formatting.Indented); - if (this.UseLocalFiles) - { - File.WriteAllText(Path.Combine(this.LocalCheckpointDirectoryPath, this.GetCheckpointCompletedBlobName()), jsonText); - } - else - { - var checkpointCompletedBlob = this.blockBlobPartitionDirectory.GetBlockBlobClient(this.GetCheckpointCompletedBlobName()); - await this.PerformWithRetriesAsync( - BlobManager.AsynchronousStorageWriteMaxConcurrency, - true, - "BlockBlobClient.UploadAsync", - "WriteCheckpointMetadata", - "", - checkpointCompletedBlob.Name, - 1000, - true, - async (numAttempts) => - { - var client = numAttempts > 1 ? checkpointCompletedBlob.Default : checkpointCompletedBlob.Aggressive; - - var azureResponse = await client.UploadAsync( - new MemoryStream(Encoding.UTF8.GetBytes(jsonText)), - new BlobUploadOptions() - { - Conditions = this.CheckpointInfoETag.HasValue ? - new BlobRequestConditions() { IfMatch = this.CheckpointInfoETag.Value } - : new BlobRequestConditions() { IfNoneMatch = Azure.ETag.All }, - HttpHeaders = new BlobHttpHeaders() { ContentType = "application/json" }, - }, - this.PartitionErrorHandler.Token); - - this.CheckpointInfoETag = azureResponse.Value.ETag; - - return jsonText.Length; - }, - async () => - { - var response = await checkpointCompletedBlob.Default.GetPropertiesAsync(); - this.CheckpointInfoETag = response.Value.ETag; - }); - } - } } } diff --git a/cs/src/devices/AzureStorageDevice/DefaultBlobManager.cs b/cs/src/devices/AzureStorageDevice/DefaultBlobManager.cs deleted file mode 100644 index cb212f547..000000000 --- a/cs/src/devices/AzureStorageDevice/DefaultBlobManager.cs +++ /dev/null @@ -1,344 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT license. - -using Microsoft.Azure.Storage; -using Microsoft.Azure.Storage.Blob; -using Microsoft.Azure.Storage.Blob.Protocol; -using Microsoft.Azure.Storage.RetryPolicies; -using System; -using System.Diagnostics; -using System.Threading; -using System.Threading.Tasks; - -namespace FASTER.devices -{ - /// - /// Default blob manager with lease support - /// - public class DefaultBlobManager : IBlobManager - { - readonly CancellationTokenSource cts; - readonly bool underLease; - string leaseId; - - readonly TimeSpan LeaseDuration = TimeSpan.FromSeconds(45); // max time the lease stays after unclean shutdown - readonly TimeSpan LeaseRenewal = TimeSpan.FromSeconds(30); // how often we renew the lease - readonly TimeSpan LeaseSafetyBuffer = TimeSpan.FromSeconds(10); // how much time we want left on the lease before issuing a protected access - volatile Stopwatch leaseTimer; - Task LeaseMaintenanceLoopTask = Task.CompletedTask; - volatile Task NextLeaseRenewalTask = Task.CompletedTask; - - private readonly CloudBlobDirectory leaseDirectory; - private const string LeaseBlobName = "lease"; - private CloudBlockBlob leaseBlob; - - static readonly SemaphoreSlim AsynchronousStorageReadMaxConcurrency = new(Environment.ProcessorCount * 25); - static readonly SemaphoreSlim AsynchronousStorageWriteMaxConcurrency = new(Environment.ProcessorCount * 25); - - /// - /// Create instance of blob manager - /// - /// Should we use blob leases - /// Directory to store lease file - public DefaultBlobManager(bool underLease, CloudBlobDirectory leaseDirectory = null) - { - this.underLease = underLease; - this.leaseDirectory = leaseDirectory; - this.cts = new CancellationTokenSource(); - - if (underLease) - { - // Start lease maintenance loop - var _ = StartAsync(); - } - } - - /// - /// Start blob manager and acquire lease - /// - /// - private async Task StartAsync() - { - this.leaseBlob = this.leaseDirectory.GetBlockBlobReference(LeaseBlobName); - await this.AcquireOwnership().ConfigureAwait(false); - } - - /// - /// Clean shutdown, wait for everything, then terminate - /// - /// - public async Task StopAsync() - { - this.cts.Cancel(); // has no effect if already cancelled - await this.LeaseMaintenanceLoopTask.ConfigureAwait(false); // wait for loop to terminate cleanly - } - - /// - public CancellationToken CancellationToken => cts.Token; - - /// - public SemaphoreSlim AsyncStorageReadMaxConcurrency => AsynchronousStorageReadMaxConcurrency; - - /// - public SemaphoreSlim AsyncStorageWriteMaxConcurrency => AsynchronousStorageWriteMaxConcurrency; - - /// - public int MaxRetries => 5; - - /// - public bool ConfigureAwaitForStorage => true; - - /// - public async ValueTask ConfirmLeaseAsync() - { - if (!underLease) - return; - - while (true) - { - if (this.leaseTimer?.Elapsed < this.LeaseDuration - this.LeaseSafetyBuffer) - return; - - Debug.WriteLine("Access is waiting for fresh lease"); - - await this.NextLeaseRenewalTask.ConfigureAwait(false); - } - } - - /// - public void HandleBlobError(string where, string message, string blobName, Exception e, bool isFatal) - { - HandleError(where, $"Encountered storage exception for blob {blobName}", e, isFatal); - } - - /// - public BlobRequestOptions GetBlobRequestOptionsWithoutRetry() - { - return new BlobRequestOptions() - { - RetryPolicy = default, // Device handles retries explicitly - NetworkTimeout = TimeSpan.FromSeconds(30), - ServerTimeout = TimeSpan.FromSeconds(20), - MaximumExecutionTime = TimeSpan.FromSeconds(30), - }; - } - - /// - public BlobRequestOptions GetBlobRequestOptionsWithRetry() - { - return new BlobRequestOptions() - { - RetryPolicy = new ExponentialRetry(TimeSpan.FromSeconds(2), MaxRetries), - NetworkTimeout = TimeSpan.FromSeconds(120), - ServerTimeout = TimeSpan.FromSeconds(120), - MaximumExecutionTime = TimeSpan.FromSeconds(120), - }; - } - - private async Task AcquireOwnership() - { - var newLeaseTimer = new Stopwatch(); - - while (true) - { - CancellationToken.ThrowIfCancellationRequested(); - - try - { - newLeaseTimer.Restart(); - - this.leaseId = await this.leaseBlob.AcquireLeaseAsync(LeaseDuration, null, - accessCondition: null, options: this.GetBlobRequestOptionsWithRetry(), operationContext: null, this.CancellationToken) - .ConfigureAwait(ConfigureAwaitForStorage); - - this.leaseTimer = newLeaseTimer; - this.LeaseMaintenanceLoopTask = Task.Run(() => this.MaintenanceLoopAsync()); - return; - } - catch (StorageException ex) when (LeaseConflictOrExpired(ex)) - { - Debug.WriteLine("Waiting for lease"); - - // the previous owner has not released the lease yet, - // try again until it becomes available, should be relatively soon - // as the transport layer is supposed to shut down the previous owner when starting this - await Task.Delay(TimeSpan.FromSeconds(1), this.CancellationToken).ConfigureAwait(false); - - continue; - } - catch (StorageException ex) when (BlobDoesNotExist(ex)) - { - try - { - // Create blob with empty content, then try again - Debug.WriteLine("Creating commit blob"); - await this.leaseBlob.UploadFromByteArrayAsync(Array.Empty(), 0, 0) - .ConfigureAwait(ConfigureAwaitForStorage); - continue; - } - catch (StorageException ex2) when (LeaseConflictOrExpired(ex2)) - { - // creation race, try from top - Debug.WriteLine("Creation race observed, retrying"); - continue; - } - } - catch (Exception e) - { - HandleError(nameof(AcquireOwnership), "Could not acquire lease", e, true); - throw; - } - } - } - - private void HandleError(string context, string message, Exception exception, bool terminate) - { - Debug.WriteLine(context + ": " + message + ", " + exception.ToString()); - - // terminate in response to the error - if (terminate && !cts.IsCancellationRequested) - { - Terminate(); - } - } - - private async Task RenewLeaseTask() - { - try - { - await Task.Delay(this.LeaseRenewal, this.CancellationToken).ConfigureAwait(false); - AccessCondition acc = new AccessCondition() { LeaseId = this.leaseId }; - var nextLeaseTimer = new Stopwatch(); - nextLeaseTimer.Start(); - await this.leaseBlob.RenewLeaseAsync(acc, this.CancellationToken) - .ConfigureAwait(ConfigureAwaitForStorage); - this.leaseTimer = nextLeaseTimer; - } - catch (OperationCanceledException) - { - // o.k. during termination or shutdown - } - catch (Exception) - { - Debug.WriteLine("Failed to renew lease"); - throw; - } - } - - private async Task MaintenanceLoopAsync() - { - try - { - while (true) - { - // save the task so storage accesses can wait for it - this.NextLeaseRenewalTask = this.RenewLeaseTask(); - - // wait for successful renewal, or exit the loop as this throws - await this.NextLeaseRenewalTask.ConfigureAwait(false); - } - } - catch (OperationCanceledException) - { - // it's o.k. to cancel while waiting - Debug.WriteLine("Lease renewal loop cleanly canceled"); - } - catch (StorageException e) when (e.InnerException != null && e.InnerException is OperationCanceledException) - { - // it's o.k. to cancel a lease renewal - Debug.WriteLine("Lease renewal storage operation canceled"); - } - catch (StorageException ex) when (LeaseConflict(ex)) - { - // We lost the lease to someone else. Terminate ownership immediately. - HandleError(nameof(MaintenanceLoopAsync), "Lost lease", ex, true); - } - catch (Exception e) when (!IsFatal(e)) - { - HandleError(nameof(MaintenanceLoopAsync), "Could not maintain lease", e, true); - } - - Debug.WriteLine("Exited lease maintenance loop"); - - if (this.CancellationToken.IsCancellationRequested) - { - // this is an unclean shutdown, so we let the lease expire to protect straggling storage accesses - Debug.WriteLine("Leaving lease to expire on its own"); - } - else - { - try - { - Debug.WriteLine("Releasing lease"); - - AccessCondition acc = new AccessCondition() { LeaseId = this.leaseId }; - - await this.leaseBlob.ReleaseLeaseAsync(accessCondition: acc, - options: this.GetBlobRequestOptionsWithRetry(), operationContext: null, cancellationToken: this.CancellationToken) - .ConfigureAwait(ConfigureAwaitForStorage); - } - catch (OperationCanceledException) - { - // it's o.k. if termination is triggered while waiting - } - catch (StorageException e) when (e.InnerException != null && e.InnerException is OperationCanceledException) - { - // it's o.k. if termination is triggered while we are releasing the lease - } - catch (Exception) - { - Debug.WriteLine("could not release lease for " + this.leaseBlob.Name); - // swallow exceptions when releasing a lease - } - } - - Terminate(); - - Debug.WriteLine("Blob manager stopped"); - } - - private void Terminate() - { - try - { - cts.Cancel(); - } - catch (AggregateException aggregate) - { - foreach (var e in aggregate.InnerExceptions) - { - HandleError("Terminate", "Encountered exeption while canceling token", e, false); - } - } - catch (Exception e) - { - HandleError("Terminate", "Encountered exeption while canceling token", e, false); - } - } - - private static bool LeaseConflict(StorageException e) - { - return (e.RequestInformation.HttpStatusCode == 409); - } - - private static bool LeaseConflictOrExpired(StorageException e) - { - return (e.RequestInformation.HttpStatusCode == 409) || (e.RequestInformation.HttpStatusCode == 412); - } - - private static bool BlobDoesNotExist(StorageException e) - { - var information = e.RequestInformation.ExtendedErrorInformation; - return (e.RequestInformation.HttpStatusCode == 404) && (information.ErrorCode.Equals(BlobErrorCodeStrings.BlobNotFound)); - } - - private static bool IsFatal(Exception exception) - { - if (exception is OutOfMemoryException || exception is StackOverflowException) - { - return true; - } - return false; - } - } -} \ No newline at end of file diff --git a/cs/src/devices/AzureStorageDevice/FasterTraceHelper.cs b/cs/src/devices/AzureStorageDevice/FasterTraceHelper.cs new file mode 100644 index 000000000..28d38006b --- /dev/null +++ b/cs/src/devices/AzureStorageDevice/FasterTraceHelper.cs @@ -0,0 +1,184 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT License. + +namespace FASTER.devices +{ + using System; + using Microsoft.Extensions.Logging; + + class FasterTraceHelper + { + readonly ILogger logger; + readonly LogLevel logLevelLimit; + readonly ILogger performanceLogger; + + public FasterTraceHelper(ILogger logger, LogLevel logLevelLimit, ILogger performanceLogger) + { + this.logger = logger; + this.logLevelLimit = logLevelLimit; + this.performanceLogger = performanceLogger; + } + + public bool IsTracingAtMostDetailedLevel => this.logLevelLimit == LogLevel.Trace; + + // ----- faster storage layer events + + public void FasterStoreCreated(long inputQueuePosition, long latencyMs) + { + if (this.logLevelLimit <= LogLevel.Information) + { + this.logger.LogInformation("Created Store, inputQueuePosition={inputQueuePosition} latencyMs={latencyMs}", inputQueuePosition, latencyMs); + } + } + public void FasterCheckpointStarted(Guid checkpointId, string details, string storeStats, long commitLogPosition, long inputQueuePosition) + { + if (this.logLevelLimit <= LogLevel.Information) + { + this.logger.LogInformation("Started Checkpoint {checkpointId}, details={details}, storeStats={storeStats}, commitLogPosition={commitLogPosition} inputQueuePosition={inputQueuePosition}", checkpointId, details, storeStats, commitLogPosition, inputQueuePosition); + } + } + + public void FasterCheckpointPersisted(Guid checkpointId, string details, long commitLogPosition, long inputQueuePosition, long latencyMs) + { + if (this.logLevelLimit <= LogLevel.Information) + { + this.logger.LogInformation("Persisted Checkpoint {checkpointId}, details={details}, commitLogPosition={commitLogPosition} inputQueuePosition={inputQueuePosition} latencyMs={latencyMs}", checkpointId, details, commitLogPosition, inputQueuePosition, latencyMs); + } + + if (latencyMs > 10000) + { + this.FasterPerfWarning($"Persisting the checkpoint {checkpointId} took {(double)latencyMs / 1000}s, which is excessive; checkpointId={checkpointId} commitLogPosition={commitLogPosition} inputQueuePosition={inputQueuePosition}"); + } + } + + public void FasterLogPersisted(long commitLogPosition, long numberEvents, long sizeInBytes, long latencyMs) + { + if (this.logLevelLimit <= LogLevel.Debug) + { + this.logger.LogDebug("Persisted Log, commitLogPosition={commitLogPosition} numberEvents={numberEvents} sizeInBytes={sizeInBytes} latencyMs={latencyMs}", commitLogPosition, numberEvents, sizeInBytes, latencyMs); + } + + if (latencyMs > 10000) + { + this.FasterPerfWarning($"Persisting the log took {(double)latencyMs / 1000}s, which is excessive; commitLogPosition={commitLogPosition} numberEvents={numberEvents} sizeInBytes={sizeInBytes}"); + } + } + + public void FasterPerfWarning(string details) + { + if (this.logLevelLimit <= LogLevel.Warning) + { + this.performanceLogger.LogWarning("Performance issue detected: {details}", details); + } + } + + public void FasterCheckpointLoaded(long commitLogPosition, long inputQueuePosition, string storeStats, long latencyMs) + { + if (this.logLevelLimit <= LogLevel.Information) + { + this.logger.LogInformation("Loaded Checkpoint, commitLogPosition={commitLogPosition} inputQueuePosition={inputQueuePosition} storeStats={storeStats} latencyMs={latencyMs}", commitLogPosition, inputQueuePosition, storeStats, latencyMs); + } + } + + public void FasterLogReplayed(long commitLogPosition, long inputQueuePosition, long numberEvents, long sizeInBytes, string storeStats, long latencyMs) + { + if (this.logLevelLimit <= LogLevel.Information) + { + this.logger.LogInformation("Replayed CommitLog, commitLogPosition={commitLogPosition} inputQueuePosition={inputQueuePosition} numberEvents={numberEvents} sizeInBytes={sizeInBytes} storeStats={storeStats} latencyMs={latencyMs}", commitLogPosition, inputQueuePosition, numberEvents, sizeInBytes, storeStats, latencyMs); + } + } + + public void FasterStorageError(string context, Exception exception) + { + if (this.logLevelLimit <= LogLevel.Error) + { + this.logger.LogError("!!! Faster Storage Error : {context} : {exception}", context, exception); + } + } + + public void FasterCacheSizeMeasured(int numPages, long numRecords, long sizeInBytes, long gcMemory, long processMemory, long discrepancy, double elapsedMs) + { + if (this.logLevelLimit <= LogLevel.Information) + { + this.logger.LogInformation("Measured CacheSize numPages={numPages} numRecords={numRecords} sizeInBytes={sizeInBytes} gcMemory={gcMemory} processMemory={processMemory} discrepancy={discrepancy} elapsedMs={elapsedMs:F2}", numPages, numRecords, sizeInBytes, gcMemory, processMemory, discrepancy, elapsedMs); + } + } + + public void FasterProgress(string details) + { + if (this.logLevelLimit <= LogLevel.Debug) + { + this.logger.LogDebug("{details}", details); + } + } + + public void FasterStorageProgress(string details) + { + if (this.logLevelLimit <= LogLevel.Trace) + { + this.logger.LogTrace("{details}", details); + } + } + + public void FasterAzureStorageAccessCompleted(string intent, long size, string operation, string target, double latency, int attempt) + { + if (this.logLevelLimit <= LogLevel.Debug) + { + this.logger.LogDebug("storage access completed intent={intent} size={size} operation={operation} target={target} latency={latency} attempt={attempt}", + intent, size, operation, target, latency, attempt); + } + } + + public enum CompactionProgress { Skipped, Started, Completed }; + + public void FasterCompactionProgress(CompactionProgress progress, string operation, long begin, long safeReadOnly, long tail, long minimalSize, long compactionAreaSize, double elapsedMs) + { + if (this.logLevelLimit <= LogLevel.Information) + { + this.logger.LogInformation("Compaction {progress} operation={operation} begin={begin} safeReadOnly={safeReadOnly} tail={tail} minimalSize={minimalSize} compactionAreaSize={compactionAreaSize} elapsedMs={elapsedMs}", progress, operation, begin, safeReadOnly, tail, minimalSize, compactionAreaSize, elapsedMs); + } + } + + // ----- lease management events + + public void LeaseAcquired() + { + if (this.logLevelLimit <= LogLevel.Information) + { + this.logger.LogInformation("PartitionLease acquired"); + } + } + + public void LeaseRenewed(double elapsedSeconds, double timing) + { + if (this.logLevelLimit <= LogLevel.Debug) + { + this.logger.LogDebug("PartitionLease renewed after {elapsedSeconds:F2}s timing={timing:F2}s", elapsedSeconds, timing); + } + } + + public void LeaseReleased(double elapsedSeconds) + { + if (this.logLevelLimit <= LogLevel.Information) + { + this.logger.LogInformation("PartitionLease released after {elapsedSeconds:F2}s", elapsedSeconds); + } + } + + public void LeaseLost(double elapsedSeconds, string operation) + { + if (this.logLevelLimit <= LogLevel.Warning) + { + this.logger.LogWarning("PartitionLease lost after {elapsedSeconds:F2}s in {operation}", elapsedSeconds, operation); + } + } + + public void LeaseProgress(string operation) + { + if (this.logLevelLimit <= LogLevel.Debug) + { + this.logger.LogDebug("PartitionLease progress: {operation}", operation); + } + } + } +} diff --git a/cs/src/devices/AzureStorageDevice/IBlobManager.cs b/cs/src/devices/AzureStorageDevice/IBlobManager.cs deleted file mode 100644 index 2844fd7ce..000000000 --- a/cs/src/devices/AzureStorageDevice/IBlobManager.cs +++ /dev/null @@ -1,69 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT license. - -using Microsoft.Azure.Storage.Blob; -using System; -using System.Threading; -using System.Threading.Tasks; - -namespace FASTER.devices -{ - /// - /// Manager for blobs, can be shared across devices. - /// - public interface IBlobManager - { - /// - /// Get blob request options (default) - /// - /// - BlobRequestOptions GetBlobRequestOptionsWithoutRetry(); - - /// - /// Get blob request options (with retry) - /// - /// - BlobRequestOptions GetBlobRequestOptionsWithRetry(); - - /// - /// Cancellation token for blob operations - /// - CancellationToken CancellationToken { get; } - - /// - /// Error handler for blob operations - /// - /// - /// - /// - /// - /// - void HandleBlobError(string where, string message, string blobName, Exception e, bool isFatal); - - /// - /// Confirm lease ownership - /// - /// - ValueTask ConfirmLeaseAsync(); - - /// - /// Max concurrency on async storage read - /// - SemaphoreSlim AsyncStorageReadMaxConcurrency { get; } - - /// - /// Max concurrency on async storage write - /// - SemaphoreSlim AsyncStorageWriteMaxConcurrency { get; } - - /// - /// Max retries - /// - int MaxRetries { get; } - - /// - /// Configure await for storage - /// - bool ConfigureAwaitForStorage { get; } - } -} \ No newline at end of file From fc6755f4dbf6252e859a3ebb807e612fdf08a965 Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Sat, 4 Feb 2023 13:11:52 -0800 Subject: [PATCH 04/29] getting there --- .../AzureStorageDevice/AzureStorageDevice.cs | 71 ++++++++++++++++++- .../devices/AzureStorageDevice/BlobEntry.cs | 2 +- .../devices/AzureStorageDevice/BlobManager.cs | 4 +- .../FASTER.devices.AzureStorageDevice.csproj | 8 ++- .../AzureStorageDevice/StorageOperations.cs | 2 +- cs/test/DeviceFasterLogTests.cs | 15 +--- cs/test/NameValidator.cs | 47 ++++++++++++ cs/test/TestUtils.cs | 8 ++- 8 files changed, 133 insertions(+), 24 deletions(-) create mode 100644 cs/test/NameValidator.cs diff --git a/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs b/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs index dee3b8ddb..187943321 100644 --- a/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs +++ b/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs @@ -19,7 +19,7 @@ namespace FASTER.devices /// A IDevice Implementation that is backed byAzure Page Blob. /// This device is slower than a local SSD or HDD, but provides scalability and shared access in the cloud. /// - class AzureStorageDevice : StorageDeviceBase + public class AzureStorageDevice : StorageDeviceBase { readonly ConcurrentDictionary blobs; readonly BlobUtilsV12.BlobDirectory pageBlobDirectory; @@ -31,6 +31,9 @@ class AzureStorageDevice : StorageDeviceBase readonly SemaphoreSlim singleWriterSemaphore; readonly TimeSpan limit; + // Whether blob files are deleted on close + private readonly bool deleteOnClose; + static long sequenceNumber; struct ReadWriteRequestInfo @@ -63,6 +66,48 @@ struct RemoveRequestInfo const long MAX_PAGEBLOB_SIZE = 512L * 1024 * 1024 * 1024; // set this at 512 GB for now TODO consider implications + /// + /// Constructs a new AzureStorageDevice instance, backed by Azure Page Blobs + /// + /// The connection string to use when estblishing connection to Azure Blobs + /// Name of the Azure Blob container to use. If there does not exist a container with the supplied name, one is created + /// Directory within blob container to use. + /// A descriptive name that will be the prefix of all blobs created with this device + /// Blob manager instance + /// Whether we use leases + /// + /// True if the program should delete all blobs created on call to Close. False otherwise. + /// The container is not deleted even if it was created in this constructor + /// + /// The maximum number of bytes this storage device can accommondate, or CAPACITY_UNSPECIFIED if there is no such limit + public AzureStorageDevice(string connectionString, string containerName, string directoryName, string blobName, BlobManager blobManager = null, bool underLease = false, bool deleteOnClose = false, long capacity = Devices.CAPACITY_UNSPECIFIED) + : base($"{connectionString}/{containerName}/{directoryName}/{blobName}", PAGE_BLOB_SECTOR_SIZE, capacity) + { + var pageBlobAccount = BlobUtilsV12.GetServiceClients(connectionString); + var pageBlobContainer = BlobUtilsV12.GetContainerClients(pageBlobAccount, containerName); + if (!pageBlobContainer.WithRetries.Exists()) + pageBlobContainer.WithRetries.CreateIfNotExists(); + var pageBlobDirectory = new BlobUtilsV12.BlobDirectory(pageBlobContainer, directoryName); + + this.deleteOnClose = deleteOnClose; + this.blobs = new ConcurrentDictionary(); + this.pendingReadWriteOperations = new ConcurrentDictionary(); + this.pendingRemoveOperations = new ConcurrentDictionary(); + this.pageBlobDirectory = pageBlobDirectory; + this.blobName = blobName; + + this.BlobManager = blobManager ?? new BlobManager(blobName, pageBlobDirectory, underLease, null, null, LogLevel.Information, null); + + this.PartitionErrorHandler = BlobManager.PartitionErrorHandler; + this.PartitionErrorHandler?.Token.Register(this.CancelAllRequests); + this.underLease = underLease; + this.hangCheckTimer = new Timer(this.DetectHangs, null, 0, 20000); + this.singleWriterSemaphore = underLease ? new SemaphoreSlim(1) : null; + this.limit = TimeSpan.FromSeconds(90); + + StartAsync().Wait(); + } + /// /// Constructs a new AzureStorageDevice instance, backed by Azure Page Blobs /// @@ -70,8 +115,8 @@ struct RemoveRequestInfo /// the directory containing the page blobs /// the blob manager handling the leases /// whether this device needs to be protected by the lease - public AzureStorageDevice(string blobName, BlobUtilsV12.BlobDirectory pageBlobDirectory, BlobManager blobManager = null, bool underLease = false) - : base($"{pageBlobDirectory}/{blobName}", PAGE_BLOB_SECTOR_SIZE, Devices.CAPACITY_UNSPECIFIED) + internal AzureStorageDevice(string blobName, BlobUtilsV12.BlobDirectory pageBlobDirectory, BlobManager blobManager = null, bool underLease = false) + : base($"{pageBlobDirectory}/{blobName}", PAGE_BLOB_SECTOR_SIZE, Devices.CAPACITY_UNSPECIFIED) { this.blobs = new ConcurrentDictionary(); this.pendingReadWriteOperations = new ConcurrentDictionary(); @@ -85,6 +130,8 @@ public AzureStorageDevice(string blobName, BlobUtilsV12.BlobDirectory pageBlobDi this.hangCheckTimer = new Timer(this.DetectHangs, null, 0, 20000); this.singleWriterSemaphore = underLease ? new SemaphoreSlim(1) : null; this.limit = TimeSpan.FromSeconds(90); + + StartAsync().Wait(); } /// @@ -252,8 +299,26 @@ public override void Dispose() { this.hangCheckTimer.Dispose(); this.singleWriterSemaphore?.Dispose(); + + // Unlike in LocalStorageDevice, we explicitly remove all page blobs if the deleteOnClose flag is set, instead of relying on the operating system + // to delete files after the end of our process. This leads to potential problems if multiple instances are sharing the same underlying page blobs. + // Since this flag is only used for testing, it is probably fine. + if (deleteOnClose) + PurgeAll(); } + /// + /// Purge all blobs related to this device. Do not use if + /// multiple instances are sharing the same underlying page blobs. + /// + public void PurgeAll() + { + foreach (var entry in blobs) + { + entry.Value.PageBlob.Default?.Delete(); + } + } + /// /// /// diff --git a/cs/src/devices/AzureStorageDevice/BlobEntry.cs b/cs/src/devices/AzureStorageDevice/BlobEntry.cs index 8e404b658..1fcce8d45 100644 --- a/cs/src/devices/AzureStorageDevice/BlobEntry.cs +++ b/cs/src/devices/AzureStorageDevice/BlobEntry.cs @@ -39,7 +39,7 @@ public BlobEntry(BlobUtilsV12.PageBlobClients pageBlob, Azure.ETag eTag, AzureSt } /// - /// Creates a new BlobEntry to represent a page blob that will be created by . + /// Creates a new BlobEntry to represent a page blob that will be created by . /// public BlobEntry(AzureStorageDevice azureStorageDevice) { diff --git a/cs/src/devices/AzureStorageDevice/BlobManager.cs b/cs/src/devices/AzureStorageDevice/BlobManager.cs index f43218795..e83e7ecd5 100644 --- a/cs/src/devices/AzureStorageDevice/BlobManager.cs +++ b/cs/src/devices/AzureStorageDevice/BlobManager.cs @@ -81,7 +81,9 @@ internal BlobManager( this.UseLocalFiles = false; this.TraceHelper = new FasterTraceHelper(logger, logLevelLimit, performanceLogger); this.PartitionErrorHandler = errorHandler; - this.shutDownOrTermination = CancellationTokenSource.CreateLinkedTokenSource(errorHandler.Token); + this.shutDownOrTermination = errorHandler == null ? + new CancellationTokenSource() : + CancellationTokenSource.CreateLinkedTokenSource(errorHandler.Token); } // For testing and debugging with local files diff --git a/cs/src/devices/AzureStorageDevice/FASTER.devices.AzureStorageDevice.csproj b/cs/src/devices/AzureStorageDevice/FASTER.devices.AzureStorageDevice.csproj index da9ff812e..e92cb0a5e 100644 --- a/cs/src/devices/AzureStorageDevice/FASTER.devices.AzureStorageDevice.csproj +++ b/cs/src/devices/AzureStorageDevice/FASTER.devices.AzureStorageDevice.csproj @@ -35,8 +35,12 @@ true bin\$(Platform)\Release\ - - + + + + + + diff --git a/cs/src/devices/AzureStorageDevice/StorageOperations.cs b/cs/src/devices/AzureStorageDevice/StorageOperations.cs index 8238e27a1..9052e21b4 100644 --- a/cs/src/devices/AzureStorageDevice/StorageOperations.cs +++ b/cs/src/devices/AzureStorageDevice/StorageOperations.cs @@ -51,7 +51,7 @@ public async Task PerformWithRetriesAsync( await this.ConfirmLeaseIsGoodForAWhileAsync(); } - this.PartitionErrorHandler.Token.ThrowIfCancellationRequested(); + this.PartitionErrorHandler?.Token.ThrowIfCancellationRequested(); this.StorageTracer?.FasterStorageProgress($"storage operation {name} ({intent}) started attempt {numAttempts}; target={target} {data}"); diff --git a/cs/test/DeviceFasterLogTests.cs b/cs/test/DeviceFasterLogTests.cs index 4e9c55531..4fbef2715 100644 --- a/cs/test/DeviceFasterLogTests.cs +++ b/cs/test/DeviceFasterLogTests.cs @@ -8,8 +8,6 @@ using FASTER.core; using FASTER.devices; using NUnit.Framework; -using Microsoft.Azure.Storage.Blob; -using Microsoft.Azure.Storage; namespace FASTER.test { @@ -25,7 +23,7 @@ internal class DeviceFasterLogTests [Category("FasterLog")] public async ValueTask PageBlobFasterLogTest1([Values] LogChecksumType logChecksum, [Values]FasterLogTestBase.IteratorType iteratorType) { - TestUtils.IgnoreIfNotRunningAzureTests(); + //TestUtils.IgnoreIfNotRunningAzureTests(); var device = new AzureStorageDevice(TestUtils.AzureEmulatedStorageString, $"{TestUtils.AzureTestContainer}", TestUtils.AzureTestDirectory, "fasterlog.log", deleteOnClose: true); var checkpointManager = new DeviceLogCommitCheckpointManager( new AzureStorageNamedDeviceFactory(TestUtils.AzureEmulatedStorageString), @@ -42,15 +40,7 @@ public async ValueTask PageBlobFasterLogTestWithLease([Values] LogChecksumType l { // Set up the blob manager so can set lease to it TestUtils.IgnoreIfNotRunningAzureTests(); - CloudStorageAccount storageAccount = CloudStorageAccount.DevelopmentStorageAccount; - var cloudBlobClient = storageAccount.CreateCloudBlobClient(); - CloudBlobContainer blobContainer = cloudBlobClient.GetContainerReference("test-container"); - blobContainer.CreateIfNotExists(); - var mycloudBlobDir = blobContainer.GetDirectoryReference(@"BlobManager/MyLeaseTest1"); - - var blobMgr = new DefaultBlobManager(true, mycloudBlobDir); - var device = new AzureStorageDevice(TestUtils.AzureEmulatedStorageString, $"{TestUtils.AzureTestContainer}", TestUtils.AzureTestDirectory, "fasterlogLease.log", deleteOnClose: true, underLease: true, blobManager: blobMgr); - + var device = new AzureStorageDevice(TestUtils.AzureEmulatedStorageString, $"{TestUtils.AzureTestContainer}", TestUtils.AzureTestDirectory, "fasterlogLease.log", deleteOnClose: true, underLease: true, blobManager: null); var checkpointManager = new DeviceLogCommitCheckpointManager( new AzureStorageNamedDeviceFactory(TestUtils.AzureEmulatedStorageString), new DefaultCheckpointNamingScheme($"{TestUtils.AzureTestContainer}/{TestUtils.AzureTestDirectory}")); @@ -58,7 +48,6 @@ public async ValueTask PageBlobFasterLogTestWithLease([Values] LogChecksumType l device.Dispose(); checkpointManager.PurgeAll(); checkpointManager.Dispose(); - blobContainer.Delete(); } diff --git a/cs/test/NameValidator.cs b/cs/test/NameValidator.cs new file mode 100644 index 000000000..04df4c25c --- /dev/null +++ b/cs/test/NameValidator.cs @@ -0,0 +1,47 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT license. + +using System; +using System.Globalization; +using System.Text.RegularExpressions; + +namespace FASTER.test +{ + static class NameValidator + { + const int ContainerShareQueueTableMinLength = 3; + const int ContainerShareQueueTableMaxLength = 63; + static readonly RegexOptions RegexOptions = RegexOptions.Singleline | RegexOptions.ExplicitCapture | RegexOptions.CultureInvariant; + static readonly Regex ShareContainerQueueRegex = new("^[a-z0-9]+(-[a-z0-9]+)*$", RegexOptions); + + /// + /// Checks if a container name is valid. + /// + /// A string representing the container name to validate. + public static void ValidateContainerName(string containerName) + { + if (!("$root".Equals(containerName, StringComparison.Ordinal) || "$logs".Equals(containerName, StringComparison.Ordinal))) + { + ValidateShareContainerQueueHelper(containerName, "Container"); + } + } + + static void ValidateShareContainerQueueHelper(string resourceName, string resourceType) + { + if (string.IsNullOrWhiteSpace(resourceName)) + { + throw new ArgumentException(string.Format(CultureInfo.InvariantCulture, "ResourceNameEmpty", resourceType)); + } + + if (resourceName.Length < ContainerShareQueueTableMinLength || resourceName.Length > ContainerShareQueueTableMaxLength) + { + throw new ArgumentException(string.Format(CultureInfo.InvariantCulture, "InvalidResourceNameLength", resourceType, ContainerShareQueueTableMinLength, ContainerShareQueueTableMaxLength)); + } + + if (!ShareContainerQueueRegex.IsMatch(resourceName)) + { + throw new ArgumentException(string.Format(CultureInfo.InvariantCulture, "InvalidResourceName", resourceType)); + } + } + } +} diff --git a/cs/test/TestUtils.cs b/cs/test/TestUtils.cs index 2b6b3cfbf..ca5873255 100644 --- a/cs/test/TestUtils.cs +++ b/cs/test/TestUtils.cs @@ -9,10 +9,12 @@ using System.Threading; using System.Runtime.InteropServices; using System.Threading.Tasks; +using System.Globalization; +using System.Text.RegularExpressions; namespace FASTER.test { - internal static class TestUtils + internal static partial class TestUtils { // Various categories used to group tests internal const string SmokeTestCategory = "Smoke"; @@ -165,11 +167,11 @@ internal static string AzureTestContainer get { var container = ConvertedClassName(forAzure: true).Replace('.', '-').ToLower(); - Microsoft.Azure.Storage.NameValidator.ValidateContainerName(container); + NameValidator.ValidateContainerName(container); return container; } } - + internal static string AzureTestDirectory => TestContext.CurrentContext.Test.MethodName; internal const string AzureEmulatedStorageString = "UseDevelopmentStorage=true;"; From 96e39b3e0cea2cac62306d9e1cb65599e5265aca Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Sat, 4 Feb 2023 19:40:47 -0800 Subject: [PATCH 05/29] Updates --- .../AzureStorageDevice/AzureStorageDevice.cs | 17 +- .../AzureStorageNamedDeviceFactory.cs | 14 +- .../devices/AzureStorageDevice/BlobManager.cs | 40 ++--- .../AzureStorageDevice/BlobUtilsV12.cs | 10 ++ ...rrorHandler.cs => IStorageErrorHandler.cs} | 10 +- .../AzureStorageDevice/StorageErrorHandler.cs | 155 ++++++++++++++++++ .../AzureStorageDevice/StorageOperations.cs | 12 +- .../AzureStorageDevice/TrackedThreads.cs | 49 ++++++ 8 files changed, 262 insertions(+), 45 deletions(-) rename cs/src/devices/AzureStorageDevice/{IPartitionErrorHandler.cs => IStorageErrorHandler.cs} (85%) create mode 100644 cs/src/devices/AzureStorageDevice/StorageErrorHandler.cs create mode 100644 cs/src/devices/AzureStorageDevice/TrackedThreads.cs diff --git a/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs b/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs index 187943321..336496c48 100644 --- a/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs +++ b/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs @@ -52,9 +52,9 @@ struct RemoveRequestInfo public DateTime TimeStamp; } - public SemaphoreSlim SingleWriterSemaphore => this.singleWriterSemaphore; + SemaphoreSlim SingleWriterSemaphore => this.singleWriterSemaphore; - internal IPartitionErrorHandler PartitionErrorHandler { get; private set; } + internal IStorageErrorHandler PartitionErrorHandler { get; private set; } // Azure Page Blobs have a fixed sector size of 512 bytes. const uint PAGE_BLOB_SECTOR_SIZE = 512; @@ -98,7 +98,7 @@ public AzureStorageDevice(string connectionString, string containerName, string this.BlobManager = blobManager ?? new BlobManager(blobName, pageBlobDirectory, underLease, null, null, LogLevel.Information, null); - this.PartitionErrorHandler = BlobManager.PartitionErrorHandler; + this.PartitionErrorHandler = BlobManager.StorageErrorHandler; this.PartitionErrorHandler?.Token.Register(this.CancelAllRequests); this.underLease = underLease; this.hangCheckTimer = new Timer(this.DetectHangs, null, 0, 20000); @@ -123,9 +123,10 @@ internal AzureStorageDevice(string blobName, BlobUtilsV12.BlobDirectory pageBlob this.pendingRemoveOperations = new ConcurrentDictionary(); this.pageBlobDirectory = pageBlobDirectory; this.blobName = blobName; - this.PartitionErrorHandler = blobManager.PartitionErrorHandler; - this.PartitionErrorHandler.Token.Register(this.CancelAllRequests); this.BlobManager = blobManager ?? new BlobManager(blobName, pageBlobDirectory, underLease, null, null, LogLevel.Information, null); + + this.PartitionErrorHandler = BlobManager.StorageErrorHandler; + this.PartitionErrorHandler.Token.Register(this.CancelAllRequests); this.underLease = underLease; this.hangCheckTimer = new Timer(this.DetectHangs, null, 0, 20000); this.singleWriterSemaphore = underLease ? new SemaphoreSlim(1) : null; @@ -140,7 +141,7 @@ public override string ToString() return $"AzureStorageDevice {this.pageBlobDirectory}{this.blobName}"; } - public async Task StartAsync() + async Task StartAsync() { try { @@ -249,7 +250,7 @@ internal void DetectHangs(object _) { if (kvp.Value.TimeStamp < threshold) { - this.BlobManager.PartitionErrorHandler.HandleError("DetectHangs", $"storage operation id={kvp.Key} has exceeded the time limit {this.limit}", null, true, false); + this.BlobManager.StorageErrorHandler.HandleError("DetectHangs", $"storage operation id={kvp.Key} has exceeded the time limit {this.limit}", null, true, false); return; } } @@ -257,7 +258,7 @@ internal void DetectHangs(object _) { if (kvp.Value.TimeStamp < threshold) { - this.BlobManager.PartitionErrorHandler.HandleError("DetectHangs", $"storage operation id={kvp.Key} has exceeded the time limit {this.limit}", null, true, false); + this.BlobManager.StorageErrorHandler.HandleError("DetectHangs", $"storage operation id={kvp.Key} has exceeded the time limit {this.limit}", null, true, false); return; } } diff --git a/cs/src/devices/AzureStorageDevice/AzureStorageNamedDeviceFactory.cs b/cs/src/devices/AzureStorageDevice/AzureStorageNamedDeviceFactory.cs index 5d8107ec7..e28a048f0 100644 --- a/cs/src/devices/AzureStorageDevice/AzureStorageNamedDeviceFactory.cs +++ b/cs/src/devices/AzureStorageDevice/AzureStorageNamedDeviceFactory.cs @@ -44,7 +44,8 @@ async Task InitializeAsync(string baseName) var dirName = string.Join("/", path.Skip(1)); this.pageBlobContainer = BlobUtilsV12.GetContainerClients(this.pageBlobAccount, containerName); - await this.pageBlobContainer.WithRetries.CreateIfNotExistsAsync(); + if (!await this.pageBlobContainer.WithRetries.ExistsAsync()) + await this.pageBlobContainer.WithRetries.CreateIfNotExistsAsync(); pageBlobDirectory = new BlobUtilsV12.BlobDirectory(pageBlobContainer, dirName); } @@ -57,14 +58,17 @@ public void Delete(FileDescriptor fileInfo) if (fileInfo.fileName != null) { - // We only delete shard 0 - dir.GetPageBlobClient(fileInfo.fileName + ".0").Default.DeleteIfExists(); + // We delete all files with fileName prefix, since shards have extensions as .0, .1, etc. + foreach (var blob in dir.GetBlobsAsync(fileInfo.fileName, default).GetAwaiter().GetResult()) + { + BlobUtilsV12.ForceDeleteAsync(pageBlobContainer.Default, blob).GetAwaiter().GetResult(); + } } else { - foreach (var blob in dir.Client.WithRetries.GetBlobs()) + foreach (var blob in dir.GetBlobsAsync(default).GetAwaiter().GetResult()) { - dir.GetPageBlobClient(blob.Name).Default.DeleteIfExists(); + BlobUtilsV12.ForceDeleteAsync(pageBlobContainer.Default, blob).GetAwaiter().GetResult(); } } } diff --git a/cs/src/devices/AzureStorageDevice/BlobManager.cs b/cs/src/devices/AzureStorageDevice/BlobManager.cs index e83e7ecd5..5368fdd0e 100644 --- a/cs/src/devices/AzureStorageDevice/BlobManager.cs +++ b/cs/src/devices/AzureStorageDevice/BlobManager.cs @@ -40,7 +40,7 @@ partial class BlobManager internal BlobUtilsV12.ContainerClients PageBlobContainer => this.pageBlobContainer; - public IPartitionErrorHandler PartitionErrorHandler { get; private set; } + public IStorageErrorHandler StorageErrorHandler { get; private set; } internal static SemaphoreSlim AsynchronousStorageReadMaxConcurrency = new SemaphoreSlim(Math.Min(100, Environment.ProcessorCount * 10)); internal static SemaphoreSlim AsynchronousStorageWriteMaxConcurrency = new SemaphoreSlim(Math.Min(50, Environment.ProcessorCount * 7)); @@ -74,13 +74,13 @@ internal BlobManager( ILogger logger, ILogger performanceLogger, LogLevel logLevelLimit, - IPartitionErrorHandler errorHandler) + IStorageErrorHandler errorHandler) { if (leaseBlobName != null) LeaseBlobName = leaseBlobName; this.pageBlobDirectory = pageBlobDirectory; this.UseLocalFiles = false; this.TraceHelper = new FasterTraceHelper(logger, logLevelLimit, performanceLogger); - this.PartitionErrorHandler = errorHandler; + this.StorageErrorHandler = errorHandler ?? new StorageErrorHandler(null, logLevelLimit, null, null); this.shutDownOrTermination = errorHandler == null ? new CancellationTokenSource() : CancellationTokenSource.CreateLinkedTokenSource(errorHandler.Token); @@ -103,11 +103,11 @@ public void HandleStorageError(string where, string message, string blobName, Ex { if (blobName == null) { - this.PartitionErrorHandler.HandleError(where, message, e, isFatal, isWarning); + this.StorageErrorHandler.HandleError(where, message, e, isFatal, isWarning); } else { - this.PartitionErrorHandler.HandleError(where, $"{message} blob={blobName}", e, isFatal, isWarning); + this.StorageErrorHandler.HandleError(where, $"{message} blob={blobName}", e, isFatal, isWarning); } } @@ -146,7 +146,7 @@ async Task AcquireOwnership() while (true) { - this.PartitionErrorHandler.Token.ThrowIfCancellationRequested(); + this.StorageErrorHandler.Token.ThrowIfCancellationRequested(); numAttempts++; try @@ -158,7 +158,7 @@ async Task AcquireOwnership() await this.leaseClient.AcquireAsync( this.LeaseDuration, null, - this.PartitionErrorHandler.Token) + this.StorageErrorHandler.Token) .ConfigureAwait(false); this.TraceHelper.LeaseAcquired(); } @@ -175,7 +175,7 @@ await this.leaseClient.AcquireAsync( // the previous owner has not released the lease yet, // try again until it becomes available, should be relatively soon // as the transport layer is supposed to shut down the previous owner when starting this - await Task.Delay(TimeSpan.FromSeconds(1), this.PartitionErrorHandler.Token); + await Task.Delay(TimeSpan.FromSeconds(1), this.StorageErrorHandler.Token); continue; } @@ -209,18 +209,18 @@ await this.PerformWithRetriesAsync( continue; } - catch (OperationCanceledException) when (this.PartitionErrorHandler.IsTerminated) + catch (OperationCanceledException) when (this.StorageErrorHandler.IsTerminated) { throw; // o.k. during termination or shutdown } - catch (Exception e) when (this.PartitionErrorHandler.IsTerminated) + catch (Exception e) when (this.StorageErrorHandler.IsTerminated) { string message = $"Lease acquisition was canceled"; this.TraceHelper.LeaseProgress(message); throw new OperationCanceledException(message, e); } catch (Exception ex) when (numAttempts < BlobManager.MaxRetries - && !this.PartitionErrorHandler.IsTerminated && BlobUtils.IsTransientStorageError(ex)) + && !this.StorageErrorHandler.IsTerminated && BlobUtils.IsTransientStorageError(ex)) { if (BlobUtils.IsTimeout(ex)) { @@ -236,7 +236,7 @@ await this.PerformWithRetriesAsync( } catch (Exception e) when (!Utils.IsFatal(e)) { - this.PartitionErrorHandler.HandleError(nameof(AcquireOwnership), "Could not acquire partition lease", e, true, false); + this.StorageErrorHandler.HandleError(nameof(AcquireOwnership), "Could not acquire partition lease", e, true, false); throw; } } @@ -254,7 +254,7 @@ public async Task RenewLeaseTask() if (!this.UseLocalFiles) { this.TraceHelper.LeaseProgress($"Renewing lease at {this.leaseTimer.Elapsed.TotalSeconds - this.LeaseDuration.TotalSeconds}s"); - await this.leaseClient.RenewAsync(null, this.PartitionErrorHandler.Token).ConfigureAwait(false); + await this.leaseClient.RenewAsync(null, this.StorageErrorHandler.Token).ConfigureAwait(false); this.TraceHelper.LeaseRenewed(this.leaseTimer.Elapsed.TotalSeconds, this.leaseTimer.Elapsed.TotalSeconds - this.LeaseDuration.TotalSeconds); if (nextLeaseTimer.ElapsedMilliseconds > 2000) @@ -265,7 +265,7 @@ public async Task RenewLeaseTask() this.leaseTimer = nextLeaseTimer; } - catch (OperationCanceledException) when (this.PartitionErrorHandler.IsTerminated) + catch (OperationCanceledException) when (this.StorageErrorHandler.IsTerminated) { throw; // o.k. during termination or shutdown } @@ -311,17 +311,17 @@ public async Task MaintenanceLoopAsync() catch (Azure.RequestFailedException ex) when (BlobUtilsV12.LeaseConflict(ex)) { // We lost the lease to someone else. Terminate ownership immediately. - this.PartitionErrorHandler.HandleError(nameof(MaintenanceLoopAsync), "Lost partition lease", ex, true, true); + this.StorageErrorHandler.HandleError(nameof(MaintenanceLoopAsync), "Lost partition lease", ex, true, true); } catch (Exception e) when (!Utils.IsFatal(e)) { - this.PartitionErrorHandler.HandleError(nameof(MaintenanceLoopAsync), "Could not maintain partition lease", e, true, false); + this.StorageErrorHandler.HandleError(nameof(MaintenanceLoopAsync), "Could not maintain partition lease", e, true, false); } this.TraceHelper.LeaseProgress("Exited lease maintenance loop"); while (this.LeaseUsers > 0 - && !this.PartitionErrorHandler.IsTerminated + && !this.StorageErrorHandler.IsTerminated && (this.leaseTimer?.Elapsed < this.LeaseDuration)) { await Task.Delay(20); // give storage accesses that are in progress and require the lease a chance to complete @@ -336,7 +336,7 @@ public async Task MaintenanceLoopAsync() { this.TraceHelper.LeaseProgress("Releasing lease"); - await this.leaseClient.ReleaseAsync(null, this.PartitionErrorHandler.Token).ConfigureAwait(false); + await this.leaseClient.ReleaseAsync(null, this.StorageErrorHandler.Token).ConfigureAwait(false); this.TraceHelper.LeaseReleased(this.leaseTimer.Elapsed.TotalSeconds); } catch (OperationCanceledException) @@ -350,11 +350,11 @@ public async Task MaintenanceLoopAsync() catch (Exception e) { // we swallow, but still report exceptions when releasing a lease - this.PartitionErrorHandler.HandleError(nameof(MaintenanceLoopAsync), "Could not release partition lease during shutdown", e, false, true); + this.StorageErrorHandler.HandleError(nameof(MaintenanceLoopAsync), "Could not release partition lease during shutdown", e, false, true); } } - this.PartitionErrorHandler.TerminateNormally(); + this.StorageErrorHandler.TerminateNormally(); this.TraceHelper.LeaseProgress("Blob manager stopped"); } diff --git a/cs/src/devices/AzureStorageDevice/BlobUtilsV12.cs b/cs/src/devices/AzureStorageDevice/BlobUtilsV12.cs index f8dad1fee..6f36e7006 100644 --- a/cs/src/devices/AzureStorageDevice/BlobUtilsV12.cs +++ b/cs/src/devices/AzureStorageDevice/BlobUtilsV12.cs @@ -155,6 +155,16 @@ public async Task> GetBlobsAsync(CancellationToken cancellationToke return list; } + public async Task> GetBlobsAsync(string filePrefix, CancellationToken cancellationToken = default) + { + var list = new List(); + await foreach (var blob in this.client.WithRetries.GetBlobsAsync(prefix: $"{this.prefix}/{filePrefix}", cancellationToken: cancellationToken)) + { + list.Add(blob.Name); + } + return list; + } + public override string ToString() { return $"{this.prefix}/"; diff --git a/cs/src/devices/AzureStorageDevice/IPartitionErrorHandler.cs b/cs/src/devices/AzureStorageDevice/IStorageErrorHandler.cs similarity index 85% rename from cs/src/devices/AzureStorageDevice/IPartitionErrorHandler.cs rename to cs/src/devices/AzureStorageDevice/IStorageErrorHandler.cs index 40560e167..cb2e73899 100644 --- a/cs/src/devices/AzureStorageDevice/IPartitionErrorHandler.cs +++ b/cs/src/devices/AzureStorageDevice/IStorageErrorHandler.cs @@ -4,18 +4,16 @@ namespace FASTER.devices { using System; - using System.Collections.Generic; - using System.Text; using System.Threading; using System.Threading.Tasks; /// - /// A handler for fatal or non-fatal errors encountered in a partition. + /// A handler for fatal or non-fatal errors encountered during storage operations. /// - public interface IPartitionErrorHandler + public interface IStorageErrorHandler { /// - /// A cancellation token that is cancelled when the partition is terminated. + /// A cancellation token that user can cancel when storage needs to terminate. /// CancellationToken Token { get; } @@ -40,7 +38,7 @@ public interface IPartitionErrorHandler Task WaitForTermination(TimeSpan timeout); /// - /// Error handling for the partition. + /// Error handling. /// /// A brief description of the component that observed the error. /// A message describing the circumstances. diff --git a/cs/src/devices/AzureStorageDevice/StorageErrorHandler.cs b/cs/src/devices/AzureStorageDevice/StorageErrorHandler.cs new file mode 100644 index 000000000..d01d95a7c --- /dev/null +++ b/cs/src/devices/AzureStorageDevice/StorageErrorHandler.cs @@ -0,0 +1,155 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT License. + +namespace FASTER.devices +{ + using System; + using System.Threading; + using System.Threading.Tasks; + using Microsoft.Extensions.Logging; + + // For indicating and initiating termination, and for tracing errors and warnings relating to a partition. + // Is is basically a wrapper around CancellationTokenSource with features for diagnostics. + class StorageErrorHandler : IStorageErrorHandler + { + readonly CancellationTokenSource cts = new CancellationTokenSource(); + readonly ILogger logger; + readonly LogLevel logLevelLimit; + readonly string account; + readonly string taskHub; + readonly TaskCompletionSource shutdownComplete; + + public event Action OnShutdown; + + public CancellationToken Token + { + get + { + try + { + return this.cts.Token; + } + catch (ObjectDisposedException) + { + return new CancellationToken(true); + } + } + } + + public bool IsTerminated => this.terminationStatus != NotTerminated; + + public bool NormalTermination => this.terminationStatus == TerminatedNormally; + + volatile int terminationStatus = NotTerminated; + const int NotTerminated = 0; + const int TerminatedWithError = 1; + const int TerminatedNormally = 2; + + public StorageErrorHandler(ILogger logger, LogLevel logLevelLimit, string storageAccountName, string taskHubName) + { + this.cts = new CancellationTokenSource(); + this.logger = logger; + this.logLevelLimit = logLevelLimit; + this.account = storageAccountName; + this.taskHub = taskHubName; + this.shutdownComplete = new TaskCompletionSource(); + } + + public void HandleError(string context, string message, Exception exception, bool terminatePartition, bool isWarning) + { + this.TraceError(isWarning, context, message, exception, terminatePartition); + + // terminate this partition in response to the error + + if (terminatePartition && this.terminationStatus == NotTerminated) + { + if (Interlocked.CompareExchange(ref this.terminationStatus, TerminatedWithError, NotTerminated) == NotTerminated) + { + this.Terminate(); + } + } + } + + public void TerminateNormally() + { + if (Interlocked.CompareExchange(ref this.terminationStatus, TerminatedNormally, NotTerminated) == NotTerminated) + { + this.Terminate(); + } + } + + void TraceError(bool isWarning, string context, string message, Exception exception, bool terminatePartition) + { + var logLevel = isWarning ? LogLevel.Warning : LogLevel.Error; + if (this.logLevelLimit <= logLevel) + { + // for warnings, do not print the entire exception message + string details = exception == null ? string.Empty : (isWarning ? $"{exception.GetType().FullName}: {exception.Message}" : exception.ToString()); + + this.logger?.Log(logLevel, "!!! {message} in {context}: {details} terminatePartition={terminatePartition}", message, context, details, terminatePartition); + } + } + + void Terminate() + { + try + { + this.logger?.LogDebug("Started PartitionCancellation"); + this.cts.Cancel(); + this.logger?.LogDebug("Completed PartitionCancellation"); + } + catch (AggregateException aggregate) + { + foreach (var e in aggregate.InnerExceptions) + { + this.HandleError("PartitionErrorHandler.Terminate", "Exception in PartitionCancellation", e, false, true); + } + } + catch (Exception e) + { + this.HandleError("PartitionErrorHandler.Terminate", "Exception in PartitionCancellation", e, false, true); + } + + // we use a dedicated shutdown thread to help debugging and to contain damage if there are hangs + Thread shutdownThread = TrackedThreads.MakeTrackedThread(Shutdown, "PartitionShutdown"); + shutdownThread.Start(); + + void Shutdown() + { + try + { + this.logger?.LogDebug("Started PartitionShutdown"); + + if (this.OnShutdown != null) + { + this.OnShutdown(); + } + + this.cts.Dispose(); + + this.logger?.LogDebug("Completed PartitionShutdown"); + } + catch (AggregateException aggregate) + { + foreach (var e in aggregate.InnerExceptions) + { + this.HandleError("PartitionErrorHandler.Shutdown", "Exception in PartitionShutdown", e, false, true); + } + } + catch (Exception e) + { + this.HandleError("PartitionErrorHandler.Shutdown", "Exception in PartitionShutdown", e, false, true); + } + + this.shutdownComplete.TrySetResult(null); + } + } + + public async Task WaitForTermination(TimeSpan timeout) + { + Task timeoutTask = Task.Delay(timeout); + var first = await Task.WhenAny(timeoutTask, this.shutdownComplete.Task); + return first == this.shutdownComplete.Task; + } + } +} diff --git a/cs/src/devices/AzureStorageDevice/StorageOperations.cs b/cs/src/devices/AzureStorageDevice/StorageOperations.cs index 9052e21b4..4f2c8ceeb 100644 --- a/cs/src/devices/AzureStorageDevice/StorageOperations.cs +++ b/cs/src/devices/AzureStorageDevice/StorageOperations.cs @@ -51,7 +51,7 @@ public async Task PerformWithRetriesAsync( await this.ConfirmLeaseIsGoodForAWhileAsync(); } - this.PartitionErrorHandler?.Token.ThrowIfCancellationRequested(); + this.StorageErrorHandler?.Token.ThrowIfCancellationRequested(); this.StorageTracer?.FasterStorageProgress($"storage operation {name} ({intent}) started attempt {numAttempts}; target={target} {data}"); @@ -71,7 +71,7 @@ public async Task PerformWithRetriesAsync( return; } - catch (Exception e) when (this.PartitionErrorHandler.IsTerminated) + catch (Exception e) when (this.StorageErrorHandler.IsTerminated) { string message = $"storage operation {name} ({intent}) was canceled"; this.StorageTracer?.FasterStorageProgress(message); @@ -101,7 +101,7 @@ public async Task PerformWithRetriesAsync( } catch (Exception exception) { - this.HandleStorageError(name, $"storage operation {name} ({intent}) failed on attempt {numAttempts}", target, exception, isCritical, this.PartitionErrorHandler.IsTerminated); + this.HandleStorageError(name, $"storage operation {name} ({intent}) failed on attempt {numAttempts}", target, exception, isCritical, this.StorageErrorHandler.IsTerminated); throw; } finally @@ -146,7 +146,7 @@ public void PerformWithRetries( this.ConfirmLeaseIsGoodForAWhile(); } - this.PartitionErrorHandler.Token.ThrowIfCancellationRequested(); + this.StorageErrorHandler.Token.ThrowIfCancellationRequested(); this.StorageTracer?.FasterStorageProgress($"storage operation {name} ({intent}) started attempt {numAttempts}; target={target} {data}"); stopwatch.Restart(); @@ -170,7 +170,7 @@ public void PerformWithRetries( return; } - catch(Exception e) when (this.PartitionErrorHandler.IsTerminated) + catch(Exception e) when (this.StorageErrorHandler.IsTerminated) { string message = $"storage operation {name} ({intent}) was canceled"; this.StorageTracer?.FasterStorageProgress(message); @@ -198,7 +198,7 @@ public void PerformWithRetries( } catch (Exception exception) { - this.HandleStorageError(name, $"storage operation {name} ({intent}) failed on attempt {numAttempts}", target, exception, isCritical, this.PartitionErrorHandler.IsTerminated); + this.HandleStorageError(name, $"storage operation {name} ({intent}) failed on attempt {numAttempts}", target, exception, isCritical, this.StorageErrorHandler.IsTerminated); throw; } finally diff --git a/cs/src/devices/AzureStorageDevice/TrackedThreads.cs b/cs/src/devices/AzureStorageDevice/TrackedThreads.cs new file mode 100644 index 000000000..4d4ef4e25 --- /dev/null +++ b/cs/src/devices/AzureStorageDevice/TrackedThreads.cs @@ -0,0 +1,49 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT License. + +namespace FASTER.devices +{ + using System; + using System.Collections.Concurrent; + using System.Linq; + using System.Threading; + + + /// + /// Functionality for tracking threads. + /// + class TrackedThreads + { + readonly static ConcurrentDictionary threads = new ConcurrentDictionary(); + + public static Thread MakeTrackedThread(Action action, string name) + { + Thread thread = null; + thread = new Thread(ThreadStart) { Name = name }; + void ThreadStart() + { + threads.TryAdd(thread.ManagedThreadId, thread); + try + { + action(); + } + finally + { + threads.TryRemove(thread.ManagedThreadId, out _); + } + } + return thread; + } + + public static int NumberThreads => threads.Count; + + public static string GetThreadNames() + { + return string.Join(",", threads + .Values + .GroupBy((thread) => thread.Name) + .OrderByDescending(group => group.Count()) + .Select(group => $"{group.Key}(x{group.Count()})")); + } + } +} From b6cd76e1ef9cb055be1a2b3cf6ebfdfca5b5ccca Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Sun, 5 Feb 2023 11:00:04 -0800 Subject: [PATCH 06/29] fixes --- .../AzureStorageDevice/FasterTraceHelper.cs | 36 +++++++++---------- cs/test/DeviceFasterLogTests.cs | 2 +- 2 files changed, 19 insertions(+), 19 deletions(-) diff --git a/cs/src/devices/AzureStorageDevice/FasterTraceHelper.cs b/cs/src/devices/AzureStorageDevice/FasterTraceHelper.cs index 28d38006b..9ddf2e53c 100644 --- a/cs/src/devices/AzureStorageDevice/FasterTraceHelper.cs +++ b/cs/src/devices/AzureStorageDevice/FasterTraceHelper.cs @@ -27,14 +27,14 @@ public void FasterStoreCreated(long inputQueuePosition, long latencyMs) { if (this.logLevelLimit <= LogLevel.Information) { - this.logger.LogInformation("Created Store, inputQueuePosition={inputQueuePosition} latencyMs={latencyMs}", inputQueuePosition, latencyMs); + this.logger?.LogInformation("Created Store, inputQueuePosition={inputQueuePosition} latencyMs={latencyMs}", inputQueuePosition, latencyMs); } } public void FasterCheckpointStarted(Guid checkpointId, string details, string storeStats, long commitLogPosition, long inputQueuePosition) { if (this.logLevelLimit <= LogLevel.Information) { - this.logger.LogInformation("Started Checkpoint {checkpointId}, details={details}, storeStats={storeStats}, commitLogPosition={commitLogPosition} inputQueuePosition={inputQueuePosition}", checkpointId, details, storeStats, commitLogPosition, inputQueuePosition); + this.logger?.LogInformation("Started Checkpoint {checkpointId}, details={details}, storeStats={storeStats}, commitLogPosition={commitLogPosition} inputQueuePosition={inputQueuePosition}", checkpointId, details, storeStats, commitLogPosition, inputQueuePosition); } } @@ -42,7 +42,7 @@ public void FasterCheckpointPersisted(Guid checkpointId, string details, long co { if (this.logLevelLimit <= LogLevel.Information) { - this.logger.LogInformation("Persisted Checkpoint {checkpointId}, details={details}, commitLogPosition={commitLogPosition} inputQueuePosition={inputQueuePosition} latencyMs={latencyMs}", checkpointId, details, commitLogPosition, inputQueuePosition, latencyMs); + this.logger?.LogInformation("Persisted Checkpoint {checkpointId}, details={details}, commitLogPosition={commitLogPosition} inputQueuePosition={inputQueuePosition} latencyMs={latencyMs}", checkpointId, details, commitLogPosition, inputQueuePosition, latencyMs); } if (latencyMs > 10000) @@ -55,7 +55,7 @@ public void FasterLogPersisted(long commitLogPosition, long numberEvents, long s { if (this.logLevelLimit <= LogLevel.Debug) { - this.logger.LogDebug("Persisted Log, commitLogPosition={commitLogPosition} numberEvents={numberEvents} sizeInBytes={sizeInBytes} latencyMs={latencyMs}", commitLogPosition, numberEvents, sizeInBytes, latencyMs); + this.logger?.LogDebug("Persisted Log, commitLogPosition={commitLogPosition} numberEvents={numberEvents} sizeInBytes={sizeInBytes} latencyMs={latencyMs}", commitLogPosition, numberEvents, sizeInBytes, latencyMs); } if (latencyMs > 10000) @@ -68,7 +68,7 @@ public void FasterPerfWarning(string details) { if (this.logLevelLimit <= LogLevel.Warning) { - this.performanceLogger.LogWarning("Performance issue detected: {details}", details); + this.performanceLogger?.LogWarning("Performance issue detected: {details}", details); } } @@ -76,7 +76,7 @@ public void FasterCheckpointLoaded(long commitLogPosition, long inputQueuePositi { if (this.logLevelLimit <= LogLevel.Information) { - this.logger.LogInformation("Loaded Checkpoint, commitLogPosition={commitLogPosition} inputQueuePosition={inputQueuePosition} storeStats={storeStats} latencyMs={latencyMs}", commitLogPosition, inputQueuePosition, storeStats, latencyMs); + this.logger?.LogInformation("Loaded Checkpoint, commitLogPosition={commitLogPosition} inputQueuePosition={inputQueuePosition} storeStats={storeStats} latencyMs={latencyMs}", commitLogPosition, inputQueuePosition, storeStats, latencyMs); } } @@ -84,7 +84,7 @@ public void FasterLogReplayed(long commitLogPosition, long inputQueuePosition, l { if (this.logLevelLimit <= LogLevel.Information) { - this.logger.LogInformation("Replayed CommitLog, commitLogPosition={commitLogPosition} inputQueuePosition={inputQueuePosition} numberEvents={numberEvents} sizeInBytes={sizeInBytes} storeStats={storeStats} latencyMs={latencyMs}", commitLogPosition, inputQueuePosition, numberEvents, sizeInBytes, storeStats, latencyMs); + this.logger?.LogInformation("Replayed CommitLog, commitLogPosition={commitLogPosition} inputQueuePosition={inputQueuePosition} numberEvents={numberEvents} sizeInBytes={sizeInBytes} storeStats={storeStats} latencyMs={latencyMs}", commitLogPosition, inputQueuePosition, numberEvents, sizeInBytes, storeStats, latencyMs); } } @@ -92,7 +92,7 @@ public void FasterStorageError(string context, Exception exception) { if (this.logLevelLimit <= LogLevel.Error) { - this.logger.LogError("!!! Faster Storage Error : {context} : {exception}", context, exception); + this.logger?.LogError("!!! Faster Storage Error : {context} : {exception}", context, exception); } } @@ -100,7 +100,7 @@ public void FasterCacheSizeMeasured(int numPages, long numRecords, long sizeInBy { if (this.logLevelLimit <= LogLevel.Information) { - this.logger.LogInformation("Measured CacheSize numPages={numPages} numRecords={numRecords} sizeInBytes={sizeInBytes} gcMemory={gcMemory} processMemory={processMemory} discrepancy={discrepancy} elapsedMs={elapsedMs:F2}", numPages, numRecords, sizeInBytes, gcMemory, processMemory, discrepancy, elapsedMs); + this.logger?.LogInformation("Measured CacheSize numPages={numPages} numRecords={numRecords} sizeInBytes={sizeInBytes} gcMemory={gcMemory} processMemory={processMemory} discrepancy={discrepancy} elapsedMs={elapsedMs:F2}", numPages, numRecords, sizeInBytes, gcMemory, processMemory, discrepancy, elapsedMs); } } @@ -108,7 +108,7 @@ public void FasterProgress(string details) { if (this.logLevelLimit <= LogLevel.Debug) { - this.logger.LogDebug("{details}", details); + this.logger?.LogDebug("{details}", details); } } @@ -116,7 +116,7 @@ public void FasterStorageProgress(string details) { if (this.logLevelLimit <= LogLevel.Trace) { - this.logger.LogTrace("{details}", details); + this.logger?.LogTrace("{details}", details); } } @@ -124,7 +124,7 @@ public void FasterAzureStorageAccessCompleted(string intent, long size, string o { if (this.logLevelLimit <= LogLevel.Debug) { - this.logger.LogDebug("storage access completed intent={intent} size={size} operation={operation} target={target} latency={latency} attempt={attempt}", + this.logger?.LogDebug("storage access completed intent={intent} size={size} operation={operation} target={target} latency={latency} attempt={attempt}", intent, size, operation, target, latency, attempt); } } @@ -135,7 +135,7 @@ public void FasterCompactionProgress(CompactionProgress progress, string operati { if (this.logLevelLimit <= LogLevel.Information) { - this.logger.LogInformation("Compaction {progress} operation={operation} begin={begin} safeReadOnly={safeReadOnly} tail={tail} minimalSize={minimalSize} compactionAreaSize={compactionAreaSize} elapsedMs={elapsedMs}", progress, operation, begin, safeReadOnly, tail, minimalSize, compactionAreaSize, elapsedMs); + this.logger?.LogInformation("Compaction {progress} operation={operation} begin={begin} safeReadOnly={safeReadOnly} tail={tail} minimalSize={minimalSize} compactionAreaSize={compactionAreaSize} elapsedMs={elapsedMs}", progress, operation, begin, safeReadOnly, tail, minimalSize, compactionAreaSize, elapsedMs); } } @@ -145,7 +145,7 @@ public void LeaseAcquired() { if (this.logLevelLimit <= LogLevel.Information) { - this.logger.LogInformation("PartitionLease acquired"); + this.logger?.LogInformation("PartitionLease acquired"); } } @@ -153,7 +153,7 @@ public void LeaseRenewed(double elapsedSeconds, double timing) { if (this.logLevelLimit <= LogLevel.Debug) { - this.logger.LogDebug("PartitionLease renewed after {elapsedSeconds:F2}s timing={timing:F2}s", elapsedSeconds, timing); + this.logger?.LogDebug("PartitionLease renewed after {elapsedSeconds:F2}s timing={timing:F2}s", elapsedSeconds, timing); } } @@ -161,7 +161,7 @@ public void LeaseReleased(double elapsedSeconds) { if (this.logLevelLimit <= LogLevel.Information) { - this.logger.LogInformation("PartitionLease released after {elapsedSeconds:F2}s", elapsedSeconds); + this.logger?.LogInformation("PartitionLease released after {elapsedSeconds:F2}s", elapsedSeconds); } } @@ -169,7 +169,7 @@ public void LeaseLost(double elapsedSeconds, string operation) { if (this.logLevelLimit <= LogLevel.Warning) { - this.logger.LogWarning("PartitionLease lost after {elapsedSeconds:F2}s in {operation}", elapsedSeconds, operation); + this.logger?.LogWarning("PartitionLease lost after {elapsedSeconds:F2}s in {operation}", elapsedSeconds, operation); } } @@ -177,7 +177,7 @@ public void LeaseProgress(string operation) { if (this.logLevelLimit <= LogLevel.Debug) { - this.logger.LogDebug("PartitionLease progress: {operation}", operation); + this.logger?.LogDebug("PartitionLease progress: {operation}", operation); } } } diff --git a/cs/test/DeviceFasterLogTests.cs b/cs/test/DeviceFasterLogTests.cs index 4fbef2715..1d30c95a8 100644 --- a/cs/test/DeviceFasterLogTests.cs +++ b/cs/test/DeviceFasterLogTests.cs @@ -23,7 +23,7 @@ internal class DeviceFasterLogTests [Category("FasterLog")] public async ValueTask PageBlobFasterLogTest1([Values] LogChecksumType logChecksum, [Values]FasterLogTestBase.IteratorType iteratorType) { - //TestUtils.IgnoreIfNotRunningAzureTests(); + TestUtils.IgnoreIfNotRunningAzureTests(); var device = new AzureStorageDevice(TestUtils.AzureEmulatedStorageString, $"{TestUtils.AzureTestContainer}", TestUtils.AzureTestDirectory, "fasterlog.log", deleteOnClose: true); var checkpointManager = new DeviceLogCommitCheckpointManager( new AzureStorageNamedDeviceFactory(TestUtils.AzureEmulatedStorageString), From b3a8fd05453d2d0f50d81e5a76ef6a37cd324579 Mon Sep 17 00:00:00 2001 From: TedHartMS <15467143+TedHartMS@users.noreply.github.com> Date: Mon, 6 Feb 2023 01:37:41 -0800 Subject: [PATCH 07/29] - Add RMW/Delete %s and logfile option to MemOnlyCache - Rename TransferReadLocksFromAndMarkSourceAtomic to CopyReadLocksFromAndMarkSourceAtomic, make it preserve destination bits other than locks, and make it fail on Sealed/Invalid records - Replace HeadAddress check with Assert in CompleteTwoPhaseCopyToTail because the record cannot have been evicted if we made it that far --- cs/samples/MemOnlyCache/Program.cs | 213 ++++++++++++++---- cs/samples/MemOnlyCache/Types.cs | 42 ++++ cs/src/core/Index/Common/RecordInfo.cs | 23 +- .../FASTER/Implementation/EphemeralLocking.cs | 34 +-- .../Index/FASTER/Implementation/ReadCache.cs | 2 +- cs/src/core/Utilities/LockTable.cs | 2 +- 6 files changed, 244 insertions(+), 72 deletions(-) diff --git a/cs/samples/MemOnlyCache/Program.cs b/cs/samples/MemOnlyCache/Program.cs index 8a93c1c35..f15cec537 100644 --- a/cs/samples/MemOnlyCache/Program.cs +++ b/cs/samples/MemOnlyCache/Program.cs @@ -4,6 +4,7 @@ using FASTER.core; using System; using System.Diagnostics; +using System.IO; using System.Threading; #pragma warning disable IDE0079 // Remove unnecessary suppression @@ -38,10 +39,26 @@ class Program const string NumThreadsArg = "-t"; /// - /// Percentage of writes in incoming workload requests (remaining are reads) + /// Percentage of Reads in incoming workload requests (remaining is updates) /// - static int WritePercent = 0; - const string WritePercentArg = "-w"; + static int ReadPercent = 100; + + /// + /// Percentage of RMWs in incoming workload requests + /// + static int RmwPercent = 0; + + /// + /// Percentage of Upserts in incoming workload requests + /// + static int UpsertPercent = 0; + + /// + /// Percentage of Deletes in incoming workload requests + /// + static int DeletePercent = 0; + + const string OpPercentArg = "-op-rmud%"; /// /// Uniform random distribution (true) or Zipf distribution (false) of requests @@ -49,6 +66,12 @@ class Program static bool UseUniform = false; const string UseUniformArg = "-u"; + /// + /// If true, create a log file in the {tempdir}\MemOnlyCacheSample + /// + static bool UseLogFile = false; + const string UseLogFileArg = "-l"; + /// /// Uniform random distribution (true) or Zipf distribution (false) of requests /// @@ -62,14 +85,21 @@ class Program const string UseReadCacheArg = "--readcache"; /// - /// Skew factor (theta) of Zipf distribution + /// Percentage of Cache-miss RMWs /// - const double Theta = 0.99; + static int CacheMissRmwPercent = 0; /// - /// Whether to upsert the key on a cache miss + /// Percentage of Cache-miss Upserts /// - const bool UpsertOnCacheMiss = true; + static int CacheMissUpsertPercent = 100; + + const string CacheMissInsertPercentArg = "--cm-mu%"; + + /// + /// Skew factor (theta) of Zipf distribution + /// + const double Theta = 0.99; static FasterKV h; static CacheSizeTracker sizeTracker; @@ -93,22 +123,25 @@ private static bool Usage() Console.WriteLine($" {MaxKeySizeArg}: Max key size; we choose actual size randomly. Default = {MaxKeySize}"); Console.WriteLine($" {MaxValueSizeArg}: Max value size; we choose actual size randomly. Default = {MaxValueSize}"); Console.WriteLine($" {NumThreadsArg}: Number of threads accessing FASTER instances. Default = {NumThreads}"); - Console.WriteLine($" {WritePercentArg}: Percentage of writes in incoming workload requests (remaining are reads). Default = {WritePercent}"); + Console.WriteLine($" {OpPercentArg}: Percentage of [(r)eads,r(m)ws,(u)pserts,(d)eletes] (summing to 0 or 100) in incoming workload requests. Default = {ReadPercent},{RmwPercent},{UpsertPercent},{DeletePercent}"); Console.WriteLine($" {UseUniformArg}: Uniform random distribution (true) or Zipf distribution (false) of requests. Default = {UseUniform}"); - Console.WriteLine($" {NoReadCTTArg}: Copy Reads from Immutable region to tail of log. Default = {!UseReadCTT}"); - Console.WriteLine($" {UseReadCacheArg}: Use the ReadCache. Default = {UseReadCache}"); + Console.WriteLine($" {UseLogFileArg}: Use log file (true) instead of NullDevice (false). Default = {UseLogFile}"); + Console.WriteLine($" {NoReadCTTArg}: Whether to copy reads from the Immutable region to tail of log. Default = {!UseReadCTT}"); + Console.WriteLine($" {UseReadCacheArg}: Whether to use the ReadCache. Default = {UseReadCache}"); + Console.WriteLine($" {CacheMissInsertPercentArg}: Whether to insert the key on a cache miss, and if so, the percentage of [r(m)ws,(u)pserts] (summing to 0 or 100) to do so. Default = {CacheMissRmwPercent},{CacheMissUpsertPercent}"); Console.WriteLine($" {HelpArg1}, {HelpArg2}, {HelpArg3}: This screen."); return false; } static bool GetArgs(string[] args) { - for (var ii = 0; ii < args.Length; ++ii) + string arg = string.Empty, val = string.Empty; + try { - var arg = args[ii].ToLower(); - var val = "n/a"; - try + for (var ii = 0; ii < args.Length; ++ii) { + arg = args[ii].ToLower(); + val = "n/a"; if (IsHelpArg(arg)) return Usage(); @@ -131,6 +164,12 @@ static bool GetArgs(string[] args) continue; } + if (arg == UseLogFileArg) + { + UseLogFile = true; + continue; + } + // Args taking a value if (ii >= args.Length - 1) { @@ -153,20 +192,55 @@ static bool GetArgs(string[] args) NumThreads = int.Parse(val); continue; } - if (arg == WritePercentArg) + if (arg == OpPercentArg) { - WritePercent = int.Parse(val); + var percents = val.Split(',', StringSplitOptions.RemoveEmptyEntries | StringSplitOptions.TrimEntries); + var success = percents.Length == 4; + if (success) + { + ReadPercent = int.Parse(percents[0]); + RmwPercent = int.Parse(percents[1]); + UpsertPercent = int.Parse(percents[2]); + DeletePercent = int.Parse(percents[3]); + var total = ReadPercent + RmwPercent + UpsertPercent + DeletePercent; + success = total == 0 || total == 100; + } + if (!success) + { + Console.WriteLine($"{arg} requires 4 values summing to 0 or 100: Percentage of [(r)eads,r(m)ws,(u)pserts,(d)eletes]"); + return false; + } + continue; + } + if (arg == CacheMissInsertPercentArg) + { + var percents = val.Split(',', StringSplitOptions.RemoveEmptyEntries | StringSplitOptions.TrimEntries); + var success = percents.Length == 2; + if (success) + { + CacheMissRmwPercent = int.Parse(percents[0]); + CacheMissUpsertPercent = int.Parse(percents[1]); + var total = CacheMissRmwPercent + CacheMissUpsertPercent; + success = total == 0 || total == 100; + } + if (!success) + { + Console.WriteLine($"{arg} requires 2 values summing to 0 or 100: Percentage of [r(m)ws,(u)pserts]"); + return false; + } continue; } Console.WriteLine($"Unknown option: {arg}"); return Usage(); } - catch (Exception ex) - { - Console.WriteLine($"Error: Arg {arg}, value {val} encountered exception: {ex.Message}"); - return false; - } + + // Note: Here we could verify parameter compatibility + } + catch (Exception ex) + { + Console.WriteLine($"Error: Arg {arg}, value {val} encountered exception: {ex.Message}"); + return false; } return true; } @@ -177,12 +251,31 @@ static void Main(string[] args) if (!GetArgs(args)) return; - var log = new NullDevice(); // no storage involved + IDevice log, objectLog; + SerializerSettings serializerSettings = null; + if (UseLogFile) + { + var path = Path.GetTempPath() + "MemOnlyCacheSample\\"; + log = Devices.CreateLogDevice(path + "hlog.log"); + objectLog = Devices.CreateLogDevice(path + "hlog_obj.log"); + + serializerSettings = new SerializerSettings + { + keySerializer = () => new CacheKeySerializer(), + valueSerializer = () => new CacheValueSerializer() + }; + } + else + { + // no storage involved + log = new NullDevice(); + objectLog = log; + } // Define settings for log var logSettings = new LogSettings { - LogDevice = log, ObjectLogDevice = log, + LogDevice = log, ObjectLogDevice = objectLog, MutableFraction = 0.9, // 10% of memory log is "read-only region" ReadFlags = UseReadCTT ? ReadFlags.CopyReadsToTail : ReadFlags.None, // reads in read-only region are copied to tail PageSizeBits = 14, // Each page is sized at 2^14 bytes @@ -199,7 +292,7 @@ static void Main(string[] args) // Set hash table size targeting 1 record per bucket var numBucketBits = (int)Math.Ceiling(Math.Log2(numRecords)); - h = new FasterKV(1L << numBucketBits, logSettings, comparer: new CacheKey()); + h = new FasterKV(1L << numBucketBits, logSettings, serializerSettings: serializerSettings, comparer: new CacheKey()); sizeTracker = new CacheSizeTracker(h, targetSize); // Initially populate store @@ -209,6 +302,8 @@ static void Main(string[] args) ContinuousRandomWorkload(); h.Dispose(); + log.Dispose(); + objectLog.Dispose(); Console.WriteLine("Press to end"); Console.ReadLine(); @@ -289,8 +384,8 @@ private static void RandomWorkload(int threadid) using var session = h.For(new CacheFunctions(sizeTracker)).NewSession(); - var rnd = new Random(threadid); - var zipf = new ZipfGenerator(rnd, DbSize, Theta); + var rng = new Random(threadid); + var zipf = new ZipfGenerator(rng, DbSize, Theta); CacheValue output = default; int localStatusFound = 0, localStatusNotFound = 0; @@ -305,40 +400,76 @@ private static void RandomWorkload(int threadid) Interlocked.Add(ref totalReads, 256); localStatusFound = localStatusNotFound = 0; } - int op = WritePercent == 0 ? 0 : rnd.Next(100); - long k = UseUniform ? rnd.Next(DbSize) : zipf.Next(); - var key = new CacheKey(k, 1 + rnd.Next(MaxKeySize - 1)); + var wantValue = RmwPercent + UpsertPercent > 0; - if (op < WritePercent) - { - var value = new CacheValue(1 + rnd.Next(MaxValueSize - 1), (byte)key.key); - session.Upsert(ref key, ref value); - } - else + int op = ReadPercent < 100 ? rng.Next(100) : 99; // rng.Next() is not inclusive of the upper bound + long k = UseUniform ? rng.Next(DbSize) : zipf.Next(); + + var key = new CacheKey(k, 1 + rng.Next(MaxKeySize - 1)); + + CacheValue createValue() => new CacheValue(1 + rng.Next(MaxValueSize - 1), (byte)key.key); + CacheValue value = wantValue ? createValue() : null; + + if (op < ReadPercent) { var status = session.Read(ref key, ref output); - + if (status.IsPending) + (status, output) = GetSinglePendingResult(session); if (!status.Found) { if (status.IsFaulted) - throw new Exception("Error!"); + throw new Exception("Unexpected Error!"); localStatusNotFound++; - if (UpsertOnCacheMiss) + if (CacheMissRmwPercent + CacheMissUpsertPercent > 0) { - var value = new CacheValue(1 + rnd.Next(MaxValueSize - 1), (byte)key.key); - session.Upsert(ref key, ref value); + value ??= createValue(); + var which = rng.Next(100); + if (which < CacheMissRmwPercent) + { + status = session.RMW(ref key, ref value); + if (status.IsPending) + session.CompletePending(wait: true); + } + else + session.Upsert(ref key, ref value); } } else { localStatusFound++; if (output.value[0] != (byte)key.key) - throw new Exception("Read error!"); + throw new Exception("Read value error!"); } } + else if (op < ReadPercent + RmwPercent) + { + var status = session.RMW(ref key, ref value); + if (status.IsPending) + session.CompletePending(wait: true); + } + else if (op < ReadPercent + RmwPercent + UpsertPercent) + { + session.Upsert(ref key, ref value); + } + else + { + session.Delete(ref key); + } i++; } } + + internal static (Status status, CacheValue output) GetSinglePendingResult(ClientSession session) + { + session.CompletePendingWithOutputs(out var completedOutputs, wait: true); + if (!completedOutputs.Next()) + throw new Exception("Expected to read one result"); + var result = (completedOutputs.Current.Status, completedOutputs.Current.Output); + if (completedOutputs.Next()) + throw new Exception("Did not expect to read a second result"); + completedOutputs.Dispose(); + return result; + } } } diff --git a/cs/samples/MemOnlyCache/Types.cs b/cs/samples/MemOnlyCache/Types.cs index 37af11a45..46c24c681 100644 --- a/cs/samples/MemOnlyCache/Types.cs +++ b/cs/samples/MemOnlyCache/Types.cs @@ -25,6 +25,26 @@ public CacheKey(long key, int extraSize = 0) public bool Equals(ref CacheKey k1, ref CacheKey k2) => k1.key == k2.key; public int GetSize => sizeof(long) + extra.Length + 48; // heap size incl. ~48 bytes ref/array overheads + + public override string ToString() => $"key = {key}, len = {extra.Length}"; + } + + public class CacheKeySerializer : BinaryObjectSerializer + { + public override void Deserialize(out CacheKey obj) + { + obj = new CacheKey(); + obj.key = reader.ReadInt64(); + int size = reader.ReadInt32(); + obj.extra = reader.ReadBytes(size); + } + + public override void Serialize(ref CacheKey obj) + { + writer.Write(obj.key); + writer.Write(obj.extra.Length); + writer.Write(obj.extra); + } } public sealed class CacheValue @@ -37,7 +57,29 @@ public CacheValue(int size, byte firstByte) value[0] = firstByte; } + public CacheValue(byte[] serializedValue) + { + value = serializedValue; + } + public int GetSize => value.Length + 48; // heap size for byte array incl. ~48 bytes ref/array overheads + + public override string ToString() => $"value[0] = {value[0]}, len = {value.Length}"; + } + + public class CacheValueSerializer : BinaryObjectSerializer + { + public override void Deserialize(out CacheValue obj) + { + int size = reader.ReadInt32(); + obj = new CacheValue(reader.ReadBytes(size)); + } + + public override void Serialize(ref CacheValue obj) + { + writer.Write(obj.value.Length); + writer.Write(obj.value); + } } /// diff --git a/cs/src/core/Index/Common/RecordInfo.cs b/cs/src/core/Index/Common/RecordInfo.cs index 158f1b052..def5c5263 100644 --- a/cs/src/core/Index/Common/RecordInfo.cs +++ b/cs/src/core/Index/Common/RecordInfo.cs @@ -95,6 +95,8 @@ public static void WriteInfo(ref RecordInfo info, bool inNewVersion, bool tombst private static bool IsIntermediateOrInvalidWord(long word) => (word & (kTentativeBitMask | kSealedBitMask | kValidBitMask)) != kValidBitMask; + private static bool IsInvalidOrSealedWord(long word) => (word & (kSealedBitMask | kValidBitMask)) != kValidBitMask; + public bool TryLock(LockType lockType) { if (lockType == LockType.Shared) @@ -269,14 +271,20 @@ public void TransferLocksFrom(ref RecordInfo source) } [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool TransferReadLocksFromAndMarkSourceAtomic(ref RecordInfo source, bool allowXLock, bool seal, bool removeEphemeralLock) + public bool CopyReadLocksFromAndMarkSourceAtomic(ref RecordInfo source, bool allowXLock, bool seal, bool removeEphemeralLock) { - // This is called when tranferring read locks from the read cache or Lock Table to a tentative log record. + // This is called when transferring read locks from the read cache or Lock Table to a tentative log record. This does not remove + // locks from the source record because if they exist it means other threads have the record locked and must be allowed to + // unlock it (and observe the 'false' return of that unlock due to the Seal/Invalid, and then go chase the record where it is now). Debug.Assert(this.Tentative, "Must only transfer locks to a tentative recordInfo"); - Debug.Assert(!this.IsLockedExclusive, "Must only transfer readlocks"); + Debug.Assert((word & (kExclusiveLockBitMask | kSharedLockMaskInWord)) != kExclusiveLockBitMask, "Must only transfer readlocks"); for (; ; Thread.Yield()) { long expected_word = source.word; + + // If this is invalid or sealed, someone else won the race. + if (IsInvalidOrSealedWord(expected_word)) + return false; var new_word = expected_word; // Fail if there is an established XLock. Having both X and S locks means the other thread is still in the read-lock draining portion @@ -296,11 +304,12 @@ public bool TransferReadLocksFromAndMarkSourceAtomic(ref RecordInfo source, bool if (removeEphemeralLock) new_word -= kSharedLockIncrement; - // Update the source record; this ensures we atomically transfer the lock count while setting the mark bit. + // Update the source record; this ensures we atomically copy the lock count while setting the mark bit. // If that succeeds, then we update our own word. if (expected_word == Interlocked.CompareExchange(ref source.word, new_word, expected_word)) { - this.word = (new_word & ~kSealedBitMask) | kValidBitMask; + this.word &= ~(kExclusiveLockBitMask | kSharedLockMaskInWord); + this.word |= new_word & (kExclusiveLockBitMask | kSharedLockMaskInWord); return true; } } @@ -352,6 +361,8 @@ public bool Tentative [MethodImpl(MethodImplOptions.AggressiveInlining)] public void ClearTentativeBitAtomic() { + Debug.Assert(this.Tentative, "Should only ClearTentative a tentative record"); + // Call this when locking or splicing may be done simultaneously while (true) { @@ -421,8 +432,6 @@ public bool InNewVersion } public void SetDirtyAndModified() => word |= kDirtyBitMask | kModifiedBitMask; - public void SetModified() => word |= kModifiedBitMask; - public void ClearModified() => word &= (~kModifiedBitMask); public void SetDirty() => word |= kDirtyBitMask; public void SetTombstone() => word |= kTombstoneBitMask; public void SetValid() => word |= kValidBitMask; diff --git a/cs/src/core/Index/FASTER/Implementation/EphemeralLocking.cs b/cs/src/core/Index/FASTER/Implementation/EphemeralLocking.cs index fea4be7cb..a27d6c64b 100644 --- a/cs/src/core/Index/FASTER/Implementation/EphemeralLocking.cs +++ b/cs/src/core/Index/FASTER/Implementation/EphemeralLocking.cs @@ -126,7 +126,7 @@ private void EphemeralXUnlockAfterUpdate( { if (fasterSession.DisableEphemeralLocking) { - Debug.Assert(!stackCtx.recSrc.HasLockTableLock, "HasLockTableLock should only be true if we are doing ephemeral locking"); + Debug.Assert(!stackCtx.recSrc.HasLock, "HasLock should only be true if we are doing ephemeral locking"); return; } @@ -178,7 +178,7 @@ private void EphemeralSUnlockAfterPendingIO(Faste ref RecordInfo srcRecordInfo, ref RecordInfo newRecordInfo, out OperationStatus status) where FasterSession : IFasterSession { - // We don't check for ephem eral xlocking here; we know we had that lock, but we don't need to actually lock the new record because + // We don't check for ephemeral xlocking here; we know we had that lock, but we don't need to actually lock the new record because // we know this is the last step and we are going to unlock it immediately; it is protected until we remove the Tentative bit. if (fasterSession.IsManualLocking) @@ -251,24 +251,14 @@ private bool CompleteTwoPhaseCopyToTail(F bool success = true; if (stackCtx.recSrc.HasInMemorySrc) { - // We're copying from immutable or readcache. If the locked record has gone below HeadAddress due to the BlockAllocate, - // we must wait until the record is closed and transferred to the lock table, then transfer the locks from there. - if (stackCtx.recSrc.LogicalAddress >= stackCtx.recSrc.Log.HeadAddress) // TODO: This may not need to be checked, since we passed VerifyInMemoryAddresses - { - // Unlock the ephemeral lock here; we mark the source so we *know* we will have an invalid unlock on srcRecordInfo and would have to chase - // through InternalLock to unlock it, so we save the time by not transferring our ephemeral lock; 'Tentative' still protects the new record. - newRecordInfo.TransferReadLocksFromAndMarkSourceAtomic(ref srcRecordInfo, allowXLock: fasterSession.IsManualLocking, - seal: stackCtx.recSrc.HasMainLogSrc, removeEphemeralLock: stackCtx.recSrc.HasInMemoryLock); - } - else - { - SpinWaitUntilRecordIsClosed(ref key, stackCtx.hei.hash, stackCtx.recSrc.LogicalAddress, stackCtx.recSrc.Log); - if (!VerifyInMemoryAddresses(ref stackCtx)) - return false; - success = !LockTable.IsActive || LockTable.CompleteTwoPhaseCopyToTail(ref key, stackCtx.hei.hash, ref newRecordInfo, - allowXLock: fasterSession.IsManualLocking, removeEphemeralLock: stackCtx.recSrc.HasInMemoryLock); // we acquired the lock via HasInMemoryLock - } - stackCtx.recSrc.HasInMemoryLock = false; + stackCtx.recSrc.AssertInMemorySourceWasNotEvicted(); + + // Unlock the ephemeral lock here; we mark the source so we *know* we will have an invalid unlock on srcRecordInfo and would have to chase + // through InternalLock to unlock it, so we save the time by not transferring our ephemeral lock; 'Tentative' still protects the new record. + success = newRecordInfo.CopyReadLocksFromAndMarkSourceAtomic(ref srcRecordInfo, allowXLock: fasterSession.IsManualLocking, + seal: stackCtx.recSrc.HasMainLogSrc, removeEphemeralLock: stackCtx.recSrc.HasInMemoryLock); + if (success) + stackCtx.recSrc.HasInMemoryLock = false; } else { @@ -285,7 +275,7 @@ private bool CompleteTwoPhaseCopyToTail(F removeEphemeralLock: stackCtx.recSrc.HasLockTableLock)) && (!UseReadCache || ReadCacheCompleteTwoPhaseCopyToTail(ref key, ref stackCtx.hei, ref newRecordInfo, allowXLock: fasterSession.IsManualLocking, - removeEphemeralLock: stackCtx.recSrc.HasLockTableLock)); + removeEphemeralLock: stackCtx.recSrc.HasLockTableLock)); // HasLockTableLock because we checked HasInMemorySrc above stackCtx.recSrc.HasLockTableLock = false; } } diff --git a/cs/src/core/Index/FASTER/Implementation/ReadCache.cs b/cs/src/core/Index/FASTER/Implementation/ReadCache.cs index 05660ced2..76072b31f 100644 --- a/cs/src/core/Index/FASTER/Implementation/ReadCache.cs +++ b/cs/src/core/Index/FASTER/Implementation/ReadCache.cs @@ -283,7 +283,7 @@ private bool ReadCacheCompleteTwoPhaseCopyToTail(ref Key key, ref HashEntryInfo if (!recordInfo.Invalid && comparer.Equals(ref key, ref readcache.GetKey(physicalAddress))) { if (SpinWaitWhileTentativeAndReturnValidity(ref recordInfo)) - return newRecordInfo.TransferReadLocksFromAndMarkSourceAtomic(ref recordInfo, allowXLock, seal: false, removeEphemeralLock); + return newRecordInfo.CopyReadLocksFromAndMarkSourceAtomic(ref recordInfo, allowXLock, seal: false, removeEphemeralLock); } entry.word = recordInfo.PreviousAddress; } diff --git a/cs/src/core/Utilities/LockTable.cs b/cs/src/core/Utilities/LockTable.cs index 741fe92eb..2a3675bf1 100644 --- a/cs/src/core/Utilities/LockTable.cs +++ b/cs/src/core/Utilities/LockTable.cs @@ -385,7 +385,7 @@ internal FindEntryFunctions_CompleteTwoPhaseCopyToTail(RecordInfo toRecordInfo, public void NotFound(ref TKey key) => success = true; public void FoundEntry(ref TKey key, ref RecordInfo recordInfo) - => success = toRecordInfo.TransferReadLocksFromAndMarkSourceAtomic(ref recordInfo, allowXLock, seal: false, this.removeEphemeralLock); + => success = toRecordInfo.CopyReadLocksFromAndMarkSourceAtomic(ref recordInfo, allowXLock, seal: false, this.removeEphemeralLock); } /// From 059044d33042a3560dddc6b5ca86bb6162d9b3bf Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Mon, 6 Feb 2023 11:29:24 -0800 Subject: [PATCH 08/29] update --- .../AzureStorageNamedDeviceFactory.cs | 38 ++++++++++++++----- 1 file changed, 28 insertions(+), 10 deletions(-) diff --git a/cs/src/devices/AzureStorageDevice/AzureStorageNamedDeviceFactory.cs b/cs/src/devices/AzureStorageDevice/AzureStorageNamedDeviceFactory.cs index e28a048f0..fa28d9040 100644 --- a/cs/src/devices/AzureStorageDevice/AzureStorageNamedDeviceFactory.cs +++ b/cs/src/devices/AzureStorageDevice/AzureStorageNamedDeviceFactory.cs @@ -83,24 +83,42 @@ public IDevice Get(FileDescriptor fileInfo) public IEnumerable ListContents(string path) { var dir = pageBlobDirectory.GetSubDirectory(path); - var client = dir.Client.WithRetries; - foreach (var item in client.GetBlobsByHierarchy() - .OrderByDescending(f => dir.GetPageBlobClient(f.Blob.Name).Default.GetProperties().Value.LastModified)) + var client = pageBlobContainer.Default; + + HashSet directories = new(); + foreach (var item in client.GetBlobs(prefix: $"{dir.Prefix}/") + .OrderByDescending(f => client.GetBlobClient(f.Name).GetProperties().Value.LastModified) + ) { - if (item.IsPrefix) + // get the directory name + var name = item.Name.Substring(dir.Prefix.Length + 1); + // get substring until first slash + var slash = name.IndexOf('/'); + if (slash > 0) { - yield return new FileDescriptor + // this is a directory + var dirName = name.Substring(0, slash); + if (!directories.Contains(dirName)) { - directoryName = item.Prefix, - fileName = "" - }; + directories.Add(dirName); + + // find file name from path + var fileName = name.Substring(name.LastIndexOf('/') + 1); + + yield return new FileDescriptor + { + directoryName = dirName, + fileName = "", + }; + } } else { + // this is a file yield return new FileDescriptor { - directoryName = item.Blob.Name, - fileName = "" + directoryName = "", + fileName = name, }; } } From 1c5fcea95789716906a3fb36b0422c0ae05a2b0d Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Mon, 6 Feb 2023 13:55:22 -0800 Subject: [PATCH 09/29] updates --- cs/src/core/FasterLog/FasterLog.cs | 25 +++++++++++++------ .../AzureStorageDevice/AzureStorageDevice.cs | 13 ++++++++-- 2 files changed, 29 insertions(+), 9 deletions(-) diff --git a/cs/src/core/FasterLog/FasterLog.cs b/cs/src/core/FasterLog/FasterLog.cs index ac63bf21c..5fac78617 100644 --- a/cs/src/core/FasterLog/FasterLog.cs +++ b/cs/src/core/FasterLog/FasterLog.cs @@ -2045,8 +2045,12 @@ private void RestoreLatest(out Dictionary iterators, out byte[] co CommittedUntilAddress = long.MaxValue; beginAddress = info.BeginAddress; allocator.HeadAddress = long.MaxValue; - using var scanIterator = Scan(info.UntilAddress, long.MaxValue, recover: false); - scanIterator.ScanForwardForCommit(ref info); + try + { + using var scanIterator = Scan(info.UntilAddress, long.MaxValue, recover: false); + scanIterator.ScanForwardForCommit(ref info); + } + catch { } } // If until address is 0, that means info is still its default value and we haven't been able to recover @@ -2130,9 +2134,13 @@ private void RestoreSpecificCommit(long requestedCommitNum, out Dictionary + public override long GetFileSize(int segment) + { + if (!this.blobs.TryGetValue(segment, out BlobEntry blobEntry)) + return 0; + long size = blobEntry.PageBlob.Default.GetProperties().Value.ContentLength; + return size; + } + /// /// /// @@ -421,7 +430,7 @@ public override unsafe void ReadAsync(int segmentId, ulong sourceAddress, IntPtr { var nonLoadedBlob = this.pageBlobDirectory.GetPageBlobClient(this.GetSegmentBlobName(segmentId)); var exception = new InvalidOperationException("Attempt to read a non-loaded segment"); - this.BlobManager?.HandleStorageError(nameof(ReadAsync), exception.Message, nonLoadedBlob.Default?.Name, exception, true, false); + this.BlobManager?.HandleStorageError(nameof(ReadAsync), exception.Message, nonLoadedBlob.Default?.Name, exception, false, true); throw exception; } From 09ef381b4707e630047005bef7002ef6a93dae7a Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Mon, 6 Feb 2023 16:29:53 -0800 Subject: [PATCH 10/29] Cleanup and minor fixes --- .../AzureStorageDevice/AzureStorageDevice.cs | 19 ++- .../devices/AzureStorageDevice/BlobManager.cs | 137 ++++++++---------- .../AzureStorageDevice/BlobUtilsV12.cs | 1 - cs/test/DeviceFasterLogTests.cs | 1 - cs/test/TestUtils.cs | 4 +- 5 files changed, 78 insertions(+), 84 deletions(-) diff --git a/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs b/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs index 1293170e6..0510c9d38 100644 --- a/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs +++ b/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs @@ -30,6 +30,7 @@ public class AzureStorageDevice : StorageDeviceBase readonly Timer hangCheckTimer; readonly SemaphoreSlim singleWriterSemaphore; readonly TimeSpan limit; + readonly bool localBlobManager; // Whether blob files are deleted on close private readonly bool deleteOnClose; @@ -69,7 +70,7 @@ struct RemoveRequestInfo /// /// Constructs a new AzureStorageDevice instance, backed by Azure Page Blobs /// - /// The connection string to use when estblishing connection to Azure Blobs + /// The connection string to use when establishing connection to Azure Blobs /// Name of the Azure Blob container to use. If there does not exist a container with the supplied name, one is created /// Directory within blob container to use. /// A descriptive name that will be the prefix of all blobs created with this device @@ -79,8 +80,9 @@ struct RemoveRequestInfo /// True if the program should delete all blobs created on call to Close. False otherwise. /// The container is not deleted even if it was created in this constructor /// - /// The maximum number of bytes this storage device can accommondate, or CAPACITY_UNSPECIFIED if there is no such limit - public AzureStorageDevice(string connectionString, string containerName, string directoryName, string blobName, BlobManager blobManager = null, bool underLease = false, bool deleteOnClose = false, long capacity = Devices.CAPACITY_UNSPECIFIED) + /// The maximum number of bytes this storage device can accommodate, or CAPACITY_UNSPECIFIED if there is no such limit + /// Logger + public AzureStorageDevice(string connectionString, string containerName, string directoryName, string blobName, BlobManager blobManager = null, bool underLease = false, bool deleteOnClose = false, long capacity = Devices.CAPACITY_UNSPECIFIED, ILogger logger = null) : base($"{connectionString}/{containerName}/{directoryName}/{blobName}", PAGE_BLOB_SECTOR_SIZE, capacity) { var pageBlobAccount = BlobUtilsV12.GetServiceClients(connectionString); @@ -96,7 +98,8 @@ public AzureStorageDevice(string connectionString, string containerName, string this.pageBlobDirectory = pageBlobDirectory; this.blobName = blobName; - this.BlobManager = blobManager ?? new BlobManager(blobName, pageBlobDirectory, underLease, null, null, LogLevel.Information, null); + if (blobManager == null) localBlobManager = true; + this.BlobManager = blobManager ?? new BlobManager(logger, logger, LogLevel.Information, null, underLease, pageBlobDirectory, blobName); this.PartitionErrorHandler = BlobManager.StorageErrorHandler; this.PartitionErrorHandler?.Token.Register(this.CancelAllRequests); @@ -115,7 +118,8 @@ public AzureStorageDevice(string connectionString, string containerName, string /// the directory containing the page blobs /// the blob manager handling the leases /// whether this device needs to be protected by the lease - internal AzureStorageDevice(string blobName, BlobUtilsV12.BlobDirectory pageBlobDirectory, BlobManager blobManager = null, bool underLease = false) + /// Logger + internal AzureStorageDevice(string blobName, BlobUtilsV12.BlobDirectory pageBlobDirectory, BlobManager blobManager = null, bool underLease = false, ILogger logger = null) : base($"{pageBlobDirectory}/{blobName}", PAGE_BLOB_SECTOR_SIZE, Devices.CAPACITY_UNSPECIFIED) { this.blobs = new ConcurrentDictionary(); @@ -123,7 +127,7 @@ internal AzureStorageDevice(string blobName, BlobUtilsV12.BlobDirectory pageBlob this.pendingRemoveOperations = new ConcurrentDictionary(); this.pageBlobDirectory = pageBlobDirectory; this.blobName = blobName; - this.BlobManager = blobManager ?? new BlobManager(blobName, pageBlobDirectory, underLease, null, null, LogLevel.Information, null); + this.BlobManager = blobManager ?? new BlobManager(logger, logger, LogLevel.Information, null, underLease, pageBlobDirectory, blobName); this.PartitionErrorHandler = BlobManager.StorageErrorHandler; this.PartitionErrorHandler.Token.Register(this.CancelAllRequests); @@ -298,6 +302,9 @@ void CancelAllRequests() /// public override void Dispose() { + if (localBlobManager) + BlobManager.StopAsync().Wait(); + this.hangCheckTimer.Dispose(); this.singleWriterSemaphore?.Dispose(); diff --git a/cs/src/devices/AzureStorageDevice/BlobManager.cs b/cs/src/devices/AzureStorageDevice/BlobManager.cs index 5368fdd0e..774f2d3bb 100644 --- a/cs/src/devices/AzureStorageDevice/BlobManager.cs +++ b/cs/src/devices/AzureStorageDevice/BlobManager.cs @@ -3,7 +3,6 @@ namespace FASTER.devices { - using FASTER.core; using Microsoft.Extensions.Logging; using System; using System.Diagnostics; @@ -15,19 +14,14 @@ namespace FASTER.devices /// /// Provides management of blobs and blob names associated with a partition, and logic for partition lease maintenance and termination. /// - partial class BlobManager + public partial class BlobManager { - readonly uint partitionId; readonly CancellationTokenSource shutDownOrTermination; - readonly string taskHubPrefix; - BlobUtilsV12.ServiceClients pageBlobAccount; - BlobUtilsV12.ContainerClients pageBlobContainer; - BlobUtilsV12.BlockBlobClients eventLogCommitBlob; + BlobUtilsV12.BlockBlobClients leaseBlob; BlobLeaseClient leaseClient; - - BlobUtilsV12.BlobDirectory pageBlobDirectory; - BlobUtilsV12.BlobDirectory blockBlobPartitionDirectory; + BlobUtilsV12.BlobDirectory leaseBlobDirectory; + readonly string LeaseBlobName = "commit-lease"; readonly TimeSpan LeaseDuration = TimeSpan.FromSeconds(45); // max time the lease stays after unclean shutdown readonly TimeSpan LeaseRenewal = TimeSpan.FromSeconds(30); // how often we renew the lease @@ -36,10 +30,9 @@ partial class BlobManager internal FasterTraceHelper TraceHelper { get; private set; } internal FasterTraceHelper StorageTracer => this.TraceHelper.IsTracingAtMostDetailedLevel ? this.TraceHelper : null; - public DateTime IncarnationTimestamp { get; private set; } - - internal BlobUtilsV12.ContainerClients PageBlobContainer => this.pageBlobContainer; - + /// + /// Error handler for storage accesses + /// public IStorageErrorHandler StorageErrorHandler { get; private set; } internal static SemaphoreSlim AsynchronousStorageReadMaxConcurrency = new SemaphoreSlim(Math.Min(100, Environment.ProcessorCount * 10)); @@ -49,53 +42,61 @@ partial class BlobManager volatile Stopwatch leaseTimer; - internal const long HashTableSize = 1L << 14; // 16 k buckets, 1 MB - internal const long HashTableSizeBytes = HashTableSize * 64; + const int MaxRetries = 10; - public const int MaxRetries = 10; + readonly bool maintainLease = false; + /// + /// Get delay between retries + /// + /// + /// public static TimeSpan GetDelayBetweenRetries(int numAttempts) => TimeSpan.FromSeconds(Math.Pow(2, (numAttempts - 1))); /// /// Create a blob manager. /// - /// - /// - /// /// A logger for logging /// /// A limit on log event level emitted /// A handler for errors encountered in this partition + /// Whether lease should be maintained by blob manager + /// Lease bob is stored in this directory + /// Name of lease blob (default is commit-lease) internal BlobManager( - string leaseBlobName, - BlobUtilsV12.BlobDirectory pageBlobDirectory, - bool underLease, ILogger logger, ILogger performanceLogger, LogLevel logLevelLimit, - IStorageErrorHandler errorHandler) + IStorageErrorHandler errorHandler, + bool maintainLease = false, + BlobUtilsV12.BlobDirectory leaseBlobDirectory = default, + string leaseBlobName = null) { - if (leaseBlobName != null) LeaseBlobName = leaseBlobName; - this.pageBlobDirectory = pageBlobDirectory; - this.UseLocalFiles = false; + this.maintainLease = maintainLease; + this.leaseBlobDirectory = leaseBlobDirectory; + this.LeaseBlobName = leaseBlobName ?? LeaseBlobName; this.TraceHelper = new FasterTraceHelper(logger, logLevelLimit, performanceLogger); this.StorageErrorHandler = errorHandler ?? new StorageErrorHandler(null, logLevelLimit, null, null); this.shutDownOrTermination = errorHandler == null ? new CancellationTokenSource() : CancellationTokenSource.CreateLinkedTokenSource(errorHandler.Token); + if (maintainLease) + StartAsync().Wait(); } - // For testing and debugging with local files - bool UseLocalFiles { get; } - string LeaseBlobName = "commit-lease"; + Task LeaseMaintenanceLoopTask = Task.CompletedTask; volatile Task NextLeaseRenewalTask = Task.CompletedTask; + /// + /// Start lease maintenance loop + /// + /// public async Task StartAsync() { - this.eventLogCommitBlob = this.pageBlobDirectory.GetBlockBlobClient(LeaseBlobName); - this.leaseClient = this.eventLogCommitBlob.WithRetries.GetBlobLeaseClient(); + this.leaseBlob = this.leaseBlobDirectory.GetBlockBlobClient(LeaseBlobName); + this.leaseClient = this.leaseBlob.WithRetries.GetBlobLeaseClient(); await this.AcquireOwnership(); } @@ -153,17 +154,13 @@ async Task AcquireOwnership() { newLeaseTimer.Restart(); - if (!this.UseLocalFiles) - { - await this.leaseClient.AcquireAsync( - this.LeaseDuration, - null, - this.StorageErrorHandler.Token) - .ConfigureAwait(false); - this.TraceHelper.LeaseAcquired(); - } + await this.leaseClient.AcquireAsync( + this.LeaseDuration, + null, + this.StorageErrorHandler.Token) + .ConfigureAwait(false); + this.TraceHelper.LeaseAcquired(); - this.IncarnationTimestamp = DateTime.UtcNow; this.leaseTimer = newLeaseTimer; this.LeaseMaintenanceLoopTask = Task.Run(() => this.MaintenanceLoopAsync()); return; @@ -188,14 +185,14 @@ await this.PerformWithRetriesAsync( "CloudBlockBlob.UploadFromByteArrayAsync", "CreateCommitLog", "", - this.eventLogCommitBlob.Default.Name, + this.leaseBlob.Default.Name, 2000, true, async (numAttempts) => { try { - var client = numAttempts > 2 ? this.eventLogCommitBlob.Default : this.eventLogCommitBlob.Aggressive; + var client = numAttempts > 2 ? this.leaseBlob.Default : this.leaseBlob.Aggressive; await client.UploadAsync(new MemoryStream()); } catch (Azure.RequestFailedException ex2) when (BlobUtilsV12.LeaseConflictOrExpired(ex2)) @@ -251,16 +248,13 @@ public async Task RenewLeaseTask() var nextLeaseTimer = new System.Diagnostics.Stopwatch(); nextLeaseTimer.Start(); - if (!this.UseLocalFiles) - { - this.TraceHelper.LeaseProgress($"Renewing lease at {this.leaseTimer.Elapsed.TotalSeconds - this.LeaseDuration.TotalSeconds}s"); - await this.leaseClient.RenewAsync(null, this.StorageErrorHandler.Token).ConfigureAwait(false); - this.TraceHelper.LeaseRenewed(this.leaseTimer.Elapsed.TotalSeconds, this.leaseTimer.Elapsed.TotalSeconds - this.LeaseDuration.TotalSeconds); + this.TraceHelper.LeaseProgress($"Renewing lease at {this.leaseTimer.Elapsed.TotalSeconds - this.LeaseDuration.TotalSeconds}s"); + await this.leaseClient.RenewAsync(null, this.StorageErrorHandler.Token).ConfigureAwait(false); + this.TraceHelper.LeaseRenewed(this.leaseTimer.Elapsed.TotalSeconds, this.leaseTimer.Elapsed.TotalSeconds - this.LeaseDuration.TotalSeconds); - if (nextLeaseTimer.ElapsedMilliseconds > 2000) - { - this.TraceHelper.FasterPerfWarning($"RenewLeaseAsync took {nextLeaseTimer.Elapsed.TotalSeconds:F1}s, which is excessive; {this.leaseTimer.Elapsed.TotalSeconds - this.LeaseDuration.TotalSeconds}s past expiry"); - } + if (nextLeaseTimer.ElapsedMilliseconds > 2000) + { + this.TraceHelper.FasterPerfWarning($"RenewLeaseAsync took {nextLeaseTimer.Elapsed.TotalSeconds:F1}s, which is excessive; {this.leaseTimer.Elapsed.TotalSeconds - this.LeaseDuration.TotalSeconds}s past expiry"); } this.leaseTimer = nextLeaseTimer; @@ -330,28 +324,25 @@ public async Task MaintenanceLoopAsync() this.TraceHelper.LeaseProgress("Waited for lease users to complete"); // release the lease - if (!this.UseLocalFiles) + try { - try - { - this.TraceHelper.LeaseProgress("Releasing lease"); + this.TraceHelper.LeaseProgress("Releasing lease"); - await this.leaseClient.ReleaseAsync(null, this.StorageErrorHandler.Token).ConfigureAwait(false); - this.TraceHelper.LeaseReleased(this.leaseTimer.Elapsed.TotalSeconds); - } - catch (OperationCanceledException) - { - // it's o.k. if termination is triggered while waiting - } - catch (Azure.RequestFailedException e) when (e.InnerException != null && e.InnerException is OperationCanceledException) - { - // it's o.k. if termination is triggered while we are releasing the lease - } - catch (Exception e) - { - // we swallow, but still report exceptions when releasing a lease - this.StorageErrorHandler.HandleError(nameof(MaintenanceLoopAsync), "Could not release partition lease during shutdown", e, false, true); - } + await this.leaseClient.ReleaseAsync(null, this.StorageErrorHandler.Token).ConfigureAwait(false); + this.TraceHelper.LeaseReleased(this.leaseTimer.Elapsed.TotalSeconds); + } + catch (OperationCanceledException) + { + // it's o.k. if termination is triggered while waiting + } + catch (Azure.RequestFailedException e) when (e.InnerException != null && e.InnerException is OperationCanceledException) + { + // it's o.k. if termination is triggered while we are releasing the lease + } + catch (Exception e) + { + // we swallow, but still report exceptions when releasing a lease + this.StorageErrorHandler.HandleError(nameof(MaintenanceLoopAsync), "Could not release partition lease during shutdown", e, false, true); } this.StorageErrorHandler.TerminateNormally(); diff --git a/cs/src/devices/AzureStorageDevice/BlobUtilsV12.cs b/cs/src/devices/AzureStorageDevice/BlobUtilsV12.cs index 6f36e7006..e038c1544 100644 --- a/cs/src/devices/AzureStorageDevice/BlobUtilsV12.cs +++ b/cs/src/devices/AzureStorageDevice/BlobUtilsV12.cs @@ -174,7 +174,6 @@ public override string ToString() /// /// Forcefully deletes a blob. /// - /// The CloudBlob to delete. /// A task that completes when the operation is finished. public static async Task ForceDeleteAsync(BlobContainerClient containerClient, string blobName) { diff --git a/cs/test/DeviceFasterLogTests.cs b/cs/test/DeviceFasterLogTests.cs index 1d30c95a8..8afb1f0f9 100644 --- a/cs/test/DeviceFasterLogTests.cs +++ b/cs/test/DeviceFasterLogTests.cs @@ -38,7 +38,6 @@ public async ValueTask PageBlobFasterLogTest1([Values] LogChecksumType logChecks [Category("FasterLog")] public async ValueTask PageBlobFasterLogTestWithLease([Values] LogChecksumType logChecksum, [Values] FasterLogTestBase.IteratorType iteratorType) { - // Set up the blob manager so can set lease to it TestUtils.IgnoreIfNotRunningAzureTests(); var device = new AzureStorageDevice(TestUtils.AzureEmulatedStorageString, $"{TestUtils.AzureTestContainer}", TestUtils.AzureTestDirectory, "fasterlogLease.log", deleteOnClose: true, underLease: true, blobManager: null); var checkpointManager = new DeviceLogCommitCheckpointManager( diff --git a/cs/test/TestUtils.cs b/cs/test/TestUtils.cs index ca5873255..296be2fe6 100644 --- a/cs/test/TestUtils.cs +++ b/cs/test/TestUtils.cs @@ -9,12 +9,10 @@ using System.Threading; using System.Runtime.InteropServices; using System.Threading.Tasks; -using System.Globalization; -using System.Text.RegularExpressions; namespace FASTER.test { - internal static partial class TestUtils + internal static class TestUtils { // Various categories used to group tests internal const string SmokeTestCategory = "Smoke"; From 94c8b1e025c02e626644ed96e4d38d8c594baa4d Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Mon, 6 Feb 2023 16:51:13 -0800 Subject: [PATCH 11/29] update --- .../devices/AzureStorageDevice/AzureStorageDevice.cs | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs b/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs index 0510c9d38..be7fd248f 100644 --- a/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs +++ b/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs @@ -118,19 +118,26 @@ public AzureStorageDevice(string connectionString, string containerName, string /// the directory containing the page blobs /// the blob manager handling the leases /// whether this device needs to be protected by the lease + /// + /// True if the program should delete all blobs created on call to Close. False otherwise. + /// The container is not deleted even if it was created in this constructor + /// /// Logger - internal AzureStorageDevice(string blobName, BlobUtilsV12.BlobDirectory pageBlobDirectory, BlobManager blobManager = null, bool underLease = false, ILogger logger = null) + internal AzureStorageDevice(string blobName, BlobUtilsV12.BlobDirectory pageBlobDirectory, BlobManager blobManager = null, bool underLease = false, bool deleteOnClose = false, ILogger logger = null) : base($"{pageBlobDirectory}/{blobName}", PAGE_BLOB_SECTOR_SIZE, Devices.CAPACITY_UNSPECIFIED) { + this.deleteOnClose = deleteOnClose; this.blobs = new ConcurrentDictionary(); this.pendingReadWriteOperations = new ConcurrentDictionary(); this.pendingRemoveOperations = new ConcurrentDictionary(); this.pageBlobDirectory = pageBlobDirectory; this.blobName = blobName; + + if (blobManager == null) localBlobManager = true; this.BlobManager = blobManager ?? new BlobManager(logger, logger, LogLevel.Information, null, underLease, pageBlobDirectory, blobName); this.PartitionErrorHandler = BlobManager.StorageErrorHandler; - this.PartitionErrorHandler.Token.Register(this.CancelAllRequests); + this.PartitionErrorHandler?.Token.Register(this.CancelAllRequests); this.underLease = underLease; this.hangCheckTimer = new Timer(this.DetectHangs, null, 0, 20000); this.singleWriterSemaphore = underLease ? new SemaphoreSlim(1) : null; From b2ed43e321041e0d90025f1192cd0b42712e8747 Mon Sep 17 00:00:00 2001 From: TedHartMS <15467143+TedHartMS@users.noreply.github.com> Date: Tue, 7 Feb 2023 00:21:17 -0800 Subject: [PATCH 12/29] - Add --RunTime and -q(uiet) args to MemOnlyCache - Refactored disk-image bit cleaning to a RecordInfo method - Revised (Save|Get)AllocationForRetry to ensure no overwrite - Improve EphemeralSUnlock and merge EphemeralSUnlockAfterPendingIO into it for the InternalLock retry loop - Add more Asserts in CompleteTwoPhase* - Moved SpinWaitWhileTentative in FindInReadCache inside key-matched block - Ensure readcache bit is appropriately set on stackCtx.newLogicalAddress --- cs/samples/MemOnlyCache/Program.cs | 94 +++++++++++++------ cs/samples/MemOnlyCache/Types.cs | 4 +- cs/src/core/Allocator/ScanIteratorBase.cs | 3 + cs/src/core/Index/Common/RecordInfo.cs | 6 ++ .../FASTER/Implementation/BlockAllocate.cs | 17 +++- .../FASTER/Implementation/ContinuePending.cs | 29 +++--- .../FASTER/Implementation/EphemeralLocking.cs | 93 +++++++++--------- .../FASTER/Implementation/InternalRead.cs | 2 - .../Index/FASTER/Implementation/ReadCache.cs | 50 +++++----- .../FASTER/Implementation/RecordSource.cs | 3 +- cs/src/core/Utilities/LockTable.cs | 2 +- cs/test/BasicFASTERTests.cs | 35 +++++++ cs/test/LockTableTests.cs | 2 - 13 files changed, 212 insertions(+), 128 deletions(-) diff --git a/cs/samples/MemOnlyCache/Program.cs b/cs/samples/MemOnlyCache/Program.cs index f15cec537..8198389b2 100644 --- a/cs/samples/MemOnlyCache/Program.cs +++ b/cs/samples/MemOnlyCache/Program.cs @@ -58,7 +58,7 @@ class Program /// static int DeletePercent = 0; - const string OpPercentArg = "-op-rmud%"; + const string OpPercentArg = "--op-rmud%"; /// /// Uniform random distribution (true) or Zipf distribution (false) of requests @@ -72,6 +72,12 @@ class Program static bool UseLogFile = false; const string UseLogFileArg = "-l"; + /// + /// Quiet; do not ask for ENTER to end + /// + static bool Quiet = false; + const string QuietArg = "-q"; + /// /// Uniform random distribution (true) or Zipf distribution (false) of requests /// @@ -93,9 +99,14 @@ class Program /// Percentage of Cache-miss Upserts /// static int CacheMissUpsertPercent = 100; - const string CacheMissInsertPercentArg = "--cm-mu%"; + /// + /// Number of seconds to run + /// + static int RunTime = 0; + const string RunTimeArg = "--runtime"; + /// /// Skew factor (theta) of Zipf distribution /// @@ -115,20 +126,24 @@ class Program const string HelpArg3 = "--help"; static bool IsHelpArg(string arg) => arg == HelpArg1 || arg == HelpArg2 || arg == HelpArg3; + static bool done; + private static bool Usage() { - Console.WriteLine("Reads 'linked lists' of records for each key by backing up the previous-address chain, including showing record versions"); + Console.WriteLine("Runs a loop that illustrates an in-memory cache with dynamic size limit"); Console.WriteLine("Usage:"); - Console.WriteLine($" {DbSizeArg}: Total database size. Default = {DbSize}"); - Console.WriteLine($" {MaxKeySizeArg}: Max key size; we choose actual size randomly. Default = {MaxKeySize}"); - Console.WriteLine($" {MaxValueSizeArg}: Max value size; we choose actual size randomly. Default = {MaxValueSize}"); - Console.WriteLine($" {NumThreadsArg}: Number of threads accessing FASTER instances. Default = {NumThreads}"); - Console.WriteLine($" {OpPercentArg}: Percentage of [(r)eads,r(m)ws,(u)pserts,(d)eletes] (summing to 0 or 100) in incoming workload requests. Default = {ReadPercent},{RmwPercent},{UpsertPercent},{DeletePercent}"); - Console.WriteLine($" {UseUniformArg}: Uniform random distribution (true) or Zipf distribution (false) of requests. Default = {UseUniform}"); - Console.WriteLine($" {UseLogFileArg}: Use log file (true) instead of NullDevice (false). Default = {UseLogFile}"); - Console.WriteLine($" {NoReadCTTArg}: Whether to copy reads from the Immutable region to tail of log. Default = {!UseReadCTT}"); + Console.WriteLine($" {DbSizeArg} #: Total database size. Default = {DbSize}"); + Console.WriteLine($" {MaxKeySizeArg} #: Max key size; we choose actual size randomly. Default = {MaxKeySize}"); + Console.WriteLine($" {MaxValueSizeArg} #: Max value size; we choose actual size randomly. Default = {MaxValueSize}"); + Console.WriteLine($" {OpPercentArg} #,#,#,#: Percentage of [(r)eads,r(m)ws,(u)pserts,(d)eletes] (summing to 0 or 100) operations in incoming workload requests. Default = {ReadPercent},{RmwPercent},{UpsertPercent},{DeletePercent}"); + Console.WriteLine($" {NoReadCTTArg}: Turn off (true) or allow (false) copying of reads from the Immutable region of the log to the tail of log. Default = {!UseReadCTT}"); Console.WriteLine($" {UseReadCacheArg}: Whether to use the ReadCache. Default = {UseReadCache}"); - Console.WriteLine($" {CacheMissInsertPercentArg}: Whether to insert the key on a cache miss, and if so, the percentage of [r(m)ws,(u)pserts] (summing to 0 or 100) to do so. Default = {CacheMissRmwPercent},{CacheMissUpsertPercent}"); + Console.WriteLine($" {CacheMissInsertPercentArg} #,#: Whether to insert the key on a cache miss, and if so, the percentage of [r(m)ws,(u)pserts] (summing to 0 or 100) operations to do so. Default = {CacheMissRmwPercent},{CacheMissUpsertPercent}"); + Console.WriteLine($" {RunTimeArg} #: If nonzero, limits the run to this many seconds. Default = {RunTime}"); + Console.WriteLine($" {NumThreadsArg} #: Number of threads accessing FASTER instances. Default = {NumThreads}"); + Console.WriteLine($" {UseUniformArg}: Uniform random distribution (true) or Zipf distribution (false) of requests. Default = {UseUniform}"); + Console.WriteLine($" {UseLogFileArg}: Use log file (true; written to '{GetLogPath()}') instead of NullDevice (false). Default = {UseLogFile}"); + Console.WriteLine($" {QuietArg}: Quiet; do not ask for ENTER to end. Default = {Quiet}"); Console.WriteLine($" {HelpArg1}, {HelpArg2}, {HelpArg3}: This screen."); return false; } @@ -151,29 +166,31 @@ static bool GetArgs(string[] args) UseUniform = true; continue; } - if (arg == NoReadCTTArg) { UseReadCTT = false; continue; } - if (arg == UseReadCacheArg) { UseReadCache = true; continue; } - if (arg == UseLogFileArg) { UseLogFile = true; continue; } + if (arg == QuietArg) + { + Quiet = true; + continue; + } // Args taking a value if (ii >= args.Length - 1) { - Console.WriteLine($"Error: End of arg list encountered while processing arg {arg}; expected argument"); + Console.WriteLine($"Error: End of arg list encountered while processing arg {arg}; either an unknown flag or a missing value"); return false; } val = args[++ii]; @@ -192,6 +209,11 @@ static bool GetArgs(string[] args) NumThreads = int.Parse(val); continue; } + if (arg == RunTimeArg) + { + RunTime = int.Parse(val); + continue; + } if (arg == OpPercentArg) { var percents = val.Split(',', StringSplitOptions.RemoveEmptyEntries | StringSplitOptions.TrimEntries); @@ -245,6 +267,8 @@ static bool GetArgs(string[] args) return true; } + static string GetLogPath() => Path.GetTempPath() + "MemOnlyCacheSample\\"; + static void Main(string[] args) { // This sample shows the use of FASTER as a concurrent pure in-memory cache @@ -255,7 +279,7 @@ static void Main(string[] args) SerializerSettings serializerSettings = null; if (UseLogFile) { - var path = Path.GetTempPath() + "MemOnlyCacheSample\\"; + var path = GetLogPath(); log = Devices.CreateLogDevice(path + "hlog.log"); objectLog = Devices.CreateLogDevice(path + "hlog_obj.log"); @@ -305,8 +329,15 @@ static void Main(string[] args) log.Dispose(); objectLog.Dispose(); - Console.WriteLine("Press to end"); - Console.ReadLine(); + if (Quiet) + { + Console.WriteLine($"Completed RunTime of {RunTime} seconds; exiting"); + } + else + { + Console.WriteLine("Press to end"); + Console.ReadLine(); + } } private static void PopulateStore(int count) @@ -339,15 +370,20 @@ private static void ContinuousRandomWorkload() Stopwatch sw = new(); sw.Start(); var _lastReads = totalReads; - var _lastTime = sw.ElapsedMilliseconds; + var _lastTimeMs = sw.ElapsedMilliseconds; int count = 0; - while (true) + while (RunTime == 0 || (_lastTimeMs / 1000) < RunTime) { Thread.Sleep(1500); - var tmp = totalReads; - var tmp2 = sw.ElapsedMilliseconds; + var currentReads = totalReads; + var currentTimeMs = sw.ElapsedMilliseconds; + var currentElapsed = currentTimeMs - _lastTimeMs; + var ts = TimeSpan.FromSeconds(currentTimeMs / 1000); + var totalElapsed = ts.ToString(); - Console.WriteLine("Throughput: {0,8:0.00}K ops/sec; Hit rate: {1:N2}; Memory footprint: {2,11:N2}KB", (_lastReads - tmp) / (double)(_lastTime - tmp2), statusFound / (double)(statusFound + statusNotFound), sizeTracker.TotalSizeBytes / 1024.0); + Console.WriteLine("Throughput: {0,8:0.00}K ops/sec; Hit rate: {1:N2}; Memory footprint: {2,12:N2}KB, elapsed: {3:c}", + (currentReads - _lastReads) / (double)(currentElapsed), statusFound / (double)(statusFound + statusNotFound), + sizeTracker.TotalSizeBytes / 1024.0, totalElapsed); Interlocked.Exchange(ref statusFound, 0); Interlocked.Exchange(ref statusNotFound, 0); @@ -373,9 +409,13 @@ private static void ContinuousRandomWorkload() Console.WriteLine("**** Setting target memory: {0,11:N2}KB", targetSize / 1024.0); } - _lastReads = tmp; - _lastTime = tmp2; + _lastReads = currentReads; + _lastTimeMs = currentTimeMs; } + + done = true; + for (int i = 0; i < NumThreads; i++) + threads[i].Join(); } private static void RandomWorkload(int threadid) @@ -391,7 +431,7 @@ private static void RandomWorkload(int threadid) int localStatusFound = 0, localStatusNotFound = 0; int i = 0; - while (true) + while (!done) { if ((i % 256 == 0) && (i > 0)) { diff --git a/cs/samples/MemOnlyCache/Types.cs b/cs/samples/MemOnlyCache/Types.cs index 46c24c681..d2c4036b3 100644 --- a/cs/samples/MemOnlyCache/Types.cs +++ b/cs/samples/MemOnlyCache/Types.cs @@ -26,7 +26,7 @@ public CacheKey(long key, int extraSize = 0) public int GetSize => sizeof(long) + extra.Length + 48; // heap size incl. ~48 bytes ref/array overheads - public override string ToString() => $"key = {key}, len = {extra.Length}"; + public override string ToString() => $"key {key}, len {extra.Length}"; } public class CacheKeySerializer : BinaryObjectSerializer @@ -64,7 +64,7 @@ public CacheValue(byte[] serializedValue) public int GetSize => value.Length + 48; // heap size for byte array incl. ~48 bytes ref/array overheads - public override string ToString() => $"value[0] = {value[0]}, len = {value.Length}"; + public override string ToString() => $"value[0] {value[0]}, len {value.Length}"; } public class CacheValueSerializer : BinaryObjectSerializer diff --git a/cs/src/core/Allocator/ScanIteratorBase.cs b/cs/src/core/Allocator/ScanIteratorBase.cs index 73f4a1acd..db3b87124 100644 --- a/cs/src/core/Allocator/ScanIteratorBase.cs +++ b/cs/src/core/Allocator/ScanIteratorBase.cs @@ -243,5 +243,8 @@ public void Reset() currentAddress = -1; nextAddress = beginAddress; } + + /// + public override string ToString() => $"BA {BeginAddress}, EA {EndAddress}, CA {CurrentAddress}, NA {NextAddress}"; } } diff --git a/cs/src/core/Index/Common/RecordInfo.cs b/cs/src/core/Index/Common/RecordInfo.cs index def5c5263..e8d4de639 100644 --- a/cs/src/core/Index/Common/RecordInfo.cs +++ b/cs/src/core/Index/Common/RecordInfo.cs @@ -97,6 +97,12 @@ public static void WriteInfo(ref RecordInfo info, bool inNewVersion, bool tombst private static bool IsInvalidOrSealedWord(long word) => (word & (kSealedBitMask | kValidBitMask)) != kValidBitMask; + public void CleanDiskImage() + { + // We ignore locks and temp bits for disk images + this.word &= ~(kExclusiveLockBitMask | kSharedLockMaskInWord | kTentativeBitMask | kSealedBitMask); + } + public bool TryLock(LockType lockType) { if (lockType == LockType.Shared) diff --git a/cs/src/core/Index/FASTER/Implementation/BlockAllocate.cs b/cs/src/core/Index/FASTER/Implementation/BlockAllocate.cs index 29f3dcdd3..89862e3e1 100644 --- a/cs/src/core/Index/FASTER/Implementation/BlockAllocate.cs +++ b/cs/src/core/Index/FASTER/Implementation/BlockAllocate.cs @@ -60,7 +60,14 @@ void SaveAllocationForRetry(ref PendingContext(ref PendingContext= minSize; } } diff --git a/cs/src/core/Index/FASTER/Implementation/ContinuePending.cs b/cs/src/core/Index/FASTER/Implementation/ContinuePending.cs index d2c2fd72a..2556a91e7 100644 --- a/cs/src/core/Index/FASTER/Implementation/ContinuePending.cs +++ b/cs/src/core/Index/FASTER/Implementation/ContinuePending.cs @@ -38,11 +38,7 @@ internal OperationStatus InternalContinuePendingRead { ref RecordInfo srcRecordInfo = ref hlog.GetInfoFromBytePointer(request.record.GetValidPointer()); - // We ignore locks and temp bits for disk images - srcRecordInfo.ClearLocks(); - srcRecordInfo.Tentative = false; - srcRecordInfo.Unseal(); - // Debug.Assert(!srcRecordInfo.IsIntermediate, "Should always retrieve a non-Tentative, non-Sealed record from disk"); + srcRecordInfo.CleanDiskImage(); if (request.logicalAddress >= hlog.BeginAddress) { @@ -123,7 +119,7 @@ internal OperationStatus InternalContinuePendingRead stackCtx = new(comparer.GetHashCode64(ref key)); OperationStatus status; @@ -337,7 +329,7 @@ internal OperationStatus InternalCopyToTailForCompaction hei.Address (the .PreviousAddress and CAS comparison value). do { if (!BlockAllocateReadCache(allocatedSize, out newLogicalAddress, ref pendingContext, out _)) return OperationStatus.SUCCESS; // We don't slow down Reads to handle allocation failure in the read cache, but don't return StatusCode.CopiedRecordToReadCache newPhysicalAddress = readcache.GetPhysicalAddress(newLogicalAddress); - localLog = readcache; - readcacheNewAddressBit = Constants.kReadCacheBitMask; if (!VerifyInMemoryAddresses(ref stackCtx)) { @@ -464,7 +457,7 @@ internal OperationStatus InternalTryCopyToTail(FasterSession faste ref Key key, ref OperationStackContext stackCtx, ref RecordInfo recordInfo) where FasterSession : IFasterSession { - if (!stackCtx.recSrc.HasInMemoryLock) + if (!stackCtx.recSrc.HasLock) return; - // Updaters (Upsert, RMW, Delete) XLock records. Readers do not, including anything calling InternalTryCopyToTail. This means the record may - // be transferred from the readcache to the main log (or even to the LockTable, if the record was in the (SafeHeadAddress, ClosedUntilAddress) - // interval when a Read started). - - // If the record dived below HeadAddress, we must wait for it to enter the lock table before unlocking; InternalLock does this (and starts - // by searching the in-memory space first, which is good because the record may have been transferred). - // If RecordInfo unlock fails, the locks were transferred to another recordInfo; do InternalLock to chase the key through the full process. - OperationStatus status; - do + // This is called on recovery from Pending Read, in which case we may have a LockTable lock. First try to unlock the in-memory record. + // If on recovery in either immediate or pending read, then we may have evicted the source address or it may have been Read/CopyToTail. + // So we fall through on a failed unlock or non-InMemory lock. + if (!stackCtx.recSrc.HasInMemoryLock || stackCtx.recSrc.LogicalAddress < stackCtx.recSrc.Log.HeadAddress || !recordInfo.TryUnlockShared()) { - if (stackCtx.recSrc.LogicalAddress >= stackCtx.recSrc.Log.HeadAddress && recordInfo.TryUnlockShared()) - break; - status = InternalLock(ref key, new(LockOperationType.Unlock, LockType.Shared), out _); - } while (HandleImmediateRetryStatus(status, currentCtx, currentCtx, fasterSession, ref pendingContext)); + // Updaters (Upsert, RMW, Delete) XLock records. Readers do not, including anything calling InternalTryCopyToTail. This means the record may + // be transferred from the readcache to the main log (or even to the LockTable, if the record was in the (SafeHeadAddress, ClosedUntilAddress) + // interval when a Read started). + + if (stackCtx.recSrc.HasLockTableLock && LockTable.Unlock(ref key, stackCtx.hei.hash, LockType.Shared)) + return; + + // If the record dived below HeadAddress, we must wait for it to enter the lock table before unlocking; InternalLock does this (and starts + // by searching the in-memory space first, which is good because the record may have been transferred). + // If RecordInfo unlock fails, the locks were transferred to another recordInfo; do InternalLock to chase the key through the full process. + OperationStatus status; + do + { + status = InternalLock(ref key, new(LockOperationType.Unlock, LockType.Shared), out _); + } while (HandleImmediateRetryStatus(status, currentCtx, currentCtx, fasterSession, ref pendingContext)); + } + stackCtx.recSrc.HasInMemoryLock = false; } @@ -170,30 +178,6 @@ private void EphemeralXUnlockAndAbandonUpdate(FasterSession fasterSession, - FasterExecutionContext currentCtx, ref PendingContext pendingContext, - ref Key key, ref OperationStackContext stackCtx, ref RecordInfo srcRecordInfo) - where FasterSession : IFasterSession - { - if (fasterSession.DisableEphemeralLocking) - { - Debug.Assert(!stackCtx.recSrc.HasLock, "recSrc.HasLock should only be true if we are doing ephemeral locking"); - return; - } - - // Unlock read locks, if any. - if (stackCtx.recSrc.HasInMemoryLock) - { - // This unlocks the source (old) record; the new record may already be operated on by other threads, which is fine. - EphemeralSUnlock(fasterSession, currentCtx, ref pendingContext, ref key, ref stackCtx, ref srcRecordInfo); - return; - } - - if (stackCtx.recSrc.HasLockTableLock) - LockTable.Unlock(ref key, stackCtx.hei.hash, LockType.Shared); - } - [MethodImpl(MethodImplOptions.AggressiveInlining)] private bool CompleteTwoPhaseUpdate(FasterSession fasterSession, ref Key key, ref OperationStackContext stackCtx, ref RecordInfo srcRecordInfo, ref RecordInfo newRecordInfo, out OperationStatus status) @@ -214,7 +198,7 @@ private bool CompleteTwoPhaseUpdate(Faste Debug.Assert(found && ltriLT.IsLocked && !ltriLT.Tentative, "Error--non-InMemorySrc expected to find a non-tentative locked locktable entry"); transferred = LockTable.IsActive && LockTable.TransferToLogRecord(ref key, stackCtx.hei.hash, ref newRecordInfo); - Debug.Assert(transferred, "ManualLocking Non-InMemory source should find a LockTable entry to transfer locks from"); + Debug.Assert(transferred, "ManualLocking Non-InMemory source should find a LockTable entry to transfer locks from in CompleteTwoPhaseUpdate"); } #if DEBUG if (this.LockTable.TryGet(ref key, stackCtx.hei.hash, out var ltri)) @@ -253,8 +237,8 @@ private bool CompleteTwoPhaseCopyToTail(F { stackCtx.recSrc.AssertInMemorySourceWasNotEvicted(); - // Unlock the ephemeral lock here; we mark the source so we *know* we will have an invalid unlock on srcRecordInfo and would have to chase - // through InternalLock to unlock it, so we save the time by not transferring our ephemeral lock; 'Tentative' still protects the new record. + // Unlock the ephemeral lock here; we mark the source, so we *know* we will have an invalid unlock on srcRecordInfo and would have to chase + // through InternalLock to unlock it, so we save the time by not transferring our ephemeral lock. 'Tentative' still protects the new record. success = newRecordInfo.CopyReadLocksFromAndMarkSourceAtomic(ref srcRecordInfo, allowXLock: fasterSession.IsManualLocking, seal: stackCtx.recSrc.HasMainLogSrc, removeEphemeralLock: stackCtx.recSrc.HasInMemoryLock); if (success) @@ -266,17 +250,28 @@ private bool CompleteTwoPhaseCopyToTail(F { // For manual locking, we should already have made sure there is at least an SLock for this; since there is no HasInMemorySrc, it is in the Lock Table. if (LockTable.IsActive) - LockTable.TransferToLogRecord(ref key, stackCtx.hei.hash, ref newRecordInfo); + { + if (!LockTable.TransferToLogRecord(ref key, stackCtx.hei.hash, ref newRecordInfo)) + Debug.Fail("ManualLocking Non-InMemory source should find a LockTable entry to transfer locks from in CompleteTwoPhaseCopyToTail"); + } } else { - // XLocks are not allowed here in the ephemeral section, because another thread owns them (ephemeral locking only takes a read lock for operations that end up here). - success = (!LockTable.IsActive || LockTable.CompleteTwoPhaseCopyToTail(ref key, stackCtx.hei.hash, ref newRecordInfo, allowXLock: fasterSession.IsManualLocking, - removeEphemeralLock: stackCtx.recSrc.HasLockTableLock)) - && - (!UseReadCache || ReadCacheCompleteTwoPhaseCopyToTail(ref key, ref stackCtx.hei, ref newRecordInfo, allowXLock: fasterSession.IsManualLocking, - removeEphemeralLock: stackCtx.recSrc.HasLockTableLock)); // HasLockTableLock because we checked HasInMemorySrc above - stackCtx.recSrc.HasLockTableLock = false; + // XLocks are not allowed here in the ephemeral section, because another thread owns them (for operations that end up here, ephemeral locking only takes a read lock). + var lt_success = !LockTable.IsActive || LockTable.CompleteTwoPhaseCopyToTail(ref key, stackCtx.hei.hash, ref newRecordInfo, allowXLock: fasterSession.IsManualLocking, + removeEphemeralLock: stackCtx.recSrc.HasLockTableLock); + + // We must check readcache even if we don't have an in-memory source, because another thread may have transferred from the LockTable to the ReadCache. + var rc_success = !UseReadCache || ReadCacheCompleteTwoPhaseCopyToTail(ref key, ref stackCtx.hei, ref newRecordInfo, allowXLock: fasterSession.IsManualLocking, + removeEphemeralLock: stackCtx.recSrc.HasLockTableLock); // HasLockTableLock because we checked HasInMemorySrc above + + // We don't have to worry about checking the main log, because a CAS by another thread would fail due to our Tentative record being there + // for both mainhash entry (CAS compares to entry.word) and readcache (splice CAS compares to LowestLogicalAddress). + success = lt_success && rc_success; + Debug.Assert(success, "Expected success releasing LockTable lock in Complete2pCTT"); + + if (success) + stackCtx.recSrc.HasLockTableLock = false; } } return success; diff --git a/cs/src/core/Index/FASTER/Implementation/InternalRead.cs b/cs/src/core/Index/FASTER/Implementation/InternalRead.cs index c3a8cfb8a..3ebaf3b00 100644 --- a/cs/src/core/Index/FASTER/Implementation/InternalRead.cs +++ b/cs/src/core/Index/FASTER/Implementation/InternalRead.cs @@ -122,8 +122,6 @@ internal OperationStatus InternalRead( #region Normal processing - var prevHA = hlog.HeadAddress; - // Mutable region (even fuzzy region is included here) if (stackCtx.recSrc.LogicalAddress >= hlog.SafeReadOnlyAddress) { diff --git a/cs/src/core/Index/FASTER/Implementation/ReadCache.cs b/cs/src/core/Index/FASTER/Implementation/ReadCache.cs index 76072b31f..31ede8af8 100644 --- a/cs/src/core/Index/FASTER/Implementation/ReadCache.cs +++ b/cs/src/core/Index/FASTER/Implementation/ReadCache.cs @@ -44,33 +44,37 @@ private bool FindInReadCache(ref Key key, ref OperationStackContext // Use a non-ref local, because we update it below to remove the readcache bit. RecordInfo recordInfo = readcache.GetInfo(stackCtx.recSrc.LowestReadCachePhysicalAddress); - // When traversing the readcache, we skip Invalid records. The semantics of Seal are that the operation is retried, so if we leave - // Sealed records in the readcache, we'll never get past them. Therefore, we go from Tentative to Invalid if the Tentative record - // has to be invalidated. There is only one scenario where we go Tentative -> Invalid in the readcache: when an updated record was - // added to the main log. This record is *after* the Invalidated one, so it is safe to proceed. We don't go Tentative -> Invalid for - // Read/CopyToReadCache; InternalContinuePendingRead makes sure there is not already a record in the readcache for a record just read - // from disk, and the usual CAS-into-hashbucket operation to add a new readcache record will catch the case a subsequent one was added. - if (recordInfo.Tentative && waitForTentative) - { - // This is not a ref, so we have to re-get it. - ref var ri = ref readcache.GetInfo(stackCtx.recSrc.LowestReadCachePhysicalAddress); - SpinWaitWhileTentativeAndReturnValidity(ref ri); - recordInfo = ri; - } - // Return true if we find a read cache entry matching the key. Skip Invalid but *not* Intermediate; that's tested as part of lock acquisition. if (!recordInfo.Invalid && stackCtx.recSrc.LatestLogicalAddress > untilAddress && !stackCtx.recSrc.HasReadCacheSrc && comparer.Equals(ref key, ref readcache.GetKey(stackCtx.recSrc.LowestReadCachePhysicalAddress))) { - // Keep these at the current readcache location; they'll be the caller's source record. - stackCtx.recSrc.LogicalAddress = stackCtx.recSrc.LowestReadCacheLogicalAddress; - stackCtx.recSrc.PhysicalAddress = stackCtx.recSrc.LowestReadCachePhysicalAddress; - stackCtx.recSrc.HasReadCacheSrc = true; - stackCtx.recSrc.Log = readcache; - - // Read() does not need to continue past the found record; updaters need to continue to find latestLogicalAddress and lowestReadCache*Address. - if (!alwaysFindLatestLA) - return true; + // When traversing the readcache, we skip Invalid records. The semantics of Seal are that the operation is retried, so if we leave + // Sealed records in the readcache, we'll never get past them. Therefore, we go from Tentative to Invalid if the Tentative record + // has to be invalidated. There is only one scenario where we go Tentative -> Invalid in the readcache: when an updated record was + // added to the main log. This record is *after* the Invalidated one, so it is safe to proceed. We don't go Tentative -> Invalid for + // Read/CopyToReadCache; InternalContinuePendingRead makes sure there is not already a record in the readcache for a record just read + // from disk, and the usual CAS-into-hashbucket operation to add a new readcache record will catch the case a subsequent one was added. + bool valid = true; + if (recordInfo.Tentative && waitForTentative) + { + // This is not a ref, so we have to re-get it. + ref var ri = ref readcache.GetInfo(stackCtx.recSrc.LowestReadCachePhysicalAddress); + valid = SpinWaitWhileTentativeAndReturnValidity(ref ri); + recordInfo = ri; + } + + if (valid) + { + // Keep these at the current readcache location; they'll be the caller's source record. + stackCtx.recSrc.LogicalAddress = stackCtx.recSrc.LowestReadCacheLogicalAddress; + stackCtx.recSrc.PhysicalAddress = stackCtx.recSrc.LowestReadCachePhysicalAddress; + stackCtx.recSrc.HasReadCacheSrc = true; + stackCtx.recSrc.Log = readcache; + + // Read() does not need to continue past the found record; updaters need to continue to find latestLogicalAddress and lowestReadCache*Address. + if (!alwaysFindLatestLA) + return true; + } } // Is the previous record a main log record? If so, break out. diff --git a/cs/src/core/Index/FASTER/Implementation/RecordSource.cs b/cs/src/core/Index/FASTER/Implementation/RecordSource.cs index 0e54310cc..a662251fc 100644 --- a/cs/src/core/Index/FASTER/Implementation/RecordSource.cs +++ b/cs/src/core/Index/FASTER/Implementation/RecordSource.cs @@ -175,7 +175,8 @@ public override string ToString() var llaRC = IsReadCache(LatestLogicalAddress) ? isRC : string.Empty; var laRC = IsReadCache(LogicalAddress) ? isRC : string.Empty; static string bstr(bool value) => value ? "T" : "F"; - return $"lla {AbsoluteAddress(LatestLogicalAddress)}{llaRC}, la {AbsoluteAddress(LogicalAddress)}{laRC}, lrcla {AbsoluteAddress(LowestReadCacheLogicalAddress)}," + return $"lla {AbsoluteAddress(LatestLogicalAddress)}{llaRC}, la {AbsoluteAddress(LogicalAddress)}{laRC}, pa {PhysicalAddress:x}" + + $" lrcla {AbsoluteAddress(LowestReadCacheLogicalAddress)}, lrcpa {LowestReadCachePhysicalAddress:x}" + $" hasMLsrc {bstr(HasMainLogSrc)}, hasRCsrc {bstr(HasReadCacheSrc)}, hasIMlock {bstr(HasInMemoryLock)}, hasLTlock {bstr(HasLockTableLock)}"; } } diff --git a/cs/src/core/Utilities/LockTable.cs b/cs/src/core/Utilities/LockTable.cs index 2a3675bf1..3a5f31c38 100644 --- a/cs/src/core/Utilities/LockTable.cs +++ b/cs/src/core/Utilities/LockTable.cs @@ -382,7 +382,7 @@ internal FindEntryFunctions_CompleteTwoPhaseCopyToTail(RecordInfo toRecordInfo, success = false; } - public void NotFound(ref TKey key) => success = true; + public void NotFound(ref TKey key) { /* If there is no entry, somone else moved it */ } public void FoundEntry(ref TKey key, ref RecordInfo recordInfo) => success = toRecordInfo.CopyReadLocksFromAndMarkSourceAtomic(ref recordInfo, allowXLock, seal: false, this.removeEphemeralLock); diff --git a/cs/test/BasicFASTERTests.cs b/cs/test/BasicFASTERTests.cs index 479c78615..efabe4e94 100644 --- a/cs/test/BasicFASTERTests.cs +++ b/cs/test/BasicFASTERTests.cs @@ -808,5 +808,40 @@ public static void KVBasicsSampleEndToEndInDocs() s.Read(ref key, ref output); Assert.AreEqual(10, output); } + + [Test] + [Category("FasterKV")] + public static void UshortKeyByteValueTest() + { + using var log = Devices.CreateLogDevice($"{TestUtils.MethodTestDir}/hlog.log", deleteOnClose: false); + using var store = new FasterKV(1L << 20, new LogSettings { LogDevice = log }); + using var s = store.NewSession(new SimpleFunctions()); + ushort key = 1024; + byte value = 1, input = 10, output = 0; + + // For blittable types, the records are not 8-byte aligned; RecordSize is sizeof(RecordInfo) + sizeof(ushort) + sizeof(byte) + const int expectedRecordSize = sizeof(long) + sizeof(ushort) + sizeof(byte); + Assert.AreEqual(11, expectedRecordSize); + long prevTailLogicalAddress = store.hlog.GetTailAddress(); + long prevTailPhysicalAddress = store.hlog.GetPhysicalAddress(prevTailLogicalAddress); + for (var ii = 0; ii < 5; ++ii, ++key, ++value, ++input) + { + output = 0; + s.Upsert(ref key, ref value); + s.Read(ref key, ref output); + Assert.AreEqual(value, output); + s.RMW(ref key, ref input); + s.Read(ref key, ref output); + Assert.AreEqual(input, output); + + var tailLogicalAddress = store.hlog.GetTailAddress(); + Assert.AreEqual(expectedRecordSize, tailLogicalAddress - prevTailLogicalAddress); + long tailPhysicalAddress = store.hlog.GetPhysicalAddress(tailLogicalAddress); + Assert.AreEqual(expectedRecordSize, tailPhysicalAddress - prevTailPhysicalAddress); + + prevTailLogicalAddress = tailLogicalAddress; + prevTailPhysicalAddress = tailPhysicalAddress; + } + } } } \ No newline at end of file diff --git a/cs/test/LockTableTests.cs b/cs/test/LockTableTests.cs index 51ac2aa63..e953e3e8c 100644 --- a/cs/test/LockTableTests.cs +++ b/cs/test/LockTableTests.cs @@ -6,8 +6,6 @@ using NUnit.Framework; using System; using System.Collections.Generic; -using System.Linq; -using System.Reflection.Metadata.Ecma335; using System.Threading; using System.Threading.Tasks; using static FASTER.test.TestUtils; From 10157a63a3018a339e9fe41756356b97ebb539bb Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Tue, 7 Feb 2023 14:49:00 -0800 Subject: [PATCH 13/29] update testutils.deletedirectory --- cs/test/TestUtils.cs | 25 ++++++------------------- 1 file changed, 6 insertions(+), 19 deletions(-) diff --git a/cs/test/TestUtils.cs b/cs/test/TestUtils.cs index 296be2fe6..9f20dbe00 100644 --- a/cs/test/TestUtils.cs +++ b/cs/test/TestUtils.cs @@ -49,11 +49,9 @@ internal static void DeleteDirectory(string path, bool wait = false) } } - bool retry = true; - while (retry) + for (; ; Thread.Yield()) { // Exceptions may happen due to a handle briefly remaining held after Dispose(). - retry = false; try { Directory.Delete(path, true); @@ -61,21 +59,10 @@ internal static void DeleteDirectory(string path, bool wait = false) catch (Exception ex) when (ex is IOException || ex is UnauthorizedAccessException) { - if (!wait) - { - try { Directory.Delete(path, true); } - catch { } - return; - } - retry = true; } + if (!wait || !Directory.Exists(path)) + break; } - - if (!wait) - return; - - while (Directory.Exists(path)) - Thread.Yield(); } /// @@ -120,7 +107,7 @@ internal static IDevice CreateTestDevice(DeviceType testDeviceType, string filen bool preallocateFile = false; long capacity = Devices.CAPACITY_UNSPECIFIED; bool recoverDevice = false; - + switch (testDeviceType) { #if WINDOWS @@ -141,7 +128,7 @@ internal static IDevice CreateTestDevice(DeviceType testDeviceType, string filen device = new ManagedLocalStorageDevice(filename, preallocateFile, deleteOnClose, capacity, recoverDevice); 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: + case DeviceType.LocalMemory: device = new LocalMemoryDevice(1L << 28, 1L << 25, 2, sector_size: 512, latencyMs: latencyMs, fileName: filename); // 64 MB (1L << 26) is enough for our test cases break; } @@ -169,7 +156,7 @@ internal static string AzureTestContainer return container; } } - + internal static string AzureTestDirectory => TestContext.CurrentContext.Test.MethodName; internal const string AzureEmulatedStorageString = "UseDevelopmentStorage=true;"; From b7dfb99e19b2f0bdded1e741a7ed7da021343244 Mon Sep 17 00:00:00 2001 From: TedHartMS <15467143+TedHartMS@users.noreply.github.com> Date: Tue, 7 Feb 2023 16:23:57 -0800 Subject: [PATCH 14/29] WIP: add memory size args to MemOnlyCache --- cs/samples/MemOnlyCache/Program.cs | 69 +++++++++++++++++++++++------- 1 file changed, 53 insertions(+), 16 deletions(-) diff --git a/cs/samples/MemOnlyCache/Program.cs b/cs/samples/MemOnlyCache/Program.cs index 8198389b2..317ec2a96 100644 --- a/cs/samples/MemOnlyCache/Program.cs +++ b/cs/samples/MemOnlyCache/Program.cs @@ -15,10 +15,10 @@ namespace MemOnlyCache class Program { /// - /// Total database size + /// Maximum number of keys in the database /// - static int DbSize = 10_000_000; - const string DbSizeArg = "--dbsize"; + static int MaxKeys = 10_000_000; + const string MaxKeyArg = "--maxkeys"; /// /// Max key size; we choose actual size randomly @@ -32,6 +32,24 @@ class Program static int MaxValueSize = 1000; const string MaxValueSizeArg = "--valuesize"; + /// + /// Total in-memory size bits + /// + static int MemorySizeBits = 25; // (2^25 / 24) = ~1.39M key-value pairs (log uses 24 bytes per KV pair) + const string MemorySizeBitsArg = "--memsizebits"; + + /// + /// Page size bits + /// + static int PageSizeBits = 14; // Each page is sized at 2^14 bytes + const string PageSizeBitsArg = "--pagesizebits"; + + /// + /// Average collisions + /// + static int TagChainLength = 1; // No collisions + const string TagChainLengthArg = "--tagchainlen"; + /// /// Number of threads accessing FASTER instances /// @@ -132,9 +150,12 @@ private static bool Usage() { Console.WriteLine("Runs a loop that illustrates an in-memory cache with dynamic size limit"); Console.WriteLine("Usage:"); - Console.WriteLine($" {DbSizeArg} #: Total database size. Default = {DbSize}"); + Console.WriteLine($" {MaxKeyArg} #: Maximum number of keys in the database. Default = {MaxKeys}"); Console.WriteLine($" {MaxKeySizeArg} #: Max key size; we choose actual size randomly. Default = {MaxKeySize}"); Console.WriteLine($" {MaxValueSizeArg} #: Max value size; we choose actual size randomly. Default = {MaxValueSize}"); + Console.WriteLine($" {MemorySizeBitsArg} #: In-memory size of the log, in bits. Default = {MemorySizeBits}"); + Console.WriteLine($" {PageSizeBitsArg} #: Page size, in bits. Default = {PageSizeBits}"); + Console.WriteLine($" {TagChainLengthArg} #: Average length of the hash collision chain for each tag. Default = {TagChainLength}"); Console.WriteLine($" {OpPercentArg} #,#,#,#: Percentage of [(r)eads,r(m)ws,(u)pserts,(d)eletes] (summing to 0 or 100) operations in incoming workload requests. Default = {ReadPercent},{RmwPercent},{UpsertPercent},{DeletePercent}"); Console.WriteLine($" {NoReadCTTArg}: Turn off (true) or allow (false) copying of reads from the Immutable region of the log to the tail of log. Default = {!UseReadCTT}"); Console.WriteLine($" {UseReadCacheArg}: Whether to use the ReadCache. Default = {UseReadCache}"); @@ -194,9 +215,9 @@ static bool GetArgs(string[] args) return false; } val = args[++ii]; - if (arg == DbSizeArg) + if (arg == MaxKeyArg) { - DbSize = int.Parse(val); + MaxKeys = int.Parse(val); continue; } if (arg == MaxKeySizeArg) @@ -204,6 +225,21 @@ static bool GetArgs(string[] args) MaxKeySize = int.Parse(val); continue; } + if (arg == MemorySizeBitsArg) + { + MemorySizeBits = int.Parse(val); + continue; + } + if (arg == PageSizeBitsArg) + { + PageSizeBits = int.Parse(val); + continue; + } + if (arg == TagChainLengthArg) + { + TagChainLength = int.Parse(val); + continue; + } if (arg == NumThreadsArg) { NumThreads = int.Parse(val); @@ -257,7 +293,7 @@ static bool GetArgs(string[] args) return Usage(); } - // Note: Here we could verify parameter compatibility + // Note: Here we could verify parameter values and compatibility } catch (Exception ex) { @@ -301,9 +337,9 @@ static void Main(string[] args) { LogDevice = log, ObjectLogDevice = objectLog, MutableFraction = 0.9, // 10% of memory log is "read-only region" - ReadFlags = UseReadCTT ? ReadFlags.CopyReadsToTail : ReadFlags.None, // reads in read-only region are copied to tail - PageSizeBits = 14, // Each page is sized at 2^14 bytes - MemorySizeBits = 25, // (2^25 / 24) = ~1.39M key-value pairs (log uses 24 bytes per KV pair) + ReadFlags = UseReadCTT ? ReadFlags.CopyReadsToTail : ReadFlags.None, // whether reads in read-only region are copied to tail + PageSizeBits = PageSizeBits, + MemorySizeBits = MemorySizeBits }; if (UseReadCache) @@ -311,10 +347,11 @@ static void Main(string[] args) // Number of records in memory, assuming class keys and values and x64 platform // (8-byte key + 8-byte value + 8-byte header = 24 bytes per record) - int numRecords = (int)(Math.Pow(2, logSettings.MemorySizeBits) / 24); + const int recordSize = 24; + int numRecords = (int)(Math.Pow(2, logSettings.MemorySizeBits) / recordSize); // Set hash table size targeting 1 record per bucket - var numBucketBits = (int)Math.Ceiling(Math.Log2(numRecords)); + var numBucketBits = (int)Math.Ceiling(Math.Log2(numRecords)) / TagChainLength; h = new FasterKV(1L << numBucketBits, logSettings, serializerSettings: serializerSettings, comparer: new CacheKey()); sizeTracker = new CacheSizeTracker(h, targetSize); @@ -349,7 +386,7 @@ private static void PopulateStore(int count) for (int i = 0; i < count; i++) { - int k = r.Next(DbSize); + int k = r.Next(MaxKeys); var key = new CacheKey(k, 1 + r.Next(MaxKeySize - 1)); var value = new CacheValue(1 + r.Next(MaxValueSize - 1), (byte)key.key); s.Upsert(ref key, ref value); @@ -420,12 +457,12 @@ private static void ContinuousRandomWorkload() private static void RandomWorkload(int threadid) { - Console.WriteLine("Issuing {0} random read workload of {1} reads from thread {2}", UseUniform ? "uniform" : "zipf", DbSize, threadid); + Console.WriteLine("Issuing {0} random read workload of {1} reads from thread {2}", UseUniform ? "uniform" : "zipf", MaxKeys, threadid); using var session = h.For(new CacheFunctions(sizeTracker)).NewSession(); var rng = new Random(threadid); - var zipf = new ZipfGenerator(rng, DbSize, Theta); + var zipf = new ZipfGenerator(rng, MaxKeys, Theta); CacheValue output = default; int localStatusFound = 0, localStatusNotFound = 0; @@ -444,7 +481,7 @@ private static void RandomWorkload(int threadid) var wantValue = RmwPercent + UpsertPercent > 0; int op = ReadPercent < 100 ? rng.Next(100) : 99; // rng.Next() is not inclusive of the upper bound - long k = UseUniform ? rng.Next(DbSize) : zipf.Next(); + long k = UseUniform ? rng.Next(MaxKeys) : zipf.Next(); var key = new CacheKey(k, 1 + rng.Next(MaxKeySize - 1)); From 44244446f11e25f408cbc2f5af983c396af2f053 Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Wed, 8 Feb 2023 08:29:01 -0800 Subject: [PATCH 15/29] testing CI error --- cs/test/DeltaLogTests.cs | 1 + 1 file changed, 1 insertion(+) diff --git a/cs/test/DeltaLogTests.cs b/cs/test/DeltaLogTests.cs index 5dac77fbe..a90c29720 100644 --- a/cs/test/DeltaLogTests.cs +++ b/cs/test/DeltaLogTests.cs @@ -38,6 +38,7 @@ public void TearDown() [Test] [Category("FasterLog")] [Category("Smoke")] + [Ignore("testing CI error")] public void DeltaLogTest1([Values] TestUtils.DeviceType deviceType) { const int TotalCount = 200; From f7b3e5b4e49882e8ebb3b70cd46266219978a64b Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Wed, 8 Feb 2023 10:15:06 -0800 Subject: [PATCH 16/29] test --- cs/test/TestUtils.cs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cs/test/TestUtils.cs b/cs/test/TestUtils.cs index 9f20dbe00..e6f8daeca 100644 --- a/cs/test/TestUtils.cs +++ b/cs/test/TestUtils.cs @@ -83,7 +83,7 @@ internal static void RecreateDirectory(string path) internal static void IgnoreIfNotRunningAzureTests() { // Need this environment variable set AND Azure Storage Emulator running - if (!IsRunningAzureTests) + //if (!IsRunningAzureTests) Assert.Ignore("Environment variable RunAzureTests is not defined"); } From d9b8bdf24674dd1bf49c05292da0b88826343698 Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Wed, 8 Feb 2023 14:26:23 -0800 Subject: [PATCH 17/29] add logging for ASD --- cs/test/BasicDiskFASTERTests.cs | 6 +-- cs/test/DeviceFasterLogTests.cs | 4 +- cs/test/LogFormatter.cs | 26 ++++++++++++ cs/test/NUnitLoggerProvider.cs | 75 +++++++++++++++++++++++++++++++++ cs/test/TestUtils.cs | 24 ++++++++++- 5 files changed, 128 insertions(+), 7 deletions(-) create mode 100644 cs/test/LogFormatter.cs create mode 100644 cs/test/NUnitLoggerProvider.cs diff --git a/cs/test/BasicDiskFASTERTests.cs b/cs/test/BasicDiskFASTERTests.cs index 43b4c006c..8e117f60e 100644 --- a/cs/test/BasicDiskFASTERTests.cs +++ b/cs/test/BasicDiskFASTERTests.cs @@ -26,7 +26,7 @@ public void LocalStorageWriteRead() public void PageBlobWriteRead() { TestUtils.IgnoreIfNotRunningAzureTests(); - TestDeviceWriteRead(new AzureStorageDevice(TestUtils.AzureEmulatedStorageString, TestUtils.AzureTestContainer, TestUtils.AzureTestDirectory, "BasicDiskFASTERTests")); + TestDeviceWriteRead(new AzureStorageDevice(TestUtils.AzureEmulatedStorageString, TestUtils.AzureTestContainer, TestUtils.AzureTestDirectory, "BasicDiskFASTERTests", logger: TestUtils.TestLoggerFactory.CreateLogger("asd"))); } [Test] @@ -35,7 +35,7 @@ public void PageBlobWriteRead() public void PageBlobWriteReadWithLease() { TestUtils.IgnoreIfNotRunningAzureTests(); - TestDeviceWriteRead(new AzureStorageDevice(TestUtils.AzureEmulatedStorageString, TestUtils.AzureTestContainer, TestUtils.AzureTestDirectory, "BasicDiskFASTERTests",null,true,true)); + TestDeviceWriteRead(new AzureStorageDevice(TestUtils.AzureEmulatedStorageString, TestUtils.AzureTestContainer, TestUtils.AzureTestDirectory, "BasicDiskFASTERTests",null,true,true, logger: TestUtils.TestLoggerFactory.CreateLogger("asd"))); } [Test] @@ -48,7 +48,7 @@ public void TieredWriteRead() IDevice localDevice = Devices.CreateLogDevice(TestUtils.MethodTestDir + "/BasicDiskFASTERTests.log", deleteOnClose: true, capacity: 1 << 30); if (TestUtils.IsRunningAzureTests) { - IDevice cloudDevice = new AzureStorageDevice(TestUtils.AzureEmulatedStorageString, TestUtils.AzureTestContainer, TestUtils.AzureTestDirectory, "BasicDiskFASTERTests"); + IDevice cloudDevice = new AzureStorageDevice(TestUtils.AzureEmulatedStorageString, TestUtils.AzureTestContainer, TestUtils.AzureTestDirectory, "BasicDiskFASTERTests", logger: TestUtils.TestLoggerFactory.CreateLogger("asd")); tested = new TieredStorageDevice(1, localDevice, cloudDevice); } else diff --git a/cs/test/DeviceFasterLogTests.cs b/cs/test/DeviceFasterLogTests.cs index 8afb1f0f9..078fb21e5 100644 --- a/cs/test/DeviceFasterLogTests.cs +++ b/cs/test/DeviceFasterLogTests.cs @@ -24,7 +24,7 @@ internal class DeviceFasterLogTests public async ValueTask PageBlobFasterLogTest1([Values] LogChecksumType logChecksum, [Values]FasterLogTestBase.IteratorType iteratorType) { TestUtils.IgnoreIfNotRunningAzureTests(); - var device = new AzureStorageDevice(TestUtils.AzureEmulatedStorageString, $"{TestUtils.AzureTestContainer}", TestUtils.AzureTestDirectory, "fasterlog.log", deleteOnClose: true); + var device = new AzureStorageDevice(TestUtils.AzureEmulatedStorageString, $"{TestUtils.AzureTestContainer}", TestUtils.AzureTestDirectory, "fasterlog.log", deleteOnClose: true, logger: TestUtils.TestLoggerFactory.CreateLogger("asd")); var checkpointManager = new DeviceLogCommitCheckpointManager( new AzureStorageNamedDeviceFactory(TestUtils.AzureEmulatedStorageString), new DefaultCheckpointNamingScheme($"{TestUtils.AzureTestContainer}/{TestUtils.AzureTestDirectory}")); @@ -39,7 +39,7 @@ public async ValueTask PageBlobFasterLogTest1([Values] LogChecksumType logChecks public async ValueTask PageBlobFasterLogTestWithLease([Values] LogChecksumType logChecksum, [Values] FasterLogTestBase.IteratorType iteratorType) { TestUtils.IgnoreIfNotRunningAzureTests(); - var device = new AzureStorageDevice(TestUtils.AzureEmulatedStorageString, $"{TestUtils.AzureTestContainer}", TestUtils.AzureTestDirectory, "fasterlogLease.log", deleteOnClose: true, underLease: true, blobManager: null); + var device = new AzureStorageDevice(TestUtils.AzureEmulatedStorageString, $"{TestUtils.AzureTestContainer}", TestUtils.AzureTestDirectory, "fasterlogLease.log", deleteOnClose: true, underLease: true, blobManager: null, logger: TestUtils.TestLoggerFactory.CreateLogger("asd")); var checkpointManager = new DeviceLogCommitCheckpointManager( new AzureStorageNamedDeviceFactory(TestUtils.AzureEmulatedStorageString), new DefaultCheckpointNamingScheme($"{TestUtils.AzureTestContainer}/{TestUtils.AzureTestDirectory}")); diff --git a/cs/test/LogFormatter.cs b/cs/test/LogFormatter.cs new file mode 100644 index 000000000..fb1f4cc76 --- /dev/null +++ b/cs/test/LogFormatter.cs @@ -0,0 +1,26 @@ +using System; +using System.Globalization; + +namespace FASTER.test +{ + /// + /// Log formatter primitives + /// + public static class LogFormatter + { + private const string TIME_FORMAT = "HH:mm:ss.ffff"; + private const string DATE_FORMAT = "yyyy-MM-dd " + TIME_FORMAT; + + /// + /// Format date + /// + /// + public static string FormatDate(DateTime dateTime) => dateTime.ToString(DATE_FORMAT, CultureInfo.InvariantCulture); + + /// + /// Format time + /// + /// + public static string FormatTime(DateTime dateTime) => dateTime.ToString(TIME_FORMAT, CultureInfo.InvariantCulture); + } +} diff --git a/cs/test/NUnitLoggerProvider.cs b/cs/test/NUnitLoggerProvider.cs new file mode 100644 index 000000000..a46a160ad --- /dev/null +++ b/cs/test/NUnitLoggerProvider.cs @@ -0,0 +1,75 @@ +using System; +using System.IO; +using System.Collections.Generic; +using Microsoft.Extensions.Logging; +using System.Threading; + +namespace FASTER.test +{ + public class NUnitLoggerProvider : ILoggerProvider + { + private readonly TextWriter textWriter; + private readonly string scope; + + readonly static string[] lvl = new string[] + { + "trce", + "dbug", + "info", + "warn", + "errr", + "crit", + }; + + public NUnitLoggerProvider(TextWriter textWriter, string scope = "") + { + this.textWriter = textWriter; + this.scope = scope; + } + + public ILogger CreateLogger(string categoryName) => new NUnitLogger(categoryName, textWriter, scope); + + public void Dispose() + { } + + private class NUnitLogger : ILogger + { + private readonly string categoryName; + private readonly TextWriter textWriter; + private readonly string scope; + private int loggerId; + + public NUnitLogger(string categoryName, TextWriter textWriter, string scope) + { + this.categoryName = $"{categoryName}:{Interlocked.Increment(ref loggerId)}"; + this.textWriter = textWriter; + this.scope = scope; + } + + public IDisposable BeginScope(TState state) => default!; + + public bool IsEnabled(LogLevel logLevel) => true; + + private string GetLevelStr(LogLevel ll) => lvl[(int)ll]; + + public void Log( + LogLevel logLevel, + EventId eventId, + TState state, + Exception exception, + Func formatter) + { + var msg = string.Format("[{0:D3}.{1}.({2})] |{3}| <{4}> {5} ^{6}^", + eventId.Id, + LogFormatter.FormatDate(DateTime.UtcNow), + GetLevelStr(logLevel), + scope, + categoryName, + exception, + state); + + textWriter.Write(msg); + } + } + } +} diff --git a/cs/test/TestUtils.cs b/cs/test/TestUtils.cs index e6f8daeca..6655d4ea0 100644 --- a/cs/test/TestUtils.cs +++ b/cs/test/TestUtils.cs @@ -9,6 +9,7 @@ using System.Threading; using System.Runtime.InteropServices; using System.Threading.Tasks; +using Microsoft.Extensions.Logging; namespace FASTER.test { @@ -27,6 +28,8 @@ internal static class TestUtils internal const string RMWTestCategory = "RMW"; internal const string ModifiedBitTestCategory = "ModifiedBitTest"; + public static ILoggerFactory TestLoggerFactory = CreateLoggerFactoryInstance(TestContext.Progress, LogLevel.Trace); + /// /// Delete a directory recursively /// @@ -78,12 +81,29 @@ internal static void RecreateDirectory(string path) Directory.CreateDirectory(path); } + /// + /// Create logger factory for given TextWriter and loglevel + /// E.g. Use with TestContext.Progress to print logs while test is running. + /// + /// + /// + /// + /// + public static ILoggerFactory CreateLoggerFactoryInstance(TextWriter textWriter, LogLevel logLevel, string scope = "") + { + return LoggerFactory.Create(builder => + { + builder.AddProvider(new NUnitLoggerProvider(textWriter, scope)); + builder.SetMinimumLevel(logLevel); + }); + } + internal static bool IsRunningAzureTests => "yes".Equals(Environment.GetEnvironmentVariable("RunAzureTests")) || "yes".Equals(Environment.GetEnvironmentVariable("RUNAZURETESTS")); internal static void IgnoreIfNotRunningAzureTests() { // Need this environment variable set AND Azure Storage Emulator running - //if (!IsRunningAzureTests) + if (!IsRunningAzureTests) Assert.Ignore("Environment variable RunAzureTests is not defined"); } @@ -122,7 +142,7 @@ internal static IDevice CreateTestDevice(DeviceType testDeviceType, string filen #endif case DeviceType.EmulatedAzure: IgnoreIfNotRunningAzureTests(); - device = new AzureStorageDevice(AzureEmulatedStorageString, AzureTestContainer, AzureTestDirectory, Path.GetFileName(filename), deleteOnClose: deleteOnClose); + device = new AzureStorageDevice(AzureEmulatedStorageString, AzureTestContainer, AzureTestDirectory, Path.GetFileName(filename), deleteOnClose: deleteOnClose, logger: TestLoggerFactory.CreateLogger("asd")); break; case DeviceType.MLSD: device = new ManagedLocalStorageDevice(filename, preallocateFile, deleteOnClose, capacity, recoverDevice); From eef29c0d48577184c566eff9bd5677a0ba8a2fdc Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Wed, 8 Feb 2023 14:33:25 -0800 Subject: [PATCH 18/29] no azure - no disposetests --- cs/test/DisposeTests.cs | 6 +++++- cs/test/TestUtils.cs | 2 +- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/cs/test/DisposeTests.cs b/cs/test/DisposeTests.cs index 47e5541b5..2247961e8 100644 --- a/cs/test/DisposeTests.cs +++ b/cs/test/DisposeTests.cs @@ -1,4 +1,5 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. +/* +// Copyright (c) Microsoft Corporation. All rights reserved. // Licensed under the MIT license. using System.Collections.Generic; @@ -527,6 +528,7 @@ public void PendingRead([Values] ReadCopyDestination copyDest) [Test] [Category("FasterKV")] [Category("Smoke")] + [Ignore("CI issue")] public void CopyToTailWithInitialReadCache([Values(ReadCopyDestination.ReadCache)] ReadCopyDestination copyDest) { // We use the ReadCopyDestination.ReadCache parameter so Setup() knows to set up the readcache, but @@ -583,6 +585,7 @@ public void DisposePendingRead2Threads([Values] ReadCopyDestination copyDest) [Test] [Category("FasterKV")] [Category("Smoke")] + [Ignore("CI issue")] public void DisposeCopyToTailWithInitialReadCache2Threads([Values(ReadCopyDestination.ReadCache)] ReadCopyDestination copyDest) { // We use the ReadCopyDestination.ReadCache parameter so Setup() knows to set up the readcache, but @@ -713,3 +716,4 @@ public void DisposePendingRmwWithNoConflictTest() } } } +*/ \ No newline at end of file diff --git a/cs/test/TestUtils.cs b/cs/test/TestUtils.cs index 6655d4ea0..6f3a9de09 100644 --- a/cs/test/TestUtils.cs +++ b/cs/test/TestUtils.cs @@ -103,7 +103,7 @@ public static ILoggerFactory CreateLoggerFactoryInstance(TextWriter textWriter, internal static void IgnoreIfNotRunningAzureTests() { // Need this environment variable set AND Azure Storage Emulator running - if (!IsRunningAzureTests) + //if (!IsRunningAzureTests) Assert.Ignore("Environment variable RunAzureTests is not defined"); } From c184a5e5b05484895c2be021c7bae05472363a65 Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Wed, 8 Feb 2023 15:15:20 -0800 Subject: [PATCH 19/29] incl azure --- cs/test/TestUtils.cs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cs/test/TestUtils.cs b/cs/test/TestUtils.cs index 6f3a9de09..6655d4ea0 100644 --- a/cs/test/TestUtils.cs +++ b/cs/test/TestUtils.cs @@ -103,7 +103,7 @@ public static ILoggerFactory CreateLoggerFactoryInstance(TextWriter textWriter, internal static void IgnoreIfNotRunningAzureTests() { // Need this environment variable set AND Azure Storage Emulator running - //if (!IsRunningAzureTests) + if (!IsRunningAzureTests) Assert.Ignore("Environment variable RunAzureTests is not defined"); } From 1c6b33c71afcfa8d5b7914f5051a16e5c409392b Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Fri, 10 Feb 2023 10:14:58 -0800 Subject: [PATCH 20/29] re-enable delta log tests --- cs/test/DeltaLogTests.cs | 1 - 1 file changed, 1 deletion(-) diff --git a/cs/test/DeltaLogTests.cs b/cs/test/DeltaLogTests.cs index a90c29720..5dac77fbe 100644 --- a/cs/test/DeltaLogTests.cs +++ b/cs/test/DeltaLogTests.cs @@ -38,7 +38,6 @@ public void TearDown() [Test] [Category("FasterLog")] [Category("Smoke")] - [Ignore("testing CI error")] public void DeltaLogTest1([Values] TestUtils.DeviceType deviceType) { const int TotalCount = 200; From 2a42cbe9671016edebe5cd898606e74b83b827df Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Fri, 10 Feb 2023 10:49:40 -0800 Subject: [PATCH 21/29] Cleanup --- .../AzureStorageDevice/AzureStorageDevice.cs | 22 +++++++++---------- .../devices/AzureStorageDevice/BlobEntry.cs | 2 +- .../devices/AzureStorageDevice/BlobManager.cs | 2 +- .../AzureStorageDevice/StorageOperations.cs | 2 +- 4 files changed, 14 insertions(+), 14 deletions(-) diff --git a/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs b/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs index be7fd248f..4a6130c3e 100644 --- a/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs +++ b/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs @@ -55,7 +55,7 @@ struct RemoveRequestInfo SemaphoreSlim SingleWriterSemaphore => this.singleWriterSemaphore; - internal IStorageErrorHandler PartitionErrorHandler { get; private set; } + internal IStorageErrorHandler StorageErrorHandler { get; private set; } // Azure Page Blobs have a fixed sector size of 512 bytes. const uint PAGE_BLOB_SECTOR_SIZE = 512; @@ -101,8 +101,8 @@ public AzureStorageDevice(string connectionString, string containerName, string if (blobManager == null) localBlobManager = true; this.BlobManager = blobManager ?? new BlobManager(logger, logger, LogLevel.Information, null, underLease, pageBlobDirectory, blobName); - this.PartitionErrorHandler = BlobManager.StorageErrorHandler; - this.PartitionErrorHandler?.Token.Register(this.CancelAllRequests); + this.StorageErrorHandler = BlobManager.StorageErrorHandler; + this.StorageErrorHandler.Token.Register(this.CancelAllRequests); this.underLease = underLease; this.hangCheckTimer = new Timer(this.DetectHangs, null, 0, 20000); this.singleWriterSemaphore = underLease ? new SemaphoreSlim(1) : null; @@ -136,8 +136,8 @@ internal AzureStorageDevice(string blobName, BlobUtilsV12.BlobDirectory pageBlob if (blobManager == null) localBlobManager = true; this.BlobManager = blobManager ?? new BlobManager(logger, logger, LogLevel.Information, null, underLease, pageBlobDirectory, blobName); - this.PartitionErrorHandler = BlobManager.StorageErrorHandler; - this.PartitionErrorHandler?.Token.Register(this.CancelAllRequests); + this.StorageErrorHandler = BlobManager.StorageErrorHandler; + this.StorageErrorHandler.Token.Register(this.CancelAllRequests); this.underLease = underLease; this.hangCheckTimer = new Timer(this.DetectHangs, null, 0, 20000); this.singleWriterSemaphore = underLease ? new SemaphoreSlim(1) : null; @@ -181,7 +181,7 @@ await this.BlobManager.PerformWithRetriesAsync( var page = await client.GetBlobsAsync( prefix: prefix, - cancellationToken: this.PartitionErrorHandler.Token) + cancellationToken: this.StorageErrorHandler.Token) .AsPages(continuationToken, 100) .FirstAsync(); @@ -209,7 +209,7 @@ await this.BlobManager.PerformWithRetriesAsync( // make sure we did not lose the lease while iterating to find the blobs await this.BlobManager.ConfirmLeaseIsGoodForAWhileAsync(); - this.PartitionErrorHandler.Token.ThrowIfCancellationRequested(); + this.StorageErrorHandler.Token.ThrowIfCancellationRequested(); // find longest contiguous sequence at end @@ -378,7 +378,7 @@ public override void RemoveSegmentAsync(int segment, AsyncCallback callback, IAs async (numAttempts) => { var client = (numAttempts > 1) ? entry.PageBlob.Default : entry.PageBlob.Aggressive; - await client.DeleteAsync(cancellationToken: this.PartitionErrorHandler.Token); + await client.DeleteAsync(cancellationToken: this.StorageErrorHandler.Token); return 1; }); } @@ -413,7 +413,7 @@ Task Delete(BlobEntry entry) async (numAttempts) => { var client = (numAttempts > 1) ? entry.PageBlob.Default : entry.PageBlob.Aggressive; - await client.DeleteAsync(cancellationToken: this.PartitionErrorHandler.Token); + await client.DeleteAsync(cancellationToken: this.StorageErrorHandler.Token); return 1; }); } @@ -543,7 +543,7 @@ await this.BlobManager.PerformWithRetriesAsync( transactionalContentHash: null, conditions: this.underLease ? new PageBlobRequestConditions() { IfMatch = blobEntry.ETag } : null, progressHandler: null, - cancellationToken: this.PartitionErrorHandler.Token).ConfigureAwait(false); + cancellationToken: this.StorageErrorHandler.Token).ConfigureAwait(false); blobEntry.ETag = response.Value.ETag; } @@ -597,7 +597,7 @@ await this.BlobManager.PerformWithRetriesAsync( range: new Azure.HttpRange(sourceAddress + offset, length), conditions: null, rangeGetContentHash: false, - cancellationToken: this.PartitionErrorHandler.Token) + cancellationToken: this.StorageErrorHandler.Token) .ConfigureAwait(false); await response.Value.Content.CopyToAsync(stream).ConfigureAwait(false); diff --git a/cs/src/devices/AzureStorageDevice/BlobEntry.cs b/cs/src/devices/AzureStorageDevice/BlobEntry.cs index 1fcce8d45..d0f6fc05c 100644 --- a/cs/src/devices/AzureStorageDevice/BlobEntry.cs +++ b/cs/src/devices/AzureStorageDevice/BlobEntry.cs @@ -76,7 +76,7 @@ await this.azureStorageDevice.BlobManager.PerformWithRetriesAsync( var response = await client.CreateAsync( size: size, conditions: new Azure.Storage.Blobs.Models.PageBlobRequestConditions() { IfNoneMatch = Azure.ETag.All }, - cancellationToken: this.azureStorageDevice.PartitionErrorHandler.Token); + cancellationToken: this.azureStorageDevice.StorageErrorHandler.Token); this.ETag = response.Value.ETag; return 1; diff --git a/cs/src/devices/AzureStorageDevice/BlobManager.cs b/cs/src/devices/AzureStorageDevice/BlobManager.cs index 774f2d3bb..2c7bc7d82 100644 --- a/cs/src/devices/AzureStorageDevice/BlobManager.cs +++ b/cs/src/devices/AzureStorageDevice/BlobManager.cs @@ -55,7 +55,7 @@ public static TimeSpan GetDelayBetweenRetries(int numAttempts) => TimeSpan.FromSeconds(Math.Pow(2, (numAttempts - 1))); /// - /// Create a blob manager. + /// Create a blob manager to handle tracing, leases, and storage operations /// /// A logger for logging /// diff --git a/cs/src/devices/AzureStorageDevice/StorageOperations.cs b/cs/src/devices/AzureStorageDevice/StorageOperations.cs index 4f2c8ceeb..0324eef39 100644 --- a/cs/src/devices/AzureStorageDevice/StorageOperations.cs +++ b/cs/src/devices/AzureStorageDevice/StorageOperations.cs @@ -51,7 +51,7 @@ public async Task PerformWithRetriesAsync( await this.ConfirmLeaseIsGoodForAWhileAsync(); } - this.StorageErrorHandler?.Token.ThrowIfCancellationRequested(); + this.StorageErrorHandler.Token.ThrowIfCancellationRequested(); this.StorageTracer?.FasterStorageProgress($"storage operation {name} ({intent}) started attempt {numAttempts}; target={target} {data}"); From 876ee6d7bcc86fdfb893897f991ec174e5fd6305 Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Fri, 10 Feb 2023 13:12:00 -0800 Subject: [PATCH 22/29] try re-enabling dispose tests --- cs/test/DisposeTests.cs | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/cs/test/DisposeTests.cs b/cs/test/DisposeTests.cs index 2247961e8..724026e80 100644 --- a/cs/test/DisposeTests.cs +++ b/cs/test/DisposeTests.cs @@ -1,5 +1,4 @@ -/* -// Copyright (c) Microsoft Corporation. All rights reserved. +// Copyright (c) Microsoft Corporation. All rights reserved. // Licensed under the MIT license. using System.Collections.Generic; @@ -715,5 +714,4 @@ public void DisposePendingRmwWithNoConflictTest() Assert.AreEqual(DisposeHandler.DeserializedFromDisk, functions.handlerQueue.Dequeue()); } } -} -*/ \ No newline at end of file +} \ No newline at end of file From 113e26519873692864d1164a4b964c76202cef14 Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Fri, 10 Feb 2023 14:23:04 -0800 Subject: [PATCH 23/29] try a fix --- cs/test/DisposeTests.cs | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/cs/test/DisposeTests.cs b/cs/test/DisposeTests.cs index 724026e80..0610d379d 100644 --- a/cs/test/DisposeTests.cs +++ b/cs/test/DisposeTests.cs @@ -114,7 +114,7 @@ void WaitForEvent() { // There should be one readcache entry for this test. Assert.IsTrue(new HashBucketEntry() { word = entry.Address }.ReadCache); - Assert.GreaterOrEqual(address, tester.fht.ReadCache.BeginAddress); + Assert.GreaterOrEqual(address, tester.fht.ReadCache.HeadAddress); var physicalAddress = tester.fht.readcache.GetPhysicalAddress(entry.AbsoluteAddress); ref RecordInfo recordInfo = ref tester.fht.readcache.GetInfo(physicalAddress); address = recordInfo.PreviousAddress; @@ -138,6 +138,7 @@ void WaitForEvent() } else { + Assert.GreaterOrEqual(address, tester.fht.ReadCache.HeadAddress); var physicalAddress = tester.fht.readcache.GetPhysicalAddress(entry.AbsoluteAddress); ref RecordInfo recordInfo = ref tester.fht.readcache.GetInfo(physicalAddress); while (recordInfo.PreviousAddress == address) @@ -527,7 +528,6 @@ public void PendingRead([Values] ReadCopyDestination copyDest) [Test] [Category("FasterKV")] [Category("Smoke")] - [Ignore("CI issue")] public void CopyToTailWithInitialReadCache([Values(ReadCopyDestination.ReadCache)] ReadCopyDestination copyDest) { // We use the ReadCopyDestination.ReadCache parameter so Setup() knows to set up the readcache, but @@ -584,7 +584,6 @@ public void DisposePendingRead2Threads([Values] ReadCopyDestination copyDest) [Test] [Category("FasterKV")] [Category("Smoke")] - [Ignore("CI issue")] public void DisposeCopyToTailWithInitialReadCache2Threads([Values(ReadCopyDestination.ReadCache)] ReadCopyDestination copyDest) { // We use the ReadCopyDestination.ReadCache parameter so Setup() knows to set up the readcache, but From bf3335089eef83fc8630c26d78fda0a122a0ed5d Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Fri, 10 Feb 2023 14:27:34 -0800 Subject: [PATCH 24/29] another fix --- cs/src/core/Index/FASTER/Implementation/BlockAllocate.cs | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/cs/src/core/Index/FASTER/Implementation/BlockAllocate.cs b/cs/src/core/Index/FASTER/Implementation/BlockAllocate.cs index 89862e3e1..49c96e3e3 100644 --- a/cs/src/core/Index/FASTER/Implementation/BlockAllocate.cs +++ b/cs/src/core/Index/FASTER/Implementation/BlockAllocate.cs @@ -57,6 +57,7 @@ private static bool TryBlockAllocate( [MethodImpl(MethodImplOptions.AggressiveInlining)] void SaveAllocationForRetry(ref PendingContext pendingContext, long logicalAddress, long physicalAddress, int allocatedSize) { +#if false ref var recordInfo = ref hlog.GetInfo(physicalAddress); recordInfo.SetInvalid(); // so log scan will skip it @@ -69,11 +70,15 @@ void SaveAllocationForRetry(ref PendingContext(ref PendingContext pendingContext, long minAddress, int minSize, out long newLogicalAddress, out long newPhysicalAddress) { + newLogicalAddress = newPhysicalAddress = 0; + return false; +#if false // Use an earlier allocation from a failed operation, if possible. newLogicalAddress = pendingContext.retryNewLogicalAddress; pendingContext.retryNewLogicalAddress = 0; @@ -87,6 +92,7 @@ bool GetAllocationForRetry(ref PendingContext= minSize; +#endif } } } From 8db4cde6e448b0fdedf5a7aae1a2b822cd4c50a3 Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Fri, 10 Feb 2023 15:32:16 -0800 Subject: [PATCH 25/29] try --- cs/src/core/Index/FASTER/Implementation/BlockAllocate.cs | 6 ------ 1 file changed, 6 deletions(-) diff --git a/cs/src/core/Index/FASTER/Implementation/BlockAllocate.cs b/cs/src/core/Index/FASTER/Implementation/BlockAllocate.cs index 49c96e3e3..89862e3e1 100644 --- a/cs/src/core/Index/FASTER/Implementation/BlockAllocate.cs +++ b/cs/src/core/Index/FASTER/Implementation/BlockAllocate.cs @@ -57,7 +57,6 @@ private static bool TryBlockAllocate( [MethodImpl(MethodImplOptions.AggressiveInlining)] void SaveAllocationForRetry(ref PendingContext pendingContext, long logicalAddress, long physicalAddress, int allocatedSize) { -#if false ref var recordInfo = ref hlog.GetInfo(physicalAddress); recordInfo.SetInvalid(); // so log scan will skip it @@ -70,15 +69,11 @@ void SaveAllocationForRetry(ref PendingContext(ref PendingContext pendingContext, long minAddress, int minSize, out long newLogicalAddress, out long newPhysicalAddress) { - newLogicalAddress = newPhysicalAddress = 0; - return false; -#if false // Use an earlier allocation from a failed operation, if possible. newLogicalAddress = pendingContext.retryNewLogicalAddress; pendingContext.retryNewLogicalAddress = 0; @@ -92,7 +87,6 @@ bool GetAllocationForRetry(ref PendingContext= minSize; -#endif } } } From 96466fa9c48428f87edda0b69f24bb0bcbf61aa9 Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Fri, 10 Feb 2023 16:27:36 -0800 Subject: [PATCH 26/29] code cleanup --- cs/src/core/Allocator/AllocatorBase.cs | 2 +- .../AzureStorageDevice/AzureStorageDevice.cs | 6 +- .../devices/AzureStorageDevice/BlobEntry.cs | 2 +- .../devices/AzureStorageDevice/BlobManager.cs | 28 +++- .../AzureStorageDevice/FasterTraceHelper.cs | 132 ++++++------------ .../AzureStorageDevice/IBlobManager.cs | 65 +++++++++ .../AzureStorageDevice/StorageOperations.cs | 99 +------------ 7 files changed, 135 insertions(+), 199 deletions(-) create mode 100644 cs/src/devices/AzureStorageDevice/IBlobManager.cs diff --git a/cs/src/core/Allocator/AllocatorBase.cs b/cs/src/core/Allocator/AllocatorBase.cs index 6b575256c..ac3c5aa79 100644 --- a/cs/src/core/Allocator/AllocatorBase.cs +++ b/cs/src/core/Allocator/AllocatorBase.cs @@ -1886,7 +1886,7 @@ private unsafe void AsyncGetFromDiskCallback(uint errorCode, uint numBytes, obje { if (errorCode != 0) { - logger?.LogError($"AsyncGetFromDiskCallback error: {errorCode}"); + logger?.LogError("AsyncGetFromDiskCallback error: {errorCode}", errorCode); } var result = (AsyncGetFromDiskResult>)context; diff --git a/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs b/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs index 4a6130c3e..209188a62 100644 --- a/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs +++ b/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs @@ -82,7 +82,7 @@ struct RemoveRequestInfo /// /// The maximum number of bytes this storage device can accommodate, or CAPACITY_UNSPECIFIED if there is no such limit /// Logger - public AzureStorageDevice(string connectionString, string containerName, string directoryName, string blobName, BlobManager blobManager = null, bool underLease = false, bool deleteOnClose = false, long capacity = Devices.CAPACITY_UNSPECIFIED, ILogger logger = null) + public AzureStorageDevice(string connectionString, string containerName, string directoryName, string blobName, IBlobManager blobManager = null, bool underLease = false, bool deleteOnClose = false, long capacity = Devices.CAPACITY_UNSPECIFIED, ILogger logger = null) : base($"{connectionString}/{containerName}/{directoryName}/{blobName}", PAGE_BLOB_SECTOR_SIZE, capacity) { var pageBlobAccount = BlobUtilsV12.GetServiceClients(connectionString); @@ -246,7 +246,7 @@ await this.BlobManager.PerformWithRetriesAsync( /// /// Is called on exceptions, if non-null; can be set by application /// - internal BlobManager BlobManager { get; set; } + internal IBlobManager BlobManager { get; set; } string GetSegmentBlobName(int segmentId) { @@ -487,7 +487,7 @@ public override void WriteAsync(IntPtr sourceAddress, int segmentId, ulong desti if (!this.blobs.TryGetValue(segmentId, out BlobEntry blobEntry)) { - BlobEntry entry = new BlobEntry(this); + BlobEntry entry = new(this); if (this.blobs.TryAdd(segmentId, entry)) { var pageBlob = this.pageBlobDirectory.GetPageBlobClient(this.GetSegmentBlobName(segmentId)); diff --git a/cs/src/devices/AzureStorageDevice/BlobEntry.cs b/cs/src/devices/AzureStorageDevice/BlobEntry.cs index d0f6fc05c..676a50d95 100644 --- a/cs/src/devices/AzureStorageDevice/BlobEntry.cs +++ b/cs/src/devices/AzureStorageDevice/BlobEntry.cs @@ -61,7 +61,7 @@ public async Task CreateAsync(long size, BlobUtilsV12.PageBlobClients pageBlob) } await this.azureStorageDevice.BlobManager.PerformWithRetriesAsync( - BlobManager.AsynchronousStorageReadMaxConcurrency, + this.azureStorageDevice.BlobManager.AsynchronousStorageReadMaxConcurrency, true, "PageBlobClient.CreateAsync", "CreateDevice", diff --git a/cs/src/devices/AzureStorageDevice/BlobManager.cs b/cs/src/devices/AzureStorageDevice/BlobManager.cs index 2c7bc7d82..aae76129a 100644 --- a/cs/src/devices/AzureStorageDevice/BlobManager.cs +++ b/cs/src/devices/AzureStorageDevice/BlobManager.cs @@ -14,13 +14,13 @@ namespace FASTER.devices /// /// Provides management of blobs and blob names associated with a partition, and logic for partition lease maintenance and termination. /// - public partial class BlobManager + internal partial class BlobManager : IBlobManager { readonly CancellationTokenSource shutDownOrTermination; BlobUtilsV12.BlockBlobClients leaseBlob; BlobLeaseClient leaseClient; - BlobUtilsV12.BlobDirectory leaseBlobDirectory; + readonly BlobUtilsV12.BlobDirectory leaseBlobDirectory; readonly string LeaseBlobName = "commit-lease"; readonly TimeSpan LeaseDuration = TimeSpan.FromSeconds(45); // max time the lease stays after unclean shutdown @@ -28,15 +28,25 @@ public partial class BlobManager readonly TimeSpan LeaseSafetyBuffer = TimeSpan.FromSeconds(10); // how much time we want left on the lease before issuing a protected access internal FasterTraceHelper TraceHelper { get; private set; } - internal FasterTraceHelper StorageTracer => this.TraceHelper.IsTracingAtMostDetailedLevel ? this.TraceHelper : null; + + /// + /// Storage tracer + /// + public FasterTraceHelper StorageTracer => this.TraceHelper.IsTracingAtMostDetailedLevel ? this.TraceHelper : null; /// /// Error handler for storage accesses /// public IStorageErrorHandler StorageErrorHandler { get; private set; } - internal static SemaphoreSlim AsynchronousStorageReadMaxConcurrency = new SemaphoreSlim(Math.Min(100, Environment.ProcessorCount * 10)); - internal static SemaphoreSlim AsynchronousStorageWriteMaxConcurrency = new SemaphoreSlim(Math.Min(50, Environment.ProcessorCount * 7)); + static readonly SemaphoreSlim AsynchronousStorageReadMaxConcurrencyStatic = new(Math.Min(100, Environment.ProcessorCount * 10)); + static readonly SemaphoreSlim AsynchronousStorageWriteMaxConcurrencyStatic = new(Math.Min(50, Environment.ProcessorCount * 7)); + + /// + public SemaphoreSlim AsynchronousStorageReadMaxConcurrency => AsynchronousStorageReadMaxConcurrencyStatic; + + /// + public SemaphoreSlim AsynchronousStorageWriteMaxConcurrency => AsynchronousStorageWriteMaxConcurrencyStatic; internal volatile int LeaseUsers; @@ -93,13 +103,14 @@ internal BlobManager( /// Start lease maintenance loop /// /// - public async Task StartAsync() + async Task StartAsync() { this.leaseBlob = this.leaseBlobDirectory.GetBlockBlobClient(LeaseBlobName); this.leaseClient = this.leaseBlob.WithRetries.GetBlobLeaseClient(); await this.AcquireOwnership(); } + /// public void HandleStorageError(string where, string message, string blobName, Exception e, bool isFatal, bool isWarning) { if (blobName == null) @@ -112,7 +123,9 @@ public void HandleStorageError(string where, string message, string blobName, Ex } } - // clean shutdown, wait for everything, then terminate + /// + /// clean shutdown, wait for everything, then terminate + /// public async Task StopAsync() { this.shutDownOrTermination.Cancel(); // has no effect if already cancelled @@ -120,6 +133,7 @@ public async Task StopAsync() await this.LeaseMaintenanceLoopTask; // wait for loop to terminate cleanly } + /// public ValueTask ConfirmLeaseIsGoodForAWhileAsync() { if (this.leaseTimer?.Elapsed < this.LeaseDuration - this.LeaseSafetyBuffer && !this.shutDownOrTermination.IsCancellationRequested) diff --git a/cs/src/devices/AzureStorageDevice/FasterTraceHelper.cs b/cs/src/devices/AzureStorageDevice/FasterTraceHelper.cs index 9ddf2e53c..edc683e41 100644 --- a/cs/src/devices/AzureStorageDevice/FasterTraceHelper.cs +++ b/cs/src/devices/AzureStorageDevice/FasterTraceHelper.cs @@ -6,12 +6,21 @@ namespace FASTER.devices using System; using Microsoft.Extensions.Logging; - class FasterTraceHelper + /// + /// FASTER trace helper + /// + public class FasterTraceHelper { readonly ILogger logger; readonly LogLevel logLevelLimit; readonly ILogger performanceLogger; + /// + /// Create a trace helper for FASTER + /// + /// + /// + /// public FasterTraceHelper(ILogger logger, LogLevel logLevelLimit, ILogger performanceLogger) { this.logger = logger; @@ -19,51 +28,16 @@ public FasterTraceHelper(ILogger logger, LogLevel logLevelLimit, ILogger perform this.performanceLogger = performanceLogger; } + /// + /// Is tracing at most detailed level + /// public bool IsTracingAtMostDetailedLevel => this.logLevelLimit == LogLevel.Trace; - // ----- faster storage layer events - - public void FasterStoreCreated(long inputQueuePosition, long latencyMs) - { - if (this.logLevelLimit <= LogLevel.Information) - { - this.logger?.LogInformation("Created Store, inputQueuePosition={inputQueuePosition} latencyMs={latencyMs}", inputQueuePosition, latencyMs); - } - } - public void FasterCheckpointStarted(Guid checkpointId, string details, string storeStats, long commitLogPosition, long inputQueuePosition) - { - if (this.logLevelLimit <= LogLevel.Information) - { - this.logger?.LogInformation("Started Checkpoint {checkpointId}, details={details}, storeStats={storeStats}, commitLogPosition={commitLogPosition} inputQueuePosition={inputQueuePosition}", checkpointId, details, storeStats, commitLogPosition, inputQueuePosition); - } - } - - public void FasterCheckpointPersisted(Guid checkpointId, string details, long commitLogPosition, long inputQueuePosition, long latencyMs) - { - if (this.logLevelLimit <= LogLevel.Information) - { - this.logger?.LogInformation("Persisted Checkpoint {checkpointId}, details={details}, commitLogPosition={commitLogPosition} inputQueuePosition={inputQueuePosition} latencyMs={latencyMs}", checkpointId, details, commitLogPosition, inputQueuePosition, latencyMs); - } - - if (latencyMs > 10000) - { - this.FasterPerfWarning($"Persisting the checkpoint {checkpointId} took {(double)latencyMs / 1000}s, which is excessive; checkpointId={checkpointId} commitLogPosition={commitLogPosition} inputQueuePosition={inputQueuePosition}"); - } - } - - public void FasterLogPersisted(long commitLogPosition, long numberEvents, long sizeInBytes, long latencyMs) - { - if (this.logLevelLimit <= LogLevel.Debug) - { - this.logger?.LogDebug("Persisted Log, commitLogPosition={commitLogPosition} numberEvents={numberEvents} sizeInBytes={sizeInBytes} latencyMs={latencyMs}", commitLogPosition, numberEvents, sizeInBytes, latencyMs); - } - - if (latencyMs > 10000) - { - this.FasterPerfWarning($"Persisting the log took {(double)latencyMs / 1000}s, which is excessive; commitLogPosition={commitLogPosition} numberEvents={numberEvents} sizeInBytes={sizeInBytes}"); - } - } + /// + /// Perf warning + /// + /// public void FasterPerfWarning(string details) { if (this.logLevelLimit <= LogLevel.Warning) @@ -72,46 +46,10 @@ public void FasterPerfWarning(string details) } } - public void FasterCheckpointLoaded(long commitLogPosition, long inputQueuePosition, string storeStats, long latencyMs) - { - if (this.logLevelLimit <= LogLevel.Information) - { - this.logger?.LogInformation("Loaded Checkpoint, commitLogPosition={commitLogPosition} inputQueuePosition={inputQueuePosition} storeStats={storeStats} latencyMs={latencyMs}", commitLogPosition, inputQueuePosition, storeStats, latencyMs); - } - } - - public void FasterLogReplayed(long commitLogPosition, long inputQueuePosition, long numberEvents, long sizeInBytes, string storeStats, long latencyMs) - { - if (this.logLevelLimit <= LogLevel.Information) - { - this.logger?.LogInformation("Replayed CommitLog, commitLogPosition={commitLogPosition} inputQueuePosition={inputQueuePosition} numberEvents={numberEvents} sizeInBytes={sizeInBytes} storeStats={storeStats} latencyMs={latencyMs}", commitLogPosition, inputQueuePosition, numberEvents, sizeInBytes, storeStats, latencyMs); - } - } - - public void FasterStorageError(string context, Exception exception) - { - if (this.logLevelLimit <= LogLevel.Error) - { - this.logger?.LogError("!!! Faster Storage Error : {context} : {exception}", context, exception); - } - } - - public void FasterCacheSizeMeasured(int numPages, long numRecords, long sizeInBytes, long gcMemory, long processMemory, long discrepancy, double elapsedMs) - { - if (this.logLevelLimit <= LogLevel.Information) - { - this.logger?.LogInformation("Measured CacheSize numPages={numPages} numRecords={numRecords} sizeInBytes={sizeInBytes} gcMemory={gcMemory} processMemory={processMemory} discrepancy={discrepancy} elapsedMs={elapsedMs:F2}", numPages, numRecords, sizeInBytes, gcMemory, processMemory, discrepancy, elapsedMs); - } - } - - public void FasterProgress(string details) - { - if (this.logLevelLimit <= LogLevel.Debug) - { - this.logger?.LogDebug("{details}", details); - } - } - + /// + /// Storage progress + /// + /// public void FasterStorageProgress(string details) { if (this.logLevelLimit <= LogLevel.Trace) @@ -120,6 +58,9 @@ public void FasterStorageProgress(string details) } } + /// + /// Azure storage access completed + /// public void FasterAzureStorageAccessCompleted(string intent, long size, string operation, string target, double latency, int attempt) { if (this.logLevelLimit <= LogLevel.Debug) @@ -129,18 +70,13 @@ public void FasterAzureStorageAccessCompleted(string intent, long size, string o } } - public enum CompactionProgress { Skipped, Started, Completed }; - - public void FasterCompactionProgress(CompactionProgress progress, string operation, long begin, long safeReadOnly, long tail, long minimalSize, long compactionAreaSize, double elapsedMs) - { - if (this.logLevelLimit <= LogLevel.Information) - { - this.logger?.LogInformation("Compaction {progress} operation={operation} begin={begin} safeReadOnly={safeReadOnly} tail={tail} minimalSize={minimalSize} compactionAreaSize={compactionAreaSize} elapsedMs={elapsedMs}", progress, operation, begin, safeReadOnly, tail, minimalSize, compactionAreaSize, elapsedMs); - } - } // ----- lease management events + + /// + /// Lease acquired + /// public void LeaseAcquired() { if (this.logLevelLimit <= LogLevel.Information) @@ -149,6 +85,9 @@ public void LeaseAcquired() } } + /// + /// Lease renewed + /// public void LeaseRenewed(double elapsedSeconds, double timing) { if (this.logLevelLimit <= LogLevel.Debug) @@ -157,6 +96,9 @@ public void LeaseRenewed(double elapsedSeconds, double timing) } } + /// + /// Lease released + /// public void LeaseReleased(double elapsedSeconds) { if (this.logLevelLimit <= LogLevel.Information) @@ -165,6 +107,9 @@ public void LeaseReleased(double elapsedSeconds) } } + /// + /// Lease lost + /// public void LeaseLost(double elapsedSeconds, string operation) { if (this.logLevelLimit <= LogLevel.Warning) @@ -173,6 +118,9 @@ public void LeaseLost(double elapsedSeconds, string operation) } } + /// + /// Lease progress + /// public void LeaseProgress(string operation) { if (this.logLevelLimit <= LogLevel.Debug) diff --git a/cs/src/devices/AzureStorageDevice/IBlobManager.cs b/cs/src/devices/AzureStorageDevice/IBlobManager.cs new file mode 100644 index 000000000..8dd2545ef --- /dev/null +++ b/cs/src/devices/AzureStorageDevice/IBlobManager.cs @@ -0,0 +1,65 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT License. + +using System.Threading.Tasks; +using System.Threading; +using System; + +namespace FASTER.devices +{ + /// + /// Blob manager interface + /// + public interface IBlobManager + { + /// + /// Stop tasks associated with blob manager + /// + Task StopAsync(); + + /// + /// Storage error handler + /// + IStorageErrorHandler StorageErrorHandler { get; } + + /// + /// Storage tracer + /// + FasterTraceHelper StorageTracer { get; } + + /// + /// Handle storage error + /// + void HandleStorageError(string where, string message, string blobName, Exception e, bool isFatal, bool isWarning); + + /// + /// Read concurrency + /// + SemaphoreSlim AsynchronousStorageReadMaxConcurrency { get; } + + /// + /// Write concurrency + /// + SemaphoreSlim AsynchronousStorageWriteMaxConcurrency { get; } + + /// + /// Perform operation with retries + /// + Task PerformWithRetriesAsync( + SemaphoreSlim semaphore, + bool requireLease, + string name, + string intent, + string data, + string target, + int expectedLatencyBound, + bool isCritical, + Func> operationAsync, + Func readETagAsync = null); + + /// + /// Confirm lease is good for a while + /// + ValueTask ConfirmLeaseIsGoodForAWhileAsync(); + } +} \ No newline at end of file diff --git a/cs/src/devices/AzureStorageDevice/StorageOperations.cs b/cs/src/devices/AzureStorageDevice/StorageOperations.cs index 0324eef39..aaa06fd41 100644 --- a/cs/src/devices/AzureStorageDevice/StorageOperations.cs +++ b/cs/src/devices/AzureStorageDevice/StorageOperations.cs @@ -8,8 +8,9 @@ namespace FASTER.devices using System.Threading; using System.Threading.Tasks; - public partial class BlobManager + internal partial class BlobManager : IBlobManager { + /// public async Task PerformWithRetriesAsync( SemaphoreSlim semaphore, bool requireLease, @@ -29,7 +30,7 @@ public async Task PerformWithRetriesAsync( await semaphore.WaitAsync(); } - Stopwatch stopwatch = new Stopwatch(); + Stopwatch stopwatch = new(); int numAttempts = 0; bool mustReadETagFirst = false; @@ -115,99 +116,7 @@ public async Task PerformWithRetriesAsync( } finally { - if (semaphore != null) - { - semaphore.Release(); - } - } - } - - public void PerformWithRetries( - bool requireLease, - string name, - string intent, - string data, - string target, - int expectedLatencyBound, - bool isCritical, - Func operation) - { - Stopwatch stopwatch = new Stopwatch(); - int numAttempts = 0; - - while (true) // retry loop - { - numAttempts++; - try - { - if (requireLease) - { - Interlocked.Increment(ref this.LeaseUsers); - this.ConfirmLeaseIsGoodForAWhile(); - } - - this.StorageErrorHandler.Token.ThrowIfCancellationRequested(); - - this.StorageTracer?.FasterStorageProgress($"storage operation {name} ({intent}) started attempt {numAttempts}; target={target} {data}"); - stopwatch.Restart(); - - (long size, bool completed) = operation(numAttempts); - - if (!completed) - { - continue; - } - - stopwatch.Stop(); - this.StorageTracer?.FasterStorageProgress($"storage operation {name} ({intent}) succeeded on attempt {numAttempts}; target={target} latencyMs={stopwatch.Elapsed.TotalMilliseconds:F1} size={size} {data} "); - - this.TraceHelper.FasterAzureStorageAccessCompleted(intent, size, name, target, stopwatch.Elapsed.TotalMilliseconds, numAttempts); - - if (stopwatch.ElapsedMilliseconds > expectedLatencyBound) - { - this.TraceHelper.FasterPerfWarning($"storage operation {name} ({intent}) took {stopwatch.Elapsed.TotalSeconds:F1}s on attempt {numAttempts}, which is excessive; {data}"); - } - - return; - } - catch(Exception e) when (this.StorageErrorHandler.IsTerminated) - { - string message = $"storage operation {name} ({intent}) was canceled"; - this.StorageTracer?.FasterStorageProgress(message); - throw new OperationCanceledException(message, e); - } - catch (Exception e) when (numAttempts < BlobManager.MaxRetries && BlobUtils.IsTransientStorageError(e)) - { - stopwatch.Stop(); - if (BlobUtils.IsTimeout(e)) - { - this.TraceHelper.FasterPerfWarning($"storage operation {name} ({intent}) timed out on attempt {numAttempts} after {stopwatch.Elapsed.TotalSeconds:F1}s, retrying now; target={target} {data}"); - } - else - { - TimeSpan nextRetryIn = BlobManager.GetDelayBetweenRetries(numAttempts); - this.HandleStorageError(name, $"storage operation {name} ({intent}) failed transiently on attempt {numAttempts}, retry in {nextRetryIn}s", target, e, false, true); - Thread.Sleep(nextRetryIn); - } - continue; - } - catch (Azure.RequestFailedException ex) when (BlobUtilsV12.PreconditionFailed(ex)) - { - // precondition failed, which indicates we are observing a rare partition race - this.HandleStorageError(name, $"storage operation {name} ({intent}) failed precondition on attempt {numAttempts}", target, ex, true, true); - } - catch (Exception exception) - { - this.HandleStorageError(name, $"storage operation {name} ({intent}) failed on attempt {numAttempts}", target, exception, isCritical, this.StorageErrorHandler.IsTerminated); - throw; - } - finally - { - if (requireLease) - { - Interlocked.Decrement(ref this.LeaseUsers); - } - } + semaphore?.Release(); } } } From cf8a5bef4686b99af29e371aeda3c4fb818b03c7 Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Sat, 11 Feb 2023 11:21:53 -0800 Subject: [PATCH 27/29] merge --- cs/samples/MemOnlyCache/Program.cs | 17 +++++++---------- cs/src/core/Allocator/AllocatorBase.cs | 10 ++++++---- cs/src/core/Index/FASTER/LogAccessor.cs | 4 +++- 3 files changed, 16 insertions(+), 15 deletions(-) diff --git a/cs/samples/MemOnlyCache/Program.cs b/cs/samples/MemOnlyCache/Program.cs index 317ec2a96..3453b78fc 100644 --- a/cs/samples/MemOnlyCache/Program.cs +++ b/cs/samples/MemOnlyCache/Program.cs @@ -45,10 +45,10 @@ class Program const string PageSizeBitsArg = "--pagesizebits"; /// - /// Average collisions + /// Hashtable size bits /// - static int TagChainLength = 1; // No collisions - const string TagChainLengthArg = "--tagchainlen"; + static int HashSizeBits = 20; // Default is 'no collisions' + const string HashSizeBitsArg = "--hashsizebits"; /// /// Number of threads accessing FASTER instances @@ -155,7 +155,7 @@ private static bool Usage() Console.WriteLine($" {MaxValueSizeArg} #: Max value size; we choose actual size randomly. Default = {MaxValueSize}"); Console.WriteLine($" {MemorySizeBitsArg} #: In-memory size of the log, in bits. Default = {MemorySizeBits}"); Console.WriteLine($" {PageSizeBitsArg} #: Page size, in bits. Default = {PageSizeBits}"); - Console.WriteLine($" {TagChainLengthArg} #: Average length of the hash collision chain for each tag. Default = {TagChainLength}"); + Console.WriteLine($" {HashSizeBitsArg} #: Number of bits in the hash table (recordSize is 24, so '{nameof(MemorySizeBitsArg)}' - '{nameof(HashSizeBitsArg)}' - 5, if positive, is a rough log2 of average tag chain length). Default = {HashSizeBits}"); Console.WriteLine($" {OpPercentArg} #,#,#,#: Percentage of [(r)eads,r(m)ws,(u)pserts,(d)eletes] (summing to 0 or 100) operations in incoming workload requests. Default = {ReadPercent},{RmwPercent},{UpsertPercent},{DeletePercent}"); Console.WriteLine($" {NoReadCTTArg}: Turn off (true) or allow (false) copying of reads from the Immutable region of the log to the tail of log. Default = {!UseReadCTT}"); Console.WriteLine($" {UseReadCacheArg}: Whether to use the ReadCache. Default = {UseReadCache}"); @@ -235,9 +235,9 @@ static bool GetArgs(string[] args) PageSizeBits = int.Parse(val); continue; } - if (arg == TagChainLengthArg) + if (arg == HashSizeBitsArg) { - TagChainLength = int.Parse(val); + HashSizeBits = int.Parse(val); continue; } if (arg == NumThreadsArg) @@ -350,10 +350,7 @@ static void Main(string[] args) const int recordSize = 24; int numRecords = (int)(Math.Pow(2, logSettings.MemorySizeBits) / recordSize); - // Set hash table size targeting 1 record per bucket - var numBucketBits = (int)Math.Ceiling(Math.Log2(numRecords)) / TagChainLength; - - h = new FasterKV(1L << numBucketBits, logSettings, serializerSettings: serializerSettings, comparer: new CacheKey()); + h = new FasterKV(1L << HashSizeBits, logSettings, serializerSettings: serializerSettings, comparer: new CacheKey()); sizeTracker = new CacheSizeTracker(h, targetSize); // Initially populate store diff --git a/cs/src/core/Allocator/AllocatorBase.cs b/cs/src/core/Allocator/AllocatorBase.cs index ac3c5aa79..b5768ffe2 100644 --- a/cs/src/core/Allocator/AllocatorBase.cs +++ b/cs/src/core/Allocator/AllocatorBase.cs @@ -930,6 +930,7 @@ public int EmptyPageCount emptyPageCount = value; headOffsetLagSize -= emptyPageCount; + // Lag addresses are the number of pages "behind" TailPageOffset (the tail in the circular buffer). ReadOnlyLagAddress = (long)(LogMutableFraction * headOffsetLagSize) << LogPageSizeBits; HeadOffsetLagAddress = (long)headOffsetLagSize << LogPageSizeBits; } @@ -942,6 +943,7 @@ public int EmptyPageCount if (!prot) epoch.Resume(); try { + // These shifts adjust via application of the lag addresses. var _tailAddress = GetTailAddress(); PageAlignedShiftReadOnlyAddress(_tailAddress); PageAlignedShiftHeadAddress(_tailAddress); @@ -1329,10 +1331,6 @@ private void OnPagesClosedWorker() long closeStartAddress = ClosedUntilAddress; long closeEndAddress = OngoingCloseUntilAddress; - // If we are using a null storage device, we must also shift BeginAddress - if (IsNullDevice) - Utility.MonotonicUpdate(ref BeginAddress, closeEndAddress, out _); - if (ReadCache) EvictCallback(closeStartAddress, closeEndAddress); @@ -1346,6 +1344,10 @@ private void OnPagesClosedWorker() if (OnEvictionObserver is not null) MemoryPageScan(start, end, OnEvictionObserver); + // If we are using a null storage device, we must also shift BeginAddress + if (IsNullDevice) + Utility.MonotonicUpdate(ref BeginAddress, end, out _); + // If the end of the closing range is at the end of the page, free the page if (end == closePageAddress + PageSize) FreePage((int)(closePageAddress >> LogPageSizeBits)); diff --git a/cs/src/core/Index/FASTER/LogAccessor.cs b/cs/src/core/Index/FASTER/LogAccessor.cs index db20e1a11..044df2764 100644 --- a/cs/src/core/Index/FASTER/LogAccessor.cs +++ b/cs/src/core/Index/FASTER/LogAccessor.cs @@ -97,7 +97,9 @@ public void SetEmptyPageCount(int pageCount, bool wait = false) public long MemorySizeBytes => ((long)(allocator.AllocatedPageCount + allocator.OverflowPageCount)) << allocator.LogPageSizeBits; /// - /// Whether we have allocated exactly the requested number of pages on the log (based on BufferSize and EmptyPageCount) + /// Whether we have allocated exactly the requested number of pages on the log (based on BufferSize and EmptyPageCount). + /// This can take some time between increasing EmptyPageCount and the actual page eviction (decrementing AllocatedPageCount), + /// or between decreasing EmptyPageCount and the time the page is allocated (increasing AllocatedPageCount). /// public bool PageAllocationStabilized() => allocator.AllocatedPageCount == allocator.BufferSize - allocator.EmptyPageCount + 1; From 32d690195131dfcac8e91814f25588921a086ea1 Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Mon, 13 Feb 2023 09:49:58 -0800 Subject: [PATCH 28/29] remove poor tests --- cs/test/DisposeTests.cs | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/cs/test/DisposeTests.cs b/cs/test/DisposeTests.cs index 0610d379d..bea0fd7d4 100644 --- a/cs/test/DisposeTests.cs +++ b/cs/test/DisposeTests.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. All rights reserved. // Licensed under the MIT license. +#if false + using System.Collections.Generic; using System.Threading; using System.Threading.Tasks; @@ -713,4 +715,5 @@ public void DisposePendingRmwWithNoConflictTest() Assert.AreEqual(DisposeHandler.DeserializedFromDisk, functions.handlerQueue.Dequeue()); } } -} \ No newline at end of file +} +#endif \ No newline at end of file From da1722dbcbda6eebee0e289cb77d970ace37b9d7 Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Mon, 13 Feb 2023 09:53:23 -0800 Subject: [PATCH 29/29] update nuspec --- .../AzureStorageDevice/FASTER.devices.AzureStorageDevice.nuspec | 1 + 1 file changed, 1 insertion(+) diff --git a/cs/src/devices/AzureStorageDevice/FASTER.devices.AzureStorageDevice.nuspec b/cs/src/devices/AzureStorageDevice/FASTER.devices.AzureStorageDevice.nuspec index f5d4092ba..f02256afd 100644 --- a/cs/src/devices/AzureStorageDevice/FASTER.devices.AzureStorageDevice.nuspec +++ b/cs/src/devices/AzureStorageDevice/FASTER.devices.AzureStorageDevice.nuspec @@ -22,6 +22,7 @@ +