Merged PR 801377: Add cache factory to construct a local cache with a remote blob cache

Use TwoLevelCache to implement a bxl-side cache factory that allows to create a local cache + plain remote blob one. Existing knobs for both caches are preserved.
This commit is contained in:
Serge Mera 2024-08-22 18:42:08 +00:00
Родитель 8e4c12e724
Коммит 27ba0692aa
10 изменённых файлов: 472 добавлений и 53 удалений

Просмотреть файл

@ -28,6 +28,10 @@ namespace BuildXL.Cache.Interfaces
/// <summary>
/// Validates a configuration object.
/// </summary>
/// <remarks>
/// This is currently only called from the VerticalAggregator, and does not account for validations that involve populating data from the build engine.
/// TODO: refactor to make it work in the rest of the scenarios.
/// </remarks>
IEnumerable<Failure> ValidateConfiguration([NotNull]ICacheConfigData cacheData);
}
}

Просмотреть файл

@ -32,7 +32,7 @@ namespace BuildXL.Cache.MemoizationStoreAdapter
/// </remarks>
public class BlobCacheFactory : BlobCacheFactoryBase<BlobCacheConfig>, ICacheFactory
{
internal override Task<MemoizationStore.Interfaces.Caches.ICache> CreateCacheAsync(ILogger logger, BlobCacheConfig configuration)
internal override Task<MemoizationStore.Interfaces.Caches.ICache> CreateInnerCacheAsync(ILogger logger, BlobCacheConfig configuration)
{
return Task.FromResult((MemoizationStore.Interfaces.Caches.ICache)CreateCache(logger, configuration).Cache);
}
@ -194,10 +194,6 @@ namespace BuildXL.Cache.MemoizationStoreAdapter
var failures = new List<Failure>();
failures.AddFailureIfNullOrWhitespace(cacheConfig.CacheLogPath, nameof(cacheConfig.CacheLogPath));
failures.AddFailureIfNullOrWhitespace(cacheConfig.CacheId, nameof(cacheConfig.CacheId));
failures.AddFailureIfNullOrWhitespace(cacheConfig.ConnectionStringEnvironmentVariableName, nameof(cacheConfig.ConnectionStringEnvironmentVariableName));
failures.AddFailureIfNullOrWhitespace(cacheConfig.ConnectionStringFileEnvironmentVariableName, nameof(cacheConfig.ConnectionStringFileEnvironmentVariableName));
failures.AddFailureIfNullOrWhitespace(cacheConfig.Universe, nameof(cacheConfig.Universe));
failures.AddFailureIfNullOrWhitespace(cacheConfig.Namespace, nameof(cacheConfig.Namespace));
return failures;
});
}

Просмотреть файл

@ -63,50 +63,20 @@ namespace BuildXL.Cache.MemoizationStoreAdapter
try
{
var logPath = new AbsolutePath(configuration.CacheLogPath);
// Logging always happens to a file (and ETW)
var etwFileLogger = new DisposeLogger(() => new EtwFileLog(logPath.Path, configuration.CacheId), configuration.LogFlushIntervalSeconds);
// If Kusto uploading is specified, in addition to the file/etw logging we want to setup an additional logger to take care of this
ILogger logger;
if (configuration.LogToKusto)
{
var workspacePath = Path.Combine(Path.GetTempPath(), $"{configuration.CacheId}Upload");
Directory.CreateDirectory(workspacePath);
var storageLog = AzureBlobStorageLog.CreateWithManagedIdentity(
etwFileLogger, configuration.LogToKustoIdentityId, configuration.LogToKustoBlobUri, workspacePath, CancellationToken.None);
await storageLog.StartupAsync().ThrowIfFailure();
var nLogger = await NLogAdapterHelper.CreateAdapterForCacheClientAsync(
etwFileLogger,
new BasicTelemetryFieldsProvider(configuration.BuildId),
configuration.Role,
new Dictionary<string, string>(),
storageLog);
// We want the blob upload to happen in addition to the regular etw/file logging, so let's compose both
logger = new CompositeLogger(etwFileLogger, nLogger);
}
else
{
logger = etwFileLogger;
}
var (logger, logPath) = await CreateLoggerAsync(configuration);
// If the retention period is not set, this is not a blocker for constructing the cache, but performance can be degraded. Report it.
var failures = new List<Failure>();
var cache = new MemoizationStoreAdapterCache(
cacheId: configuration.CacheId,
innerCache: await CreateCacheAsync(logger, configuration),
innerCache: await CreateInnerCacheAsync(logger, configuration),
logger: logger,
statsFile: new AbsolutePath(logPath.Path + ".stats"),
isReadOnly: configuration.IsReadOnly,
implicitPin: ImplicitPin.None,
precedingStateDegradationFailures: failures);
var startupResult = await cache.StartupAsync();
if (!startupResult.Succeeded)
{
@ -121,6 +91,41 @@ namespace BuildXL.Cache.MemoizationStoreAdapter
}
}
internal abstract Task<MemoizationStore.Interfaces.Caches.ICache> CreateCacheAsync(ILogger logger, TConfig configuration);
internal static async Task<(ILogger logger, AbsolutePath logPath)> CreateLoggerAsync(TConfig configuration)
{
var logPath = new AbsolutePath(configuration.CacheLogPath);
ILogger logger;
// Logging always happens to a file (and ETW)
var etwFileLogger = new DisposeLogger(() => new EtwFileLog(logPath.Path, configuration.CacheId), configuration.LogFlushIntervalSeconds);
if (configuration.LogToKusto)
{
var workspacePath = Path.Combine(Path.GetTempPath(), $"{configuration.CacheId}Upload");
Directory.CreateDirectory(workspacePath);
var storageLog = AzureBlobStorageLog.CreateWithManagedIdentity(
etwFileLogger, configuration.LogToKustoIdentityId, configuration.LogToKustoBlobUri, workspacePath, CancellationToken.None);
await storageLog.StartupAsync().ThrowIfFailure();
var nLogger = await NLogAdapterHelper.CreateAdapterForCacheClientAsync(
etwFileLogger,
new BasicTelemetryFieldsProvider(configuration.BuildId),
configuration.Role,
new Dictionary<string, string>(),
storageLog);
// We want the blob upload to happen in addition to the regular etw/file logging, so let's compose both
logger = new CompositeLogger(etwFileLogger, nLogger);
}
else
{
logger = etwFileLogger;
}
return (logger, logPath);
}
internal abstract Task<MemoizationStore.Interfaces.Caches.ICache> CreateInnerCacheAsync(ILogger logger, TConfig configuration);
}
}

Просмотреть файл

@ -0,0 +1,38 @@
// Copyright (c) Microsoft Corporation.
// Licensed under the MIT License.
using System;
using System.Diagnostics.ContractsLight;
using BuildXL.Cache.Interfaces;
using BuildXL.Utilities.Configuration;
using BuildXL.Utilities.Core;
namespace BuildXL.Cache.MemoizationStoreAdapter
{
/// <summary>
/// Configuration object for <see cref="BlobWithLocalCacheFactory"/>
/// </summary>
public class BlobWithLocalCacheConfig : IEngineDependentSettingsConfiguration
{
/// <summary>
/// The local cache configuration
/// </summary>
public MemoizationStoreCacheFactory.Config LocalCache { get; set; }
/// <summary>
/// The blob-based remote cache configuration
/// </summary>
public BlobCacheConfig RemoteCache { get; set; }
/// <inheritdoc/>
public bool TryPopulateFrom(Guid activityId, IConfiguration configuration, PathTable pathTable, out Failure failure)
{
// The local cache config does not depend on engine configurations. Asserting that in case things change and this goes unnoticed.
#pragma warning disable CS0184 // 'is' expression's given expression is never of the provided type
Contract.Assert(!(LocalCache is IEngineDependentSettingsConfiguration));
#pragma warning restore CS0184 // 'is' expression's given expression is never of the provided type
return RemoteCache.TryPopulateFrom(activityId, configuration, pathTable, out failure);
}
}
}

Просмотреть файл

@ -0,0 +1,112 @@
// Copyright (c) Microsoft Corporation.
// Licensed under the MIT License.
using System;
using System.Collections.Generic;
using System.Diagnostics.CodeAnalysis;
using System.Diagnostics.ContractsLight;
using System.Threading.Tasks;
using BuildXL.Cache.ContentStore.Interfaces.Logging;
using BuildXL.Cache.Interfaces;
using BuildXL.Cache.MemoizationStore.Distributed.Stores;
using BuildXL.Cache.MemoizationStore.Interfaces.Sessions;
using BuildXL.Utilities.Configuration;
using BuildXL.Utilities.Core;
namespace BuildXL.Cache.MemoizationStoreAdapter
{
/// <summary>
/// A cache factory that uses <see cref="TwoLevelCache"/> to implement a composite cache with a local cache (via <see cref="MemoizationStoreCacheFactory"/>
/// and a remote blob-based cache (via <see cref="BlobCacheFactory"/>)
/// </summary>
public class BlobWithLocalCacheFactory : ICacheFactory
{
/// <inheritdoc/>
public async Task<Possible<ICache, Failure>> InitializeCacheAsync(ICacheConfigData cacheData, Guid activityId = default, IConfiguration configuration = null, PathTable pathTable = null)
{
Contract.Requires(cacheData != null);
var possibleCacheConfig = cacheData.Create<BlobWithLocalCacheConfig>(activityId, configuration, pathTable);
if (!possibleCacheConfig.Succeeded)
{
return possibleCacheConfig.Failure;
}
return await InitializeCacheAsync(possibleCacheConfig.Result);
}
private static async Task<Possible<ICache, Failure>> InitializeCacheAsync(BlobWithLocalCacheConfig blobWithLocalCacheConfig)
{
// Initialize remote cache
var remoteCacheConfig = blobWithLocalCacheConfig.RemoteCache;
MemoizationStore.Interfaces.Caches.ICache remoteCache;
ILogger combinedLogger;
ContentStore.Interfaces.FileSystem.AbsolutePath logPath;
try
{
// This call will also set up Kusto logging if configured
(combinedLogger, logPath) = await BlobCacheFactory.CreateLoggerAsync(remoteCacheConfig);
remoteCache = await new BlobCacheFactory().CreateInnerCacheAsync(combinedLogger, remoteCacheConfig);
}
catch (Exception e)
{
return new CacheConstructionFailure(remoteCacheConfig.CacheId, e);
}
// Initialize local cache
var localCacheConfig = blobWithLocalCacheConfig.LocalCache;
MemoizationStore.Interfaces.Caches.ICache localCache;
try
{
// Let's use a single logger for both local and remote, so we follow the same user experience as the ephemeral cache, where local and remote logs go to the same file
localCache = new MemoizationStoreCacheFactory().CreateInnerCache(combinedLogger, localCacheConfig);
}
catch (Exception e)
{
return new CacheConstructionFailure(localCacheConfig.CacheId, e);
}
var twoLevelCacheConfig = new TwoLevelCacheConfiguration
{
RemoteCacheIsReadOnly = remoteCacheConfig.IsReadOnly,
AlwaysUpdateFromRemote = true,
BatchRemotePinsOnPut = false,
SkipRemotePutIfAlreadyExistsInLocal = false,
SkipRemotePinOnPut = true,
};
var innerCache = new TwoLevelCache(localCache, remoteCache, twoLevelCacheConfig);
var statsFilePath = new ContentStore.Interfaces.FileSystem.AbsolutePath(logPath.Path + ".stats");
var cache = new MemoizationStoreAdapterCache(
new CacheId(localCacheConfig.CacheId, remoteCacheConfig.CacheId),
innerCache,
combinedLogger,
statsFilePath,
// Not really used at this level. Each local & remote cache carry their own config at this respect.
isReadOnly: false,
replaceExistingOnPlaceFile: false,
implicitPin: ContentStore.Interfaces.Stores.ImplicitPin.None);
var startupResult = await cache.StartupAsync();
if (!startupResult.Succeeded)
{
return startupResult.Failure;
}
return cache;
}
/// <inheritdoc/>
public IEnumerable<Failure> ValidateConfiguration(ICacheConfigData cacheData)
{
// These validation callbacks are currently not working. Check comment on the base interface.
return new Failure[] { };
}
}
}

Просмотреть файл

@ -48,7 +48,7 @@ public class EphemeralCacheFactory : BlobCacheFactoryBase<EphemeralCacheConfig>,
return base.InitializeCacheAsync(configuration);
}
internal override async Task<MemoizationStore.Interfaces.Caches.ICache> CreateCacheAsync(ILogger logger, EphemeralCacheConfig configuration)
internal override async Task<MemoizationStore.Interfaces.Caches.ICache> CreateInnerCacheAsync(ILogger logger, EphemeralCacheConfig configuration)
{
var tracingContext = new Context(logger);
var context = new OperationContext(tracingContext);
@ -131,14 +131,6 @@ public class EphemeralCacheFactory : BlobCacheFactoryBase<EphemeralCacheConfig>,
var failures = new List<Failure>();
failures.AddFailureIfNullOrWhitespace(cacheConfig.CacheLogPath, nameof(cacheConfig.CacheLogPath));
failures.AddFailureIfNullOrWhitespace(cacheConfig.CacheId, nameof(cacheConfig.CacheId));
failures.AddFailureIfNullOrWhitespace(
cacheConfig.ConnectionStringEnvironmentVariableName,
nameof(cacheConfig.ConnectionStringEnvironmentVariableName));
failures.AddFailureIfNullOrWhitespace(
cacheConfig.ConnectionStringFileEnvironmentVariableName,
nameof(cacheConfig.ConnectionStringFileEnvironmentVariableName));
failures.AddFailureIfNullOrWhitespace(cacheConfig.Universe, nameof(cacheConfig.Universe));
failures.AddFailureIfNullOrWhitespace(cacheConfig.Namespace, nameof(cacheConfig.Namespace));
if (cacheConfig.IsReadOnly)
{
failures.Add(new Failure<string>(ReadOnlyModeError));

Просмотреть файл

@ -8,6 +8,7 @@ using System.Diagnostics.ContractsLight;
using System.Threading.Tasks;
using BuildXL.Cache.ContentStore.Distributed.NuCache;
using BuildXL.Cache.ContentStore.Grpc;
using BuildXL.Cache.ContentStore.Interfaces.Logging;
using BuildXL.Cache.ContentStore.Interfaces.Stores;
using BuildXL.Cache.ContentStore.Sessions;
using BuildXL.Cache.ContentStore.Stores;
@ -269,9 +270,7 @@ namespace BuildXL.Cache.MemoizationStoreAdapter
var logPath = new AbsolutePath(cacheConfig.CacheLogPath);
var logger = new DisposeLogger(() => new EtwFileLog(logPath.Path, cacheConfig.CacheId), cacheConfig.LogFlushIntervalSeconds);
var localCache = cacheConfig.UseStreamCAS
? CreateLocalCacheWithStreamPathCas(cacheConfig, logger)
: CreateGrpcCache(cacheConfig, logger);
var localCache = CreateInnerCache(logger, cacheConfig);
var statsFilePath = new AbsolutePath(logPath.Path + ".stats");
@ -298,6 +297,13 @@ namespace BuildXL.Cache.MemoizationStoreAdapter
}
}
internal MemoizationStore.Interfaces.Caches.ICache CreateInnerCache(ILogger logger, Config cacheConfig)
{
return cacheConfig.UseStreamCAS
? CreateLocalCacheWithStreamPathCas(cacheConfig, logger)
: CreateGrpcCache(cacheConfig, logger);
}
private static CasConfig GetCasConfig(Config config)
{
return new CasConfig
@ -359,7 +365,7 @@ namespace BuildXL.Cache.MemoizationStoreAdapter
};
}
private static MemoizationStore.Interfaces.Caches.ICache CreateGrpcCache(Config config, DisposeLogger logger)
private static MemoizationStore.Interfaces.Caches.ICache CreateGrpcCache(Config config, ILogger logger)
{
Contract.Requires(config.RetryIntervalSeconds >= 0);
Contract.Requires(config.RetryCount >= 0);
@ -406,7 +412,7 @@ namespace BuildXL.Cache.MemoizationStoreAdapter
}
private static LocalCache CreateLocalCacheWithStreamPathCas(Config config, DisposeLogger logger)
private static LocalCache CreateLocalCacheWithStreamPathCas(Config config, ILogger logger)
{
Contract.Requires(config.UseStreamCAS);

Просмотреть файл

@ -0,0 +1,134 @@
// Copyright (c) Microsoft Corporation.
// Licensed under the MIT License.
using System;
using System.IO;
using System.Linq;
using System.Threading.Tasks;
using BuildXL.Cache.ContentStore.InterfacesTest;
using BuildXL.Cache.Interfaces;
using BuildXL.Utilities.Configuration.Mutable;
using BuildXL.Utilities.Core;
using ContentStoreTest.Distributed.Redis;
using ContentStoreTest.Test;
using Test.BuildXL.TestUtilities.Xunit;
using Xunit;
using Xunit.Abstractions;
#nullable enable
namespace BuildXL.Cache.Tests
{
[Collection("Redis-based tests")]
[TestClassIfSupported(requiresWindowsBasedOperatingSystem: true)]
public class BlobWithLocalCacheTests : TestWithOutput
{
private readonly LocalRedisFixture _fixture;
private readonly PathTable _pathTable;
protected string ScratchPath => Path.Combine(Path.GetTempPath(), GetType().ToString());
public BlobWithLocalCacheTests(LocalRedisFixture fixture, ITestOutputHelper output)
: base(output)
{
_fixture = fixture;
_pathTable = new PathTable();
}
[Fact]
public async Task TestEndToEnd()
{
var (config, logPath) = SetupTest(nameof(TestEndToEnd));
FullCacheRecord cacheRecord = await CreateCacheAndRunPip(config);
// This check serves two purposes: we want to check the log file is properly setup and reflects both local and remote cache activity.
// We also want to check that the fake build resulted in adding (a CompareExchange call) content to both caches.
#if NET
var lines = await File.ReadAllLinesAsync(logPath);
#else
var lines = File.ReadAllLines(logPath);
#endif
Assert.True(lines.Any(line => line.Contains(cacheRecord.StrongFingerprint.WeakFingerprint.ToString()) && line.Contains("AzureBlobStorageMetadataStore.CompareExchangeAsync")));
Assert.True(lines.Any(line => line.Contains(cacheRecord.StrongFingerprint.WeakFingerprint.ToString()) && line.Contains("RocksDbMemoizationDatabase.CompareExchangeAsync")));
}
[Fact]
public async Task TestRemoteIsReadonly()
{
var (config, logPath) = SetupTest(nameof(TestRemoteIsReadonly), remoteIsReadOnly: true);
FullCacheRecord cacheRecord = await CreateCacheAndRunPip(config);
// We want to check that the fake build resulted in adding (a CompareExchange call) content to the local cache, but not to the remote cache.
#if NET
var lines = await File.ReadAllLinesAsync(logPath);
#else
var lines = File.ReadAllLines(logPath);
#endif
Assert.False(lines.Any(line => line.Contains(cacheRecord.StrongFingerprint.WeakFingerprint.ToString()) && line.Contains("AzureBlobStorageMetadataStore.CompareExchangeAsync")));
Assert.True(lines.Any(line => line.Contains(cacheRecord.StrongFingerprint.WeakFingerprint.ToString()) && line.Contains("RocksDbMemoizationDatabase.CompareExchangeAsync")));
}
private async Task<FullCacheRecord> CreateCacheAndRunPip(string config)
{
ICache? cache = null;
FullCacheRecord? cacheRecord = null;
try
{
var configuration = new ConfigurationImpl();
cache = await CacheFactory.InitializeCacheAsync(config, default(Guid), configuration, _pathTable).SuccessAsync();
var session = await cache.CreateSessionAsync().SuccessAsync();
cacheRecord = await FakeBuild.DoPipAsync(session, "TestPip");
await session.CloseAsync().SuccessAsync();
}
finally
{
if (cache != null)
{
await cache.ShutdownAsync().SuccessAsync();
}
}
return cacheRecord;
}
/// <summary>
/// Each test will get clean caches, for both remote and local. The universe is based on the test name.
/// </summary>
private (string configuration, string logPath) SetupTest(string testName, bool remoteIsReadOnly = false)
{
// Start emulator to get a blob storage account
using var storage = AzuriteStorageProcess.CreateAndStartEmpty(_fixture, TestGlobal.Logger);
// Let's use an env var to communicate the connection string
var connectionStringEnvVarName = $"ConnectionString{GetType()}{testName}";
Environment.SetEnvironmentVariable(connectionStringEnvVarName, storage.ConnectionString);
var logDir = Path.Combine(ScratchPath, testName, "logs");
var baseLogPath = Path.Combine(logDir, "blobWithLocal");
var rootDir = Path.Combine(ScratchPath, testName, "root");
var config = $@"{{
""Assembly"": ""BuildXL.Cache.MemoizationStoreAdapter"",
""Type"": ""BuildXL.Cache.MemoizationStoreAdapter.BlobWithLocalCacheFactory"",
""LocalCache"": {{
""MaxCacheSizeInMB"": 40480,
""CacheLogPath"": ""{baseLogPath.Replace("\\", "\\\\")}.local.log"",
""CacheRootPath"": ""{rootDir.Replace("\\", "\\\\")}"",
""CacheId"": ""TestLocal""
}},
""RemoteCache"": {{
""CacheLogPath"": ""{baseLogPath.Replace("\\", "\\\\")}"",
""CacheId"": ""TestBlob"",
""Universe"" : ""{testName.ToLowerInvariant()}"",
""RetentionPolicyInDays"": 6,
""ConnectionStringEnvironmentVariableName"" : ""{connectionStringEnvVarName}""
{(remoteIsReadOnly? @",""IsReadOnly"" : true" : string.Empty)}
}}
}}
";
return (config, baseLogPath);
}
}
}

Просмотреть файл

@ -0,0 +1,22 @@
// Copyright (c) Microsoft Corporation.
// Licensed under the MIT License.
using ContentStoreTest.Distributed.Redis;
using Xunit;
namespace BuildXL.Cache.Tests
{
/// <summary>
/// Custom collection that uses <see cref="LocalRedisFixture"/>.
/// </summary>
/// <remarks>
/// WARNING: there needs to be one of these per assembly that needs to usee <see cref="LocalRedisFixture"/>
/// </remarks>
[CollectionDefinition("Redis-based tests")]
public class LocalRedisCollection : ICollectionFixture<LocalRedisFixture>
{
// This class has no code, and is never created. Its purpose is simply
// to be the place to apply [CollectionDefinition] and all the
// ICollectionFixture<> interfaces.
}
}

Просмотреть файл

@ -0,0 +1,110 @@
// Copyright (c) Microsoft. All rights reserved.
// Licensed under the MIT license. See LICENSE file in the project root for full license information.
import * as XUnit from "Sdk.Managed.Testing.XUnit";
import * as ManagedSdk from "Sdk.Managed";
import { Transformer } from "Sdk.Transformers";
import * as ContentStore from "BuildXL.Cache.ContentStore";
namespace BlobWithLocalCache {
export declare const qualifier : BuildXLSdk.DefaultQualifierWithNet472;
@@public
export const dll = BuildXLSdk.cacheTest({
assemblyName: "BuildXL.Cache.BlobWithLocalCache.Test",
sources: globR(d`.`, "*.cs"),
allowUnsafeBlocks: true,
runTestArgs: {
untrackTestDirectory: true, // GRPC server may create memory-mapped files in this directory
unsafeTestRunArguments: {
untrackedPaths: [
...addIfLazy(Context.isWindowsOS() && Environment.getDirectoryValue("CommonProgramFiles") !== undefined,
() => [f`${Environment.getDirectoryValue("CommonProgramFiles")}/SSL/openssl.cnf`])
]
}
},
assemblyBindingRedirects: BuildXLSdk.cacheBindingRedirects(),
references: [
...addIf(BuildXLSdk.isFullFramework, importFrom("System.Collections.Immutable").pkg),
...addIf(BuildXLSdk.isFullFramework,
NetFx.System.IO.dll,
NetFx.System.Net.Primitives.dll,
NetFx.System.Xml.dll,
NetFx.System.Xml.Linq.dll
),
...addIf(BuildXLSdk.isFullFramework,
importFrom("Azure.Core.Amqp").pkg,
importFrom("Microsoft.Azure.Amqp").pkg,
importFrom("System.IdentityModel.Tokens.Jwt").pkg
),
...ContentStore.getSerializationPackages(true),
ContentStore.Distributed.dll,
ContentStore.DistributedTest.dll,
...ContentStore.Distributed.eventHubPackages,
ContentStore.UtilitiesCore.dll,
ContentStore.Hashing.dll,
ContentStore.Interfaces.dll,
Interfaces.dll,
ContentStore.InterfacesTest.dll,
ContentStore.Library.dll,
ContentStore.Test.dll,
importFrom("BuildXL.Cache.MemoizationStore").Interfaces.dll,
importFrom("BuildXL.Cache.MemoizationStore").Library.dll,
importFrom("BuildXL.Cache.MemoizationStore").Distributed.dll,
importFrom("BuildXL.Cache.DistributedCache.Host").Service.dll,
importFrom("BuildXL.Cache.DistributedCache.Host").Configuration.dll,
importFrom("BuildXL.Utilities").dll,
importFrom("BuildXL.Utilities").KeyValueStore.dll,
importFrom("BuildXL.Utilities").Native.dll,
importFrom("BuildXL.Utilities").Utilities.Core.dll,
importFrom("BuildXL.Utilities").Configuration.dll,
importFrom("BuildXL.Cache.VerticalStore").Interfaces.dll,
// Using gRPC.NET hosting implementation from the launcher.
...addIfLazy(!BuildXLSdk.isFullFramework, () => [importFrom("BuildXL.Cache.DistributedCache.Host").LauncherServer.exe]),
ContentStore.Grpc.dll,
...importFrom("Sdk.Selfhost.RocksDbSharp").pkgs,
...BuildXLSdk.fluentAssertionsWorkaround,
...BuildXLSdk.systemThreadingTasksDataflowPackageReference,
...ContentStore.getAzureBlobStorageSdkPackages(true),
...ContentStore.getGrpcPackages(true),
...ContentStore.getProtobufNetPackages(true),
...BuildXLSdk.getSystemMemoryPackages(true),
importFrom("System.ServiceModel.Http").pkg,
importFrom("System.ServiceModel.Primitives").pkg,
...(BuildXLSdk.isFullFramework
? [
importFrom("System.IO.Pipelines").pkg,
importFrom("System.Runtime.CompilerServices.Unsafe").pkg,
importFrom("Pipelines.Sockets.Unofficial").pkg,
]
: [
importFrom("System.IO.Pipelines").pkg,
...(BuildXLSdk.isDotNetCore ? [] : [
importFrom("System.Runtime.CompilerServices.Unsafe").pkg,
]),
importFrom("Pipelines.Sockets.Unofficial").pkg,
]),
...BuildXLSdk.systemThreadingChannelsPackages,
...BuildXLSdk.bclAsyncPackages,
// Needed because of snipped dependencies for System.IO.Pipelines and System.Threading.Channels
importFrom("System.Threading.Tasks.Extensions").pkg,
],
runtimeContent: [
{
subfolder: r`azurite`,
contents: [
importFrom("BuildXL.Azurite.Executables").Contents.all
]
},
importFrom("BuildXL.Cache.VerticalStore").MemoizationStoreAdapter.dll
],
});
}