From 241e99912059bb58de9c12047d59ec757e5213b7 Mon Sep 17 00:00:00 2001 From: Lance Collins Date: Mon, 20 Jun 2022 20:17:55 +0000 Subject: [PATCH] Merged PR 666481: Add sst file creation to BlobContentLocationRegistry. Add sst file creation to BlobContentLocationRegistry. --- .../Distributed/Blob/BlobFolderStorage.cs | 83 +- .../Distributed/Blob/BlobWrapper.cs | 66 +- .../MetadataService/IRoleObserver.cs | 4 +- .../ObservableMasterElectionMechanism.cs | 8 +- .../ResilientGlobalCacheService.cs | 25 +- .../RocksDbContentMetadataDatabase.cs | 58 +- .../MetadataService/RocksDbOperations.cs | 23 +- .../AzureBlobStorageCheckpointRegistry.cs | 2 +- ...AzureBlobStorageMasterElectionMechanism.cs | 4 +- ...bContentLocationRegistry.DatabaseWriter.cs | 225 +++++ .../NuCache/BlobContentLocationRegistry.cs | 834 ++++++++++++++---- .../Distributed/NuCache/CheckpointManager.cs | 17 +- .../ClusterStateManager.cs | 6 +- .../Distributed/NuCache/ContentListing.cs | 126 ++- .../NuCache/IMasterElectionMechanism.cs | 6 +- .../NuCache/MachineContentEntry.cs | 4 +- .../NuCache/NuCacheCollectionUtilities.cs | 5 +- .../Distributed/NuCache/PartitionId.cs | 84 ++ .../TransitioningContentLocationStore.cs | 2 +- .../Services/ContentLocationStoreServices.cs | 12 +- .../DistributedContentStoreServices.cs | 5 +- .../Stores/DistributedContentStore.cs | 47 +- .../Distributed/Utilities/JsonUtilities.cs | 51 +- ...BlobStorageMasterElectionMechanismTests.cs | 12 +- .../BlobContentLocationRegistryTests.cs | 164 +++- .../ContentLocation/BlobFolderStorageTests.cs | 31 + ...LocationRegistryDistributedContentTests.cs | 113 +++ .../ContentLocation/ContentListingTests.cs | 18 + ...entMetadataStoreDistributedContentTests.cs | 4 +- ...calLocationStoreDistributedContentTests.cs | 46 +- ...ocationStoreDistributedContentTestsBase.cs | 79 ++ .../ContentLocation/NuCache/ShortHashTests.cs | 21 + .../ResilientContentMetadataServiceTests.cs | 8 + .../Sessions/DistributedContentTests.cs | 5 + .../Cache/ContentStore/Hashing/ContentHash.cs | 21 +- .../Cache/ContentStore/Hashing/ShortHash.cs | 7 +- .../Interfaces/Results/ResultsExtensions.cs | 4 +- .../Hashing/ContentHashTests.cs | 132 +-- .../Extensions/EnumerableExtensions.cs | 13 +- .../Extensions/EnumerableExtensionsTests.cs | 6 +- .../DistributedContentSettings.cs | 22 +- .../LocalLocationStoreSettings.cs | 36 +- .../DistributedOneLevelCacheTests.cs | 1 + .../Extensions/CollectionExtensions.cs | 26 - .../TypeScript.Net/Scanning/Scanner.cs | 1 + .../Collections/CollectionUtilities.cs | 51 ++ .../FactIfSupportedAttribute.cs | 2 + .../TestUtilities.XUnit/TestRequirements.cs | 5 + Public/Src/Utilities/Utilities/AsyncOut.cs | 16 + cg/nuget/cgmanifest.json | 6 +- config.dsc | 6 +- 51 files changed, 2044 insertions(+), 509 deletions(-) create mode 100644 Public/Src/Cache/ContentStore/Distributed/NuCache/BlobContentLocationRegistry.DatabaseWriter.cs create mode 100644 Public/Src/Cache/ContentStore/Distributed/NuCache/PartitionId.cs create mode 100644 Public/Src/Cache/ContentStore/DistributedTest/ContentLocation/BlobLocationRegistryDistributedContentTests.cs diff --git a/Public/Src/Cache/ContentStore/Distributed/Blob/BlobFolderStorage.cs b/Public/Src/Cache/ContentStore/Distributed/Blob/BlobFolderStorage.cs index 0a6420ea7..6d52c609d 100644 --- a/Public/Src/Cache/ContentStore/Distributed/Blob/BlobFolderStorage.cs +++ b/Public/Src/Cache/ContentStore/Distributed/Blob/BlobFolderStorage.cs @@ -10,6 +10,7 @@ using System.Net; using System.Runtime.CompilerServices; using System.Text; using System.Text.RegularExpressions; +using System.Threading; using System.Threading.Tasks; using BuildXL.Cache.ContentStore.Distributed.Utilities; using BuildXL.Cache.ContentStore.Interfaces.Results; @@ -50,11 +51,18 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache return new BlobName(fileName, IsRelative: true); } + public static implicit operator BlobName?(string? fileName) + { + return fileName != null + ? new BlobName(fileName, IsRelative: true) + : default(BlobName?); + } + public static BlobName CreateAbsolute(string name) => new BlobName(name, false); public override string ToString() { - return Name; + return ToDisplayName(); } public string ToDisplayName() @@ -147,26 +155,38 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache timeout: _configuration.StorageInteractionTimeout); } + public BlobWrapper GetBlob(CancellationToken token, BlobName fileName) + { + var blob = GetBlockBlobReference(fileName); + return WrapBlob(token, fileName, blob); + } + + internal BlobWrapper WrapBlob(CancellationToken token, BlobName fileName, CloudBlockBlob blob) + { + return new BlobWrapper(this, blob, fileName, token, DefaultBlobStorageRequestOptions); + } + public Task UseBlockBlobAsync( OperationContext context, BlobName fileName, Func> useAsync, [CallerMemberName] string? caller = null, Func? endMessageSuffix = null, - TimeSpan? timeout = null) + TimeSpan? timeout = null, + bool isCritical = false) where T : ResultBase { return context.PerformOperationWithTimeoutAsync( Tracer, context => { - var blob = GetBlockBlobReference(fileName); - var wrapperBlob = new BlobWrapper(blob, fileName, context.Token, DefaultBlobStorageRequestOptions); + var wrapperBlob = GetBlob(context.Token, fileName); return useAsync(context, wrapperBlob); }, extraEndMessage: r => $"FileName=[{GetDisplayPath(fileName)}]{endMessageSuffix?.Invoke(r)}", traceOperationStarted: false, caller: caller, + isCritical: isCritical, timeout: timeout ?? _configuration.StorageInteractionTimeout); } @@ -496,6 +516,18 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache } } + private CloudBlobDirectory GetDirectoryReference(BlobName fileName) + { + if (fileName.IsRelative) + { + return Directory.GetDirectoryReference(fileName.Name); + } + else + { + return _container.GetDirectoryReference(fileName.Name); + } + } + public Task TouchAsync(OperationContext context, BlobName fileName) { return context.PerformOperationWithTimeoutAsync(Tracer, async context => @@ -519,13 +551,13 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache /// /// Lists blobs in folder /// - public IAsyncEnumerable ListBlobsAsync( + public IAsyncEnumerable ListBlobNamesAsync( OperationContext context, Regex? regex = null, string? subDirectoryPath = null, int? maxResults = null) { - return ListBlobsCoreAsync(context, regex, subDirectoryPath, maxResults: maxResults).Select(blob => BlobName.CreateAbsolute(blob.Name)); + return ListBlobsAsync(context, regex, subDirectoryPath, maxResults: maxResults).Select(blob => BlobName.CreateAbsolute(blob.Name)); } /// @@ -537,11 +569,11 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache Regex? regex = null, string? subDirectoryPath = null) { - var blobs = await ListBlobsCoreAsync( + var blobs = await ListBlobsAsync( context, regex, subDirectoryPath, - getMetadata: true, + blobListingDetails: BlobListingDetails.Metadata, maxResults: maxResults).ToListAsync(); blobs.Sort(LruCompareBlobs); @@ -578,39 +610,46 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache /// /// Lists blobs in folder /// - private async IAsyncEnumerable ListBlobsCoreAsync( + internal async IAsyncEnumerable ListBlobsAsync( OperationContext context, Regex? regex = null, - string? subDirectoryPath = null, - bool getMetadata = false, - int? maxResults = null) + BlobName? prefix = null, + BlobListingDetails blobListingDetails = BlobListingDetails.None, + int? maxResults = null, + bool listingSingleBlobSnapshots = false) { BlobContinuationToken? continuation = null; + var directory = Directory; + if (prefix != null) + { + directory = GetDirectoryReference(prefix.Value); + } + + var delimiter = directory.ServiceClient.DefaultDelimiter; + var listingPrefix = listingSingleBlobSnapshots && directory.Prefix.EndsWith(delimiter) + ? directory.Prefix.Substring(0, directory.Prefix.Length - delimiter.Length) + : directory.Prefix; + while (!context.Token.IsCancellationRequested) { - var directory = Directory; - if (subDirectoryPath != null) - { - directory = directory.GetDirectoryReference(subDirectoryPath); - } - var blobs = await context.PerformOperationWithTimeoutAsync( Tracer, async context => { - var result = await directory.ListBlobsSegmentedAsync( + var result = await _container.ListBlobsSegmentedAsync( + prefix: listingPrefix, useFlatBlobListing: true, - blobListingDetails: getMetadata ? BlobListingDetails.Metadata : BlobListingDetails.None, + blobListingDetails: blobListingDetails, maxResults: maxResults, currentToken: continuation, - options: null, + options: DefaultBlobStorageRequestOptions, operationContext: null, cancellationToken: context.Token); return Result.Success(result); }, timeout: _configuration.StorageInteractionTimeout, - extraEndMessage: r => $"ItemCount={r.GetValueOrDefault()?.Results.Count()}").ThrowIfFailureAsync(); + extraEndMessage: r => $"Prefix={directory.Prefix} ItemCount={r.GetValueOrDefault()?.Results.Count()}").ThrowIfFailureAsync(); continuation = blobs.ContinuationToken; diff --git a/Public/Src/Cache/ContentStore/Distributed/Blob/BlobWrapper.cs b/Public/Src/Cache/ContentStore/Distributed/Blob/BlobWrapper.cs index af9369efa..9b15135de 100644 --- a/Public/Src/Cache/ContentStore/Distributed/Blob/BlobWrapper.cs +++ b/Public/Src/Cache/ContentStore/Distributed/Blob/BlobWrapper.cs @@ -5,12 +5,14 @@ using System; using System.Collections.Generic; using System.Diagnostics.ContractsLight; using System.IO; +using System.Linq; using System.Runtime.CompilerServices; using System.Threading; using System.Threading.Tasks; using BuildXL.Cache.ContentStore.Hashing; using Microsoft.WindowsAzure.Storage; using Microsoft.WindowsAzure.Storage.Blob; +using OperationContext = BuildXL.Cache.ContentStore.Tracing.Internal.OperationContext; #nullable enable @@ -19,8 +21,16 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache /// /// Wrapper which passes common arguments to APIs /// - public record struct BlobWrapper(CloudBlockBlob Blob, BlobName Name, CancellationToken Token, BlobRequestOptions Options, Microsoft.WindowsAzure.Storage.OperationContext? Context = null) + public record BlobWrapper( + BlobFolderStorage Storage, + CloudBlockBlob Blob, + BlobName Name, + CancellationToken Token, + BlobRequestOptions Options, + Microsoft.WindowsAzure.Storage.OperationContext? Context = null) { + public AccessCondition? DefaultAccessCondition { get; set; } + public T? GetMetadataOrDefault(Func parse, T? defaultValue = default, [CallerMemberName] string key = null!) { T parseCore(string value) @@ -35,7 +45,7 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache } } - return Blob?.Metadata.TryGetValue(key, out var value) == true && !string.IsNullOrEmpty(value) + return Blob.Metadata.TryGetValue(key, out var value) == true && !string.IsNullOrEmpty(value) ? parseCore(value) : defaultValue; } @@ -44,7 +54,7 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache { if (value == null) { - Blob?.Metadata.Remove(key); + Blob.Metadata.Remove(key); } else if (Blob != null) { @@ -52,19 +62,24 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache } } + private AccessCondition? Apply(AccessCondition? condition) + { + return condition ?? DefaultAccessCondition; + } + internal Task AcquireLeaseAsync(TimeSpan leaseTime, AccessCondition? accessCondition = null) { - return Blob.AcquireLeaseAsync(leaseTime, proposedLeaseId: null, accessCondition, Options, Context, Token); + return Blob.AcquireLeaseAsync(leaseTime, proposedLeaseId: null, Apply(accessCondition), Options, Context, Token); } internal Task> DownloadBlockListAsync(BlockListingFilter filter, AccessCondition? accessCondition = null) { - return Blob.DownloadBlockListAsync(filter, accessCondition, Options, Context, Token); + return Blob.DownloadBlockListAsync(filter, Apply(accessCondition), Options, Context, Token); } internal Task DownloadRangeToStreamAsync(Stream stream, long? offset, long? length, AccessCondition? accessCondition = null) { - return Blob.DownloadRangeToStreamAsync(stream, offset, length, accessCondition, Options, Context, Token); + return Blob.DownloadRangeToStreamAsync(stream, offset, length, Apply(accessCondition), Options, Context, Token); } internal Task ExistsAsync() @@ -74,29 +89,56 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache internal Task FetchAttributesAsync(AccessCondition? accessCondition = null) { - return Blob.FetchAttributesAsync(accessCondition, Options, Context, Token); + return Blob.FetchAttributesAsync(Apply(accessCondition), Options, Context, Token); } internal Task PutBlockAsync(string blockId, Stream stream, AccessCondition? accessCondition = null, ContentHash? md5Hash = null) { Contract.Requires(md5Hash == null || md5Hash.Value.HashType == HashType.MD5); var contentMD5 = md5Hash == null ? null : Convert.ToBase64String(md5Hash.Value.ToHashByteArray()); - return Blob.PutBlockAsync(blockId, stream, contentMD5, accessCondition, Options, Context, Token); + return Blob.PutBlockAsync(blockId, stream, contentMD5, Apply(accessCondition), Options, Context, Token); } internal Task PutBlockListAsync(IEnumerable blockList, AccessCondition? accessCondition = null) { - return Blob.PutBlockListAsync(blockList, accessCondition, Options, Context, Token); + return Blob.PutBlockListAsync(blockList, Apply(accessCondition), Options, Context, Token); } - internal Task ReleaseLeaseAsync(AccessCondition accessCondition) + internal Task ReleaseLeaseAsync(AccessCondition leaseCondition) { - return Blob.ReleaseLeaseAsync(accessCondition, Options, Context, Token); + return Blob.ReleaseLeaseAsync(leaseCondition, Options, Context, Token); } internal Task UploadFromByteArrayAsync(ArraySegment buffer, AccessCondition? accessCondition = null) { - return Blob.UploadFromByteArrayAsync(buffer.Array, buffer.Offset, buffer.Count, accessCondition, Options, Context, Token); + return Blob.UploadFromByteArrayAsync(buffer.Array, buffer.Offset, buffer.Count, Apply(accessCondition), Options, Context, Token); + } + + internal async Task> ListSnapshotsAsync( + OperationContext context, + Func transformBlob) + { + return await Storage.ListBlobsAsync( + context, + regex: null, + prefix: Name, + listingSingleBlobSnapshots: true, + blobListingDetails: BlobListingDetails.Metadata | BlobListingDetails.Snapshots) + .Where(blob => blob.IsSnapshot) + .OfType() + .Select(blob => transformBlob(Storage.WrapBlob(context.Token, Name with { SnapshotTime = blob.SnapshotTime }, blob))) + .ToListAsync(); + } + + internal Task DeleteIfExistsAsync(DeleteSnapshotsOption option = DeleteSnapshotsOption.None, AccessCondition? accessCondition = null, CancellationToken? token = default) + { + return Blob.DeleteIfExistsAsync(option, Apply(accessCondition), Options, Context, token ?? Token); + } + + internal async Task SnapshotAsync(AccessCondition? accessCondition = null) + { + var result = await Blob.SnapshotAsync(metadata: null, Apply(accessCondition), Options, Context, Token); + return Storage.GetBlob(Token, Name with { SnapshotTime = result.SnapshotTime }); } } } \ No newline at end of file diff --git a/Public/Src/Cache/ContentStore/Distributed/MetadataService/IRoleObserver.cs b/Public/Src/Cache/ContentStore/Distributed/MetadataService/IRoleObserver.cs index eb7819991..2143372ca 100644 --- a/Public/Src/Cache/ContentStore/Distributed/MetadataService/IRoleObserver.cs +++ b/Public/Src/Cache/ContentStore/Distributed/MetadataService/IRoleObserver.cs @@ -1,7 +1,9 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT License. +using System; using System.Threading.Tasks; +using BuildXL.Cache.ContentStore.Distributed.NuCache; using BuildXL.Cache.ContentStore.Interfaces.Stores; using BuildXL.Cache.ContentStore.Tracing.Internal; @@ -12,6 +14,6 @@ namespace BuildXL.Cache.ContentStore.Distributed.MetadataService /// public interface IRoleObserver : IStartupShutdownSlim { - public Task OnRoleUpdatedAsync(OperationContext context, Role role); + Task OnRoleUpdatedAsync(OperationContext context, MasterElectionState electionState); } } diff --git a/Public/Src/Cache/ContentStore/Distributed/MetadataService/ObservableMasterElectionMechanism.cs b/Public/Src/Cache/ContentStore/Distributed/MetadataService/ObservableMasterElectionMechanism.cs index 9d7cb2278..6113e40dc 100644 --- a/Public/Src/Cache/ContentStore/Distributed/MetadataService/ObservableMasterElectionMechanism.cs +++ b/Public/Src/Cache/ContentStore/Distributed/MetadataService/ObservableMasterElectionMechanism.cs @@ -97,7 +97,7 @@ namespace BuildXL.Cache.ContentStore.Distributed.MetadataService if (result.Succeeded) { _lastGetRoleTime = _clock.UtcNow; - OnRoleUpdated(context, result.Value.Role); + OnRoleUpdated(context, result.Value); } return result; @@ -108,17 +108,17 @@ namespace BuildXL.Cache.ContentStore.Distributed.MetadataService var result = await _inner.ReleaseRoleIfNecessaryAsync(context); if (result.Succeeded) { - OnRoleUpdated(context, result.Value); + OnRoleUpdated(context, MasterElectionState.DefaultWorker with { Role = result.Value }); } return result; } - private void OnRoleUpdated(OperationContext context, Role role) + private void OnRoleUpdated(OperationContext context, MasterElectionState electionState) { if (_observer != null) { - _observer.OnRoleUpdatedAsync(context, role).FireAndForget(context); + _observer.OnRoleUpdatedAsync(context, electionState).FireAndForget(context); } } } diff --git a/Public/Src/Cache/ContentStore/Distributed/MetadataService/ResilientGlobalCacheService.cs b/Public/Src/Cache/ContentStore/Distributed/MetadataService/ResilientGlobalCacheService.cs index 1ab6beaea..51d774051 100644 --- a/Public/Src/Cache/ContentStore/Distributed/MetadataService/ResilientGlobalCacheService.cs +++ b/Public/Src/Cache/ContentStore/Distributed/MetadataService/ResilientGlobalCacheService.cs @@ -129,6 +129,8 @@ namespace BuildXL.Cache.ContentStore.Distributed.MetadataService private ActionQueue _concurrencyLimitingQueue; private readonly IClock _clock; + private readonly BlobContentLocationRegistry _registry; + protected override Tracer Tracer { get; } = new Tracer(nameof(ResilientGlobalCacheService)); internal ContentMetadataEventStream EventStream => _eventStream; @@ -198,7 +200,8 @@ namespace BuildXL.Cache.ContentStore.Distributed.MetadataService CheckpointManager checkpointManager, RocksDbContentMetadataStore store, ContentMetadataEventStream eventStream, - IClock clock = null) + IClock clock = null, + BlobContentLocationRegistry registry = null) : base(store) { _configuration = configuration; @@ -206,9 +209,11 @@ namespace BuildXL.Cache.ContentStore.Distributed.MetadataService _checkpointManager = checkpointManager; _eventStream = eventStream; _clock = clock ?? SystemClock.Instance; + _registry = registry; LinkLifetime(_eventStream); LinkLifetime(_checkpointManager); + LinkLifetime(registry); RunInBackground(nameof(CreateCheckpointLoopAsync), CreateCheckpointLoopAsync, fireAndForget: true); @@ -240,6 +245,9 @@ namespace BuildXL.Cache.ContentStore.Distributed.MetadataService { if (!ShouldRetry(out var retryReason, out var errorMessage, isShutdown: true)) { + // Stop database updates + _registry?.SetDatabaseUpdateLeaseExpiry(null); + // Stop logging _eventStream.SetIsLogging(false); @@ -254,8 +262,9 @@ namespace BuildXL.Cache.ContentStore.Distributed.MetadataService return BoolResult.Success; } - public async Task OnRoleUpdatedAsync(OperationContext context, Role role) + public async Task OnRoleUpdatedAsync(OperationContext context, MasterElectionState electionState) { + var role = electionState.Role; if (!StartupCompleted || ShutdownStarted) { return; @@ -267,11 +276,20 @@ namespace BuildXL.Cache.ContentStore.Distributed.MetadataService _lastSuccessfulHeartbeat = _clock.UtcNow; if (_role != role) { + // Stop database updates + _registry?.SetDatabaseUpdateLeaseExpiry(null); + _eventStream.SetIsLogging(false); _hasRestoredCheckpoint = false; _role = role; } + if (!ShouldRetry(out _, out _)) + { + // Notify registry that master lease is still held to ensure database is updated. + _registry?.SetDatabaseUpdateLeaseExpiry(electionState.MasterLeaseExpiryUtc); + } + if (_role == Role.Master) { // Acquire mutex to ensure we cancel the actual outstanding background restore @@ -296,6 +314,9 @@ namespace BuildXL.Cache.ContentStore.Distributed.MetadataService { _hasRestoredCheckpoint = true; _eventStream.SetIsLogging(true); + + // Resume database updates + _registry?.SetDatabaseUpdateLeaseExpiry(electionState.MasterLeaseExpiryUtc); } } finally diff --git a/Public/Src/Cache/ContentStore/Distributed/MetadataService/RocksDbContentMetadataDatabase.cs b/Public/Src/Cache/ContentStore/Distributed/MetadataService/RocksDbContentMetadataDatabase.cs index ebc8c9c7d..e2bebca97 100644 --- a/Public/Src/Cache/ContentStore/Distributed/MetadataService/RocksDbContentMetadataDatabase.cs +++ b/Public/Src/Cache/ContentStore/Distributed/MetadataService/RocksDbContentMetadataDatabase.cs @@ -9,6 +9,7 @@ using System.Diagnostics.SymbolStore; using System.Globalization; using System.IO; using System.Linq; +using System.Runtime.CompilerServices; using System.Runtime.InteropServices; using System.Text.Json; using System.Threading; @@ -250,6 +251,11 @@ namespace BuildXL.Cache.ContentStore.Distributed.MetadataService return result; } + private MergeOperator MergeContentMergeOperator { get; } = MergeOperators.CreateAssociative( + "MergeContent", + merge: RocksDbOperations.MergeLocations, + transformSingle: RocksDbOperations.ProcessSingleLocationEntry); + private bool IsStoredEpochInvalid([NotNullWhen(true)] out string? epoch) { TryGetGlobalEntry(nameof(GlobalKeys.StoredEpoch), out epoch); @@ -263,10 +269,13 @@ namespace BuildXL.Cache.ContentStore.Distributed.MetadataService yield return ( Columns.MergeContent, - MergeOperators.CreateAssociative( - "MergeContent", - merge: RocksDbOperations.MergeLocations, - transformSingle: RocksDbOperations.ProcessSingleLocationEntry) + MergeContentMergeOperator + ); + + yield return + ( + Columns.SstMergeContent, + MergeContentMergeOperator ); } @@ -690,6 +699,11 @@ namespace BuildXL.Cache.ContentStore.Distributed.MetadataService return ColumnNames[(int)columnFamily][(int)resolvedGroup]; } + internal RocksDbStore UnsafeGetStore() + { + return _keyValueStore.Use(store => store).ToResult().Value!; + } + /// /// Ingests sst files from the given paths for the column /// @@ -697,7 +711,8 @@ namespace BuildXL.Cache.ContentStore.Distributed.MetadataService { return _keyValueStore.Use(store => store.Database.IngestExternalFiles( files.Select(f => f.Path).ToArray(), - new IngestExternalFileOptions().SetMoveFiles(true), + new IngestExternalFileOptions().SetMoveFiles(true) + , store.GetColumn(NameOf(Columns.SstMergeContent)))) .ToBoolResult(); } @@ -1119,7 +1134,7 @@ namespace BuildXL.Cache.ContentStore.Distributed.MetadataService || store.TryReadValue(key, valueBuffer, NameOf(columns, GetFormerColumnGroup(columns))) >= 0; } - private bool TryDeserializeValue(RocksDbStore store, ReadOnlySpan key, Columns columns, DeserializeValue deserializer, [NotNullWhen(true)] out TResult? result) + public bool TryDeserializeValue(RocksDbStore store, ReadOnlySpan key, Columns columns, DeserializeValue deserializer, [NotNullWhen(true)] out TResult? result) { return TryDeserializeValue(store, key, NameOf(columns), deserializer, out result) || IsRotatedColumn(columns) && TryDeserializeValue(store, key, NameOf(columns, GetFormerColumnGroup(columns)), deserializer, out result); @@ -1302,6 +1317,37 @@ namespace BuildXL.Cache.ContentStore.Distributed.MetadataService }); } + public Result IterateSstMergeContentEntries(OperationContext context, Action onEntry) + { + return _keyValueStore.Use( + static (store, state) => + { + var hashKeySize = Unsafe.SizeOf(); + return store.IterateDbContent( + iterator => + { + var key = iterator.Key(); + if (key.Length != hashKeySize) + { + return; + } + + var hash = MemoryMarshal.Read(key); + RocksDbOperations.ReadMergedContentLocationEntry(iterator.Value(), out var machines, out var info); + foreach (var machine in machines) + { + var entry = new MachineContentEntry(hash, machine, info.Size!.Value, info.LatestAccessTime ?? CompactTime.Zero); + state.onEntry(entry); + } + }, + state.@this.NameOf(Columns.SstMergeContent), + startValue: (byte[]?)null, + state.context.Token); + + }, (@this: this, onEntry, context)) + .ToResult(); + } + /// public override IEnumerable> EnumerateStrongFingerprints(OperationContext context) { diff --git a/Public/Src/Cache/ContentStore/Distributed/MetadataService/RocksDbOperations.cs b/Public/Src/Cache/ContentStore/Distributed/MetadataService/RocksDbOperations.cs index 0e65996c3..29d2be0c5 100644 --- a/Public/Src/Cache/ContentStore/Distributed/MetadataService/RocksDbOperations.cs +++ b/Public/Src/Cache/ContentStore/Distributed/MetadataService/RocksDbOperations.cs @@ -9,6 +9,7 @@ using BuildXL.Cache.ContentStore.Distributed.NuCache; using BuildXL.Cache.ContentStore.Hashing; using BuildXL.Cache.ContentStore.Service; using BuildXL.Cache.ContentStore.Utils; +using BuildXL.Utilities.Collections; using BuildXL.Utilities.Serialization; using RocksDbSharp; @@ -33,17 +34,31 @@ namespace BuildXL.Cache.ContentStore.Distributed.MetadataService /// /// Deletes the range of keys in the given partition (i.e. all keys with partition as prefix). /// - public static void DeleteLocationEntryPartitionRange(this TWriter writer, byte partition) + public static void DeleteLocationEntryPartitionRange(this TWriter writer, PartitionId partition) where TWriter : IRocksDbColumnWriter { // Create a key after all keys with the given partition by taking partition // and suffixing with byte.MaxValue greater to maximum key length. // Next partition id can't be used because there is no way to represent the next for partition 255. - Span rangeEnd = stackalloc byte[ShortHash.SerializedLength + 1]; - rangeEnd[0] = partition; + Span rangeEnd = stackalloc byte[ShortHash.SerializedLength + 2]; + rangeEnd[0] = partition.EndValue; rangeEnd.Slice(1).Fill(byte.MaxValue); - writer.DeleteRange(stackalloc[] { partition }, rangeEnd); + writer.DeleteRange(stackalloc[] { partition.StartValue }, rangeEnd); + } + + /// + /// Gets a key for the partition record in the partition's range after all valid shard hash + /// keys but within the range which would be deleted by a DeleteRange operation. + /// + public static ReadOnlyArray GetPartitionRecordKey(this PartitionId partition) + { + var key = new byte[ShortHash.SerializedLength + 1]; + key[0] = partition.EndValue; + key.AsSpan().Slice(1).Fill(byte.MaxValue); + key[ShortHash.SerializedLength] = 0; + + return key; } /// diff --git a/Public/Src/Cache/ContentStore/Distributed/NuCache/AzureBlobStorageCheckpointRegistry.cs b/Public/Src/Cache/ContentStore/Distributed/NuCache/AzureBlobStorageCheckpointRegistry.cs index 01c2f8bd9..ac1b20f50 100644 --- a/Public/Src/Cache/ContentStore/Distributed/NuCache/AzureBlobStorageCheckpointRegistry.cs +++ b/Public/Src/Cache/ContentStore/Distributed/NuCache/AzureBlobStorageCheckpointRegistry.cs @@ -240,7 +240,7 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache private IAsyncEnumerable ListBlobsRecentFirstAsync(OperationContext context) { - var blobs = _storage.ListBlobsAsync(context, _blobNameRegex) + var blobs = _storage.ListBlobNamesAsync(context, _blobNameRegex) .Select(name => { // This should never fail, because ListBlobsAsync returns blobs that we know already match. diff --git a/Public/Src/Cache/ContentStore/Distributed/NuCache/AzureBlobStorageMasterElectionMechanism.cs b/Public/Src/Cache/ContentStore/Distributed/NuCache/AzureBlobStorageMasterElectionMechanism.cs index 4e2fab4e4..cf00535e8 100644 --- a/Public/Src/Cache/ContentStore/Distributed/NuCache/AzureBlobStorageMasterElectionMechanism.cs +++ b/Public/Src/Cache/ContentStore/Distributed/NuCache/AzureBlobStorageMasterElectionMechanism.cs @@ -119,7 +119,7 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache if (r.Succeeded) { // We don't know who the master is any more - _lastElection = new MasterElectionState(Master: default, Role: Role.Worker); + _lastElection = MasterElectionState.DefaultWorker; } return Result.Success(Role.Worker); @@ -236,7 +236,7 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache master = default(MachineLocation); } - return new MasterElectionState(master, master.Equals(_primaryMachineLocation) ? Role.Master : Role.Worker); + return new MasterElectionState(master, master.Equals(_primaryMachineLocation) ? Role.Master : Role.Worker, MasterLeaseExpiryUtc: lease?.LeaseExpiryTimeUtc); } private Task> UpdateRoleAsync(OperationContext context, TryUpdateLease tryUpdateLease) diff --git a/Public/Src/Cache/ContentStore/Distributed/NuCache/BlobContentLocationRegistry.DatabaseWriter.cs b/Public/Src/Cache/ContentStore/Distributed/NuCache/BlobContentLocationRegistry.DatabaseWriter.cs new file mode 100644 index 000000000..28910bcf2 --- /dev/null +++ b/Public/Src/Cache/ContentStore/Distributed/NuCache/BlobContentLocationRegistry.DatabaseWriter.cs @@ -0,0 +1,225 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT License. + +using System; +using System.Collections.Generic; +using System.Diagnostics.ContractsLight; +using System.IO; +using System.Runtime.CompilerServices; +using System.Runtime.InteropServices; +using BuildXL.Cache.ContentStore.Distributed.MetadataService; +using BuildXL.Cache.ContentStore.Hashing; +using BuildXL.Cache.ContentStore.Interfaces.Extensions; +using BuildXL.Cache.ContentStore.Interfaces.Results; +using BuildXL.Cache.ContentStore.Stores; +using BuildXL.Cache.ContentStore.Tracing.Internal; +using BuildXL.Cache.ContentStore.Utils; +using BuildXL.Utilities.Collections; +using BuildXL.Utilities.Serialization; +using RocksDbSharp; +using static BuildXL.Cache.ContentStore.Distributed.MetadataService.RocksDbOperations; + +namespace BuildXL.Cache.ContentStore.Distributed.NuCache +{ + public partial class BlobContentLocationRegistry + { + /// + /// Writes the full and diff entries based on the baseline and current content list + /// + public static BoolResult WriteSstFiles( + OperationContext context, + PartitionId partitionId, + ContentListing baseline, + ContentListing current, + IRocksDbColumnWriter fullSstWriter, + IRocksDbColumnWriter diffSstWriter, + PartitionUpdateStatistics counters) + { + // Computes and writes per hash entries of the current listing + // and returns the enumerator so subsequent step can consume the + // entries + var fullEntries = EnumerateAndWriteFullSstFile( + fullSstWriter, + partitionId, + current.EnumerateEntries()); + + // Compute the differences + var diffEntries = baseline.EnumerateChanges( + current.EnumerateEntries(), + counters.DiffStats, + synthesizeUniqueHashEntries: true); + + // Write the differences + WriteDiffSstFile( + diffSstWriter, + diffEntries: ComputeDatabaseEntries(diffEntries), + currentEntries: fullEntries, + baselineEntries: ComputeDatabaseEntries(baseline.EnumerateEntries()), + counters); + + return BoolResult.Success; + } + + /// + /// Computes and writes per hash entries of the current listing + /// and returns the enumerator so subsequent step can consume the + /// entries + /// + private static IEnumerator EnumerateAndWriteFullSstFile( + IRocksDbColumnWriter writer, + PartitionId partitionId, + IEnumerable entries) + { + writer.DeleteLocationEntryPartitionRange(partitionId); + + foreach (var entry in ComputeDatabaseEntries(entries)) + { + entry.Write(writer, merge: false); + yield return entry; + } + } + + /// + /// Writes the diff sst file + /// + private static void WriteDiffSstFile( + IRocksDbColumnWriter writer, + IEnumerable diffEntries, + IEnumerator currentEntries, + IEnumerable baselineEntries, + PartitionUpdateStatistics counters) + { + // Enumerates the diff entries and writes deletions and returns the existing entries + IEnumerable enumerateDiffEntriesAndWriteDeletions() + { + foreach (var item in NuCacheCollectionUtilities.DistinctMergeSorted(diffEntries.GetEnumerator(), currentEntries, e => e.Hash, e => e.Hash)) + { + var diffEntry = item.left; + var fullEntry = item.right; + if (item.mode == MergeMode.LeftOnly) + { + // Entry no long exists in current entries. + // Just put a delete + writer.DeleteLocationEntry(item.left.Hash); + counters.DiffStats.Deletes.Add(new MachineContentEntry(item.left.Hash, default, item.left.Info.Size ?? 0, default)); + } + else if (item.mode == MergeMode.Both) + { + yield return diffEntry; + } + else + { + // RightOnly case not handled because this is only concerned with entries which appear in the diff. + // This case should probably not happen since entries are synthezized into diff for every unique hash in the + // current entries. + } + } + } + + // Enumerates the existing diff entries and writes minimal data to database. (i.e. size, creation time, and last access time + // are excluded respectively if they are present in the base entry) + foreach (var item in NuCacheCollectionUtilities.DistinctMergeSorted(enumerateDiffEntriesAndWriteDeletions(), baselineEntries, e => e.Hash, e => e.Hash)) + { + // RightOnly case not handled because this is only concerned with entries which appear in the diff. + if (item.mode != MergeMode.RightOnly) + { + var diffEntry = item.left; + if (item.mode == MergeMode.Both) + { + var baselineEntry = item.right; + diffEntry.Info = MachineContentInfo.Diff(baseline: baselineEntry.Info, current: diffEntry.Info); + } + + if (diffEntry.Entries.Count == 0) + { + // This is a touch-only entry + // Don't need to write size for touch-only entry + diffEntry.Info.Size = null; + + if (diffEntry.Info.LatestAccessTime == null) + { + // Don't need to write touch-only entry if last access time is not set + continue; + } + } + + diffEntry.Write(writer, merge: true); + } + } + } + + private static IEnumerable ComputeDatabaseEntries(IEnumerable entries) + { + var group = new LocationEntryBuilder(); + foreach (var entry in entries) + { + if (group.HasInfo && group.Hash != entry.ShardHash) + { + yield return group; + group.Reset(); + } + + group.Add(entry); + } + + if (group.HasInfo) + { + yield return group; + } + } + + /// + /// Accumulates values into a per-hash entry + /// + private class LocationEntryBuilder + { + public bool HasInfo { get; set; } + public Buffer Entries = new(); + public ShardHash Hash { get; set; } = default; + public MachineContentInfo Info = MachineContentInfo.Default; + + public void Write(IRocksDbColumnWriter writer, bool merge) + { + MergeOrPutContentLocationEntry(writer, + Hash, + Entries.ItemSpan, + static l => l, + Info, + merge); + } + + public void Add(MachineContentEntry entry) + { + HasInfo = true; + if (Entries.Count == 0) + { + Hash = entry.ShardHash; + } + + if (!entry.Location.IsRemove) + { + Info.Merge(entry); + } + + // Need to check for valid location in this case + // because entry may just be a synthetic entry + // which is added for the purpose of touches + if (entry.Location.IsValid) + { + Entries.Add(entry.Location); + } + } + + /// + /// Reset the entry + /// + public void Reset() + { + HasInfo = false; + Hash = default; + Entries.Reset(); + Info = MachineContentInfo.Default; + } + } + } +} diff --git a/Public/Src/Cache/ContentStore/Distributed/NuCache/BlobContentLocationRegistry.cs b/Public/Src/Cache/ContentStore/Distributed/NuCache/BlobContentLocationRegistry.cs index fcae3b627..cb2d9c44b 100644 --- a/Public/Src/Cache/ContentStore/Distributed/NuCache/BlobContentLocationRegistry.cs +++ b/Public/Src/Cache/ContentStore/Distributed/NuCache/BlobContentLocationRegistry.cs @@ -2,6 +2,7 @@ // Licensed under the MIT License. using System; +using System.Collections.Concurrent; using System.Collections.Generic; using System.Diagnostics.ContractsLight; using System.Globalization; @@ -9,10 +10,17 @@ using System.IO; using System.Linq; using System.Net; using System.Runtime.CompilerServices; +using System.Text.Json; +using System.Text.Json.Serialization; +using System.Threading; using System.Threading.Tasks; +using BuildXL.Cache.ContentStore.Distributed.MetadataService; using BuildXL.Cache.ContentStore.Distributed.Utilities; +using BuildXL.Cache.ContentStore.Extensions; +using BuildXL.Cache.ContentStore.FileSystem; using BuildXL.Cache.ContentStore.Hashing; using BuildXL.Cache.ContentStore.Interfaces.Extensions; +using BuildXL.Cache.ContentStore.Interfaces.FileSystem; using BuildXL.Cache.ContentStore.Interfaces.Results; using BuildXL.Cache.ContentStore.Interfaces.Secrets; using BuildXL.Cache.ContentStore.Interfaces.Stores; @@ -22,11 +30,17 @@ using BuildXL.Cache.ContentStore.Stores; using BuildXL.Cache.ContentStore.Tracing; using BuildXL.Cache.ContentStore.Utils; using BuildXL.Cache.Host.Configuration; +using BuildXL.Utilities; +using BuildXL.Utilities.Collections; +using BuildXL.Utilities.ParallelAlgorithms; +using BuildXL.Utilities.Serialization; using BuildXL.Utilities.Tasks; using Microsoft.WindowsAzure.Storage; using Microsoft.WindowsAzure.Storage.Blob; +using RocksDbSharp; using static BuildXL.Cache.ContentStore.Distributed.NuCache.BlobFolderStorage; using static BuildXL.Cache.ContentStore.Utils.DateTimeUtilities; +using AbsolutePath = BuildXL.Cache.ContentStore.Interfaces.FileSystem.AbsolutePath; using EnumerableExtensions = BuildXL.Cache.ContentStore.Extensions.EnumerableExtensions; using OperationContext = BuildXL.Cache.ContentStore.Tracing.Internal.OperationContext; @@ -36,6 +50,7 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache { public record BlobContentLocationRegistryConfiguration : BlobContentLocationRegistrySettings, IBlobFolderStorageConfiguration { + [JsonIgnore] public AzureBlobStorageCredentials? Credentials { get; set; } public RetryPolicyConfiguration RetryPolicy { get; set; } = BlobFolderStorage.DefaultRetryPolicy; @@ -58,14 +73,13 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache /// Partition submission blob - contains listing of all machine content in the partition (one block per machine) /// Partition output blob - contains blocks for each processed output based on the partition submission blob. /// FullListing - full sorted listing of content for all machines. (Basically just the sorted partition input sorted by hash, then machine). - /// DiffListing - listing of adds/removes from base snapshot of FullListing blob - /// FullSst - same information as FullListing but stored as entries in RocksDb sst file (not yet implemented) - /// DiffSst - same information as DiffListing but stored as entries in RocksDb sst file (not yet implemented) + /// FullSst - same information as FullListing but stored as entries in RocksDb sst file + /// DiffSst - listing of adds/removes from prior snapshot's content as RocksDb merge operators /// /// Content is partitioned based on first byte of hash into 256 partitions. /// Machines push their content for a partition as a block in the partition input blob (this can be done in parallel). /// - public class BlobContentLocationRegistry : StartupShutdownComponentBase + public partial class BlobContentLocationRegistry : StartupShutdownComponentBase { protected override Tracer Tracer { get; } = new Tracer(nameof(BlobContentLocationRegistry)); @@ -84,17 +98,38 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache private ClusterState ClusterState => _clusterStateManager.ClusterState; private MachineRecord LocalMachineRecord { get; set; } + private RocksDbContentMetadataDatabase Database { get; set; } + + private readonly ReadOnlyArray _partitionIds; + + private readonly IAbsFileSystem _fileSystem = PassThroughFileSystem.Default; + + /// + /// Gets when the master lease will expire for the current machine. Updates to database are only + /// allowed when the master lease is valid. + /// + private DateTime _databaseUpdateLeaseExpiry = DateTime.MinValue; + + /// + /// Action queue for executing concurrent storage operations + /// Namely used for download sst files in parallel. + /// + private readonly ActionQueue _actionQueue; + /// /// Retry policy which retries PutBlock operations which fail because the lease is taken. /// Lease is only taken for short periods of time to commit the block list /// private readonly IRetryPolicy _putBlockRetryPolicy; + private bool ShouldUpdateDatabase => Configuration.UpdateDatabase && _databaseUpdateLeaseExpiry > _clock.UtcNow; + public BlobContentLocationRegistry( BlobContentLocationRegistryConfiguration configuration, ClusterStateManager clusterStateManager, - ILocalContentStore? localContentStore, MachineLocation primaryMachineLocation, + RocksDbContentMetadataDatabase database, + ILocalContentStore? localContentStore = null, IClock? clock = null) { Contract.RequiresNotNull(configuration.Credentials); @@ -102,14 +137,21 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache _localContentStore = localContentStore; _clusterStateManager = clusterStateManager; _clock = clock ?? SystemClock.Instance; - + Database = database; Storage = new BlobFolderStorage(Tracer, configuration); + _partitionIds = PartitionId.GetPartitions(configuration.PartitionCount); + + if (localContentStore != null) + { + SetLocalContentStore(localContentStore); + } LinkLifetime(Storage); + LinkLifetime(database); LinkLifetime(clusterStateManager); - LinkLifetime(localContentStore as IStartupShutdownSlim); _putBlockRetryPolicy = RetryPolicyFactory.GetExponentialPolicy(shouldRetry: ex => IsPreconditionFailedError(ex)); + _actionQueue = new ActionQueue(Configuration.MaxDegreeOfParallelism, useChannel: true); if (configuration.UpdateInBackground) { @@ -130,6 +172,11 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache LinkLifetime(localContentStore as IStartupShutdownSlim); } + internal void SetDatabaseUpdateLeaseExpiry(DateTime? expiry) + { + _databaseUpdateLeaseExpiry = expiry ?? DateTime.MinValue; + } + /// /// Background loop which submitted machine content and processes partition content blobs /// @@ -145,7 +192,7 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache return BoolResult.Success; } - internal Task UpdatePartitionsAsync(OperationContext context, Func? excludePartition = null) + internal Task UpdatePartitionsAsync(OperationContext context, Func? excludePartition = null) { LocalMachineRecord = _clusterStateManager.ClusterState.PrimaryMachineRecord; if (LocalMachineRecord == null || _localContentStore == null) @@ -153,36 +200,288 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache return BoolResult.SuccessTask; } + int databaseUpdates = 0; + bool updatedDatabaseManifest = false; + context = context.CreateNested(Tracer.Name); return context.PerformOperationAsync( Tracer, async () => { var sortedEntries = await GetLocalSortedEntriesAsync(context); - var partitions = sortedEntries.GetPartitionSlices().ToList(); + var partitions = sortedEntries.GetPartitionSlices(_partitionIds).ToList(); // Randomly enumerate partitions so machines do not all operation on the same partition concurrently - foreach (var partitionId in EnumerableExtensions.PseudoRandomEnumerate(partitions.Count).Select(p => (byte)p)) + foreach (var rangeIndex in EnumerableExtensions.PseudoRandomEnumerateRange(_partitionIds.Length)) { + var partitionId = _partitionIds[rangeIndex]; if (excludePartition?.Invoke(partitionId) == true) { continue; } - var partition = partitions[partitionId]; + var partition = partitions[rangeIndex]; // Add a block with the machine's content to the corresponding partition submission block await SubmitMachinePartitionContentAsync(context, partitionId, partition); - // Attempt to process the submission block for the partition into a new partition output blob - await ProcessPartitionAsync(context, partitionId); + if (Configuration.ProcessPartitions) + { + // Attempt to process the submission block for the partition into a new partition output blob + await ProcessPartitionAsync(context, partitionId); + } + + if (ShouldUpdateDatabase) + { + databaseUpdates++; + await UpdateDatabasePartitionAsync(context, partitionId); + } // Add a delay to not have a hot loop of storage interaction await Task.Delay(Configuration.PerPartitionDelayInterval, context.Token); } + if (ShouldUpdateDatabase) + { + updatedDatabaseManifest = true; + await UpdateManifestFromDatabaseRecordAsync(context).ThrowIfFailureAsync(); + } + return BoolResult.Success; - }); + }, + extraEndMessage: r => $"DatabaseUpdates=[{databaseUpdates}] UpdatedDatabaseManifest=[{updatedDatabaseManifest}] DatabaseUpdateLeaseExpiry=[{_databaseUpdateLeaseExpiry}]"); + } + + /// + /// Updates the manifest in blob storage to match the current records in the database + /// + internal Task> UpdateManifestFromDatabaseRecordAsync(OperationContext context, bool force = false) + { + PartitionId lastPartition = default; + return context.PerformOperationAsync( + Tracer, + () => Storage.ReadModifyWriteAsync>( + context, + Configuration.PartitionCheckpointManifestFileName, + manifest => + { + var updatedPartitions = new List(); + manifest.SetPartitionCount(_partitionIds.Length); + + if (ShouldUpdateDatabase || force) + { + foreach (var partitionId in _partitionIds) + { + lastPartition = partitionId; + var databaseRecord = TryGetDatabasePartitionRecord(partitionId) + .ThrowIfFailure(); + + var manifestRecord = manifest[partitionId]; + if (databaseRecord != manifestRecord) + { + manifest[partitionId] = databaseRecord; + updatedPartitions.Add(partitionId); + } + } + } + + return (manifest, Result: updatedPartitions, Updated: updatedPartitions.Count != 0); + }), + extraEndMessage: r => $" LastPartition={lastPartition} UpdatedPartitions=({r.GetValueOrDefault().Result?.Count ?? -1})[{string.Join(", ", r.GetValueOrDefault().Result ?? Enumerable.Empty())}]") + .AsAsync(r => r.NextState); + } + + /// + /// Update the partition in the database if it the update interval has elapsed by + /// ingesting diff ssts. The flow is: + /// - Compute the chain of diffs needed to synchronize the database with the current state. + /// If chain cannot be found, a full sst with all information for partition is applied. + /// - Download the snapshot chain of sst files and delete stale snapshots + /// - Ingest the snapshot sst files + /// + private Task UpdateDatabasePartitionAsync(OperationContext context, PartitionId partitionId) + { + return Storage.UseBlockBlobAsync>(context, + GetPartitionOutputBlobName(partitionId), + timeout: TimeSpan.FromSeconds(120), + endMessageSuffix: r => $" Partition={partitionId}, UpdateResult={r.GetValueOrDefault()}", + isCritical: true, + useAsync: async (context, blobWrapper) => + { + var blob = new PartitionBlob(partitionId, blobWrapper); + + // Check database which snapshot the database is currently at for the partition + var databaseRecordResult = TryGetDatabasePartitionRecord(partitionId); + if (!databaseRecordResult.TryGetValue(out var databaseRecord) + || databaseRecord == null) + { + // Error reading from database or no prior partition record found. Log warning and continue and use full sst. + Tracer.Warning(context, $"Partition {partitionId} record missing from database. RetrieveResult={databaseRecordResult}"); + } + + // Check whether its time for an update + var lastUpdateTime = databaseRecord?.CreationTime; + if (lastUpdateTime?.IsRecent(_clock.UtcNow, Configuration.PartitionsUpdateInterval) == true) + { + // Database was updated recently, skip updating partition. + return Result.Success(new UpdatePartitionResult(Updated: false, Record: databaseRecord, LastUpdateTime: lastUpdateTime)) + .WithSuccessDiagnostics($"Database record from '{databaseRecord?.CreationTime}' is up to date. Next update will be >= {databaseRecord?.CreationTime + Configuration.PartitionsUpdateInterval}"); + } + + // Get the chain of snapshots to the database + var snapshotChain = await ComputeSnapshotChainLatestLastAsync(context, blob, databaseRecord, AsyncOut.Var>(out var staleSnapshots)); + if (snapshotChain.Count == 0) + { + return Result.Success(new UpdatePartitionResult(Updated: false, Record: databaseRecord, LastUpdateTime: lastUpdateTime)) + .WithSuccessDiagnostics($"No new data is available for update."); + } + + var directory = Database.CreateTempDirectory($"in_sst.{DateTime.Now.Ticks}"); + long totalLength = 0; + + // Download snapshot files + var downloadBlobsTask = _actionQueue.ForEachAsync(snapshotChain, (snapshotFile, index) => + { + var snapshot = snapshotFile.Blob; + var messagePrefix = $"Partition={partitionId} SnapshotId={snapshot.SnapshotId} SnapshotTime={snapshot.Blob.SnapshotTime} BaseSnapshot={snapshot.BaseSnapshotId}"; + + return context.PerformOperationAsync( + Tracer, + async () => + { + var length = await ReadContentPartitionAsync( + context, + snapshot, + new FileReader(directory.Path / snapshotFile.FileName), + snapshotFile.Kind, + requiresMetadataFetch: false); + + Interlocked.Add(ref totalLength, length); + + return Result.Success(length); + }, + caller: "DownloadPartitionSstFileAsync", + extraEndMessage: r => $"{messagePrefix} Path={snapshotFile.FileName} Length={r.GetValueOrDefault()} ChainIndex={index}") + .ThrowIfFailureAsync(); + }); + + // Garbage collect stale snapshots + var garbageCollectSnapshotsTask = _actionQueue.ForEachAsync(staleSnapshots.Value, (snapshot, index) => + { + var messagePrefix = $"Partition={partitionId} SnapshotId={snapshot.SnapshotId} SnapshotTime={snapshot.Blob.SnapshotTime} BaseSnapshot={snapshot.BaseSnapshotId}"; + + return context.PerformOperationWithTimeoutAsync>( + Tracer, + timeout: Configuration.StorageInteractionTimeout, + operation: async context => + { + bool deleted = await snapshot.DeleteIfExistsAsync(token: context.Token); + return Result.Success(deleted); + }, + caller: "DeletePartitionSnapshotAsync", + extraEndMessage: r => $"{messagePrefix} Deleted={r.GetValueOrDefault()}") + .IgnoreFailure(); + }); + + await Task.WhenAll(downloadBlobsTask, garbageCollectSnapshotsTask); + + var currentSnapshot = snapshotChain.LastOrDefault()?.Blob; + + // Ingest the snaphot files + context.PerformOperation( + Tracer, + () => + { + // Snapshot chain is latest first. For RocksDb ingestion, later files + // take precedence so we should reverse the list + return Database.IngestMergeContentSstFiles(context, snapshotChain.Select(f => directory.Path / f.FileName)); + }, + caller: "IngestPartitionSstFiles", + messageFactory: r => $"Partition={partitionId} SnapshotId={snapshotChain.LastOrDefault()?.Blob.SnapshotId} Files={snapshotChain.Count} ByteLength={totalLength}") + .ThrowIfFailure(); + + // + TryGetDatabasePartitionRecord(partitionId).TryGetValue(out databaseRecord); + Contract.Check(databaseRecord != null && databaseRecord?.SnapshotId == currentSnapshot.SnapshotId) + ?.Assert($"Expected database snapshot id '{currentSnapshot.SnapshotId}' but found '{databaseRecord?.SnapshotId}'"); + + return Result.Success(new UpdatePartitionResult(Updated: true, Record: databaseRecord, LastUpdateTime: lastUpdateTime)); + }) + .IgnoreFailure>(); + } + + /// + /// Computes the chain of snapshot sst files to apply to the base record in order + /// to match the current content registration in blob storage + /// + private async Task> ComputeSnapshotChainLatestLastAsync( + OperationContext context, + PartitionBlob blob, + PartitionRecord? baseSnapshot, + AsyncOut> snapshotsToDelete) + { + var (snapshots, currentSnapshot) = await blob.ListSnapshotsAndCurrentAsync(context, baseSnapshot?.SnapshotId); + var snapshotChain = new List(); + if (currentSnapshot?.SnapshotId is not Guid snapshotId) + { + // No updates available for partition. + return snapshotChain; + } + + var isChainComplete = false; + Dictionary snapshotsById = snapshots + .Where(s => s.Blob.IsSnapshot && s.SnapshotId != null) + .ToDictionarySafe(s => s.SnapshotId.Value); + + // Trace back from current snapshot to the base snapshot + while (snapshotsById.TryGetValue(snapshotId, out var snapshot)) + { + isChainComplete = baseSnapshot != null && snapshot.BaseSnapshotId == baseSnapshot?.SnapshotId; + snapshotChain.Add(new SnapshotSstFile(snapshot, IsFull: snapshot.BaseSnapshotId == null)); + if (isChainComplete + || snapshot.BaseSnapshotId == null + || baseSnapshot == null + || snapshotChain.Count > Configuration.MaxSnapshotChainLength) + { + break; + } + + snapshotId = snapshot.BaseSnapshotId.Value; + } + + Contract.Assert(snapshotChain.Count > 0); + + if (!isChainComplete) + { + // Chain is incomplete (i.e. could not trace back to base snapshot). Just use latest snaphot + // as a full snapshot. + snapshotChain.Clear(); + snapshotChain.Add(new SnapshotSstFile(currentSnapshot, IsFull: true)); + + Tracer.Warning(context, $"Partition {blob.PartitionId} could not complete snapshot chain. Using full snapshot={currentSnapshot.SnapshotId}."); + } + + // Compute the timestamp of the earliest snapshot which should be retained + var minRetainedTime = snapshots + .Select(s => s.Blob.SnapshotTime) + .Where(time => time != null) + .OrderByDescending(s => s) + .Take(Configuration.MaxRetainedSnapshots) + .LastOrDefault(); + + snapshotsToDelete.Value = snapshots + // Don't delete blobs in the snapshot chain + .Except(snapshotChain.Select(s => s.Blob)) + .Where(s => s.Blob.SnapshotTime != null) + // Select the stale snapshots + .Where(s => s.Blob.SnapshotTime < minRetainedTime) + .ToList(); + + // Snapshots are ordered latest to earliest at this point. + // Reverse the snapshots so they are in the correct order for ingestion. + // (i.e. snapshot with latest data should appear last). + snapshotChain.Reverse(); + return snapshotChain; } private Task ExecuteWithRetryAsync( @@ -220,7 +519,7 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache /// private Task SubmitMachinePartitionContentAsync( OperationContext context, - byte partitionId, + PartitionId partitionId, ContentListing partition) { Observer.OnPutBlock(partitionId, partition); @@ -241,8 +540,7 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache return BoolResult.Success; }); }, - caller: "WritePartitionBlock", - endMessageSuffix: _ => $"Partition={partitionId}, BlockId={LocalMachineRecord.MachineBlockId}, {partition}") + endMessageSuffix: _ => $" Partition={partitionId}, BlockId={LocalMachineRecord.MachineBlockId}, {partition}") .IgnoreFailure(); } @@ -253,14 +551,15 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache /// public Task ComputeSortedPartitionContentAsync( OperationContext context, - byte partitionId, + PartitionId partitionId, bool takeLease = true) { return Storage.UseBlockBlobAsync>(context, GetPartitionSubmissionBlobName(partitionId), timeout: TimeSpan.FromSeconds(120), - useAsync: async (context, blob) => + useAsync: async (context, b) => { + var blob = new PartitionBlob(partitionId, b); AccessCondition leaseCondition = null; if (takeLease) { @@ -281,7 +580,7 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache retainedBlockList, leaseCondition); - var listing = await ReadContentPartitionAsync(context, partitionId, blob); + var listing = await ReadContentPartitionAsync(context, blob, ContentListingReader.Instance); if (leaseCondition != null) { @@ -302,95 +601,58 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache .ThrowIfFailureAsync(unwrap: true); } - /// - /// Reads the content listing from the blob block - /// - internal Task ReadContentPartitionAsync( - OperationContext context, - byte partitionId, - BlobName name, - PartitionOutputKind kind, - AccessCondition accessCondition = null) - { - return Storage.UseBlockBlobAsync( - context, - name, - async (context, blob) => - { - bool exists = await blob.ExistsAsync(); - if (!exists) - { - return ContentListing.CreateFromByteLength(0); - } - - var blockList = await blob.DownloadBlockListAsync( - BlockListingFilter.Committed, - accessCondition); - - var blockId = GetPartitionOutputBlockId(kind); - - // Find the range for the block id - BlobRange range = new BlobRange() { Length = -1 }; - foreach (var block in blockList) - { - if (block.Name == blockId) - { - range.Length = block.Length; - break; - } - else - { - range.Offset += block.Length; - } - } - - if (range.Length < 0) - { - return new ErrorResult($"Block '{blockId}' not found in blob: Existing blocks: {string.Join(", ", blockList.Select(b => b.Name))}"); - } - - var result = await ReadContentPartitionAsync(context, partitionId, blob, range, accessCondition); - return Result.Success(result); - }).ThrowIfFailureAsync(unwrap: true); - } - /// /// Reads the content listing from the blob (with optional range) /// - private Task ReadContentPartitionAsync( + private Task ReadContentPartitionAsync( OperationContext context, - byte partitionId, - BlobWrapper blob, - BlobRange? range = null, - AccessCondition accessCondition = null) + PartitionBlob blob, + IReader reader, + PartitionOutputKind? blockKind = null, + AccessCondition accessCondition = null, + bool requiresMetadataFetch = true) { return context.PerformOperationAsync( Tracer, async () => { - await blob.FetchAttributesAsync(); - var listing = ContentListing.CreateFromByteLength((int)(range?.Length ?? blob.Blob.Properties.Length)); + if (requiresMetadataFetch) + { + await blob.FetchAttributesAsync(); + } + + BlobRange? range = null; + if (blockKind != null) + { + range = blob[blockKind.Value]; + if (range == null) + { + return new ErrorResult($"Block range '{blockKind}' not found in blob."); + } + } + + var value = reader.Create(range?.Length ?? blob.Blob.Properties.Length); bool disposeOnException() { - listing.Dispose(); + (value as IDisposable)?.Dispose(); return false; } try { - using var stream = listing.AsStream(); + using var stream = reader.GetStream(value); await blob.DownloadRangeToStreamAsync( stream, range?.Offset, range?.Length, accessCondition); - return Result.Success(listing); + return Result.Success(value); } catch when (disposeOnException()) { throw; } }, - extraEndMessage: r => $"Partition={partitionId}, Blob={blob.Name}, Value={r.GetValueOrDefault()}") + extraEndMessage: r => $"Partition={blob.PartitionId}, Blob={blob.Name}, Block={blockKind} Value={r.GetValueOrDefault()}") .ThrowIfFailureAsync(unwrap: true); } @@ -399,16 +661,15 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache /// public Task ProcessPartitionAsync( OperationContext context, - byte partitionId) + PartitionId partitionId) { + string leaseId = null; return Storage.UseBlockBlobAsync>(context, GetPartitionOutputBlobName(partitionId), - caller: "WritePartitionBlobs", timeout: TimeSpan.FromSeconds(120), - useAsync: async (context, blob) => + useAsync: async (context, blobWrapper) => { - var metadata = new BlobMetadata(blob); - var updateResult = new UpdatePartitionResult(new PartitionChangeCounters(), metadata, Updated: false); + var blob = new PartitionBlob(partitionId, blobWrapper); if (!await blob.ExistsAsync()) { @@ -417,68 +678,147 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache AccessCondition.GenerateIfNotExistsCondition()); } - if (metadata.LastUpdateTime is DateTime lastUpdateTime - && lastUpdateTime.IsRecent(_clock.UtcNow, Configuration.PartitionsUpdateInterval)) + bool wasLeased = false; + var lastUpdateTime = blob.LastUpdateTime; + if (lastUpdateTime?.IsRecent(_clock.UtcNow, Configuration.PartitionsUpdateInterval) == true + || (wasLeased = blob.Inner.Blob.Properties.LeaseState == LeaseState.Leased)) { - return Result.Success(updateResult); + // Do nothing if blob is up to date or current being updated (currently leased) + return Result.Success(new UpdatePartitionResult(Updated: false, Record: null, LastUpdateTime: lastUpdateTime, WasLeased: wasLeased)); } - updateResult.Updated = true; - - // Acquire lease for maximum non-infinite duration. - var leaseId = await blob.AcquireLeaseAsync(TimeSpan.FromSeconds(60)); + // Acquire lease for maximum non-infinite duration (60s). + leaseId = await blob.AcquireLeaseAsync(TimeSpan.FromSeconds(60)); + var leaseCondition = AccessCondition.GenerateLeaseCondition(leaseId); + blob = blob with { DefaultAccessCondition = leaseCondition }; var manifest = await Storage .ReadAsync(context, Configuration.PartitionCheckpointManifestFileName) .ThrowIfFailureAsync(); - var baseRecord = manifest.Records[partitionId]; - AccessCondition accessCondition = AccessCondition.GenerateLeaseCondition(leaseId); + PartitionBlob baseSnapshot = null; + if (manifest[partitionId] is PartitionRecord baseRecord) + { + var snapshots = await blob.ListSnapshotsAsync(context); + baseSnapshot = snapshots.FirstOrDefault(b => b.SnapshotId == baseRecord.SnapshotId); + } // Get base listing - using var baseContentListing = baseRecord.SnapshotTime == null + using var baselineListing = baseSnapshot == null ? ContentListing.CreateFromByteLength(0) : await ReadContentPartitionAsync( context, - partitionId, - GetPartitionOutputBlobName(partitionId, baseRecord.SnapshotTime), + baseSnapshot, + ContentListingReader.Instance, PartitionOutputKind.FullListing, - accessCondition); + AccessCondition.GenerateEmptyCondition()); - using var nextContentListing = await ComputeSortedPartitionContentAsync(context, partitionId); + using var currentListing = await ComputeSortedPartitionContentAsync(context, partitionId); - var nextEntries = nextContentListing.EnumerateEntries(); + var blockList = new List(); - var blockList = new List(); + var partitionRecord = new PartitionRecord( + PartitionId: partitionId, + CreationTime: _clock.UtcNow, + SnapshotId: Guid.NewGuid(), + BaseSnapshotCreationTime: baseSnapshot?.LastUpdateTime, + BaseSnapshotId: baseSnapshot?.SnapshotId, + new PartitionUpdateStatistics()); - // Compute the differences (use LastOrDefault to force enumeration of entire enumerable) - baseContentListing.EnumerateChanges(nextEntries, updateResult.UpdateState).LastOrDefault(); + // Write the full and diff sst files + await CreateAndUploadSstFileBlocksAsync( + context, + blob, + blockList, + partitionRecord, + baseline: baselineListing, + current: currentListing).ThrowIfFailureAsync(); // Write the full listing blob await PutPartitionOutputBlockAsync( context, - partitionId, blob, PartitionOutputKind.FullListing, blockList, - () => nextContentListing.AsStream(), - accessCondition); + () => currentListing.AsStream()); // Finalize the blob // Update metadata which will be associated with the block during PutBlockList - metadata.BaseSnapshot = baseRecord.SnapshotTime; - metadata.LastUpdateTime = _clock.UtcNow; + blob.SetInfo(partitionRecord); await blob.PutBlockListAsync( - blockList, - accessCondition); + blockList.Select(b => b.BlockId)); - await blob.ReleaseLeaseAsync(accessCondition); + await blob.ReleaseLeaseAsync(leaseCondition); - return Result.Success(updateResult); + return Result.Success(new UpdatePartitionResult(Updated: true, partitionRecord, lastUpdateTime)); }, - endMessageSuffix: r => $" Partition={partitionId}, UpdateResult={r.GetValueOrDefault()}") - .IgnoreFailure(); + endMessageSuffix: r => $" Partition={partitionId}, Lease={leaseId}, UpdateResult={r.GetValueOrDefault()}") + .IgnoreFailure>(); + } + + /// + /// Write out and upload the full and diff sst files given the baseline and current content listing. + /// + private Task CreateAndUploadSstFileBlocksAsync( + OperationContext context, + PartitionBlob blob, + List blockList, + PartitionRecord record, + ContentListing baseline, + ContentListing current) + { + Contract.Assert(Database != null); + + return context.PerformOperationAsync( + Tracer, + async () => + { + using var dir = Database.CreateTempDirectory("out_sst"); + + var fullSstPath = dir.Path / $"{blob.PartitionId}.full.sst"; + var diffSstPath = dir.Path / $"{blob.PartitionId}.diff.sst"; + + context.PerformOperation( + Tracer, + () => + { + using SstFileWriter fullSstWriter = Database.CreateContentSstWriter(context, fullSstPath).ThrowIfFailure(); + using SstFileWriter diffSstWriter = Database.CreateContentSstWriter(context, diffSstPath).ThrowIfFailure(); + + var result = WriteSstFiles( + context, + blob.PartitionId, + baseline: baseline, + current: current, + fullSstWriter: fullSstWriter, + diffSstWriter: diffSstWriter, + record.Statistics); + + // Write the database record as the last entry which contains the snapshot id and summary information + // about the content contained in the current listing and the difference with the baseline + WriteDatabasePartitionRecord(fullSstWriter, record); + WriteDatabasePartitionRecord(diffSstWriter, record); + + fullSstWriter.Finish(); + diffSstWriter.Finish(); + + return result; + + }).ThrowIfFailure(); + + Task putSstFileBlockAsync(AbsolutePath path, PartitionOutputKind kind) + { + return PutPartitionOutputBlockAsync(context, blob, kind, blockList, () => _fileSystem.OpenReadOnly(path, FileShare.Read)); + } + + await Task.WhenAll( + putSstFileBlockAsync(fullSstPath, PartitionOutputKind.FullSst), + putSstFileBlockAsync(diffSstPath, PartitionOutputKind.DiffSst)); + + return BoolResult.Success; + }, + extraEndMessage: r => $" Partition={blob.PartitionId}, Counters={record.Statistics}"); } /// @@ -486,12 +826,10 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache /// private Task PutPartitionOutputBlockAsync( OperationContext context, - byte partitionId, - BlobWrapper blob, + PartitionBlob blob, PartitionOutputKind kind, - List blockList, - Func getStream, - AccessCondition accessCondition) + List blockList, + Func getStream) { return context.PerformOperationAsync( Tracer, @@ -503,29 +841,32 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache await blob.PutBlockAsync( blockId, - stream, - accessCondition); + stream); - blockList.Add(blockId); + lock (blockList) + { + var last = blockList.LastOrDefault().Range; + var range = new BlobRange(Offset: last.Offset + last.Length, Length: stream.Length); + blockList.Add(new(kind, range, blockId)); + blob[kind] = range; + } return BoolResult.Success; } }, - caller: $"WritePartition{kind}Block", - extraEndMessage: _ => $"Partition={partitionId}") + caller: $"{nameof(PutPartitionOutputBlockAsync)}.{kind}", + extraEndMessage: _ => $"Partition={blob.PartitionId} Range={blob[kind]}") .ThrowIfFailureAsync(); } - private static BlobName GetPartitionSubmissionBlobName(byte partitionId) + private static BlobName GetPartitionSubmissionBlobName(PartitionId partitionId) { - var hexPartitionId = HexUtilities.BytesToHex(new[] { partitionId }); - return $"{hexPartitionId[0]}/{hexPartitionId}.bin"; + return $"{partitionId.BlobPrefix}.bin"; } - private static BlobName GetPartitionOutputBlobName(byte partitionId, DateTimeOffset? snapshotTime = null) + private static BlobName GetPartitionOutputBlobName(PartitionId partitionId, DateTimeOffset? snapshotTime = null) { - var hexPartitionId = HexUtilities.BytesToHex(new[] { partitionId }); - BlobName blobName = $"{hexPartitionId[0]}/{hexPartitionId}.out.bin"; + BlobName blobName = $"{partitionId.BlobPrefix}.out.bin"; blobName = blobName with { SnapshotTime = snapshotTime }; return blobName; } @@ -557,34 +898,119 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache /// /// Manifest tracking the snapshots used for each partition in a checkpoint. /// - private record PartitionCheckpointManifest + internal record PartitionCheckpointManifest { - public PartitionRecord[] Records { get; } = new PartitionRecord[ContentListing.PartitionCount]; + public PartitionRecord?[] Records { get; set; } = new PartitionRecord?[0]; + + public void SetPartitionCount(int partitionCount) + { + if (Records.Length != partitionCount) + { + Records = new PartitionRecord?[partitionCount]; + } + } + + public PartitionRecord? this[PartitionId partitionId] + { + get + { + if (Records.Length == partitionId.PartitionCount) + { + return Records[partitionId.Index]; + } + + return null; + } + set + { + Contract.Check(partitionId.PartitionCount == Records.Length)? + .Assert($"{partitionId.PartitionCount} does not match {Records.Length}"); + + Records[partitionId.Index] = value; + } + } } /// /// Record of snapshot of a partition used for the checkpoint (blob storage uses this a the snapshot id) /// - private record struct PartitionRecord(DateTimeOffset? SnapshotTime); + internal record struct PartitionRecord( + PartitionId PartitionId, + DateTime CreationTime, + Guid SnapshotId, + DateTime? BaseSnapshotCreationTime, + Guid? BaseSnapshotId, + PartitionUpdateStatistics Statistics); private record struct BlobRange(long Offset, long Length); - private record struct UpdatePartitionResult(PartitionChangeCounters UpdateState, BlobMetadata Metadata, bool Updated); + private record struct UpdatePartitionResult(bool Updated, PartitionRecord? Record, DateTime? LastUpdateTime, bool? WasLeased = null); + + private record FileReader(AbsolutePath Path) : IReader + { + public long Create(long byteLength) + { + return byteLength; + } + + public Stream GetStream(long length) + { + return PassThroughFileSystem.Default.OpenForWrite(Path, length, FileMode.Create, FileShare.Delete); + } + } + + private class ContentListingReader : IReader + { + public static readonly ContentListingReader Instance = new ContentListingReader(); + + public ContentListing Create(long byteLength) + { + return ContentListing.CreateFromByteLength((int)byteLength); + } + + public Stream GetStream(ContentListing value) + { + return value.AsStream(); + } + } + + private interface IReader + { + T Create(long byteLength); + + Stream GetStream(T value); + } + + private record SnapshotSstFile(PartitionBlob Blob, bool IsFull) : IKeyedItem + { + public string FileName + { + get + { + var kind = IsFull ? "Full" : "Diff"; + return $"{Blob.PartitionId}.{Blob.LastUpdateTime:o}.{kind}.sst".Replace(":", ""); + } + } + + public PartitionOutputKind Kind => IsFull ? PartitionOutputKind.FullSst : PartitionOutputKind.DiffSst; + + public Guid GetKey() => Blob.SnapshotId.Value; + } /// /// Wrapper for accessing typed values of blob metadata. /// /// DO NOT RENAME properties as these are used as keys in metadata dictionary. /// - private record struct BlobMetadata(BlobWrapper Blob) + private sealed record PartitionBlob(PartitionId PartitionId, BlobWrapper Inner) : BlobWrapper(Inner), IKeyedItem { /// - /// The base snapshot used when computing the difference blocks. + /// The base snapshot id used when computing the difference blocks. /// - public DateTimeOffset? BaseSnapshot + public Guid? BaseSnapshotId { - get => Blob.GetMetadataOrDefault(s => DateTimeOffset.Parse(s, null, DateTimeStyles.RoundtripKind)); - set => Blob.SetMetadata(value?.ToString("O")); + get => GetMetadataOrDefault(s => Guid.Parse(s)); + set => SetMetadata(value?.ToString()); } /// @@ -592,9 +1018,120 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache /// public DateTime? LastUpdateTime { - get => Blob.GetMetadataOrDefault(s => DateTime.Parse(s, null, DateTimeStyles.RoundtripKind)); - set => Blob.SetMetadata(value?.ToString("O")); + get => GetMetadataOrDefault(s => DateTime.Parse(s, null, DateTimeStyles.RoundtripKind)); + set => SetMetadata(value?.ToString("O")); } + + public BlobRange? this[PartitionOutputKind kind] + { + get => GetMetadataOrDefault(s => JsonUtilities.JsonDeserialize(s), key: $"{kind}BlockRange"); + set => SetMetadata(value == null ? null : JsonUtilities.JsonSerialize(value.Value), key: $"{kind}BlockRange"); + } + + /// + /// The unique identifier of the current snapshot + /// + public Guid? SnapshotId + { + get => GetMetadataOrDefault(s => Guid.Parse(s)); + set => SetMetadata(value?.ToString()); + } + + public void SetInfo(PartitionRecord record) + { + BaseSnapshotId = record.BaseSnapshotId; + SnapshotId = record.SnapshotId; + LastUpdateTime = record.CreationTime; + } + + public async Task<(List snapshots, PartitionBlob currentSnapshot)> ListSnapshotsAndCurrentAsync(OperationContext context, Guid? excludedSnapshotId) + { + if (!await ExistsAsync() + || SnapshotId is not Guid currentSnapshotId + || currentSnapshotId == excludedSnapshotId) + { + // No updates are available. + return (new List(), null); + } + + var snapshots = await ListSnapshotsAsync(context); + + var currentSnapshot = snapshots.FirstOrDefault(s => s.SnapshotId == currentSnapshotId); + if (currentSnapshot == null) + { + (await TrySnapshotAsync(knownToExist: true)).TryGetValue(out currentSnapshot); + if (currentSnapshot != null) + { + snapshots.Add(currentSnapshot); + } + } + + return (snapshots, currentSnapshot); + } + + public Task> ListSnapshotsAsync(OperationContext context) + { + return ListSnapshotsAsync(context, wrapper => new PartitionBlob(PartitionId, wrapper)); + } + + public PartitionBlob GetSnaphot(DateTimeOffset? snapshotTime) + { + return new PartitionBlob(PartitionId, Inner with + { + Blob = new CloudBlockBlob(Blob.Uri, snapshotTime, Blob.ServiceClient), + Name = Inner.Name with { SnapshotTime = snapshotTime } + }); + } + + public async Task> TrySnapshotAsync(bool knownToExist = false) + { + if (!knownToExist && !await Blob.ExistsAsync()) + { + return default; + } + + var result = new PartitionBlob(PartitionId, await SnapshotAsync()); + + // Perform another existence query to fetch attributes + if (!await result.ExistsAsync()) + { + return default; + } + + return result; + } + + public override int GetHashCode() + { + return RuntimeHelpers.GetHashCode(this); + } + + public bool Equals(PartitionBlob? other) + { + return ReferenceEquals(this, other); + } + + public Guid? GetKey() + { + return SnapshotId; + } + } + + private record struct OutputBlock(PartitionOutputKind Kind, BlobRange Range, string BlockId); + + private Result TryGetDatabasePartitionRecord(PartitionId partitionId) + { + return Database.TryDeserializeValue( + partitionId.GetPartitionRecordKey(), + RocksDbContentMetadataDatabase.Columns.SstMergeContent, + static reader => JsonSerializer.Deserialize(reader.Span, JsonUtilities.DefaultSerializationOptions)) + .Select(o => o.HasValue ? o.Value : default(PartitionRecord?), isNullAllowed: true); + } + + private void WriteDatabasePartitionRecord(IRocksDbColumnWriter writer, PartitionRecord record) + { + var recordBytes = JsonSerializer.SerializeToUtf8Bytes(record, JsonUtilities.DefaultSerializationOptions); + writer.Put(record.PartitionId.GetPartitionRecordKey(), recordBytes); } internal enum PartitionOutputKind @@ -614,12 +1151,7 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache /// /// Content listing file containing entries for all content in the partition /// - FullListing, - - /// - /// Content listing file containing entries for added/removed content in the partition since last iteration - /// - DiffListing + FullListing } /// @@ -627,7 +1159,7 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache /// internal class TestObserver { - internal virtual void OnPutBlock(byte partitionId, ContentListing partition) + internal virtual void OnPutBlock(PartitionId partitionId, ContentListing partition) { } } diff --git a/Public/Src/Cache/ContentStore/Distributed/NuCache/CheckpointManager.cs b/Public/Src/Cache/ContentStore/Distributed/NuCache/CheckpointManager.cs index c78cd27f3..618b85b69 100644 --- a/Public/Src/Cache/ContentStore/Distributed/NuCache/CheckpointManager.cs +++ b/Public/Src/Cache/ContentStore/Distributed/NuCache/CheckpointManager.cs @@ -92,7 +92,7 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache { while (!context.Token.IsCancellationRequested) { - await PeriodicRestoreCheckpointAsync(context); + await PeriodicRestoreCheckpointAsync(context).IgnoreFailure(); await Task.Delay(_configuration.RestoreCheckpointInterval, context.Token); } @@ -100,7 +100,7 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache return BoolResult.Success; } - private Task PeriodicRestoreCheckpointAsync(OperationContext context) + public Task PeriodicRestoreCheckpointAsync(OperationContext context) { return context.PerformOperationAsync( Tracer, @@ -109,8 +109,7 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache var checkpointState = await CheckpointRegistry.GetCheckpointStateAsync(context).ThrowIfFailureAsync(); return await RestoreCheckpointAsync(context, checkpointState); - }) - .IgnoreFailure(); + }); } private record DatabaseStats @@ -255,6 +254,9 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache long uploadSize = 0; long retainedSize = 0; + int uploadCount = 0; + int retainedCount = 0; + var newManifest = new CheckpointManifest(); await ParallelAlgorithms.WhenDoneAsync( _configuration.IncrementalCheckpointDegreeOfParallelism, @@ -278,6 +280,7 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache AddEntry(newManifest, relativePath, storageId); attemptUpload = false; + Interlocked.Increment(ref retainedCount); Interlocked.Add(ref retainedSize, _fileSystem.GetFileSize(file)); Counters[ContentLocationStoreCounters.IncrementalCheckpointFilesUploadSkipped].Increment(); } @@ -291,6 +294,7 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache incrementalCheckpointsPrefix + relativePath).ThrowIfFailureAsync(); AddEntry(newManifest, relativePath, storageId); + Interlocked.Increment(ref uploadCount); Interlocked.Add(ref uploadSize, _fileSystem.GetFileSize(file)); Counters[ContentLocationStoreCounters.IncrementalCheckpointFilesUploaded].Increment(); } @@ -299,6 +303,11 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache Tracer.TrackMetric(context, "CheckpointUploadSize", uploadSize); Tracer.TrackMetric(context, "CheckpointRetainedSize", retainedSize); + Tracer.TrackMetric(context, "CheckpointTotalSize", uploadSize + retainedSize); + + Tracer.TrackMetric(context, "CheckpointUploadCount", uploadCount); + Tracer.TrackMetric(context, "CheckpointRetainedCount", retainedCount); + Tracer.TrackMetric(context, "CheckpointTotalCount", uploadCount + retainedCount); DatabaseWriteManifest(context, newManifest); diff --git a/Public/Src/Cache/ContentStore/Distributed/NuCache/ClusterStateManagement/ClusterStateManager.cs b/Public/Src/Cache/ContentStore/Distributed/NuCache/ClusterStateManagement/ClusterStateManager.cs index 6de29e085..340cb2d11 100644 --- a/Public/Src/Cache/ContentStore/Distributed/NuCache/ClusterStateManagement/ClusterStateManager.cs +++ b/Public/Src/Cache/ContentStore/Distributed/NuCache/ClusterStateManagement/ClusterStateManager.cs @@ -10,6 +10,7 @@ using BuildXL.Cache.ContentStore.Utils; using OperationContext = BuildXL.Cache.ContentStore.Tracing.Internal.OperationContext; using System.Collections.Generic; using System.Diagnostics.ContractsLight; +using System; #nullable enable @@ -40,7 +41,10 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache LinkLifetime(_storage); - RunInBackground(nameof(BackgroundUpdateAsync), BackgroundUpdateAsync, fireAndForget: true); + if (_configuration.Checkpoint?.UpdateClusterStateInterval > TimeSpan.Zero) + { + RunInBackground(nameof(BackgroundUpdateAsync), BackgroundUpdateAsync, fireAndForget: true); + } } protected override async Task StartupComponentAsync(OperationContext context) diff --git a/Public/Src/Cache/ContentStore/Distributed/NuCache/ContentListing.cs b/Public/Src/Cache/ContentStore/Distributed/NuCache/ContentListing.cs index 11e024102..eb7ff9ecb 100644 --- a/Public/Src/Cache/ContentStore/Distributed/NuCache/ContentListing.cs +++ b/Public/Src/Cache/ContentStore/Distributed/NuCache/ContentListing.cs @@ -5,8 +5,11 @@ using System; using System.Collections.Generic; using System.Diagnostics.ContractsLight; using System.IO; +using System.Linq; +using System.Numerics; using System.Runtime.CompilerServices; using System.Runtime.InteropServices; +using BuildXL.Cache.ContentStore.Hashing; using BuildXL.Cache.ContentStore.Stores; using BuildXL.Utilities.Collections; @@ -17,8 +20,6 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache /// public record ContentListing : IDisposable { - internal const int PartitionCount = 256; - private readonly SafeAllocHHandle _handle; private readonly int _offset; private int _length; @@ -134,12 +135,11 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache /// /// Enumerates the partition slices from the listing /// - public IEnumerable GetPartitionSlices() + public IEnumerable GetPartitionSlices(IEnumerable ids) { int start = 0; - for (int i = 0; i < PartitionCount; i++) + foreach (var partitionId in ids) { - var partitionId = (byte)i; var partition = GetPartitionContentSlice(partitionId, ref start); yield return partition; } @@ -148,13 +148,13 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache /// /// Gets the slice of the full listing containing the partition's content. /// - private ContentListing GetPartitionContentSlice(byte partitionId, ref int start) + private ContentListing GetPartitionContentSlice(PartitionId partitionId, ref int start) { var entrySpan = EntrySpan; for (int i = start; i < entrySpan.Length; i++) { var entry = entrySpan[i]; - if (entry.PartitionId != partitionId) + if (!partitionId.Contains(entry.PartitionId)) { var result = GetSlice(start, i - start); start = i; @@ -164,14 +164,15 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache return GetSlice(start, entrySpan.Length - start); } - + /// /// Gets an unmanaged stream over the listing. /// - public Stream AsStream() + public StreamWithLength AsStream() { - return new UnmanagedMemoryStream(_handle, _offset * (long)MachineContentEntry.ByteLength, _length * (long)MachineContentEntry.ByteLength, FileAccess.ReadWrite); + var stream = new UnmanagedMemoryStream(_handle, _offset * (long)MachineContentEntry.ByteLength, _length * (long)MachineContentEntry.ByteLength, FileAccess.ReadWrite); + return stream.WithLength(stream.Length); } /// @@ -202,24 +203,36 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache /// /// Computes the difference from this listing to . /// - public IEnumerable EnumerateChanges(IEnumerable nextEntries, - PartitionChangeCounters counters = null) + public IEnumerable EnumerateChanges( + IEnumerable nextEntries, + BoxRef counters, + bool synthesizeUniqueHashEntries) { - counters ??= new PartitionChangeCounters(); - foreach (var diff in NuCacheCollectionUtilities.DistinctDiffSorted(EnumerateEntries(), nextEntries, i => i)) + counters ??= new DiffContentStatistics(); + foreach (var diff in NuCacheCollectionUtilities.DistinctMergeSorted(EnumerateEntries(), nextEntries, i => i, i => i)) { if (diff.mode == MergeMode.LeftOnly) { - counters.Removes++; - counters.RemoveContentSize += diff.item.Size.Value; - yield return diff.item with { Location = diff.item.Location.AsRemove() }; + counters.Value.Removes.Add(diff.left); + yield return diff.left with { Location = diff.left.Location.AsRemove() }; } else { - counters.Adds++; - counters.AddContentSize += diff.item.Size.Value; - Contract.Assert(!diff.item.Location.IsRemove); - yield return diff.item; + var entry = diff.Either(); + bool isUnique = counters.Value.Total.Add(entry); + if (diff.mode == MergeMode.RightOnly) + { + counters.Value.Adds.Add(entry, isUnique); + yield return diff.right; + } + else if (isUnique && synthesizeUniqueHashEntries) + { + // Synthesize fake entry for hash + // This ensures that touches happen even when no content adds/removes + // have happened for the hash. + // NOTE: This should not be written to the database. + yield return new MachineContentEntry() with { ShardHash = entry.ShardHash }; + } } } } @@ -243,12 +256,73 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache /// /// Counters for /// - public record PartitionChangeCounters + public record PartitionUpdateStatistics { - public long Adds; - public long Removes; - public long AddContentSize; - public long RemoveContentSize; + // TODO: Log(size) statistics? + public DiffContentStatistics DiffStats { get; init; } = new DiffContentStatistics(); + } + + public record ContentStatistics + { + public long TotalCount { get; set; } + public long TotalSize { get; set; } + public long UniqueCount { get; set; } + public long UniqueSize { get; set; } + public ShardHash? First { get; set; } + public ShardHash? Last { get; set; } + public int MaxHashFirstByteDifference { get; set; } + public MachineContentInfo Info = MachineContentInfo.Default; + + public bool Add(MachineContentEntry entry, bool? isUnique = default) + { + var last = Last; + Last = entry.ShardHash; + First ??= Last; + + Info.Merge(entry); + var size = entry.Size.Value; + + TotalCount++; + TotalSize += size; + + isUnique ??= last != Last; + if (isUnique.Value) + { + if (last != null) + { + var bytes1 = MemoryMarshal.AsBytes(stackalloc[] { last.Value }); + var bytes2 = MemoryMarshal.AsBytes(stackalloc[] { Last.Value }); + MaxHashFirstByteDifference = Math.Max(MaxHashFirstByteDifference, GetFirstByteDifference(bytes1, bytes2)); + } + + UniqueCount++; + UniqueSize += size; + } + + return isUnique.Value; + } + + private static int GetFirstByteDifference(Span bytes1, Span bytes2) + { + for (int i = 0; i < bytes1.Length; i++) + { + if (bytes1[i] != bytes2[i]) + { + return i; + } + } + + return bytes1.Length; + } + } + + public record DiffContentStatistics() + { + public ContentStatistics Adds { get; init; } = new(); + public ContentStatistics Removes { get; init; } = new(); + public ContentStatistics Deletes { get; init; } = new(); + public ContentStatistics Touchs { get; init; } = new(); + public ContentStatistics Total { get; init; } = new(); } /// diff --git a/Public/Src/Cache/ContentStore/Distributed/NuCache/IMasterElectionMechanism.cs b/Public/Src/Cache/ContentStore/Distributed/NuCache/IMasterElectionMechanism.cs index 66faf5829..9ecd24f57 100644 --- a/Public/Src/Cache/ContentStore/Distributed/NuCache/IMasterElectionMechanism.cs +++ b/Public/Src/Cache/ContentStore/Distributed/NuCache/IMasterElectionMechanism.cs @@ -1,6 +1,7 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT License. +using System; using System.Threading.Tasks; using BuildXL.Cache.ContentStore.Interfaces.Results; using BuildXL.Cache.ContentStore.Interfaces.Stores; @@ -12,9 +13,10 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache /// Current master machine MachineLocation Master, /// Role of the current machine - Role Role) + Role Role, + DateTime? MasterLeaseExpiryUtc) { - public static MasterElectionState DefaultWorker = new MasterElectionState(Master: default, Role: Role.Worker); + public static MasterElectionState DefaultWorker = new MasterElectionState(Master: default, Role: Role.Worker, MasterLeaseExpiryUtc: null); } /// diff --git a/Public/Src/Cache/ContentStore/Distributed/NuCache/MachineContentEntry.cs b/Public/Src/Cache/ContentStore/Distributed/NuCache/MachineContentEntry.cs index 75d20cce7..78b10bd6f 100644 --- a/Public/Src/Cache/ContentStore/Distributed/NuCache/MachineContentEntry.cs +++ b/Public/Src/Cache/ContentStore/Distributed/NuCache/MachineContentEntry.cs @@ -304,7 +304,7 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache /// public CompactTime? LatestAccessTime { - get => _latestAccessTime > 0 ? new CompactTime(_latestAccessTime) : null; + get => _latestAccessTime != DefaultInvalidLatestAccessTime ? new CompactTime(_latestAccessTime) : null; } /// @@ -313,7 +313,7 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache /// public CompactTime? EarliestAccessTime { - get => _earliestAccessTime > 0 ? new CompactTime(_earliestAccessTime) : null; + get => _earliestAccessTime != DefaultInvalidEarliestTime ? new CompactTime(_earliestAccessTime) : null; } /// diff --git a/Public/Src/Cache/ContentStore/Distributed/NuCache/NuCacheCollectionUtilities.cs b/Public/Src/Cache/ContentStore/Distributed/NuCache/NuCacheCollectionUtilities.cs index f035c3ba9..1e1f882b8 100644 --- a/Public/Src/Cache/ContentStore/Distributed/NuCache/NuCacheCollectionUtilities.cs +++ b/Public/Src/Cache/ContentStore/Distributed/NuCache/NuCacheCollectionUtilities.cs @@ -109,7 +109,10 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache return DistinctMergeSorted(leftEnumerator, rightEnumerator, getLeftComparable, getRightComparable); } - private static IEnumerable<(TLeft left, TRight right, MergeMode mode)> DistinctMergeSorted( + /// + /// Merges two sorted sequences with no duplicates. + /// + public static IEnumerable<(TLeft left, TRight right, MergeMode mode)> DistinctMergeSorted( IEnumerator leftEnumerator, IEnumerator rightEnumerator, Func getLeftComparable, diff --git a/Public/Src/Cache/ContentStore/Distributed/NuCache/PartitionId.cs b/Public/Src/Cache/ContentStore/Distributed/NuCache/PartitionId.cs new file mode 100644 index 000000000..daa8cd819 --- /dev/null +++ b/Public/Src/Cache/ContentStore/Distributed/NuCache/PartitionId.cs @@ -0,0 +1,84 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT License. + +using System; +using System.Linq; +using System.Text.Json.Serialization; +using BuildXL.Cache.ContentStore.Distributed.MetadataService; +using BuildXL.Cache.ContentStore.Interfaces.Results; +using BuildXL.Cache.ContentStore.Interfaces.Utils; +using BuildXL.Utilities; +using BuildXL.Utilities.Collections; +using BuildXL.Utilities.Tasks; + +namespace BuildXL.Cache.ContentStore.Distributed.NuCache +{ + /// + /// Defines a hash partition encompassing hashes with prefixes from + /// to inclusive. + /// + /// The first included hash prefix of the range + /// The last included hash prefix of the range + public record struct PartitionId(byte StartValue, byte EndValue) + { + private const int MaxPartitionCount = 256; + + /// + /// Number of a hash prefixes included in the partition + /// + public int Width { get; } = (EndValue - StartValue) + 1; + + /// + /// The index of the partition in the ordered list of partitions + /// + public int Index => StartValue / Width; + + /// + /// The total number of partitions. + /// + public int PartitionCount => MaxPartitionCount / Width; + + /// + /// The prefix used for blobs representing this partition + /// + public string BlobPrefix => $"{PartitionCount}/{HexUtilities.BytesToHex(new[] { StartValue })}-{HexUtilities.BytesToHex(new[] { EndValue })}"; + + /// + /// Gets whether the partition contains the hash prefix (i.e. first byte of the hash) + /// + public bool Contains(byte hashPrefix) + { + return StartValue <= hashPrefix && hashPrefix <= EndValue; + } + + /// + public override string ToString() + { + return $"[{StartValue}, {EndValue}]"; + } + + /// + /// Gets an ordered array of partitions for the given count. NOTE: The input is coerced into a valid + /// value (i.e. a power of 2 between 1 and 256 inclusive) + /// + public static ReadOnlyArray GetPartitions(int partitionCount) + { + partitionCount = Math.Min(Math.Max(1, partitionCount), MaxPartitionCount); + + // Round to power of two + var powerOfTwo = Bits.HighestBitSet((uint)partitionCount); + partitionCount = (int)(powerOfTwo < partitionCount ? powerOfTwo << 1 : powerOfTwo); + + var perRangeCount = (MaxPartitionCount + partitionCount - 1) / partitionCount; + + return Enumerable.Range(0, partitionCount) + .Select(i => + { + var start = i * perRangeCount; + var end = Math.Min(byte.MaxValue, (start + perRangeCount) - 1); + return new PartitionId((byte)start, (byte)end); + }) + .ToArray(); + } + } +} diff --git a/Public/Src/Cache/ContentStore/Distributed/NuCache/TransitioningContentLocationStore.cs b/Public/Src/Cache/ContentStore/Distributed/NuCache/TransitioningContentLocationStore.cs index d06a8389a..0bd98615f 100644 --- a/Public/Src/Cache/ContentStore/Distributed/NuCache/TransitioningContentLocationStore.cs +++ b/Public/Src/Cache/ContentStore/Distributed/NuCache/TransitioningContentLocationStore.cs @@ -27,7 +27,7 @@ namespace BuildXL.Cache.ContentStore.Distributed.NuCache /// /// implementation that supports old redis and new local location store. /// - internal class TransitioningContentLocationStore : StartupShutdownBase, IContentLocationStore, IDistributedLocationStore, IDistributedMachineInfo + public class TransitioningContentLocationStore : StartupShutdownBase, IContentLocationStore, IDistributedLocationStore, IDistributedMachineInfo { /// public ILocalContentStore LocalContentStore { get; } diff --git a/Public/Src/Cache/ContentStore/Distributed/Services/ContentLocationStoreServices.cs b/Public/Src/Cache/ContentStore/Distributed/Services/ContentLocationStoreServices.cs index 81c95648e..c3372beb0 100644 --- a/Public/Src/Cache/ContentStore/Distributed/Services/ContentLocationStoreServices.cs +++ b/Public/Src/Cache/ContentStore/Distributed/Services/ContentLocationStoreServices.cs @@ -129,21 +129,25 @@ namespace BuildXL.Cache.ContentStore.Distributed.Services CentralStorage = Create(() => CreateCentralStorage()); BlobContentLocationRegistry = CreateOptional( - () => Dependencies.DistributedContentSettings.InstanceOrDefault()?.LocationStoreSettings?.EnableBlobContentLocationRegistry == true, + () => Dependencies.DistributedContentSettings.InstanceOrDefault()?.LocationStoreSettings?.EnableBlobContentLocationRegistry == true + && !Configuration.DistributedContentConsumerOnly, () => CreateBlobContentLocationRegistry()); } private BlobContentLocationRegistry CreateBlobContentLocationRegistry() { + var baseConfiguration = Arguments.Dependencies.DistributedContentSettings.GetRequiredInstance().LocationStoreSettings.BlobContentLocationRegistrySettings ?? new(); + var database = (RocksDbContentMetadataDatabase)Dependencies.GlobalCacheCheckpointManager.GetRequiredInstance().Database; return new BlobContentLocationRegistry( - new BlobContentLocationRegistryConfiguration(Arguments.Dependencies.DistributedContentSettings.GetRequiredInstance().LocationStoreSettings.BlobContentLocationRegistrySettings ?? new()) + new BlobContentLocationRegistryConfiguration(baseConfiguration) { Credentials = Configuration.AzureBlobStorageCheckpointRegistryConfiguration!.Credentials, }, ClusterStateManager.Instance, - localContentStore: null, // Set to null initially. Will be populated when content location store is initialized similar to LLS. Configuration.PrimaryMachineLocation, - Arguments.Clock); + database, + localContentStore: null, // Set to null initially. Will be populated when content location store is initialized similar to LLS. + clock: Arguments.Clock); } private ClientGlobalCacheStore CreateClientGlobalCacheStore() diff --git a/Public/Src/Cache/ContentStore/Distributed/Services/DistributedContentStoreServices.cs b/Public/Src/Cache/ContentStore/Distributed/Services/DistributedContentStoreServices.cs index b529fc686..3f9a87fb8 100644 --- a/Public/Src/Cache/ContentStore/Distributed/Services/DistributedContentStoreServices.cs +++ b/Public/Src/Cache/ContentStore/Distributed/Services/DistributedContentStoreServices.cs @@ -284,7 +284,7 @@ namespace BuildXL.Cache.ContentStore.Distributed.Services }; if (!GlobalCacheService.IsAvailable - && DistributedContentSettings.EnableGlobalCacheLocationStoreValidation + && DistributedContentSettings.GlobalCacheDatabaseValidationMode != DatabaseValidationMode.None && DistributedContentSettings.GlobalCacheBackgroundRestore) { // Restore checkpoints in checkpoint manager when GCS is unavailable since @@ -371,7 +371,8 @@ namespace BuildXL.Cache.ContentStore.Distributed.Services checkpointManager, RocksDbContentMetadataStore.Instance, eventStream, - clock); + clock, + registry: ContentLocationStoreServices.Instance.BlobContentLocationRegistry.InstanceOrDefault()); return service; } diff --git a/Public/Src/Cache/ContentStore/Distributed/Stores/DistributedContentStore.cs b/Public/Src/Cache/ContentStore/Distributed/Stores/DistributedContentStore.cs index 864e4c5f4..5e3da1948 100644 --- a/Public/Src/Cache/ContentStore/Distributed/Stores/DistributedContentStore.cs +++ b/Public/Src/Cache/ContentStore/Distributed/Stores/DistributedContentStore.cs @@ -170,7 +170,8 @@ namespace BuildXL.Cache.ContentStore.Distributed.Stores string IDistributedContentCopierHost2.ReportCopyResult(OperationContext context, ContentLocation info, CopyFileResult result) { - if (_distributedContentSettings?.EnableGlobalCacheLocationStoreValidation != true + if (_distributedContentSettings == null + || _distributedContentSettings.GlobalCacheDatabaseValidationMode == DatabaseValidationMode.None || GlobalCacheCheckpointManager == null || LocalLocationStore == null || !LocalLocationStore.ClusterState.TryResolveMachineId(info.Machine, out var machineId)) @@ -195,25 +196,41 @@ namespace BuildXL.Cache.ContentStore.Distributed.Stores }; } + bool shouldError = _distributedContentSettings.GlobalCacheDatabaseValidationMode == DatabaseValidationMode.LogAndError + && result.Succeeded + && info.Origin == GetBulkOrigin.Local + && !isPresentInGcs; + + // Represent various aspects as parameters in operation so + // they show up as separate dimensions on the metric in MDM. var presence = isPresentInGcs ? "GcsContains" : "GcsMissing"; + var operationResult = new OperationResult( + message: presence, + operationName: presence, + tracerName: $"{nameof(DistributedContentStore)}.{nameof(IDistributedContentCopierHost2.ReportCopyResult)}", + status: result.GetStatus(), + duration: result.Duration, + operationKind: originToKind(info.Origin), + exception: result.Exception, + operationId: context.TracingContext.TraceId, + severity: Severity.Debug); + // Directly calls IOperationLogger interface because we don't want to log a message, just surface a complex metric - if (context.TracingContext.Logger is IOperationLogger logger) + if (shouldError && context.TracingContext.Logger is IStructuredLogger slog) { - // Represent various aspects as parameters in operation so - // they show up as separate dimensions on the metric in MDM. - logger.OperationFinished(new OperationResult( - message: "", - operationName: presence, - tracerName: $"{nameof(DistributedContentStore)}.{nameof(IDistributedContentCopierHost2.ReportCopyResult)}", - status: result.GetStatus(), - duration: result.Duration, - operationKind: originToKind(info.Origin), - exception: result.Exception, - operationId: context.TracingContext.TraceId, - severity: Severity.Debug)); + slog.LogOperationFinished(operationResult); + } + else if (context.TracingContext.Logger is IOperationLogger logger) + { + logger.OperationFinished(operationResult); } - return $"IsPresentInGcs=[{isPresentInGcs}]"; + if (shouldError) + { + new ErrorResult($"Content found in LLS but not GCS DB. {info.Hash} CopyResult={result}").ThrowIfFailure(); + } + + return $"Origin=[{info.Origin}] IsPresentInGcs=[{isPresentInGcs}]"; } private Task> CreateCopySession(Context context) diff --git a/Public/Src/Cache/ContentStore/Distributed/Utilities/JsonUtilities.cs b/Public/Src/Cache/ContentStore/Distributed/Utilities/JsonUtilities.cs index 41179892b..adc9bc80f 100644 --- a/Public/Src/Cache/ContentStore/Distributed/Utilities/JsonUtilities.cs +++ b/Public/Src/Cache/ContentStore/Distributed/Utilities/JsonUtilities.cs @@ -35,10 +35,20 @@ namespace BuildXL.Cache.ContentStore.Distributed.Utilities public static JsonSerializerOptions IndentedSerializationOptions { get; } = GetOptions(indent: true); + /// + /// Options used when reading deployment configuration + /// + public static JsonDocumentOptions DefaultDocumentOptions { get; } = new JsonDocumentOptions() + { + AllowTrailingCommas = true, + CommentHandling = JsonCommentHandling.Skip + }; + private static JsonSerializerOptions GetOptions(bool indent) { return new JsonSerializerOptions() { + IgnoreReadOnlyProperties = true, AllowTrailingCommas = true, ReadCommentHandling = JsonCommentHandling.Skip, WriteIndented = indent, @@ -51,6 +61,9 @@ namespace BuildXL.Cache.ContentStore.Distributed.Utilities FuncJsonConverter.Create(ReadMachineId, (writer, value) => writer.WriteNumberValue(value.Index)), FuncJsonConverter.Create(ReadMachineLocation, (writer, value) => writer.WriteStringValue(value.Path)), FuncJsonConverter.Create(ReadShortHash, (writer, value) => writer.WriteStringValue(value.HashType == HashType.Unknown ? null : value.ToString())), + FuncJsonConverter.Create(ReadShardHash, (writer, value) => writer.WriteStringValue(value.ToShortHash().HashType == HashType.Unknown ? null : value.ToShortHash().ToString())), + FuncJsonConverter.Create(ReadCompactTime, (writer, value) => writer.WriteStringValue(value.ToDateTime())), + FuncJsonConverter.Create(ReadCompactSize, (writer, value) => writer.WriteNumberValue(value.Value)), } }; } @@ -68,7 +81,7 @@ namespace BuildXL.Cache.ContentStore.Distributed.Utilities return new MachineLocation(property.GetString()); } - + var data = reader.GetString(); return data == null ? default : new MachineLocation(data); } @@ -78,6 +91,21 @@ namespace BuildXL.Cache.ContentStore.Distributed.Utilities return new MachineId(reader.GetInt32()); } + private static ShardHash ReadShardHash(ref Utf8JsonReader reader) + { + return ReadShortHash(ref reader).AsEntryKey(); + } + + private static CompactSize ReadCompactSize(ref Utf8JsonReader reader) + { + return reader.GetInt64(); + } + + private static CompactTime ReadCompactTime(ref Utf8JsonReader reader) + { + return reader.GetDateTime(); + } + private static ShortHash ReadShortHash(ref Utf8JsonReader reader) { if (reader.TokenType == JsonTokenType.StartObject) @@ -90,15 +118,6 @@ namespace BuildXL.Cache.ContentStore.Distributed.Utilities return data == null ? default : new ShortHash(data); } - /// - /// Options used when reading deployment configuration - /// - public static JsonDocumentOptions DefaultDocumentOptions { get; } = new JsonDocumentOptions() - { - AllowTrailingCommas = true, - CommentHandling = JsonCommentHandling.Skip - }; - /// /// Serialize the value to json using /// @@ -120,6 +139,18 @@ namespace BuildXL.Cache.ContentStore.Distributed.Utilities /// public static ValueTask JsonDeserializeAsync(Stream value) { +#if NETCOREAPP + if (value is MemoryStream memoryStream && memoryStream.TryGetBuffer(out var buffer)) + { + // JsonSerializer.DeserializeAsync can fail on reading large streams if there is a value which + // must be skipped. Workaround this in some cases where the data is in memory + // and using the synchronous API. + var doc = JsonDocument.Parse(buffer.AsMemory(), DefaultDocumentOptions); + var result = JsonSerializer.Deserialize(doc, DefaultSerializationOptions); + return new ValueTask(result); + } +#endif + return JsonSerializer.DeserializeAsync(value, DefaultSerializationOptions); } diff --git a/Public/Src/Cache/ContentStore/DistributedTest/ContentLocation/AzureBlobStorageMasterElectionMechanismTests.cs b/Public/Src/Cache/ContentStore/DistributedTest/ContentLocation/AzureBlobStorageMasterElectionMechanismTests.cs index d53172653..8a89a40d7 100644 --- a/Public/Src/Cache/ContentStore/DistributedTest/ContentLocation/AzureBlobStorageMasterElectionMechanismTests.cs +++ b/Public/Src/Cache/ContentStore/DistributedTest/ContentLocation/AzureBlobStorageMasterElectionMechanismTests.cs @@ -343,9 +343,9 @@ namespace BuildXL.Cache.ContentStore.Distributed.Test.ContentLocation Clock = clock; } - public Task OnRoleUpdatedAsync(OperationContext context, Role role) + public Task OnRoleUpdatedAsync(OperationContext context, MasterElectionState electionState) { - return RoleQueue.Writer.WriteAsync((role, Clock.UtcNow)).AsTask(); + return RoleQueue.Writer.WriteAsync((electionState.Role, Clock.UtcNow)).AsTask(); } public async Task WaitTillCurrentRoleAsync() @@ -455,4 +455,12 @@ namespace BuildXL.Cache.ContentStore.Distributed.Test.ContentLocation await machine.ShutdownAsync(operationContext).ThrowIfFailureAsync(); } } + + public static class TestRoleObserverExtensions + { + public static Task OnRoleUpdatedAsync(this IRoleObserver observer, OperationContext context, Role role) + { + return observer.OnRoleUpdatedAsync(context, new MasterElectionState(default, role, default)); + } + } } diff --git a/Public/Src/Cache/ContentStore/DistributedTest/ContentLocation/BlobContentLocationRegistryTests.cs b/Public/Src/Cache/ContentStore/DistributedTest/ContentLocation/BlobContentLocationRegistryTests.cs index a3d9dbe6d..7b943694c 100644 --- a/Public/Src/Cache/ContentStore/DistributedTest/ContentLocation/BlobContentLocationRegistryTests.cs +++ b/Public/Src/Cache/ContentStore/DistributedTest/ContentLocation/BlobContentLocationRegistryTests.cs @@ -4,16 +4,22 @@ using System; using System.Collections.Concurrent; using System.Collections.Generic; +using System.ComponentModel; using System.Diagnostics; using System.IO; using System.Linq; using System.Runtime.CompilerServices; using System.Runtime.InteropServices; +using System.Text; +using System.Text.Json; using System.Threading; using System.Threading.Tasks; +using BuildXL.Cache.ContentStore.Distributed.MetadataService; using BuildXL.Cache.ContentStore.Distributed.NuCache; using BuildXL.Cache.ContentStore.Distributed.Utilities; +using BuildXL.Cache.ContentStore.FileSystem; using BuildXL.Cache.ContentStore.Hashing; +using BuildXL.Cache.ContentStore.Interfaces.FileSystem; using BuildXL.Cache.ContentStore.Interfaces.Results; using BuildXL.Cache.ContentStore.Interfaces.Secrets; using BuildXL.Cache.ContentStore.Interfaces.Time; @@ -26,13 +32,20 @@ using BuildXL.Cache.ContentStore.Tracing; using BuildXL.Cache.ContentStore.Tracing.Internal; using BuildXL.Cache.ContentStore.UtilitiesCore; using BuildXL.Cache.ContentStore.Utils; +using BuildXL.Cache.Host.Service; +using BuildXL.Engine.Cache.KeyValueStores; +using BuildXL.Utilities.Collections; using ContentStoreTest.Distributed.Redis; using ContentStoreTest.Test; using FluentAssertions; +using Microsoft.WindowsAzure.Storage; +using Microsoft.WindowsAzure.Storage.Blob; +using RocksDbSharp; using Test.BuildXL.TestUtilities.Xunit; using Xunit; using Xunit.Abstractions; using static BuildXL.Cache.ContentStore.Distributed.NuCache.BlobContentLocationRegistry; +using OperationContext = BuildXL.Cache.ContentStore.Tracing.Internal.OperationContext; #nullable enable annotations @@ -40,26 +53,32 @@ namespace BuildXL.Cache.ContentStore.Distributed.Test.ContentLocation { [Collection("Redis-based tests")] [Trait("Category", "WindowsOSOnly")] // 'redis-server' executable no longer exists - public class BlobContentLocationRegistryTests : TestWithOutput +#if !NETCOREAPP + [TestClassIfSupported(TestRequirements.NotSupported)] +#endif + public class BlobContentLocationRegistryTests : TestBase { private readonly static MachineLocation M1 = new MachineLocation("M1"); private readonly LocalRedisFixture _fixture; public BlobContentLocationRegistryTests(LocalRedisFixture fixture, ITestOutputHelper output) - : base(output) + : base(TestGlobal.Logger, output) { _fixture = fixture; } + [Fact] public void TestCompat() { - // Changing this value is a breaking change!! An epoch reset is needed if this changed. + // Changing these values are a breaking change!! An epoch reset is needed if they are changed. MachineRecord.MaxBlockLength.Should().Be(56); + + Unsafe.SizeOf().Should().Be(24); } [Theory] - [MemberData(nameof(TruthTable.GetTable), 2, MemberType = typeof(TruthTable))] - public async Task TestBasicRegister(bool concurrent, bool expireMachine) + [MemberData(nameof(TruthTable.GetTable), 3, MemberType = typeof(TruthTable))] + public async Task TestBasicRegister(bool concurrent, bool expireMachine, bool useMaxPartitionCount) { // Increase to find flaky test if needed int iterationCount = 1; @@ -68,6 +87,9 @@ namespace BuildXL.Cache.ContentStore.Distributed.Test.ContentLocation await RunTest(async context => { var excludedMachine = context.Machines[1]; + var firstHash = new ContentHash(HashType.MD5, Guid.Empty.ToByteArray()); + excludedMachine.Store.Add(firstHash, DateTime.UtcNow, 42); + bool excludeHeartbeat(TestMachine machine) { return expireMachine && machine == excludedMachine; @@ -82,6 +104,8 @@ namespace BuildXL.Cache.ContentStore.Distributed.Test.ContentLocation await context.HeartbeatAllMachines(); await context.UpdateAllMachinePartitionsAsync(concurrent); + await CheckAllPartitionsAsync(context, isExcluded: null); + context.Arguments.Clock.UtcNow += TimeSpan.FromHours(5); // Heartbeat all machine so all machines are aware of each other and the selected machine has expired @@ -92,44 +116,75 @@ namespace BuildXL.Cache.ContentStore.Distributed.Test.ContentLocation await CheckAllPartitionsAsync(context, excludeContentByLocation); }, - machineCount: concurrent ? 10 : 3); + machineCount: concurrent ? 10 : 3, + useMaxPartitionCount: useMaxPartitionCount); } } // Only include some partitions to keep test run times reasonable - private static readonly byte[] IncludedPartitions = new byte[] { 0, 4, 5, 6, 25, 255 }; - //private static readonly byte[] IncludedPartitions = new byte[] { 4 }; + private static readonly ReadOnlyArray MaxPartitionCountIncludedPartitions = new byte[] { 0, 4, 5, 6, 25, 255 } + .Select(i => PartitionId.GetPartitions(256)[i]) + .ToArray(); + + private static readonly ReadOnlyArray FewPartitionCountIncludedPartitions = PartitionId.GetPartitions(1).ToArray(); private static async Task CheckAllPartitionsAsync(TestContext context, Func isExcluded) { + var primary = context.PrimaryMachine; + context.ActualContent.Clear(); + context.Arguments.Context.TracingContext.Debug("--- Checking all partitions ---", nameof(BlobContentLocationRegistryTests)); - foreach (var partition in IncludedPartitions) + foreach (var partition in context.IncludedPartitions) { - var listing = await context.PrimaryMachine.Registry.ComputeSortedPartitionContentAsync(context, partition); + var listing = await primary.Registry.ComputeSortedPartitionContentAsync(context, partition); listing.EntrySpan.Length.Should().Be(listing.FullSpanForTests.Length, $"Partition={partition}"); context.CheckListing(listing, partition); context.Arguments.Context.TracingContext.Debug($"Partition={partition} Entries={listing.EntrySpan.Length}", nameof(BlobContentLocationRegistryTests)); } - var expectedContent = context.ExpectedContent.Where(t => isExcluded?.Invoke(t.Machine) != true); + var expectedContent = context.ExpectedContent.Where(t => isExcluded?.Invoke(t.Machine) != true).ToHashSet(); // Using this instead for set equivalence since the built-in equivalence is very slow. context.ActualContent.Except(expectedContent).Should().BeEmpty("Found unexpected content. Actual content minus expected content should be empty"); expectedContent.Except(context.ActualContent).Should().BeEmpty("Could not find expected content. Expected content minus actual content should be empty"); + + // Ensure database is updated by setting time after interval and triggering update + context.Clock.UtcNow += TestContext.PartitionsUpdateInterval + TimeSpan.FromSeconds(10); + await context.UpdateMachinePartitionsAsync(primary).ShouldBeSuccess(); + + var clusterState = primary.ClusterStateManager.ClusterState; + var actualDbContent = context.ExpectedContent.Take(0).ToHashSet(); + + primary.Database.IterateSstMergeContentEntries(context, entry => + { + entry.Location.IsAdd.Should().BeTrue(); + clusterState.TryResolve(entry.Location.AsMachineId(), out var location).Should().BeTrue(); + var actualEntry = (entry.Hash, location, entry.Size.Value); + expectedContent.Should().Contain(actualEntry); + actualDbContent.Add(actualEntry).Should().BeTrue(); + + }).ShouldBeSuccess().Value.ReachedEnd.Should().BeTrue(); + + expectedContent.Except(actualDbContent).Should().BeEmpty("Could not find expected content in database. Expected content minus actual content should be empty"); } private async Task RunTest( Func runTest, int machineCount = 3, double machineContentFraction = 0.5, - int totalUniqueContent = 10000) + int totalUniqueContent = 10000, + bool useMaxPartitionCount = true) { + var includedPartitions = useMaxPartitionCount + ? MaxPartitionCountIncludedPartitions + : FewPartitionCountIncludedPartitions; + var tracingContext = new Context(TestGlobal.Logger); var context = new OperationContext(tracingContext); var clock = new MemoryClock(); using var storage = AzuriteStorageProcess.CreateAndStartEmpty(_fixture, TestGlobal.Logger); - var arguments = new TestContextArguments(context, storage.ConnectionString, clock); + var arguments = new TestContextArguments(context, storage.ConnectionString, clock, TestRootDirectoryPath, includedPartitions); var testContext = new TestContext(arguments); await testContext.StartupAsync(context).ThrowIfFailureAsync(); @@ -158,11 +213,6 @@ namespace BuildXL.Cache.ContentStore.Distributed.Test.ContentLocation var maxSizeMask = (1L << maxSizeOffset) - 1; var size = MemoryMarshal.Read(MemoryMarshal.AsBytes(stackalloc[] { hash.ToFixedBytes() })) & maxSizeMask; - if (IncludedPartitions.Contains(hash[0])) - { - testContext.ExpectedContent.Add(((ShortHash)hash, machine.Location, size)); - } - machine.Store.Add( hashes[hashIndex], accessTime, @@ -174,7 +224,7 @@ namespace BuildXL.Cache.ContentStore.Distributed.Test.ContentLocation await testContext.ShutdownAsync(context).ThrowIfFailureAsync(); } - private class MockContentStore : TestObserver, ILocalContentStore + private record MockContentStore(TestContext Context, MachineLocation Location) : ILocalContentStore { private ConcurrentDictionary InfoMap { get; } = new(); private int[] PartitionCounts { get; } = new int[256]; @@ -210,26 +260,28 @@ namespace BuildXL.Cache.ContentStore.Distributed.Test.ContentLocation public void Add(ContentHash hash, DateTime accessTime, long size) { + ShortHash shortHash = hash; + var partition = shortHash[0]; if (InfoMap.TryAdd(hash, new ContentInfo(hash, size, accessTime))) { - var partition = MemoryMarshal.Read(MemoryMarshal.AsBytes(stackalloc[] { hash.ToFixedBytes() })); PartitionCounts[partition]++; - } - } - - internal override void OnPutBlock(byte partitionId, ContentListing partition) - { - if (PartitionCounts[partitionId] != partition.EntrySpan.Length) - { - + if (Context.IncludedPartitionIndices.Contains(hash[0])) + { + Context.ExpectedContent.Add((shortHash, Location, size)); + } } } } - private record TestContextArguments(OperationContext Context, string ConnectionString, MemoryClock Clock); + private record TestContextArguments(OperationContext Context, + string ConnectionString, + MemoryClock Clock, + AbsolutePath Path, + ReadOnlyArray IncludedPartitions); private record TestMachine( int Index, + RocksDbContentMetadataDatabase Database, MockContentStore Store, ClusterStateManager ClusterStateManager, BlobContentLocationRegistry Registry, @@ -247,8 +299,16 @@ namespace BuildXL.Cache.ContentStore.Distributed.Test.ContentLocation public KeyedList Machines { get; } = new(); + public static readonly TimeSpan PartitionsUpdateInterval = TimeSpan.FromMinutes(5); + public TestMachine PrimaryMachine => Machines[0]; + public ReadOnlyArray IncludedPartitions => Arguments.IncludedPartitions; + public byte[] IncludedPartitionIndices => Arguments.IncludedPartitions + .SelectMany(p => Enumerable.Range(p.StartValue, (p.EndValue - p.StartValue) + 1)) + .Select(i => (byte)i) + .ToArray(); + public HashSet<(ShortHash Hash, MachineLocation Machine, long Size)> ExpectedContent = new(); public HashSet<(ShortHash Hash, MachineLocation Machine, long Size)> ActualContent = new(); @@ -256,24 +316,39 @@ namespace BuildXL.Cache.ContentStore.Distributed.Test.ContentLocation public MemoryClock Clock => Arguments.Clock; + public DisposableDirectory Directory { get; } + + public Guid TestUniqueId = Guid.NewGuid(); + public TestContext(TestContextArguments data) { Arguments = data; + + Directory = new DisposableDirectory(PassThroughFileSystem.Default, data.Path); + } + + protected override void DisposeCore() + { + Directory.Dispose(); } public Task CreateAndStartAsync() { + var index = Machines.Count; var configuration = new BlobContentLocationRegistryConfiguration() { + FolderName = TestUniqueId.ToString(), Credentials = new AzureBlobStorageCredentials(Arguments.ConnectionString), PerPartitionDelayInterval = TimeSpan.Zero, - PartitionsUpdateInterval = TimeSpan.FromMinutes(5), - UpdateInBackground = false + PartitionsUpdateInterval = PartitionsUpdateInterval, + UpdateInBackground = false, + PartitionCount = IncludedPartitions[0].PartitionCount, + UpdateDatabase = index == 0 }; var location = GetRandomLocation(); - var store = new MockContentStore(); + var store = new MockContentStore(this, location); var clusterStateManager = new ClusterStateManager( new LocalLocationStoreConfiguration() @@ -289,15 +364,22 @@ namespace BuildXL.Cache.ContentStore.Distributed.Test.ContentLocation Clock), Clock); + var database = new RocksDbContentMetadataDatabase( + Clock, + new RocksDbContentMetadataDatabaseConfiguration(Directory.Path / Machines.Count.ToString()) + { + UseMergeOperators = true + }); + var registry = new BlobContentLocationRegistry( configuration, clusterStateManager, - store, location, - Arguments.Clock); + database, + store, + Clock); - var machine = new TestMachine(Machines.Count, store, clusterStateManager, registry, location); - registry.Observer = store; + var machine = new TestMachine(index, database, store, clusterStateManager, registry, location); Machines.Add(machine); @@ -318,7 +400,6 @@ namespace BuildXL.Cache.ContentStore.Distributed.Test.ContentLocation // Heartbeat machine so it knows about other machines await machine.ClusterStateManager.HeartbeatAsync(this, MachineState.Open).ShouldBeSuccess(); - } } } @@ -349,6 +430,9 @@ namespace BuildXL.Cache.ContentStore.Distributed.Test.ContentLocation public Task UpdateMachinePartitionsAsync(TestMachine machine) { + // Ensure the primary machine is updating its database + PrimaryMachine.Registry.SetDatabaseUpdateLeaseExpiry(Clock.UtcNow + TimeSpan.FromMinutes(5)); + return machine.Registry.UpdatePartitionsAsync(this, excludePartition: partition => !IncludedPartitions.Contains(partition)) .ThrowIfFailureAsync(s => s); @@ -379,7 +463,7 @@ namespace BuildXL.Cache.ContentStore.Distributed.Test.ContentLocation return result; } - internal void CheckListing(ContentListing listing, byte partitionId) + internal void CheckListing(ContentListing listing, PartitionId partitionId) { var clusterState = PrimaryMachine.ClusterStateManager.ClusterState; var entries = listing.EntrySpan.ToArray(); @@ -387,8 +471,8 @@ namespace BuildXL.Cache.ContentStore.Distributed.Test.ContentLocation { var entry = entries[i]; - entry.PartitionId.Should().Be(partitionId); - entry.Hash[0].Should().Be(partitionId); + entry.PartitionId.Should().Be(entry.Hash[0]); + partitionId.Contains(entry.PartitionId).Should().BeTrue(); clusterState.TryResolve(entry.Location.AsMachineId(), out var machineLocation).Should().BeTrue(); ActualContent.Add((entry.Hash, machineLocation, entry.Size.Value)).Should().BeTrue(); diff --git a/Public/Src/Cache/ContentStore/DistributedTest/ContentLocation/BlobFolderStorageTests.cs b/Public/Src/Cache/ContentStore/DistributedTest/ContentLocation/BlobFolderStorageTests.cs index 52abb7179..e99a495b9 100644 --- a/Public/Src/Cache/ContentStore/DistributedTest/ContentLocation/BlobFolderStorageTests.cs +++ b/Public/Src/Cache/ContentStore/DistributedTest/ContentLocation/BlobFolderStorageTests.cs @@ -2,7 +2,9 @@ // Licensed under the MIT License. using System; +using System.IO; using System.Runtime.CompilerServices; +using System.Text; using System.Threading; using System.Threading.Tasks; using BuildXL.Cache.ContentStore.Distributed.NuCache; @@ -57,6 +59,35 @@ namespace BuildXL.Cache.ContentStore.Distributed.Test.ContentLocation _fixture = fixture; } + /// + /// This test is for a bug in Azurite (the Azure storage emulator) + /// where creating a snapshot causes PutBlock operations with a lease to fail. + /// + [Fact] + public async Task TestStorage() + { + using var storage = AzuriteStorageProcess.CreateAndStartEmpty(_fixture, TestGlobal.Logger); + + var creds = new AzureBlobStorageCredentials(storage.ConnectionString); + + var client = creds.CreateCloudBlobClient(); + + var container = client.GetContainerReference("test"); + + await container.CreateIfNotExistsAsync(); + + var blob = container.GetBlockBlobReference("test/sub/blob.out.bin"); + + var bytes = Encoding.UTF8.GetBytes("hello"); + await blob.UploadFromByteArrayAsync(bytes, 0, bytes.Length); + + var leaseId = await blob.AcquireLeaseAsync(TimeSpan.FromSeconds(60)); + + var snapshot = await blob.SnapshotAsync(); + + await blob.PutBlockAsync("0000", new MemoryStream(), null, Microsoft.WindowsAzure.Storage.AccessCondition.GenerateLeaseCondition(leaseId), null, null); + } + [Fact] public Task CreatesMissingContainerOnWrite() { diff --git a/Public/Src/Cache/ContentStore/DistributedTest/ContentLocation/BlobLocationRegistryDistributedContentTests.cs b/Public/Src/Cache/ContentStore/DistributedTest/ContentLocation/BlobLocationRegistryDistributedContentTests.cs new file mode 100644 index 000000000..dde49b26d --- /dev/null +++ b/Public/Src/Cache/ContentStore/DistributedTest/ContentLocation/BlobLocationRegistryDistributedContentTests.cs @@ -0,0 +1,113 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT License. + +using System; +using System.Text.RegularExpressions; +using System.Threading.Tasks; +using BuildXL.Cache.ContentStore.Interfaces.Results; +using BuildXL.Cache.ContentStore.InterfacesTest.Results; +using BuildXL.Cache.Host.Configuration; +using ContentStoreTest.Distributed.Redis; +using FluentAssertions; +using Test.BuildXL.TestUtilities.Xunit; +using Xunit; +using Xunit.Abstractions; + +namespace ContentStoreTest.Distributed.Sessions +{ + [Trait("Category", "Integration")] + [Trait("Category", "LongRunningTest")] + [Collection("Redis-based tests")] +#if !NETCOREAPP + [TestClassIfSupported(TestRequirements.NotSupported)] +#endif + public class BlobLocationRegistryDistributedContentTests : LocalLocationStoreDistributedContentTests + { + /// + public BlobLocationRegistryDistributedContentTests(LocalRedisFixture redis, ITestOutputHelper output) + : base(redis, output) + { + } + + // Disable flaky test. + public override Task PinWithUnverifiedCountAndStartCopyWithThreshold(int threshold) + { + return Task.CompletedTask; + } + + protected override TestDistributedContentSettings ModifySettings(TestDistributedContentSettings dcs) + { + if (dcs.IsMasterEligible) + { + // Enable GCS on the master machine so checkpoint can be created. + dcs.MemoizationContentMetadataStoreModeOverride = ContentMetadataStoreMode.WriteBothPreferRedis; + dcs.ContentMetadataEnableResilience = true; + } + else + { + // Prevent workers from processing partition to simplify test logging so that + // partition writes only come from one machine (typically during CreateCheckpointAsync below) + dcs.LocationStoreSettings.BlobContentLocationRegistrySettings.ProcessPartitions = false; + } + + dcs.LocationStoreSettings.BlobContentLocationRegistrySettings.PartitionsUpdateInterval = "1m"; + dcs.LocationStoreSettings.BlobContentLocationRegistrySettings.PartitionCount = 2; + dcs.LocationStoreSettings.BlobContentLocationRegistrySettings.PerPartitionDelayInterval = TimeSpan.Zero; + dcs.LocationStoreSettings.BlobContentLocationRegistrySettings.UpdateInBackground = false; + dcs.LocationStoreSettings.BlobContentLocationRegistrySettings.UpdateDatabase = true; + dcs.LocationStoreSettings.EnableBlobContentLocationRegistry = true; + dcs.GlobalCacheDatabaseValidationMode = DatabaseValidationMode.LogAndError; + dcs.ContentMetadataUseMergeWrites = true; + dcs.EnableIndependentBackgroundMasterElection = true; + return base.ModifySettings(dcs); + } + + protected override async Task RestoreCheckpointAsync(InstanceRef storeRef, TestContext context) + { + await base.RestoreCheckpointAsync(storeRef, context).ShouldBeSuccess(); + + var index = storeRef.ResolveIndex(context); + + var checkpointManager = context.GetServices(index).Dependencies.GlobalCacheCheckpointManager.GetRequiredInstance(); + + await checkpointManager.PeriodicRestoreCheckpointAsync(context).ShouldBeSuccess(); + + return BoolResult.Success; + } + + protected override async Task CreateCheckpointAsync(InstanceRef storeRef, TestContext context) + { + var index = storeRef.ResolveIndex(context); + index.Should().Be(context.GetMasterIndex()); + + await UpdateAllRegistriesAsync(context, index); + + await base.CreateCheckpointAsync(storeRef, context).ShouldBeSuccess(); + + var registry = context.GetBlobContentLocationRegistry(index); + + // Increment time to ensure database is updated on master + TestClock.UtcNow += TimeSpan.FromMinutes(2); + await registry.UpdatePartitionsAsync(context).ShouldBeSuccess(); + + var services = context.GetServices(index); + var service = context.GetContentMetadataService(index); + await service.CreateCheckpointAsync(context).ShouldBeSuccess(); + + return BoolResult.Success; + } + + private static async Task UpdateAllRegistriesAsync(TestContext context, int? excludeIndex = null) + { + for (int i = 0; i < context.Stores.Count; i++) + { + if (i != excludeIndex) + { + var registry = context.GetBlobContentLocationRegistry(i); + + await registry.UpdatePartitionsAsync(context).ShouldBeSuccess(); + } + } + } + } +} \ No newline at end of file diff --git a/Public/Src/Cache/ContentStore/DistributedTest/ContentLocation/ContentListingTests.cs b/Public/Src/Cache/ContentStore/DistributedTest/ContentLocation/ContentListingTests.cs index cb3bb4fa6..37952dc7f 100644 --- a/Public/Src/Cache/ContentStore/DistributedTest/ContentLocation/ContentListingTests.cs +++ b/Public/Src/Cache/ContentStore/DistributedTest/ContentLocation/ContentListingTests.cs @@ -51,6 +51,24 @@ namespace BuildXL.Cache.ContentStore.Distributed.Test.ContentLocation _fixture = fixture; } + [Fact] + public void TestPartitionIds() + { + for (int i = 1; i < 256; i++) + { + var ids = PartitionId.GetPartitions(i); + ids.Length.Should().BeGreaterOrEqualTo(i); + ids.Length.Should().BeLessOrEqualTo(i * 2); + ids.All(id => id.Width == ids[0].Width).Should().BeTrue(); + ids.All(id => id.EndValue >= id.StartValue).Should().BeTrue(); + + for (int j = 1; j < ids.Length; j++) + { + ids[j].StartValue.Should().BeGreaterThan(ids[j - 1].EndValue); + } + } + } + [Fact] public void TestEntryFormat() { diff --git a/Public/Src/Cache/ContentStore/DistributedTest/ContentLocation/ContentMetadataStoreDistributedContentTests.cs b/Public/Src/Cache/ContentStore/DistributedTest/ContentLocation/ContentMetadataStoreDistributedContentTests.cs index ef8780132..488e792ca 100644 --- a/Public/Src/Cache/ContentStore/DistributedTest/ContentLocation/ContentMetadataStoreDistributedContentTests.cs +++ b/Public/Src/Cache/ContentStore/DistributedTest/ContentLocation/ContentMetadataStoreDistributedContentTests.cs @@ -8,6 +8,7 @@ using System.Threading.Tasks; using BuildXL.Cache.ContentStore.Distributed; using BuildXL.Cache.ContentStore.Distributed.MetadataService; using BuildXL.Cache.ContentStore.Distributed.NuCache; +using BuildXL.Cache.ContentStore.Distributed.Test.MetadataService; using BuildXL.Cache.ContentStore.Interfaces.Results; using BuildXL.Cache.ContentStore.Interfaces.Sessions; using BuildXL.Cache.ContentStore.Interfaces.Time; @@ -51,8 +52,9 @@ namespace ContentStoreTest.Distributed.Sessions protected override DistributedCacheServiceArguments ModifyArguments(DistributedCacheServiceArguments arguments) { - arguments.Configuration.DistributedContentSettings.EnableGlobalCacheLocationStoreValidation = true; + arguments.Configuration.DistributedContentSettings.GlobalCacheDatabaseValidationMode = DatabaseValidationMode.Log; arguments.Configuration.DistributedContentSettings.ContentMetadataUseMergeWrites = UseMergeOperators; + arguments.Configuration.DistributedContentSettings.EnableIndependentBackgroundMasterElection = true; return base.ModifyArguments(arguments); } diff --git a/Public/Src/Cache/ContentStore/DistributedTest/ContentLocation/LocalLocationStoreDistributedContentTests.cs b/Public/Src/Cache/ContentStore/DistributedTest/ContentLocation/LocalLocationStoreDistributedContentTests.cs index 2551cc0ab..df13ac3b2 100644 --- a/Public/Src/Cache/ContentStore/DistributedTest/ContentLocation/LocalLocationStoreDistributedContentTests.cs +++ b/Public/Src/Cache/ContentStore/DistributedTest/ContentLocation/LocalLocationStoreDistributedContentTests.cs @@ -476,7 +476,7 @@ namespace ContentStoreTest.Distributed.Sessions [Theory] [InlineData(1)] [InlineData(0)] - public Task PinWithUnverifiedCountAndStartCopy(int threshold) + public virtual Task PinWithUnverifiedCountAndStartCopyWithThreshold(int threshold) { _overrideDistributed = s => { @@ -1615,12 +1615,12 @@ namespace ContentStoreTest.Distributed.Sessions await workerStore.RegisterLocalLocationAsync(context, new[] { new ContentHashWithSize(hash, 120) }, Token, UrgencyHint.Nominal, touch: true).ShouldBeSuccess(); TestClock.UtcNow += TimeSpan.FromMinutes(2); - await masterStore.LocalLocationStore.HeartbeatAsync(context).ShouldBeSuccess(); + await CreateCheckpointAsync(masterStore, context).ShouldBeSuccess(); for (int sessionIndex = 0; sessionIndex < storeCount; sessionIndex++) { // Heartbeat to ensure machine receives checkpoint - await context.GetLocationStore(sessionIndex).LocalLocationStore.HeartbeatAsync(context).ShouldBeSuccess(); + await RestoreCheckpointAsync(sessionIndex, context).ShouldBeSuccess(); // Pin the content in the session which should fail with content not found await PinContentForSession(sessionIndex).ShouldBeContentNotFound(); @@ -2006,13 +2006,13 @@ namespace ContentStoreTest.Distributed.Sessions TestClock.UtcNow += TimeSpan.FromMinutes(masterLeaseExpiryTime.TotalMinutes / 2); // Save checkpoint by heartbeating master - await masterRedisStore.LocalLocationStore.HeartbeatAsync(context).ShouldBeSuccess(); + await CreateCheckpointAsync(masterRedisStore, context).ShouldBeSuccess(); // Verify file was uploaded // Verify file was skipped (if not first iteration) // Restore checkpoint by heartbeating worker - await workerRedisStore.LocalLocationStore.HeartbeatAsync(context).ShouldBeSuccess(); + await RestoreCheckpointAsync(workerRedisStore, context).ShouldBeSuccess(); // Files should be uploaded by master and downloaded by worker diff(masterRedisStore.LocalLocationStore.Counters, masterCounters, ContentLocationStoreCounters.IncrementalCheckpointFilesUploaded).Should().BePositive(); @@ -2152,10 +2152,10 @@ namespace ContentStoreTest.Distributed.Sessions TestClock.UtcNow += TimeSpan.FromMinutes(masterLeaseExpiryTime.TotalMinutes / 2); // Save checkpoint by heartbeating master - await masterRedisStore.LocalLocationStore.HeartbeatAsync(context).ShouldBeSuccess(); + await CreateCheckpointAsync(masterRedisStore, context).ShouldBeSuccess(); // Restore checkpoint by heartbeating worker - await workerRedisStore.LocalLocationStore.HeartbeatAsync(context).ShouldBeSuccess(); + await RestoreCheckpointAsync(workerRedisStore, context).ShouldBeSuccess(); // Files should be uploaded by master and downloaded by worker diff(masterRedisStore.LocalLocationStore.Counters, masterCounters, ContentLocationStoreCounters.IncrementalCheckpointFilesUploaded).Should().BePositive(); @@ -3046,38 +3046,6 @@ namespace ContentStoreTest.Distributed.Sessions return putResult0.ContentHash; } - private async Task UploadCheckpointOnMasterAndRestoreOnWorkers(TestContext context, bool reconcile = false, string clearStoragePrefix = null) - { - // Update time to trigger checkpoint upload and restore on master and workers respectively - TestClock.UtcNow += TimeSpan.FromMinutes(2); - - var masterStore = context.GetMaster(); - - // Heartbeat master first to upload checkpoint - await masterStore.LocalLocationStore.HeartbeatAsync(context).ShouldBeSuccess(); - - if (reconcile) - { - await masterStore.ReconcileAsync(context, force: true).ShouldBeSuccess(); - } - - if (clearStoragePrefix != null) - { - await StorageProcess.ClearAsync(clearStoragePrefix); - } - - // Next heartbeat workers to restore checkpoint - foreach (var workerStore in context.EnumerateWorkers()) - { - await workerStore.LocalLocationStore.HeartbeatAsync(context).ShouldBeSuccess(); - - if (reconcile) - { - await workerStore.ReconcileAsync(context, force: true).ShouldBeSuccess(); - } - } - } - #region SAS Tokens Tests [Fact(Skip = "For manual testing only. Requires storage account credentials")] public async Task BlobCentralStorageCredentialsUpdate() diff --git a/Public/Src/Cache/ContentStore/DistributedTest/ContentLocation/LocalLocationStoreDistributedContentTestsBase.cs b/Public/Src/Cache/ContentStore/DistributedTest/ContentLocation/LocalLocationStoreDistributedContentTestsBase.cs index 15c6b2ea7..fa7c9ea37 100644 --- a/Public/Src/Cache/ContentStore/DistributedTest/ContentLocation/LocalLocationStoreDistributedContentTestsBase.cs +++ b/Public/Src/Cache/ContentStore/DistributedTest/ContentLocation/LocalLocationStoreDistributedContentTestsBase.cs @@ -4,12 +4,14 @@ using System; using System.Collections.Concurrent; using System.Collections.Generic; +using System.Diagnostics.ContractsLight; using System.IO; using System.Linq; using System.Threading; using System.Threading.Tasks; using BuildXL.Cache.ContentStore.Distributed; using BuildXL.Cache.ContentStore.Distributed.MetadataService; +using BuildXL.Cache.ContentStore.Distributed.NuCache; using BuildXL.Cache.ContentStore.Distributed.NuCache.EventStreaming; using BuildXL.Cache.ContentStore.Distributed.Redis; using BuildXL.Cache.ContentStore.Distributed.Stores; @@ -360,6 +362,7 @@ namespace ContentStoreTest.Distributed.Sessions }; } + settings = ModifySettings(settings); var configuration = new DistributedCacheServiceConfiguration(localCasSettings, settings); var arguments = new DistributedCacheServiceArguments( @@ -383,6 +386,8 @@ namespace ContentStoreTest.Distributed.Sessions return CreateStore(context, arguments); } + protected virtual TestDistributedContentSettings ModifySettings(TestDistributedContentSettings dcs) => dcs; + protected virtual TestServerProvider CreateStore(Context context, DistributedCacheServiceArguments arguments) { if (UseGrpcServer) @@ -397,6 +402,80 @@ namespace ContentStoreTest.Distributed.Sessions } } + protected virtual Task CreateCheckpointAsync(InstanceRef storeRef, TestContext context) + { + return context.GetLocalLocationStore(storeRef.ResolveIndex(context)).HeartbeatAsync(context); + } + + protected virtual Task RestoreCheckpointAsync(InstanceRef storeRef, TestContext context) + { + return context.GetLocalLocationStore(storeRef.ResolveIndex(context)).HeartbeatAsync(context); + } + + protected async Task UploadCheckpointOnMasterAndRestoreOnWorkers(TestContext context, bool reconcile = false, string clearStoragePrefix = null) + { + // Update time to trigger checkpoint upload and restore on master and workers respectively + TestClock.UtcNow += TimeSpan.FromMinutes(2); + + var masterStore = context.GetMaster(); + + // Heartbeat master first to upload checkpoint + await CreateCheckpointAsync(masterStore, context).ShouldBeSuccess(); + + if (reconcile) + { + await masterStore.ReconcileAsync(context, force: true).ShouldBeSuccess(); + } + + if (clearStoragePrefix != null) + { + await StorageProcess.ClearAsync(clearStoragePrefix); + } + + // Next heartbeat workers to restore checkpoint + foreach (var workerStore in context.EnumerateWorkers()) + { + await RestoreCheckpointAsync(workerStore, context).ShouldBeSuccess(); + + if (reconcile) + { + await workerStore.ReconcileAsync(context, force: true).ShouldBeSuccess(); + } + } + } + + protected record struct InstanceRef( + LocalLocationStore LocalLocationStore = null, + TransitioningContentLocationStore LocationStore = null, + int? Index = null) + { + public int ResolveIndex(TestContext context) + { + return Index + ?? ResolveIndex(context, LocalLocationStore, (c, i) => c.GetLocalLocationStore(i)) + ?? ResolveIndex(context, LocationStore, (c, i) => c.GetLocationStore(i)) + ?? throw Contract.AssertFailure("Could not find instance"); + } + + public int? ResolveIndex(TestContext context, T instance, Func getIndexInstance) + where T : class + { + for (int i = 0; i < context.Stores.Count; i++) + { + if (getIndexInstance(context, i) == instance) + { + return i; + } + } + + return null; + } + + public static implicit operator InstanceRef(LocalLocationStore value) => new InstanceRef(LocalLocationStore: value); + public static implicit operator InstanceRef(int value) => new InstanceRef(Index: value); + public static implicit operator InstanceRef(TransitioningContentLocationStore value) => new InstanceRef(LocationStore: value); + } + protected async Task OpenStreamAndDisposeAsync(IContentSession session, Context context, ContentHash hash) { var openResult = await session.OpenStreamAsync(context, hash, Token).ShouldBeSuccess(); diff --git a/Public/Src/Cache/ContentStore/DistributedTest/ContentLocation/NuCache/ShortHashTests.cs b/Public/Src/Cache/ContentStore/DistributedTest/ContentLocation/NuCache/ShortHashTests.cs index f6170e62e..b4bfd035b 100644 --- a/Public/Src/Cache/ContentStore/DistributedTest/ContentLocation/NuCache/ShortHashTests.cs +++ b/Public/Src/Cache/ContentStore/DistributedTest/ContentLocation/NuCache/ShortHashTests.cs @@ -1,8 +1,10 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT License. +using System.Collections.Generic; using System.Diagnostics.ContractsLight; using System.IO; +using System.Linq; using System.Runtime.InteropServices; using System.Text; using BuildXL.Cache.ContentStore.Distributed.NuCache; @@ -21,6 +23,25 @@ namespace ContentStoreTest.Distributed.ContentLocation.NuCache public ShortHashTests(ITestOutputHelper helper) => _helper = helper; + public static IEnumerable HashTypes => HashInfoLookup.All().Distinct().Select(i => new object[] { i.HashType }); + + [Theory] + [MemberData(nameof(HashTypes))] + public void ParseAllHashTypes(HashType hashType) + { + var hash = ContentHash.Random(hashType); + var stringHash = hash.ToShortString(); + + // Test using TryParse + ShortHash.TryParse(stringHash, out var shortHash).Should().BeTrue(); + var expectedShortHash = hash.AsShortHash(); + shortHash.Should().Be(expectedShortHash); + + // Test using constructor + shortHash = new ShortHash(stringHash); + shortHash.Should().Be(expectedShortHash); + } + [Fact] public void ShortHashBinaryRoundtrip() { diff --git a/Public/Src/Cache/ContentStore/DistributedTest/MetadataService/ResilientContentMetadataServiceTests.cs b/Public/Src/Cache/ContentStore/DistributedTest/MetadataService/ResilientContentMetadataServiceTests.cs index 9744c961a..af4d823b9 100644 --- a/Public/Src/Cache/ContentStore/DistributedTest/MetadataService/ResilientContentMetadataServiceTests.cs +++ b/Public/Src/Cache/ContentStore/DistributedTest/MetadataService/ResilientContentMetadataServiceTests.cs @@ -269,4 +269,12 @@ namespace BuildXL.Cache.ContentStore.Distributed.Test.MetadataService _redisFixture.Dispose(); } } + + public static partial class TestExtensions + { + public static Task OnRoleUpdatedAsync(this ResilientGlobalCacheService service, OperationContext context, Role role) + { + return service.OnRoleUpdatedAsync(context, new MasterElectionState(default, role, default)); + } + } } diff --git a/Public/Src/Cache/ContentStore/DistributedTest/Sessions/DistributedContentTests.cs b/Public/Src/Cache/ContentStore/DistributedTest/Sessions/DistributedContentTests.cs index 181492954..3b27adaaf 100644 --- a/Public/Src/Cache/ContentStore/DistributedTest/Sessions/DistributedContentTests.cs +++ b/Public/Src/Cache/ContentStore/DistributedTest/Sessions/DistributedContentTests.cs @@ -363,6 +363,11 @@ namespace ContentStoreTest.Distributed.Sessions return GetServices(idx).RedisGlobalStore.Instance; } + internal BlobContentLocationRegistry GetBlobContentLocationRegistry(int idx) + { + return GetServices(idx).BlobContentLocationRegistry.GetRequiredInstance(); + } + internal TransitioningContentLocationStore GetLocationStore(int idx) => ((TransitioningContentLocationStore)GetDistributedSession(idx).ContentLocationStore); diff --git a/Public/Src/Cache/ContentStore/Hashing/ContentHash.cs b/Public/Src/Cache/ContentStore/Hashing/ContentHash.cs index ccfbb5794..81aa05c50 100644 --- a/Public/Src/Cache/ContentStore/Hashing/ContentHash.cs +++ b/Public/Src/Cache/ContentStore/Hashing/ContentHash.cs @@ -428,7 +428,7 @@ namespace BuildXL.Cache.ContentStore.Hashing /// public static bool TryParse(string serialized, out ContentHash contentHash) { - return TryParse(serialized, out contentHash, null); + return TryParse(serialized, out contentHash, isShortHash: false); } /// @@ -436,13 +436,13 @@ namespace BuildXL.Cache.ContentStore.Hashing /// public static bool TryParse(HashType hashType, string serialized, out ContentHash contentHash) { - return TryParse(hashType, serialized, out contentHash, null); + return TryParse(hashType, serialized, out contentHash, isShortHash: false); } /// /// Attempt to create from a known type string. /// - internal static bool TryParse(string serialized, out ContentHash contentHash, int? expectedStringLength) + internal static bool TryParse(string serialized, out ContentHash contentHash, bool isShortHash) { Contract.Requires(serialized != null); @@ -473,17 +473,17 @@ namespace BuildXL.Cache.ContentStore.Hashing return false; } - return TryParse(hashType, hash, out contentHash, expectedStringLength); + return TryParse(hashType, hash, out contentHash, isShortHash); } /// /// Attempt to create from a known type and string (without type). /// - internal static bool TryParse(HashType hashType, string serialized, out ContentHash contentHash, int? expectedStringLength) + internal static bool TryParse(HashType hashType, string serialized, out ContentHash contentHash, bool isShortHash) { Contract.Requires(serialized != null); - if (serialized.Length != (expectedStringLength ?? HashInfoLookup.Find(hashType).StringLength)) + if (serialized.Length != (isShortHash ? ShortHash.HashStringLength : HashInfoLookup.Find(hashType).StringLength)) { contentHash = default(ContentHash); return false; @@ -497,10 +497,13 @@ namespace BuildXL.Cache.ContentStore.Hashing contentHash = new ContentHash(hashType, bytes); - if (HashInfoLookup.Find(hashType) is TaggedHashInfo && !AlgorithmIdHelpers.IsHashTagValid(contentHash)) + if (!isShortHash) { - contentHash = default(ContentHash); - return false; + if (HashInfoLookup.Find(hashType) is TaggedHashInfo && !AlgorithmIdHelpers.IsHashTagValid(contentHash)) + { + contentHash = default(ContentHash); + return false; + } } return true; diff --git a/Public/Src/Cache/ContentStore/Hashing/ShortHash.cs b/Public/Src/Cache/ContentStore/Hashing/ShortHash.cs index 7dde9dbd8..28e9a365f 100644 --- a/Public/Src/Cache/ContentStore/Hashing/ShortHash.cs +++ b/Public/Src/Cache/ContentStore/Hashing/ShortHash.cs @@ -29,6 +29,11 @@ namespace BuildXL.Cache.ContentStore.Hashing /// public const int HashLength = SerializedLength - 1; + /// + /// The length in hex characters of the hash portion of a short hash. NOTE: This does NOT include characters for the hash type + /// + public const int HashStringLength = HashLength * 2; + /// public readonly ShortReadOnlyFixedBytes Value; @@ -78,7 +83,7 @@ namespace BuildXL.Cache.ContentStore.Hashing /// public static bool TryParse(string str, out ShortHash result) { - if (ContentHash.TryParse(str, out var longHash, expectedStringLength: HashLength * 2)) + if (ContentHash.TryParse(str, out var longHash, isShortHash: true)) { result = longHash.AsShortHash(); return true; diff --git a/Public/Src/Cache/ContentStore/Interfaces/Results/ResultsExtensions.cs b/Public/Src/Cache/ContentStore/Interfaces/Results/ResultsExtensions.cs index 0a3c07d18..f7f44c62b 100644 --- a/Public/Src/Cache/ContentStore/Interfaces/Results/ResultsExtensions.cs +++ b/Public/Src/Cache/ContentStore/Interfaces/Results/ResultsExtensions.cs @@ -235,11 +235,11 @@ namespace BuildXL.Cache.ContentStore.Interfaces.Results /// /// Maps result into different result type or propagates error to result type /// - public static Result Select(this Result result, Func selector) + public static Result Select(this Result result, Func selector, bool isNullAllowed = false) { if (result.Succeeded) { - return Result.Success(selector(result.Value)); + return Result.Success(selector(result.Value), isNullAllowed: isNullAllowed); } else { diff --git a/Public/Src/Cache/ContentStore/InterfacesTest/Hashing/ContentHashTests.cs b/Public/Src/Cache/ContentStore/InterfacesTest/Hashing/ContentHashTests.cs index ce8df965b..0347f7d70 100644 --- a/Public/Src/Cache/ContentStore/InterfacesTest/Hashing/ContentHashTests.cs +++ b/Public/Src/Cache/ContentStore/InterfacesTest/Hashing/ContentHashTests.cs @@ -20,6 +20,12 @@ namespace BuildXL.Cache.ContentStore.InterfacesTest.Hashing private static readonly byte[] B2 = new List {2}.Concat(Enumerable.Repeat((byte)0, ContentHash.MaxHashByteLength - 1)).ToArray(); + public static IEnumerable HashTypes => HashInfoLookup.All().Distinct().Select(i => new object[] { i.HashType }); + + public static IEnumerable HashTypesWithByteLengths => HashInfoLookup.All().Distinct().Select(i => new object[] { i.HashType, i.ByteLength }); + + public static IEnumerable HashTypesWithStringLengths => HashInfoLookup.All().Distinct().Select(i => new object[] { i.HashType, i.StringLength }); + [Fact] public void TestGetHashCodeWithDefaultInstanceShouldNotThrow() { @@ -29,12 +35,7 @@ namespace BuildXL.Cache.ContentStore.InterfacesTest.Hashing } [Theory] - [InlineData(HashType.MD5, 16)] - [InlineData(HashType.SHA1, 20)] - [InlineData(HashType.SHA256, 32)] - [InlineData(HashType.Vso0, 33)] - [InlineData(HashType.Dedup64K, 33)] - [InlineData(HashType.Dedup1024K, 33)] + [MemberData(nameof(HashTypesWithByteLengths))] public void ValidByteLength(HashType hashType, int length) { var randomHash = ContentHash.Random(hashType); @@ -43,24 +44,14 @@ namespace BuildXL.Cache.ContentStore.InterfacesTest.Hashing } [Theory] - [InlineData(HashType.MD5, 32)] - [InlineData(HashType.SHA1, 40)] - [InlineData(HashType.SHA256, 64)] - [InlineData(HashType.Vso0, 66)] - [InlineData(HashType.Dedup64K, 66)] - [InlineData(HashType.Dedup1024K, 66)] + [MemberData(nameof(HashTypesWithStringLengths))] public void ValidStringLength(HashType hashType, int length) { Assert.Equal(length, ContentHash.Random(hashType).StringLength); } [Theory] - [InlineData(HashType.MD5)] - [InlineData(HashType.SHA1)] - [InlineData(HashType.SHA256)] - [InlineData(HashType.Vso0)] - [InlineData(HashType.Dedup64K)] - [InlineData(HashType.Dedup1024K)] + [MemberData(nameof(HashTypes))] public void RandomValue(HashType hashType) { var v = ContentHash.Random(hashType); @@ -71,15 +62,17 @@ namespace BuildXL.Cache.ContentStore.InterfacesTest.Hashing { Assert.Equal(v[hashInfo.ByteLength - 1], taggedHashInfo.AlgorithmId); } + + var stringHash = v.Serialize(); + Assert.True(ContentHash.TryParse(stringHash, out var parsedHash)); + Assert.Equal(v, parsedHash); + + parsedHash = new ContentHash(stringHash); + Assert.Equal(v, parsedHash); } [Theory] - [InlineData(HashType.MD5)] - [InlineData(HashType.SHA1)] - [InlineData(HashType.SHA256)] - [InlineData(HashType.Vso0)] - [InlineData(HashType.Dedup64K)] - [InlineData(HashType.Dedup1024K)] + [MemberData(nameof(HashTypes))] public void MismatchLengthThrows(HashType hashType) { var b = Enumerable.Repeat((byte)0, 15).ToArray(); @@ -104,12 +97,7 @@ namespace BuildXL.Cache.ContentStore.InterfacesTest.Hashing } [Theory] - [InlineData(HashType.MD5)] - [InlineData(HashType.SHA1)] - [InlineData(HashType.SHA256)] - [InlineData(HashType.Vso0)] - [InlineData(HashType.Dedup64K)] - [InlineData(HashType.Dedup1024K)] + [MemberData(nameof(HashTypes))] public void EqualsTrueReferenceType(HashType hashType) { var hash = ContentHash.Random(hashType); @@ -117,12 +105,7 @@ namespace BuildXL.Cache.ContentStore.InterfacesTest.Hashing } [Theory] - [InlineData(HashType.MD5)] - [InlineData(HashType.SHA1)] - [InlineData(HashType.SHA256)] - [InlineData(HashType.Vso0)] - [InlineData(HashType.Dedup64K)] - [InlineData(HashType.Dedup1024K)] + [MemberData(nameof(HashTypes))] public void EqualsFalseReferenceType(HashType hashType) { var h1 = ContentHash.Random(hashType); @@ -131,12 +114,7 @@ namespace BuildXL.Cache.ContentStore.InterfacesTest.Hashing } [Theory] - [InlineData(HashType.MD5)] - [InlineData(HashType.SHA1)] - [InlineData(HashType.SHA256)] - [InlineData(HashType.Vso0)] - [InlineData(HashType.Dedup64K)] - [InlineData(HashType.Dedup1024K)] + [MemberData(nameof(HashTypes))] public void GetHashCodeEqual(HashType hashType) { var h1 = new ContentHash(hashType, Zeros); @@ -145,12 +123,7 @@ namespace BuildXL.Cache.ContentStore.InterfacesTest.Hashing } [Theory] - [InlineData(HashType.MD5)] - [InlineData(HashType.SHA1)] - [InlineData(HashType.SHA256)] - [InlineData(HashType.Vso0)] - [InlineData(HashType.Dedup64K)] - [InlineData(HashType.Dedup1024K)] + [MemberData(nameof(HashTypes))] public void GetHashCodeNotEqual(HashType hashType) { var h1 = ContentHash.Random(hashType); @@ -159,12 +132,7 @@ namespace BuildXL.Cache.ContentStore.InterfacesTest.Hashing } [Theory] - [InlineData(HashType.MD5)] - [InlineData(HashType.SHA1)] - [InlineData(HashType.SHA256)] - [InlineData(HashType.Vso0)] - [InlineData(HashType.Dedup64K)] - [InlineData(HashType.Dedup1024K)] + [MemberData(nameof(HashTypes))] public void CompareToEqual(HashType hashType) { var h1 = new ContentHash(hashType, Zeros); @@ -173,12 +141,7 @@ namespace BuildXL.Cache.ContentStore.InterfacesTest.Hashing } [Theory] - [InlineData(HashType.MD5)] - [InlineData(HashType.SHA1)] - [InlineData(HashType.SHA256)] - [InlineData(HashType.Vso0)] - [InlineData(HashType.Dedup64K)] - [InlineData(HashType.Dedup1024K)] + [MemberData(nameof(HashTypes))] public void CompareToLessThan(HashType hashType) { var h1 = new ContentHash(hashType, B1); @@ -187,12 +150,7 @@ namespace BuildXL.Cache.ContentStore.InterfacesTest.Hashing } [Theory] - [InlineData(HashType.MD5)] - [InlineData(HashType.SHA1)] - [InlineData(HashType.SHA256)] - [InlineData(HashType.Vso0)] - [InlineData(HashType.Dedup64K)] - [InlineData(HashType.Dedup1024K)] + [MemberData(nameof(HashTypes))] public void CompareToGreaterThan(HashType hashType) { var h1 = new ContentHash(hashType, B1); @@ -216,12 +174,7 @@ namespace BuildXL.Cache.ContentStore.InterfacesTest.Hashing } [Theory] - [InlineData(HashType.MD5)] - [InlineData(HashType.SHA1)] - [InlineData(HashType.SHA256)] - [InlineData(HashType.Vso0)] - [InlineData(HashType.Dedup64K)] - [InlineData(HashType.Dedup1024K)] + [MemberData(nameof(HashTypes))] public void EqualityOperatorTrue(HashType hashType) { var hash1 = new ContentHash(hashType, B1); @@ -230,12 +183,7 @@ namespace BuildXL.Cache.ContentStore.InterfacesTest.Hashing } [Theory] - [InlineData(HashType.MD5)] - [InlineData(HashType.SHA1)] - [InlineData(HashType.SHA256)] - [InlineData(HashType.Vso0)] - [InlineData(HashType.Dedup64K)] - [InlineData(HashType.Dedup1024K)] + [MemberData(nameof(HashTypes))] public void EqualityOperatorFalse(HashType hashType) { var h1 = new ContentHash(hashType, B1); @@ -244,12 +192,7 @@ namespace BuildXL.Cache.ContentStore.InterfacesTest.Hashing } [Theory] - [InlineData(HashType.MD5)] - [InlineData(HashType.SHA1)] - [InlineData(HashType.SHA256)] - [InlineData(HashType.Vso0)] - [InlineData(HashType.Dedup64K)] - [InlineData(HashType.Dedup1024K)] + [MemberData(nameof(HashTypes))] public void InequalityOperatorFalse(HashType hashType) { var h1 = new ContentHash(hashType, B1); @@ -258,12 +201,7 @@ namespace BuildXL.Cache.ContentStore.InterfacesTest.Hashing } [Theory] - [InlineData(HashType.MD5)] - [InlineData(HashType.SHA1)] - [InlineData(HashType.SHA256)] - [InlineData(HashType.Vso0)] - [InlineData(HashType.Dedup64K)] - [InlineData(HashType.Dedup1024K)] + [MemberData(nameof(HashTypes))] public void InequalityOperatorTrue(HashType hashType) { var h1 = new ContentHash(hashType, B1); @@ -276,12 +214,7 @@ namespace BuildXL.Cache.ContentStore.InterfacesTest.Hashing } [Theory] - [InlineData(HashType.MD5)] - [InlineData(HashType.SHA1)] - [InlineData(HashType.SHA256)] - [InlineData(HashType.Vso0)] - [InlineData(HashType.Dedup64K)] - [InlineData(HashType.Dedup1024K)] + [MemberData(nameof(HashTypes))] public void EqualContentHashRoundTripViaSpan(HashType hashType) { var h1 = ContentHash.Random(hashType); @@ -294,12 +227,7 @@ namespace BuildXL.Cache.ContentStore.InterfacesTest.Hashing } [Theory] - [InlineData(HashType.MD5)] - [InlineData(HashType.SHA1)] - [InlineData(HashType.SHA256)] - [InlineData(HashType.Vso0)] - [InlineData(HashType.Dedup64K)] - [InlineData(HashType.Dedup1024K)] + [MemberData(nameof(HashTypes))] public void EqualContentHashRoundTripViaHexString(HashType hashType) { var h1 = ContentHash.Random(hashType); diff --git a/Public/Src/Cache/ContentStore/Library/Extensions/EnumerableExtensions.cs b/Public/Src/Cache/ContentStore/Library/Extensions/EnumerableExtensions.cs index 3c6338f0a..00dfbf406 100644 --- a/Public/Src/Cache/ContentStore/Library/Extensions/EnumerableExtensions.cs +++ b/Public/Src/Cache/ContentStore/Library/Extensions/EnumerableExtensions.cs @@ -167,7 +167,7 @@ namespace BuildXL.Cache.ContentStore.Extensions /// /// Pseudorandomly enumerates the range from [0, ) /// - public static IEnumerable PseudoRandomEnumerate(int length) + public static IEnumerable PseudoRandomEnumerateRange(int length) { var offset = ThreadSafeRandom.Generator.Next(0, length); var current = ThreadSafeRandom.Generator.Next(0, length); @@ -178,6 +178,17 @@ namespace BuildXL.Cache.ContentStore.Extensions } } + /// + /// Pseudorandomly enumerates the items in the list + /// + public static IEnumerable PseudoRandomEnumerate(this IReadOnlyList list) + { + foreach (var index in PseudoRandomEnumerateRange(list.Count)) + { + yield return list[index]; + } + } + /// /// Gets a unique, pseudorandom value between [0, length). /// diff --git a/Public/Src/Cache/ContentStore/Test/Extensions/EnumerableExtensionsTests.cs b/Public/Src/Cache/ContentStore/Test/Extensions/EnumerableExtensionsTests.cs index fd7135806..954386026 100644 --- a/Public/Src/Cache/ContentStore/Test/Extensions/EnumerableExtensionsTests.cs +++ b/Public/Src/Cache/ContentStore/Test/Extensions/EnumerableExtensionsTests.cs @@ -24,18 +24,20 @@ namespace ContentStoreTest.Extensions { for (int length = 0; length < 1024; length++) { - var randomRange = EnumerableExtensions.PseudoRandomEnumerate(length).ToArray(); - var sortedRange = Enumerable.Range(0, length).ToArray(); + var randomRange = EnumerableExtensions.PseudoRandomEnumerateRange(length).ToArray(); + var randomEnumeration = sortedRange.PseudoRandomEnumerate().ToArray(); // Technically, there is a possibility that the random range will match the sorted // range. Use a high enough length so that possibility is excluded. if (length > 10) { Assert.NotEqual(sortedRange, randomRange); + Assert.NotEqual(sortedRange, randomEnumeration); } Assert.Equal(sortedRange, randomRange.OrderBy(i => i)); + Assert.Equal(sortedRange, randomEnumeration.OrderBy(i => i)); } } diff --git a/Public/Src/Cache/DistributedCache.Host/Configuration/DistributedContentSettings.cs b/Public/Src/Cache/DistributedCache.Host/Configuration/DistributedContentSettings.cs index df0134165..fbfe116e3 100644 --- a/Public/Src/Cache/DistributedCache.Host/Configuration/DistributedContentSettings.cs +++ b/Public/Src/Cache/DistributedCache.Host/Configuration/DistributedContentSettings.cs @@ -681,7 +681,7 @@ namespace BuildXL.Cache.Host.Configuration #endregion [DataMember] - public bool EnableGlobalCacheLocationStoreValidation { get; set; } = false; + public EnumSetting GlobalCacheDatabaseValidationMode { get; set; } = DatabaseValidationMode.None; [DataMember] public bool IsMasterEligible { get; set; } = false; @@ -1162,7 +1162,7 @@ namespace BuildXL.Cache.Host.Configuration public bool ContentMetadataDisableDatabaseRegisterLocation { get; set; } [DataMember] - public bool ContentMetadataEnableResilience { get; set; } + public bool ContentMetadataEnableResilience { get; set; } = true; [DataMember] public bool ContentMetadataOptimizeWrites { get; set; } @@ -1325,6 +1325,24 @@ namespace BuildXL.Cache.Host.Configuration public LocalLocationStoreSettings LocationStoreSettings { get; set; } = new(); } + public enum DatabaseValidationMode + { + /// + /// No validation + /// + None, + + /// + /// Log metrics and presence in GCS for all copies + /// + Log, + + /// + /// Log and raise error. This mainly for unit test validation. + /// + LogAndError, + } + [Flags] public enum RegisterHintHandling { diff --git a/Public/Src/Cache/DistributedCache.Host/Configuration/LocalLocationStoreSettings.cs b/Public/Src/Cache/DistributedCache.Host/Configuration/LocalLocationStoreSettings.cs index 4030fd111..c848d5cad 100644 --- a/Public/Src/Cache/DistributedCache.Host/Configuration/LocalLocationStoreSettings.cs +++ b/Public/Src/Cache/DistributedCache.Host/Configuration/LocalLocationStoreSettings.cs @@ -42,7 +42,7 @@ namespace BuildXL.Cache.Host.Configuration /// public bool EnableBlobContentLocationRegistry { get; set; } - public BlobContentLocationRegistrySettings BlobContentLocationRegistrySettings { get; } + public BlobContentLocationRegistrySettings BlobContentLocationRegistrySettings { get; set; } = new BlobContentLocationRegistrySettings(); } public record BlobContentLocationRegistrySettings @@ -53,7 +53,7 @@ namespace BuildXL.Cache.Host.Configuration public string FolderName { get; set; } = "partitions"; - public string PartitionCheckpointManifestFileName { get; set; } = "manifest.json"; + public string PartitionCheckpointManifestFileName { get; set; } = "manifest.v2.json"; /// /// Indicates whether partitions are updated in the background on a timer loop @@ -68,6 +68,36 @@ namespace BuildXL.Cache.Host.Configuration /// /// Interval between updates of partitions output blob /// - public TimeSpanSetting PartitionsUpdateInterval { get; set; } = TimeSpan.FromMinutes(5); + public TimeSpanSetting PartitionsUpdateInterval { get; set; } = TimeSpan.FromMinutes(10); + + /// + /// Gets whether partitions should be processed into output blobs (i.e. containing sst files and content listings) + /// + public bool ProcessPartitions { get; set; } = true; + + /// + /// Maximum number of diff sst snapshots for a particular partition allowed before using full sst snapshot instead. + /// + public int MaxSnapshotChainLength { get; set; } = 5; + + /// + /// Maximum number of diff sst snapshots for a particular partition allowed before using full sst snapshot instead. + /// + public int MaxRetainedSnapshots => Math.Max(1, (MaxSnapshotChainLength * 2)); + + /// + /// Maximum parallelism for sst file download + /// + public int MaxDegreeOfParallelism { get; set; } = 4; + + /// + /// Gets whether the local database should be updated with sst files + /// + public bool UpdateDatabase { get; set; } = false; + + /// + /// The number of partitions to create. Changing this number causes partition to be recomputed + /// + public int PartitionCount { get; set; } = 256; } } diff --git a/Public/Src/Cache/MemoizationStore/DistributedTest/DistributedOneLevelCacheTests.cs b/Public/Src/Cache/MemoizationStore/DistributedTest/DistributedOneLevelCacheTests.cs index 4c6a45c9f..42e9ebdec 100644 --- a/Public/Src/Cache/MemoizationStore/DistributedTest/DistributedOneLevelCacheTests.cs +++ b/Public/Src/Cache/MemoizationStore/DistributedTest/DistributedOneLevelCacheTests.cs @@ -12,6 +12,7 @@ using BuildXL.Cache.ContentStore.Distributed; using BuildXL.Cache.ContentStore.Distributed.MetadataService; using BuildXL.Cache.ContentStore.Distributed.NuCache; using BuildXL.Cache.ContentStore.Distributed.Stores; +using BuildXL.Cache.ContentStore.Distributed.Test.ContentLocation; using BuildXL.Cache.ContentStore.Hashing; using BuildXL.Cache.ContentStore.Interfaces.Results; using BuildXL.Cache.ContentStore.Interfaces.Sessions; diff --git a/Public/Src/FrontEnd/TypeScript.Net/TypeScript.Net/Extensions/CollectionExtensions.cs b/Public/Src/FrontEnd/TypeScript.Net/TypeScript.Net/Extensions/CollectionExtensions.cs index 38bedb27e..82fc3dfe4 100644 --- a/Public/Src/FrontEnd/TypeScript.Net/TypeScript.Net/Extensions/CollectionExtensions.cs +++ b/Public/Src/FrontEnd/TypeScript.Net/TypeScript.Net/Extensions/CollectionExtensions.cs @@ -472,32 +472,6 @@ namespace TypeScript.Net.Extensions } } - /// - /// Converts sequence to dictionary, but accepts duplicate keys. First will win. - /// - public static Dictionary ToDictionarySafe(this IEnumerable source, Func keySelector, - Func valueSelector) - { - Contract.Requires(source != null); - Contract.Requires(keySelector != null); - Contract.Requires(valueSelector != null); - - Dictionary result = new Dictionary(); - - foreach (var element in source) - { - var key = keySelector(element); - var value = valueSelector(element); - - if (!result.ContainsKey(key)) - { - result.Add(key, value); - } - } - - return result; - } - /// /// Returns whether a is null or empty. /// diff --git a/Public/Src/FrontEnd/TypeScript.Net/TypeScript.Net/Scanning/Scanner.cs b/Public/Src/FrontEnd/TypeScript.Net/TypeScript.Net/Scanning/Scanner.cs index 3a619f283..e5ec347d4 100644 --- a/Public/Src/FrontEnd/TypeScript.Net/TypeScript.Net/Scanning/Scanner.cs +++ b/Public/Src/FrontEnd/TypeScript.Net/TypeScript.Net/Scanning/Scanner.cs @@ -9,6 +9,7 @@ using System.Linq; using System.Numerics; using BuildXL.FrontEnd.Script.Constants; using BuildXL.Utilities; +using BuildXL.Utilities.Collections; using TypeScript.Net.Diagnostics; using TypeScript.Net.Extensions; using TypeScript.Net.Types; diff --git a/Public/Src/Utilities/Collections/CollectionUtilities.cs b/Public/Src/Utilities/Collections/CollectionUtilities.cs index a23d93554..fab796d74 100644 --- a/Public/Src/Utilities/Collections/CollectionUtilities.cs +++ b/Public/Src/Utilities/Collections/CollectionUtilities.cs @@ -190,6 +190,57 @@ namespace BuildXL.Utilities.Collections return array; } + /// + /// Converts sequence to dictionary, but accepts duplicate keys. First will win. + /// + public static Dictionary ToDictionarySafe(this IEnumerable source, Func keySelector) + where TKey : notnull + { + Contract.Requires(source != null); + Contract.Requires(keySelector != null); + + Dictionary result = new Dictionary(); + + foreach (var value in source) + { + var key = keySelector(value); + + if (!result.ContainsKey(key)) + { + result.Add(key, value); + } + } + + return result; + } + + /// + /// Converts sequence to dictionary, but accepts duplicate keys. First will win. + /// + public static Dictionary ToDictionarySafe(this IEnumerable source, Func keySelector, + Func valueSelector) + where TKey : notnull + { + Contract.Requires(source != null); + Contract.Requires(keySelector != null); + Contract.Requires(valueSelector != null); + + Dictionary result = new Dictionary(); + + foreach (var element in source) + { + var key = keySelector(element); + var value = valueSelector(element); + + if (!result.ContainsKey(key)) + { + result.Add(key, value); + } + } + + return result; + } + /// /// Clones the existing dictionary with no enumerator allocations. /// diff --git a/Public/Src/Utilities/UnitTests/TestUtilities.XUnit/FactIfSupportedAttribute.cs b/Public/Src/Utilities/UnitTests/TestUtilities.XUnit/FactIfSupportedAttribute.cs index 4c63d9005..875afe40d 100644 --- a/Public/Src/Utilities/UnitTests/TestUtilities.XUnit/FactIfSupportedAttribute.cs +++ b/Public/Src/Utilities/UnitTests/TestUtilities.XUnit/FactIfSupportedAttribute.cs @@ -76,6 +76,8 @@ namespace Test.BuildXL.TestUtilities.Xunit return; } + CheckRequirement(TestRequirements.NotSupported, () => "Test is marked not supported."); + CheckRequirement( TestRequirements.Admin, () => diff --git a/Public/Src/Utilities/UnitTests/TestUtilities.XUnit/TestRequirements.cs b/Public/Src/Utilities/UnitTests/TestUtilities.XUnit/TestRequirements.cs index 3f5e3c145..0c6ff9e8a 100644 --- a/Public/Src/Utilities/UnitTests/TestUtilities.XUnit/TestRequirements.cs +++ b/Public/Src/Utilities/UnitTests/TestUtilities.XUnit/TestRequirements.cs @@ -67,5 +67,10 @@ namespace Test.BuildXL.TestUtilities.Xunit /// Requires running on either Windows or Mac operating system (excluding Linux) /// WindowsOrMacOs = 1 << 9, + + /// + /// Used to disable a test. Typically used with #ifdef + /// + NotSupported = 1 << 10, } } diff --git a/Public/Src/Utilities/Utilities/AsyncOut.cs b/Public/Src/Utilities/Utilities/AsyncOut.cs index b655cd45f..5381e86fd 100644 --- a/Public/Src/Utilities/Utilities/AsyncOut.cs +++ b/Public/Src/Utilities/Utilities/AsyncOut.cs @@ -19,4 +19,20 @@ namespace BuildXL.Utilities return value.Value; } } + + /// + /// Helper methods for + /// + public static class AsyncOut + { + /// + /// Allows inline declaration of patterns like the + /// (out T parameter) pattern. Usage: await ExecuteAsync(out AsyncOut.Var<T>(out var outParam)); + /// + public static AsyncOut Var(out AsyncOut value) + { + value = new AsyncOut(); + return value; + } + } } diff --git a/cg/nuget/cgmanifest.json b/cg/nuget/cgmanifest.json index 294b004fc..ee331a4bc 100644 --- a/cg/nuget/cgmanifest.json +++ b/cg/nuget/cgmanifest.json @@ -141,7 +141,7 @@ "Type": "NuGet", "NuGet": { "Name": "BuildXL.Azurite.Executables", - "Version": "1.0.0-CI-20220125-034149" + "Version": "1.0.0-CI-20220612-002122" } } }, @@ -2868,7 +2868,7 @@ "Type": "NuGet", "NuGet": { "Name": "RocksDbNative", - "Version": "6.10.2-b20220610.3" + "Version": "6.10.2-b20220610.4" } } }, @@ -2877,7 +2877,7 @@ "Type": "NuGet", "NuGet": { "Name": "RocksDbSharp", - "Version": "6.10.2-b20220610.3" + "Version": "6.10.2-b20220610.4" } } }, diff --git a/config.dsc b/config.dsc index 14e0d654b..efaee903d 100644 --- a/config.dsc +++ b/config.dsc @@ -170,11 +170,11 @@ config({ { id: "Microsoft.Windows.ProjFS", version: "1.2.19351.1" }, // RocksDb - { id: "RocksDbSharp", version: "6.10.2-b20220610.3", alias: "RocksDbSharpSigned", + { id: "RocksDbSharp", version: "6.10.2-b20220610.4", alias: "RocksDbSharpSigned", dependentPackageIdsToSkip: [ "System.Memory" ], dependentPackageIdsToIgnore: [ "System.Memory" ] }, - { id: "RocksDbNative", version: "6.10.2-b20220610.3" }, + { id: "RocksDbNative", version: "6.10.2-b20220610.4" }, { id: "JsonDiffPatch.Net", version: "2.1.0" }, @@ -257,7 +257,7 @@ config({ // Azurite node app compiled to standalone executable // Sources for this package are: https://github.com/Azure/Azurite // This packaged is produced by the pipeline: https://dev.azure.com/mseng/Domino/_build?definitionId=13199 - { id: "BuildXL.Azurite.Executables", version: "1.0.0-CI-20220125-034149" }, + { id: "BuildXL.Azurite.Executables", version: "1.0.0-CI-20220612-002122" }, // It turns out Redis-64 ( https://www.nuget.org/packages/redis-64/ ) was deprecated several years // ago, and so we can't even build with it due to component governance. We don't actually care about