Initial Hosting/DI model implementation

Co-authored-by: Fabio Cavalcante <fabio.cavalcante@microsoft.com>
Co-authored-by: Brett Samblanet <brettsam@microsoft.com>
This commit is contained in:
Fabio Cavalcante 2017-11-01 19:47:38 -07:00
Родитель 68eb063918
Коммит 7f130355c4
234 изменённых файлов: 4279 добавлений и 4352 удалений

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

@ -12,7 +12,7 @@ branches:
- dev
- master
- next
image: Visual Studio 2017
image: Visual Studio 2017 Preview
max_jobs: 1
install:
- ps: |

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

@ -17,25 +17,25 @@ namespace SampleHost
public static class Functions
{
public static void BlobTrigger(
[BlobTrigger("test")] string blob)
[BlobTrigger("test")] string blob, ILogger logger)
{
Console.WriteLine("Processed blob: " + blob);
logger.LogInformation("Processed blob: " + blob);
}
public static void BlobPoisonBlobHandler(
[QueueTrigger("webjobs-blobtrigger-poison")] JObject blobInfo)
[QueueTrigger("webjobs-blobtrigger-poison")] JObject blobInfo, ILogger logger)
{
string container = (string)blobInfo["ContainerName"];
string blobName = (string)blobInfo["BlobName"];
Console.WriteLine($"Poison blob: {container}/{blobName}");
logger.LogInformation($"Poison blob: {container}/{blobName}");
}
[WorkItemValidator]
public static void ProcessWorkItem(
[QueueTrigger("test")] WorkItem workItem)
[QueueTrigger("test")] WorkItem workItem, ILogger logger)
{
Console.WriteLine($"Processed work item {workItem.ID}");
logger.LogInformation($"Processed work item {workItem.ID}");
}
public static async Task ProcessWorkItem_ServiceBus(

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

@ -1,53 +1,52 @@
// Copyright (c) .NET Foundation. All rights reserved.
// Licensed under the MIT License. See License.txt in the project root for license information.
using System;
using Microsoft.Azure.WebJobs;
using Microsoft.Azure.WebJobs.Logging;
using Microsoft.Extensions.Logging;
using Microsoft.Azure.WebJobs.Hosting;
using Microsoft.Extensions.Hosting;
using System.Threading.Tasks;
using Microsoft.Extensions.DependencyInjection;
using Microsoft.Azure.WebJobs.Host;
using Microsoft.Extensions.Configuration;
using System.Collections.Generic;
namespace SampleHost
{
class Program
{
static void Main(string[] args)
public static async Task Main(string[] args)
{
var config = new JobHostConfiguration();
config.Queues.VisibilityTimeout = TimeSpan.FromSeconds(15);
config.Queues.MaxDequeueCount = 3;
config.LoggerFactory = new LoggerFactory().AddConsole();
var builder = new HostBuilder()
.UseEnvironment("Development")
.ConfigureWebJobsHost(o =>
{
// Example setting options properties:
// o.HostId = "testhostid";
})
.ConfigureAppConfiguration(config =>
{
// Adding command line as a configuration source
config.AddCommandLine(args);
config.AddInMemoryCollection(new Dictionary<string, string>()
{
// Configuration options set from configuration providers:
{ "HostId", "testhostidfromprovider" }
});
})
.ConfigureLogging(b =>
{
b.SetMinimumLevel(LogLevel.Debug);
b.AddConsole();
b.AddApplicationInsights();
})
.UseConsoleLifetime();
if (config.IsDevelopment)
var jobHost = builder.Build();
using (jobHost)
{
config.UseDevelopmentSettings();
}
config.UseEventHub();
config.UseServiceBus();
CheckAndEnableAppInsights(config);
var host = new JobHost(config);
host.RunAndBlock();
}
private static void CheckAndEnableAppInsights(JobHostConfiguration config)
{
// If AppInsights is enabled, build up a LoggerFactory
string instrumentationKey = Environment.GetEnvironmentVariable("APPINSIGHTS_INSTRUMENTATIONKEY");
if (!string.IsNullOrEmpty(instrumentationKey))
{
var filter = new LogCategoryFilter();
filter.DefaultLevel = LogLevel.Debug;
filter.CategoryLevels[LogCategories.Results] = LogLevel.Debug;
filter.CategoryLevels[LogCategories.Aggregator] = LogLevel.Debug;
// Adjust the LogLevel for a specific Function.
filter.CategoryLevels[LogCategories.CreateFunctionCategory(nameof(Functions.ProcessWorkItem))] = LogLevel.Debug;
config.LoggerFactory = new LoggerFactory()
.AddApplicationInsights(instrumentationKey, filter.Filter)
.AddConsole(filter.Filter);
await jobHost.RunAsync();
}
}
}

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

@ -6,6 +6,10 @@
<CodeAnalysisRuleSet>$(SolutionDir)\src\src.ruleset</CodeAnalysisRuleSet>
</PropertyGroup>
<PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Debug|AnyCPU'">
<LangVersion>7.1</LangVersion>
</PropertyGroup>
<ItemGroup>
<AdditionalFiles Include="..\..\stylecop.json" Link="stylecop.json" />
<None Include="*.json" CopyToOutputDirectory="PreserveNewest" />

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

@ -0,0 +1,33 @@
// Copyright (c) .NET Foundation. All rights reserved.
// Licensed under the MIT License. See License.txt in the project root for license information.
using System;
using Microsoft.Azure.WebJobs.Logging;
using Microsoft.Extensions.Logging;
using Microsoft.ApplicationInsights.WindowsServer.TelemetryChannel.Implementation;
using Microsoft.Azure.WebJobs.Logging.ApplicationInsights;
namespace SampleHost
{
public static class SampleLoggingExtensions
{
public static void AddApplicationInsights(this ILoggingBuilder builder)
{
// If AppInsights is enabled, build up a LoggerFactory
string instrumentationKey = Environment.GetEnvironmentVariable("APPINSIGHTS_INSTRUMENTATIONKEY");
if (!string.IsNullOrEmpty(instrumentationKey))
{
var filter = new LogCategoryFilter
{
DefaultLevel = LogLevel.Debug
};
filter.CategoryLevels[LogCategories.Results] = LogLevel.Debug;
filter.CategoryLevels[LogCategories.Aggregator] = LogLevel.Debug;
ITelemetryClientFactory defaultFactory = new DefaultTelemetryClientFactory(instrumentationKey, new SamplingPercentageEstimatorSettings(), filter.Filter);
builder.AddProvider(new ApplicationInsightsLoggerProvider(defaultFactory));
}
}
}
}

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

@ -2,18 +2,12 @@
// Licensed under the MIT License. See License.txt in the project root for license information.
using System;
using System.Collections.Concurrent;
using System.Collections.Generic;
using System.Globalization;
using System.Text;
using System.Threading.Tasks;
using Microsoft.Azure.EventHubs;
using Microsoft.Azure.EventHubs.Processor;
using Microsoft.Azure.WebJobs.Host.Bindings;
using Microsoft.Azure.WebJobs.Host.Config;
using Microsoft.Azure.WebJobs.Logging;
using Newtonsoft.Json;
using Microsoft.Extensions.Logging;
using Microsoft.Azure.WebJobs.Host.Executors;
namespace Microsoft.Azure.WebJobs.ServiceBus
{
@ -21,19 +15,17 @@ namespace Microsoft.Azure.WebJobs.ServiceBus
/// Provide configuration for event hubs.
/// This is primarily mapping names to underlying EventHub listener and receiver objects from the EventHubs SDK.
/// </summary>
public class EventHubConfiguration : IExtensionConfigProvider
public class EventHubConfiguration
{
// Event Hub Names are case-insensitive.
// The same path can have multiple connection strings with different permissions (sending and receiving),
// so we track senders and receivers separately and infer which one to use based on the EventHub (sender) vs. EventHubTrigger (receiver) attribute.
// Connection strings may also encapsulate different endpoints.
//
// The client cache must be thread safe because clients are accessed/added on the function
// invocation path (BuildFromAttribute)
private readonly ConcurrentDictionary<string, EventHubClient> _clients = new ConcurrentDictionary<string, EventHubClient>(StringComparer.OrdinalIgnoreCase);
// Connection strings may also encapsulate different endpoints.
private readonly Dictionary<string, EventHubClient> _senders = new Dictionary<string, EventHubClient>(StringComparer.OrdinalIgnoreCase);
private readonly Dictionary<string, ReceiverCreds> _receiverCreds = new Dictionary<string, ReceiverCreds>(StringComparer.OrdinalIgnoreCase);
private readonly Dictionary<string, EventProcessorHost> _explicitlyProvidedHosts = new Dictionary<string, EventProcessorHost>(StringComparer.OrdinalIgnoreCase);
private readonly EventProcessorOptions _options;
private string _defaultStorageString;
private int _batchCheckpointFrequency = 1;
@ -44,17 +36,27 @@ namespace Microsoft.Azure.WebJobs.ServiceBus
public const string LeaseContainerName = "azure-webjobs-eventhub";
/// <summary>
/// Constructs a new instance.
/// default constructor. Callers can reference this without having any assembly references to service bus assemblies.
/// </summary>
public EventHubConfiguration()
: this(null, null)
{
// Our default options will delegate to our own exception
// logger. Customers can override this completely by setting their
// own EventProcessorOptions instance.
EventProcessorOptions = EventProcessorOptions.DefaultOptions;
EventProcessorOptions.MaxBatchSize = 64;
EventProcessorOptions.PrefetchCount = EventProcessorOptions.MaxBatchSize * 4;
EventProcessorOptions.SetExceptionHandler(ExceptionReceivedHandler);
}
/// <summary>
/// Constructs a new instance.
/// </summary>
/// <param name="options">The optional <see cref="EventProcessorOptions"/> to use when receiving events.</param>
public EventHubConfiguration(IStorageAccountProvider accountProvider, EventProcessorOptions options = null)
{
if (options == null)
{
options = EventProcessorOptions.DefaultOptions;
options.MaxBatchSize = 64;
options.PrefetchCount = options.MaxBatchSize * 4;
}
_options = options;
_defaultStorageString = accountProvider?.StorageConnectionString;
}
/// <summary>
@ -106,8 +108,8 @@ namespace Microsoft.Azure.WebJobs.ServiceBus
{
throw new ArgumentNullException("client");
}
_clients[eventHubName] = client;
_senders[eventHubName] = client;
}
/// <summary>
@ -130,7 +132,7 @@ namespace Microsoft.Azure.WebJobs.ServiceBus
if (string.IsNullOrWhiteSpace(sb.EntityPath))
{
sb.EntityPath = eventHubName;
}
}
var client = EventHubClient.CreateFromConnectionString(sb.ToString());
AddEventHubClient(eventHubName, client);
@ -178,7 +180,7 @@ namespace Microsoft.Azure.WebJobs.ServiceBus
this._receiverCreds[eventHubName] = new ReceiverCreds
{
EventHubConnectionString = receiverConnectionString
EventHubConnectionString = receiverConnectionString
};
}
@ -209,21 +211,18 @@ namespace Microsoft.Azure.WebJobs.ServiceBus
StorageConnectionString = storageConnectionString
};
}
internal EventHubClient GetEventHubClient(string eventHubName, string connection)
{
EventHubClient client;
if (_clients.TryGetValue(eventHubName, out client))
if (_senders.TryGetValue(eventHubName, out client))
{
return client;
}
else if (!string.IsNullOrWhiteSpace(connection))
{
return _clients.GetOrAdd(eventHubName, k =>
{
AddSender(k, connection);
return _clients[k];
});
AddSender(eventHubName, connection);
return _senders[eventHubName];
}
throw new InvalidOperationException("No event hub sender named " + eventHubName);
}
@ -264,9 +263,9 @@ namespace Microsoft.Azure.WebJobs.ServiceBus
EventProcessorHost host = new EventProcessorHost(
hostName: eventProcessorHostName,
eventHubPath: actualPath,
consumerGroupName: consumerGroup,
consumerGroupName: consumerGroup,
eventHubConnectionString: sb.ToString(),
storageConnectionString: storageConnectionString,
storageConnectionString: storageConnectionString,
leaseContainerName: LeaseContainerName,
storageBlobPrefix: blobPrefix);
@ -296,7 +295,7 @@ namespace Microsoft.Azure.WebJobs.ServiceBus
return string.Format(CultureInfo.InvariantCulture, "::{0:X4}", ordinalValue);
}
}
// Escape a blob path.
// For diagnostics, we want human-readble strings that resemble the input.
// Inputs are most commonly alphanumeric with a fex extra chars (dash, underscore, dot).
@ -311,7 +310,7 @@ namespace Microsoft.Azure.WebJobs.ServiceBus
{
sb.Append(c);
}
else if (c == '-' || c == '_' || c == '.')
else if (c == '-' || c == '_' || c == '.')
{
// Potentially common carahcters.
sb.Append(c);
@ -367,105 +366,9 @@ namespace Microsoft.Azure.WebJobs.ServiceBus
string key = EscapeBlobPath(serviceBusNamespace) + "/" + EscapeBlobPath(eventHubName) + "/";
return key;
}
public EventProcessorOptions EventProcessorOptions { get; set; }
void IExtensionConfigProvider.Initialize(ExtensionConfigContext context)
{
if (context == null)
{
throw new ArgumentNullException("context");
}
// apply at eventProcessorOptions level (maxBatchSize, prefetchCount)
context.ApplyConfig(EventProcessorOptions, "eventHub");
// apply at config level (batchCheckpointFrequency)
context.ApplyConfig(this, "eventHub");
_defaultStorageString = context.Config.StorageConnectionString;
context
.AddConverter<string, EventData>(ConvertString2EventData)
.AddConverter<EventData, string>(ConvertEventData2String)
.AddConverter<byte[], EventData>(ConvertBytes2EventData)
.AddConverter<EventData, byte[]>(ConvertEventData2Bytes)
.AddOpenConverter<OpenType.Poco, EventData>(ConvertPocoToEventData);
// register our trigger binding provider
INameResolver nameResolver = context.Config.NameResolver;
IConverterManager cm = context.Config.GetService<IConverterManager>();
var triggerBindingProvider = new EventHubTriggerAttributeBindingProvider(nameResolver, cm, this, context.Config.LoggerFactory);
context.AddBindingRule<EventHubTriggerAttribute>()
.BindToTrigger(triggerBindingProvider);
// register our binding provider
context.AddBindingRule<EventHubAttribute>()
.BindToCollector(BuildFromAttribute);
// Set the default exception handler for background exceptions
// coming from the EventProcessorHost.
ExceptionHandler = (e =>
{
LogExceptionReceivedEvent(e, context.Config.LoggerFactory);
});
}
internal static void LogExceptionReceivedEvent(ExceptionReceivedEventArgs e, ILoggerFactory loggerFactory)
{
try
{
var logger = loggerFactory?.CreateLogger(LogCategories.Executor);
string message = $"EventProcessorHost error (Action={e.Action}, HostName={e.Hostname}, PartitionId={e.PartitionId})";
var ehex = e.Exception as EventHubsException;
if (!(e.Exception is OperationCanceledException) && (ehex == null || !ehex.IsTransient))
{
// any non-transient exceptions or unknown exception types
// we want to log as errors
logger?.LogError(0, e.Exception, message);
}
else
{
// transient errors we log as verbose so we have a record
// of them, but we don't treat them as actual errors
logger?.LogDebug(0, e.Exception, message);
}
}
catch
{
// best effort logging
}
}
private IAsyncCollector<EventData> BuildFromAttribute(EventHubAttribute attribute)
{
EventHubClient client = this.GetEventHubClient(attribute.EventHubName, attribute.Connection);
return new EventHubAsyncCollector(client);
}
private static string ConvertEventData2String(EventData x)
=> Encoding.UTF8.GetString(ConvertEventData2Bytes(x));
private static EventData ConvertBytes2EventData(byte[] input)
=> new EventData(input);
private static byte[] ConvertEventData2Bytes(EventData input)
=> input.Body.Array;
private static EventData ConvertString2EventData(string input)
=> ConvertBytes2EventData(Encoding.UTF8.GetBytes(input));
private static Task<object> ConvertPocoToEventData(object arg, Attribute attrResolved, ValueBindingContext context)
{
return Task.FromResult<object>(ConvertString2EventData(JsonConvert.SerializeObject(arg)));
}
internal Action<ExceptionReceivedEventArgs> ExceptionHandler { get; set; }
private void ExceptionReceivedHandler(ExceptionReceivedEventArgs args)
{
ExceptionHandler?.Invoke(args);
}
// Get the eventhub options, used by the EventHub SDK for listening on event.
internal EventProcessorOptions GetOptions() => _options;
// Hold credentials for a given eventHub name.
// Multiple consumer groups (and multiple listeners) on the same hub can share the same credentials.

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

@ -1,54 +0,0 @@
// Copyright (c) .NET Foundation. All rights reserved.
// Licensed under the MIT License. See License.txt in the project root for license information.
using System;
using Microsoft.Azure.WebJobs.Host;
using Microsoft.Azure.WebJobs.Host.Config;
using Microsoft.Azure.WebJobs.ServiceBus;
namespace Microsoft.Azure.WebJobs
{
/// <summary>
/// Extension for registering an event hub configuration with the JobHostConfiguration.
/// </summary>
public static class EventHubJobHostConfigurationExtensions
{
/// <summary>
/// Enable connecting to event hubs for sending and receiving events. This call is required to the <see cref="EventHubAttribute"/>
/// and <see cref="EventHubTriggerAttribute"/> attributes on parameter bindings.
/// </summary>
/// <param name="config">job host configuration</param>
public static void UseEventHub(this JobHostConfiguration config)
{
if (config == null)
{
throw new ArgumentNullException("config");
}
var eventHubConfig = new EventHubConfiguration();
config.UseEventHub(eventHubConfig);
}
/// <summary>
/// Enable connecting to event hubs for sending and receiving events. This call is required to the <see cref="EventHubAttribute"/>
/// and <see cref="EventHubTriggerAttribute"/> attributes on parameter bindings.
/// </summary>
/// <param name="config">job host configuration</param>
/// <param name="eventHubConfig">event hub configuration containing connection strings to the event hubs.</param>
[System.Diagnostics.CodeAnalysis.SuppressMessage("Microsoft.Design", "CA1011:ConsiderPassingBaseTypesAsParameters")]
public static void UseEventHub(this JobHostConfiguration config, EventHubConfiguration eventHubConfig)
{
if (config == null)
{
throw new ArgumentNullException("config");
}
if (eventHubConfig == null)
{
throw new ArgumentNullException("eventHubConfig");
}
IExtensionRegistry extensions = config.GetService<IExtensionRegistry>();
extensions.RegisterExtension<IExtensionConfigProvider>(eventHubConfig);
}
}
}

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

@ -1,6 +1,8 @@
// Copyright (c) .NET Foundation. All rights reserved.
// Licensed under the MIT License. See License.txt in the project root for license information.
using Microsoft.Extensions.Configuration;
namespace Microsoft.Azure.WebJobs.Host
{
/// <summary>
@ -9,20 +11,12 @@ namespace Microsoft.Azure.WebJobs.Host
/// </summary>
public class AmbientConnectionStringProvider : IConnectionStringProvider
{
private static readonly AmbientConnectionStringProvider Singleton = new AmbientConnectionStringProvider();
internal const string Prefix = "AzureWebJobs";
private IConfiguration _configuration;
internal static readonly string Prefix = "AzureWebJobs";
private AmbientConnectionStringProvider()
public AmbientConnectionStringProvider(IConfiguration configuration)
{
}
/// <summary>
/// Gets the singleton instance.
/// </summary>
public static AmbientConnectionStringProvider Instance
{
get { return Singleton; }
_configuration = configuration;
}
/// <summary>
@ -35,12 +29,12 @@ namespace Microsoft.Azure.WebJobs.Host
{
// first try prefixing
string prefixedConnectionStringName = GetPrefixedConnectionStringName(connectionStringName);
string connectionString = ConfigurationUtility.GetConnectionString(prefixedConnectionStringName);
string connectionString = FindConnectionString(prefixedConnectionStringName);
if (string.IsNullOrEmpty(connectionString))
{
// next try a direct unprefixed lookup
connectionString = ConfigurationUtility.GetConnectionString(connectionStringName);
connectionString = FindConnectionString(connectionStringName);
}
return connectionString;
@ -50,5 +44,11 @@ namespace Microsoft.Azure.WebJobs.Host
{
return Prefix + connectionStringName;
}
private string FindConnectionString(string connectionName) =>
_configuration.GetConnectionString(connectionName) ?? _configuration[connectionName];
}
}

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

@ -209,7 +209,7 @@ namespace Microsoft.Azure.WebJobs.Host.Bindings
public ParameterDescriptor ToParameterDescriptor()
{
throw new NotImplementedException();
return new ParameterDescriptor();
}
}
}

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

@ -87,7 +87,7 @@ namespace Microsoft.Azure.WebJobs
});
}
// If somebody registered a converter from Src-->Dest, then both those types can be used to
// If somebody registered a converter from Src-->Dest, then both those types can be used to
// resolve assemblies.
// The attribute type always points to the extension's assembly.
// Whereas some of the Src,Dest types will point to the resource's "native sdk"
@ -99,6 +99,7 @@ namespace Microsoft.Azure.WebJobs
AddType(entry.Dest, funcAddType);
}
}
private void AddType(OpenType type, Action<Type> funcAddType)
{
if (type is OpenType.ExactMatch x)

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

@ -0,0 +1,14 @@
// Copyright (c) .NET Foundation. All rights reserved.
// Licensed under the MIT License. See License.txt in the project root for license information.
using System;
using System.Collections.Generic;
using System.Text;
namespace Microsoft.Azure.WebJobs.Host.Bindings
{
public interface IBindingProviderFactory
{
IBindingProvider Create();
}
}

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

@ -30,6 +30,16 @@ namespace Microsoft.Azure.WebJobs.Host.Blobs.Bindings
private INameResolver _nameResolver;
private IConverterManager _converterManager;
public BlobExtensionConfig(IStorageAccountProvider accountProvider,
IContextGetter<IBlobWrittenWatcher> contextAccessor,
INameResolver nameResolver,
IConverterManager converterManager)
{
_accountProvider = accountProvider;
_blobWrittenWatcherGetter = contextAccessor;
_nameResolver = nameResolver;
_converterManager = converterManager;
}
#region Container rules
async Task<CloudBlobContainer> IAsyncConverter<BlobAttribute, CloudBlobContainer>.ConvertAsync(
@ -236,12 +246,6 @@ namespace Microsoft.Azure.WebJobs.Host.Blobs.Bindings
public void Initialize(ExtensionConfigContext context)
{
_accountProvider = context.Config.GetService<IStorageAccountProvider>();
_blobWrittenWatcherGetter = context.PerHostServices.GetService<ContextAccessor<IBlobWrittenWatcher>>();
_nameResolver = context.Config.NameResolver;
_converterManager = context.Config.ConverterManager;
var rule = context.AddBindingRule<BlobAttribute>();
// Bind to multiple blobs (either via a container; a blob directory, an IEnumerable<T>)

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

@ -20,8 +20,8 @@ namespace Microsoft.Azure.WebJobs.Host.Blobs.Listeners
{
private const string SingletonBlobListenerScopeId = "WebJobs.Internal.Blobs";
private readonly IHostIdProvider _hostIdProvider;
private readonly IQueueConfiguration _queueConfiguration;
private readonly JobHostBlobsConfiguration _blobsConfiguration;
private readonly JobHostQueuesOptions _queueOptions;
private readonly JobHostBlobsOptions _blobsOptions;
private readonly IWebJobsExceptionHandler _exceptionHandler;
private readonly IContextSetter<IBlobWrittenWatcher> _blobWrittenWatcherSetter;
private readonly IContextSetter<IMessageEnqueuedWatcher> _messageEnqueuedWatcherSetter;
@ -36,8 +36,8 @@ namespace Microsoft.Azure.WebJobs.Host.Blobs.Listeners
private readonly SingletonManager _singletonManager;
public BlobListenerFactory(IHostIdProvider hostIdProvider,
IQueueConfiguration queueConfiguration,
JobHostBlobsConfiguration blobsConfiguration,
JobHostQueuesOptions queueOptions,
JobHostBlobsOptions blobsOptions,
IWebJobsExceptionHandler exceptionHandler,
IContextSetter<IBlobWrittenWatcher> blobWrittenWatcherSetter,
IContextSetter<IMessageEnqueuedWatcher> messageEnqueuedWatcherSetter,
@ -52,8 +52,8 @@ namespace Microsoft.Azure.WebJobs.Host.Blobs.Listeners
SingletonManager singletonManager)
{
_hostIdProvider = hostIdProvider ?? throw new ArgumentNullException(nameof(hostIdProvider));
_queueConfiguration = queueConfiguration ?? throw new ArgumentNullException(nameof(queueConfiguration));
_blobsConfiguration = blobsConfiguration ?? throw new ArgumentNullException(nameof(blobsConfiguration));
_queueOptions = queueOptions ?? throw new ArgumentNullException(nameof(queueOptions));
_blobsOptions = blobsOptions ?? throw new ArgumentNullException(nameof(blobsOptions));
_exceptionHandler = exceptionHandler ?? throw new ArgumentNullException(nameof(exceptionHandler));
_blobWrittenWatcherSetter = blobWrittenWatcherSetter ?? throw new ArgumentNullException(nameof(blobWrittenWatcherSetter));
_messageEnqueuedWatcherSetter = messageEnqueuedWatcherSetter ?? throw new ArgumentNullException(nameof(messageEnqueuedWatcherSetter));
@ -99,7 +99,7 @@ namespace Microsoft.Azure.WebJobs.Host.Blobs.Listeners
// notification queue and dispatch to the target job function.
SharedBlobQueueListener sharedBlobQueueListener = _sharedContextProvider.GetOrCreateInstance<SharedBlobQueueListener>(
new SharedBlobQueueListenerFactory(_hostAccount, sharedQueueWatcher, hostBlobTriggerQueue,
_queueConfiguration, _exceptionHandler, _loggerFactory, sharedBlobListener.BlobWritterWatcher));
_queueOptions, _exceptionHandler, _loggerFactory, sharedBlobListener.BlobWritterWatcher));
var queueListener = new BlobListener(sharedBlobQueueListener);
// determine which client to use for the poison queue
@ -107,7 +107,7 @@ namespace Microsoft.Azure.WebJobs.Host.Blobs.Listeners
// as the blob container we're monitoring
var poisonQueueClient = targetQueueClient;
if (_dataAccount.Type != StorageAccountType.GeneralPurpose ||
_blobsConfiguration.CentralizedPoisonQueue)
_blobsOptions.CentralizedPoisonQueue)
{
// use the primary storage account if the centralize flag is true,
// or if the target storage account doesn't support queues

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

@ -20,7 +20,7 @@ namespace Microsoft.Azure.WebJobs.Host.Blobs.Listeners
{
private readonly SharedQueueWatcher _sharedQueueWatcher;
private readonly IStorageQueue _hostBlobTriggerQueue;
private readonly IQueueConfiguration _queueConfiguration;
private readonly JobHostQueuesOptions _queueOptions;
private readonly IWebJobsExceptionHandler _exceptionHandler;
private readonly IBlobWrittenWatcher _blobWrittenWatcher;
private readonly IStorageAccount _hostAccount;
@ -30,7 +30,7 @@ namespace Microsoft.Azure.WebJobs.Host.Blobs.Listeners
IStorageAccount hostAccount,
SharedQueueWatcher sharedQueueWatcher,
IStorageQueue hostBlobTriggerQueue,
IQueueConfiguration queueConfiguration,
JobHostQueuesOptions queueOptions,
IWebJobsExceptionHandler exceptionHandler,
ILoggerFactory loggerFactory,
IBlobWrittenWatcher blobWrittenWatcher)
@ -38,7 +38,7 @@ namespace Microsoft.Azure.WebJobs.Host.Blobs.Listeners
_hostAccount = hostAccount ?? throw new ArgumentNullException(nameof(hostAccount));
_sharedQueueWatcher = sharedQueueWatcher ?? throw new ArgumentNullException(nameof(sharedQueueWatcher));
_hostBlobTriggerQueue = hostBlobTriggerQueue ?? throw new ArgumentNullException(nameof(hostBlobTriggerQueue));
_queueConfiguration = queueConfiguration ?? throw new ArgumentNullException(nameof(queueConfiguration));
_queueOptions = queueOptions ?? throw new ArgumentNullException(nameof(queueOptions));
_exceptionHandler = exceptionHandler ?? throw new ArgumentNullException(nameof(exceptionHandler));
_loggerFactory = loggerFactory;
_blobWrittenWatcher = blobWrittenWatcher ?? throw new ArgumentNullException(nameof(blobWrittenWatcher));
@ -60,11 +60,11 @@ namespace Microsoft.Azure.WebJobs.Host.Blobs.Listeners
// this special queue bypasses the QueueProcessorFactory - we don't want people to
// override this
QueueProcessorFactoryContext context = new QueueProcessorFactoryContext(_hostBlobTriggerQueue.SdkObject, _loggerFactory,
_queueConfiguration, defaultPoisonQueue.SdkObject);
_queueOptions, defaultPoisonQueue.SdkObject);
SharedBlobQueueProcessor queueProcessor = new SharedBlobQueueProcessor(context, triggerExecutor);
IListener listener = new QueueListener(_hostBlobTriggerQueue, defaultPoisonQueue, triggerExecutor, _exceptionHandler, _loggerFactory,
_sharedQueueWatcher, _queueConfiguration, queueProcessor);
_sharedQueueWatcher, _queueOptions, queueProcessor);
return new SharedBlobQueueListener(listener, triggerExecutor);
}

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

@ -24,10 +24,13 @@ namespace Microsoft.Azure.WebJobs.Host.Blobs.Triggers
{
private IStorageAccountProvider _accountProvider;
public BlobTriggerExtensionConfig(IStorageAccountProvider accountProvider)
{
_accountProvider = accountProvider;
}
public void Initialize(ExtensionConfigContext context)
{
_accountProvider = context.Config.GetService<IStorageAccountProvider>();
var rule = context.AddBindingRule<BlobTriggerAttribute>();
rule.BindToTrigger<IStorageBlob>();
@ -74,8 +77,8 @@ namespace Microsoft.Azure.WebJobs.Host.Blobs.Triggers
private readonly INameResolver _nameResolver;
private readonly IStorageAccountProvider _accountProvider;
private readonly IHostIdProvider _hostIdProvider;
private readonly IQueueConfiguration _queueConfiguration;
private readonly JobHostBlobsConfiguration _blobsConfiguration;
private readonly JobHostQueuesOptions _queueOptions;
private readonly JobHostBlobsOptions _blobsOptions;
private readonly IWebJobsExceptionHandler _exceptionHandler;
private readonly IContextSetter<IBlobWrittenWatcher> _blobWrittenWatcherSetter;
private readonly IContextSetter<IMessageEnqueuedWatcher> _messageEnqueuedWatcherSetter;
@ -87,8 +90,8 @@ namespace Microsoft.Azure.WebJobs.Host.Blobs.Triggers
IStorageAccountProvider accountProvider,
IExtensionTypeLocator extensionTypeLocator,
IHostIdProvider hostIdProvider,
IQueueConfiguration queueConfiguration,
JobHostBlobsConfiguration blobsConfiguration,
JobHostQueuesOptions queueOptions,
JobHostBlobsOptions blobsConfiguration,
IWebJobsExceptionHandler exceptionHandler,
IContextSetter<IBlobWrittenWatcher> blobWrittenWatcherSetter,
IContextSetter<IMessageEnqueuedWatcher> messageEnqueuedWatcherSetter,
@ -103,8 +106,8 @@ namespace Microsoft.Azure.WebJobs.Host.Blobs.Triggers
_accountProvider = accountProvider ?? throw new ArgumentNullException(nameof(accountProvider));
_hostIdProvider = hostIdProvider ?? throw new ArgumentNullException(nameof(hostIdProvider));
_queueConfiguration = queueConfiguration ?? throw new ArgumentNullException(nameof(queueConfiguration));
_blobsConfiguration = blobsConfiguration ?? throw new ArgumentNullException(nameof(blobsConfiguration));
_queueOptions = queueOptions ?? throw new ArgumentNullException(nameof(queueOptions));
_blobsOptions = blobsConfiguration ?? throw new ArgumentNullException(nameof(blobsConfiguration));
_exceptionHandler = exceptionHandler ?? throw new ArgumentNullException(nameof(exceptionHandler));
_blobWrittenWatcherSetter = blobWrittenWatcherSetter ?? throw new ArgumentNullException(nameof(blobWrittenWatcherSetter));
_messageEnqueuedWatcherSetter = messageEnqueuedWatcherSetter ?? throw new ArgumentNullException(nameof(messageEnqueuedWatcherSetter));
@ -134,7 +137,7 @@ namespace Microsoft.Azure.WebJobs.Host.Blobs.Triggers
dataAccount.AssertTypeOneOf(StorageAccountType.GeneralPurpose, StorageAccountType.BlobOnly);
ITriggerBinding binding = new BlobTriggerBinding(parameter, hostAccount, dataAccount, path,
_hostIdProvider, _queueConfiguration, _blobsConfiguration, _exceptionHandler, _blobWrittenWatcherSetter,
_hostIdProvider, _queueOptions, _blobsOptions, _exceptionHandler, _blobWrittenWatcherSetter,
_messageEnqueuedWatcherSetter, _sharedContextProvider, _singletonManager, _loggerFactory);
return binding;

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

@ -31,8 +31,8 @@ namespace Microsoft.Azure.WebJobs.Host.Blobs.Triggers
private readonly string _accountName;
private readonly IBlobPathSource _path;
private readonly IHostIdProvider _hostIdProvider;
private readonly IQueueConfiguration _queueConfiguration;
private readonly JobHostBlobsConfiguration _blobsConfiguration;
private readonly JobHostQueuesOptions _queueOptions;
private readonly JobHostBlobsOptions _blobsConfiguration;
private readonly IWebJobsExceptionHandler _exceptionHandler;
private readonly IContextSetter<IBlobWrittenWatcher> _blobWrittenWatcherSetter;
private readonly IContextSetter<IMessageEnqueuedWatcher> _messageEnqueuedWatcherSetter;
@ -47,8 +47,8 @@ namespace Microsoft.Azure.WebJobs.Host.Blobs.Triggers
IStorageAccount dataAccount,
IBlobPathSource path,
IHostIdProvider hostIdProvider,
IQueueConfiguration queueConfiguration,
JobHostBlobsConfiguration blobsConfiguration,
JobHostQueuesOptions queueOptions,
JobHostBlobsOptions blobsConfiguration,
IWebJobsExceptionHandler exceptionHandler,
IContextSetter<IBlobWrittenWatcher> blobWrittenWatcherSetter,
IContextSetter<IMessageEnqueuedWatcher> messageEnqueuedWatcherSetter,
@ -69,7 +69,7 @@ namespace Microsoft.Azure.WebJobs.Host.Blobs.Triggers
_accountName = BlobClient.GetAccountName(_blobClient);
_path = path ?? throw new ArgumentNullException(nameof(path));
_hostIdProvider = hostIdProvider ?? throw new ArgumentNullException(nameof(hostIdProvider));
_queueConfiguration = queueConfiguration ?? throw new ArgumentNullException(nameof(queueConfiguration));
_queueOptions = queueOptions ?? throw new ArgumentNullException(nameof(queueOptions));
_blobsConfiguration = blobsConfiguration ?? throw new ArgumentNullException(nameof(blobsConfiguration));
_exceptionHandler = exceptionHandler ?? throw new ArgumentNullException(nameof(exceptionHandler));
_blobWrittenWatcherSetter = blobWrittenWatcherSetter ?? throw new ArgumentNullException(nameof(blobWrittenWatcherSetter));
@ -183,7 +183,7 @@ namespace Microsoft.Azure.WebJobs.Host.Blobs.Triggers
IStorageBlobContainer container = _blobClient.GetContainerReference(_path.ContainerNamePattern);
var factory = new BlobListenerFactory(_hostIdProvider, _queueConfiguration, _blobsConfiguration, _exceptionHandler,
var factory = new BlobListenerFactory(_hostIdProvider, _queueOptions, _blobsConfiguration, _exceptionHandler,
_blobWrittenWatcherSetter, _messageEnqueuedWatcherSetter, _sharedContextProvider, _loggerFactory,
context.Descriptor.Id, _hostAccount, _dataAccount, container, _path, context.Executor, _singletonManager);

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

@ -14,29 +14,28 @@ namespace Microsoft.Azure.WebJobs.Host.Config
/// </summary>
public class ExtensionConfigContext : FluentConverterRules<Attribute, ExtensionConfigContext>
{
// List of actions to flush from the fluent configuration.
private List<Action> _updates = new List<Action>();
// Map of tyepof(TAttribute) --> FluentBindingRule<TAttribute>
private Dictionary<Type, object> _rules = new Dictionary<Type, object>();
private readonly IConverterManager _converterManager;
private readonly IWebHookProvider _webHookProvider;
private readonly IExtensionRegistry _extensionRegistry;
private readonly INameResolver _nameResolver;
public ExtensionConfigContext(INameResolver nameResolver, IConverterManager converterManager, IWebHookProvider webHookProvider, IExtensionRegistry extensionRegistry)
{
_converterManager = converterManager;
_webHookProvider = webHookProvider;
_extensionRegistry = extensionRegistry;
_nameResolver = nameResolver;
}
internal IExtensionConfigProvider Current { get; set; }
/// <summary>
/// Gets or sets the <see cref="JobHostConfiguration"/>.
/// </summary>
public JobHostConfiguration Config { get; set; }
internal override ConverterManager ConverterManager
{
get
{
return (ConverterManager) this.Config.ConverterManager;
}
}
internal ServiceProviderWrapper PerHostServices { get; set; }
internal override ConverterManager ConverterManager => (ConverterManager)_converterManager;
/// <summary>
/// Get a fully qualified URL that the host will resolve to this extension
@ -45,12 +44,11 @@ namespace Microsoft.Azure.WebJobs.Host.Config
[Obsolete("preview")]
public Uri GetWebhookHandler()
{
var webhook = this.Config.GetService<IWebHookProvider>();
if (webhook == null)
if (_webHookProvider == null)
{
return null;
}
return webhook.GetUrl(this.Current);
return _webHookProvider.GetUrl(this.Current);
}
// Ensure that multiple attempts bind to the same attribute, they get the same rule object.
@ -62,7 +60,7 @@ namespace Microsoft.Azure.WebJobs.Host.Config
if (!this._rules.TryGetValue(typeof(TAttribute), out temp))
{
// Create and register
rule = new FluentBindingRule<TAttribute>(this.Config);
rule = new FluentBindingRule<TAttribute>(_nameResolver, _converterManager, _extensionRegistry);
this._rules[typeof(TAttribute)] = rule;
_updates.Add(rule.ApplyRules);

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

@ -1,88 +0,0 @@
// Copyright (c) .NET Foundation. All rights reserved.
// Licensed under the MIT License. See License.txt in the project root for license information.
using System;
using Newtonsoft.Json;
using Newtonsoft.Json.Linq;
namespace Microsoft.Azure.WebJobs.Host.Config
{
/// <summary>
/// Helper Extension methods for extension configuration.
/// </summary>
[Obsolete("Not ready for public consumption.")]
public static class ExtensionConfigContextExtensions
{
/// <summary>
/// Get the configuration object for this extension.
/// </summary>
/// <param name="context"></param>
/// /// <param name="target"></param>
/// <param name="section"></param>
/// <returns></returns>
public static JObject GetConfig(this ExtensionConfigContext context, object target, string section = null)
{
if (context == null)
{
throw new ArgumentNullException(nameof(context));
}
JObject hostMetadata = context.Config.HostConfigMetadata;
if (hostMetadata == null)
{
return null;
}
if (section == null)
{
// By convention, an extension named "FooConfiguration" reads the "Foo" section.
if (target is IExtensionConfigProvider)
{
string name = target.GetType().Name;
var suffix = "Configuration";
if (name.EndsWith(suffix, StringComparison.Ordinal))
{
section = name.Substring(0, name.Length - suffix.Length);
}
}
}
if (section != null)
{
JToken value;
if (hostMetadata.TryGetValue(section, StringComparison.OrdinalIgnoreCase, out value))
{
hostMetadata = value as JObject;
}
}
return hostMetadata;
}
/// <summary>
/// Apply configuration for this extension.
/// </summary>
/// <param name="context"></param>
/// <param name="target">an existing object that the settings will get JSON serialized onto</param>
/// <param name="section">specifies a section from the overall config that should apply to this object.</param>
public static void ApplyConfig(this ExtensionConfigContext context, object target, string section = null)
{
if (target == null)
{
throw new ArgumentNullException(nameof(target));
}
if (context == null)
{
throw new ArgumentNullException(nameof(context));
}
JObject hostMetadata = context.GetConfig(target, section);
if (hostMetadata == null)
{
return;
}
JsonConvert.PopulateObject(hostMetadata.ToString(), target);
}
}
}

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

@ -22,8 +22,9 @@ namespace Microsoft.Azure.WebJobs.Host.Config
public class FluentBindingRule<TAttribute> : FluentConverterRules<TAttribute, FluentBindingRule<TAttribute>>
where TAttribute : Attribute
{
private readonly JobHostConfiguration _parent;
private readonly INameResolver _nameResolver;
private readonly IConverterManager _converterManager;
private readonly IExtensionRegistry _extensionRegistry;
private List<FluentBinder> _binders = new List<FluentBinder>();
// Filters to apply to current binder
@ -33,16 +34,18 @@ namespace Microsoft.Azure.WebJobs.Host.Config
private Action<TAttribute, Type> _validator;
internal FluentBindingRule(JobHostConfiguration parent)
internal FluentBindingRule(INameResolver nameResolver, IConverterManager converterManager, IExtensionRegistry extensionRegistry)
{
_parent = parent;
_nameResolver = nameResolver;
_converterManager = converterManager;
_extensionRegistry = extensionRegistry;
}
internal override ConverterManager ConverterManager
{
get
{
return (ConverterManager) _parent.ConverterManager;
return (ConverterManager)_converterManager;
}
}
@ -62,7 +65,7 @@ namespace Microsoft.Azure.WebJobs.Host.Config
{
_filterDescription.Add(filter);
}
/// <summary>
/// The subsequent Bind* operations only apply when the Attribute's property is null.
/// </summary>
@ -103,7 +106,7 @@ namespace Microsoft.Azure.WebJobs.Host.Config
{
throw new InvalidOperationException($"Rule filter for '{propertyName}' can only be used with enums.");
}
AppendFilter(FilterNode.IsEqual(prop, expectedEnumValue));
return this;
@ -165,7 +168,7 @@ namespace Microsoft.Azure.WebJobs.Host.Config
/// <typeparam name="TType"></typeparam>
/// <param name="builder"></param>
/// <returns></returns>
public FluentBinder BindToInput<TType>(Func<TAttribute, ValueBindingContext, Task<TType>> builder)
public FluentBinder BindToInput<TType>(Func<TAttribute, ValueBindingContext, Task<TType>> builder)
{
var pm = PatternMatcher.New(builder);
return BindToInput<TType>(pm);
@ -200,7 +203,7 @@ namespace Microsoft.Azure.WebJobs.Host.Config
// Common worker for BindToInput rules.
private FluentBinder BindToInput<TType>(PatternMatcher pm)
{
var rule = new BindToInputBindingProvider<TAttribute, TType>(_parent.NameResolver, _parent.ConverterManager, pm);
var rule = new BindToInputBindingProvider<TAttribute, TType>(_nameResolver, _converterManager, pm);
return Bind(rule);
}
@ -250,7 +253,7 @@ namespace Microsoft.Azure.WebJobs.Host.Config
private void BindToStream(PatternMatcher patternMatcher, FileAccess fileAccess)
{
// This will throw immediately if it can't match an ATtribute-->Stream converter.
var rule = new BindToStreamBindingProvider<TAttribute>(patternMatcher, fileAccess, _parent.NameResolver, _parent.ConverterManager);
var rule = new BindToStreamBindingProvider<TAttribute>(patternMatcher, fileAccess, _nameResolver, _converterManager);
Bind(rule);
}
@ -263,7 +266,7 @@ namespace Microsoft.Azure.WebJobs.Host.Config
/// </summary>
/// <param name="builder">Builder function to create a IValueBinder given a resolved attribute and the user parameter type. </param>
/// <returns>A binding provider that applies these semantics.</returns>
public FluentBinder BindToValueProvider(Func<TAttribute, Type, Task<IValueBinder>> builder)
public FluentBinder BindToValueProvider(Func<TAttribute, Type, Task<IValueBinder>> builder)
{
return BindToValueProvider<object>(builder);
}
@ -278,9 +281,9 @@ namespace Microsoft.Azure.WebJobs.Host.Config
public FluentBinder BindToValueProvider<TType>(Func<TAttribute, Type, Task<IValueBinder>> builder)
{
var ot = OpenType.FromType<TType>();
var nameResolver = this._parent.NameResolver;
var nameResolver = _nameResolver;
var binder = new ItemBindingProvider<TAttribute>(nameResolver, builder, ot);
return Bind(binder);
return Bind(binder);
}
@ -302,14 +305,14 @@ namespace Microsoft.Azure.WebJobs.Host.Config
if (this._filterDescription.Count > 0)
{
binder = new FilteringBindingProvider<TAttribute>(
this._parent.NameResolver,
binder,
this._nameResolver,
binder,
FilterNode.And(this._filterDescription));
this._filterDescription.Clear();
}
var opts = new FluentBinder(_parent, binder);
var opts = new FluentBinder(this._nameResolver, binder);
_binders.Add(opts);
return opts;
}
@ -371,7 +374,7 @@ namespace Microsoft.Azure.WebJobs.Host.Config
private void BindToCollector<TMessage>(PatternMatcher pm)
{
var rule = new AsyncCollectorBindingProvider<TAttribute, TMessage>(_parent.NameResolver, _parent.ConverterManager, pm);
var rule = new AsyncCollectorBindingProvider<TAttribute, TMessage>(_nameResolver, _converterManager, pm);
Bind(rule);
}
@ -387,8 +390,8 @@ namespace Microsoft.Azure.WebJobs.Host.Config
{
throw new InvalidOperationException($"The same attribute can't be bound to trigger and non-trigger bindings");
}
IExtensionRegistry extensions = _parent.GetService<IExtensionRegistry>();
extensions.RegisterExtension<ITriggerBindingProvider>(trigger);
_extensionRegistry.RegisterExtension<ITriggerBindingProvider>(trigger);
}
public void BindToTrigger<TTriggerValue>(ITriggerBindingProvider trigger = null)
@ -400,8 +403,8 @@ namespace Microsoft.Azure.WebJobs.Host.Config
}
var triggerBinder = new TriggerAdapterBindingProvider<TAttribute, TTriggerValue>(
this._parent.NameResolver,
this._parent.ConverterManager);
this._nameResolver,
_converterManager);
Bind(triggerBinder);
}
@ -431,7 +434,7 @@ namespace Microsoft.Azure.WebJobs.Host.Config
private IBindingProvider CreateBinding()
{
IBindingProvider[] bindings = _binders.Select(x => x.Binder).ToArray();
var all = new GenericCompositeBindingProvider<TAttribute>(_validator, this._parent.NameResolver, bindings);
var all = new GenericCompositeBindingProvider<TAttribute>(_validator, this._nameResolver, bindings);
return all;
}
@ -450,9 +453,8 @@ namespace Microsoft.Azure.WebJobs.Host.Config
if (_binders.Count > 0)
{
IExtensionRegistry extensions = _parent.GetService<IExtensionRegistry>();
var binding = CreateBinding();
extensions.RegisterExtension<IBindingProvider>(binding);
_extensionRegistry.RegisterExtension<IBindingProvider>(binding);
_binders.Clear();
}
}
@ -461,11 +463,11 @@ namespace Microsoft.Azure.WebJobs.Host.Config
// These apply after the binder has matched.
public class FluentBinder
{
private readonly JobHostConfiguration _parent;
private readonly INameResolver _nameResolver;
internal FluentBinder(JobHostConfiguration parent, IBindingProvider binder)
internal FluentBinder(INameResolver nameResolver, IBindingProvider binder)
{
this._parent = parent;
this._nameResolver = nameResolver;
this.Binder = binder;
}
@ -477,7 +479,7 @@ namespace Microsoft.Azure.WebJobs.Host.Config
public void AddValidator(Action<TAttribute, Type> validator)
{
var inner = this.Binder;
var binder = new ValidatingWrapperBindingProvider<TAttribute>(validator, _parent.NameResolver, inner);
var binder = new ValidatingWrapperBindingProvider<TAttribute>(validator, _nameResolver, inner);
this.Binder = binder;
}

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

@ -6,12 +6,12 @@ namespace Microsoft.Azure.WebJobs.Host
/// <summary>
/// Represents configuration for <see cref="BlobTriggerAttribute"/>.
/// </summary>
public class JobHostBlobsConfiguration
public class JobHostBlobsOptions
{
/// <summary>
/// Constructs a new instance.
/// </summary>
public JobHostBlobsConfiguration()
public JobHostBlobsOptions()
{
CentralizedPoisonQueue = false;
}

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

@ -10,7 +10,7 @@ namespace Microsoft.Azure.WebJobs.Host
/// <summary>
/// Configuration options for controlling function execution timeout behavior.
/// </summary>
public class JobHostFunctionTimeoutConfiguration
public class JobHostFunctionTimeoutOptions
{
/// <summary>
/// Gets the timeout value.

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

@ -10,7 +10,7 @@ namespace Microsoft.Azure.WebJobs
/// For example, this can support <see cref="SingletonAttribute"/>, blob leases, timers, etc.
/// This provides a common place to set storage that the various subsequent services can use.
/// </summary>
public class JobHostInternalStorageConfiguration
public class JobHostInternalStorageOptions
{
/// <summary>
/// A SAS to a Blob Container. This allows services to create blob leases and do distributed locking.

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

@ -0,0 +1,32 @@
// Copyright (c) .NET Foundation. All rights reserved.
// Licensed under the MIT License. See License.txt in the project root for license information.
using System;
using Microsoft.Azure.WebJobs.Host.Executors;
using Microsoft.Extensions.Options;
using Microsoft.WindowsAzure.Storage.Blob;
namespace Microsoft.Azure.WebJobs.Host.Configuration
{
internal class JobHostInternalStorageOptionsSetup : IConfigureOptions<JobHostInternalStorageOptions>
{
private IStorageAccountProvider _storageAccountProvider;
public JobHostInternalStorageOptionsSetup(IStorageAccountProvider storageAccountProvider)
{
_storageAccountProvider = storageAccountProvider;
}
public void Configure(JobHostInternalStorageOptions options)
{
var sasBlobContainer = _storageAccountProvider.InternalSasStorage;
if (sasBlobContainer != null)
{
var uri = new Uri(sasBlobContainer);
var sdkContainer = new CloudBlobContainer(uri);
options.InternalContainer = sdkContainer;
}
}
}
}

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

@ -0,0 +1,70 @@
// Copyright (c) .NET Foundation. All rights reserved.
// Licensed under the MIT License. See License.txt in the project root for license information.
using System;
using System.Collections.Concurrent;
using System.Diagnostics;
using System.Reflection;
using Microsoft.Azure.WebJobs.Host;
using Microsoft.Azure.WebJobs.Host.Bindings;
using Microsoft.Azure.WebJobs.Host.Config;
using Microsoft.Azure.WebJobs.Host.Executors;
using Microsoft.Azure.WebJobs.Host.Indexers;
using Microsoft.Azure.WebJobs.Host.Loggers;
using Microsoft.Azure.WebJobs.Host.Queues;
using Microsoft.Azure.WebJobs.Host.Timers;
using Microsoft.Azure.WebJobs.Logging;
using Microsoft.Extensions.Logging;
using Newtonsoft.Json.Linq;
using Microsoft.Extensions.Configuration;
namespace Microsoft.Azure.WebJobs
{
/// <summary>
/// Represents the configuration settings for a <see cref="JobHost"/>.
/// </summary>
public sealed class JobHostOptions
{
private string _hostId;
/// <summary>
/// Returns true if <see cref="UseDevelopmentSettings"/> has been called on this instance.
/// </summary>
internal bool UsingDevelopmentSettings { get; set; }
/// <summary>Gets or sets the host ID.</summary>
/// <remarks>
/// <para>
/// All host instances that share the same host ID must be homogeneous. For example, they must use the same
/// storage accounts and have the same list of functions. Host instances with the same host ID will scale out
/// and share handling of work such as BlobTrigger and run from dashboard processing and providing a heartbeat
/// to the dashboard indicating that an instance of the host running.
/// </para>
/// <para>
/// If this value is <see langword="null"/> on startup, a host ID will automatically be generated based on the assembly
/// name of the first function, and that host ID will be made available via this property after the host has fully started.
/// </para>
/// <para>
/// If non-homogeneous host instances share the same first function assembly,
/// this property must be set explicitly; otherwise, the host instances will incorrectly try to share work as if
/// they were homogeneous.
/// </para>
/// </remarks>
public string HostId
{
get
{
return _hostId;
}
set
{
if (value != null && !HostIdValidator.IsValid(value))
{
throw new ArgumentException(HostIdValidator.ValidationMessage, "value");
}
_hostId = value;
}
}
}
}

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

@ -12,7 +12,7 @@ namespace Microsoft.Azure.WebJobs.Host
/// <summary>
/// Represents configuration for <see cref="QueueTriggerAttribute"/>.
/// </summary>
public sealed class JobHostQueuesConfiguration : IQueueConfiguration
public class JobHostQueuesOptions
{
private const int DefaultMaxDequeueCount = 5;
private const int DefaultBatchSize = 16;
@ -28,9 +28,9 @@ namespace Microsoft.Azure.WebJobs.Host
private int _maxDequeueCount = DefaultMaxDequeueCount;
/// <summary>
/// Initializes a new instance of the <see cref="JobHostQueuesConfiguration"/> class.
/// Initializes a new instance of the <see cref="JobHostQueuesOptions"/> class.
/// </summary>
internal JobHostQueuesConfiguration()
public JobHostQueuesOptions()
{
_newBatchThreshold = -1;
QueueProcessorFactory = new DefaultQueueProcessorFactory();

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

@ -8,7 +8,7 @@ using System.Linq;
namespace Microsoft.Azure.WebJobs.Host
{
internal class DefaultExtensionRegistry : IExtensionRegistry
public class DefaultExtensionRegistry : IExtensionRegistry
{
private ConcurrentDictionary<Type, ConcurrentBag<object>> _registry = new ConcurrentDictionary<Type, ConcurrentBag<object>>();

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

@ -0,0 +1,43 @@
// Copyright (c) .NET Foundation. All rights reserved.
// Licensed under the MIT License. See License.txt in the project root for license information.
using System.Collections.Generic;
using Microsoft.Azure.WebJobs.Host.Config;
namespace Microsoft.Azure.WebJobs.Host
{
public class DefaultExtensionRegistryFactory : IExtensionRegistryFactory
{
private readonly IEnumerable<IExtensionConfigProvider> _registeredExtensions;
private readonly IConverterManager _converterManager;
private readonly IWebHookProvider _webHookProvider;
private readonly INameResolver _nameResolver;
public DefaultExtensionRegistryFactory(IEnumerable<IExtensionConfigProvider> registeredExtensions, IConverterManager converterManager,
INameResolver nameResolver, IWebHookProvider webHookProvider = null)
{
_registeredExtensions = registeredExtensions;
_converterManager = converterManager;
_webHookProvider = webHookProvider;
_nameResolver = nameResolver;
}
public IExtensionRegistry Create()
{
IExtensionRegistry registry = new DefaultExtensionRegistry();
ExtensionConfigContext context = new ExtensionConfigContext(_nameResolver, _converterManager, _webHookProvider, registry);
foreach (IExtensionConfigProvider extension in _registeredExtensions)
{
registry.RegisterExtension<IExtensionConfigProvider>(extension);
context.Current = extension;
extension.Initialize(context);
}
context.ApplyRules();
return registry;
}
}
}

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

@ -15,6 +15,7 @@ using Microsoft.Azure.WebJobs.Host.Storage.Queue;
using Microsoft.Azure.WebJobs.Host.Timers;
using Microsoft.Azure.WebJobs.Logging;
using Microsoft.Extensions.Logging;
using Microsoft.Extensions.Options;
using Newtonsoft.Json;
using Newtonsoft.Json.Linq;
@ -28,7 +29,7 @@ namespace Microsoft.Azure.WebJobs.Host.Dispatch
private readonly IHostIdProvider _hostIdProvider;
private readonly IWebJobsExceptionHandler _exceptionHandler;
private readonly ILoggerFactory _loggerFactory;
private readonly IQueueConfiguration _queueConfiguration;
private readonly IOptions<JobHostQueuesOptions> _queueOptions;
private readonly ISharedContextProvider _sharedContextProvider;
private readonly IContextSetter<IMessageEnqueuedWatcher> _messageEnqueuedWatcherSetter;
@ -38,11 +39,11 @@ namespace Microsoft.Azure.WebJobs.Host.Dispatch
private QueueListener _sharedQueuelistener;
private SharedQueueWriter _sharedQueueWriter;
internal SharedQueueHandler(IStorageAccountProvider accountProvider,
public SharedQueueHandler(IStorageAccountProvider accountProvider,
IHostIdProvider hostIdProvider,
IWebJobsExceptionHandler exceptionHandler,
ILoggerFactory loggerFactory,
IQueueConfiguration queueConfiguration,
IOptions<JobHostQueuesOptions> queueOptions,
ISharedContextProvider sharedContextProvider,
IContextSetter<IMessageEnqueuedWatcher> messageEnqueuedWatcherSetter)
{
@ -50,7 +51,7 @@ namespace Microsoft.Azure.WebJobs.Host.Dispatch
_hostIdProvider = hostIdProvider;
_exceptionHandler = exceptionHandler;
_loggerFactory = loggerFactory;
_queueConfiguration = queueConfiguration;
_queueOptions = queueOptions;
_sharedContextProvider = sharedContextProvider;
_messageEnqueuedWatcherSetter = messageEnqueuedWatcherSetter;
_state = State.Created;
@ -118,7 +119,7 @@ namespace Microsoft.Azure.WebJobs.Host.Dispatch
// use default poisonQueue setup
_sharedQueuelistener = new QueueListener(sharedQueue, sharedPoisonQueue, _triggerExecutor,
_exceptionHandler, _loggerFactory, sharedQueueWatcher, _queueConfiguration);
_exceptionHandler, _loggerFactory, sharedQueueWatcher, _queueOptions.Value);
}
catch (Exception ex)
{

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

@ -7,17 +7,6 @@ namespace Microsoft.Azure.WebJobs.Host.Executors
{
internal class DefaultJobActivator : IJobActivator
{
private static readonly DefaultJobActivator Singleton = new DefaultJobActivator();
private DefaultJobActivator()
{
}
public static DefaultJobActivator Instance
{
get { return Singleton; }
}
public T CreateInstance<T>()
{
return Activator.CreateInstance<T>();

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

@ -3,7 +3,6 @@
using System;
using System.Collections.Concurrent;
using System.Globalization;
using System.Threading;
using System.Threading.Tasks;
using Microsoft.Azure.WebJobs.Host.Storage;
@ -15,65 +14,19 @@ namespace Microsoft.Azure.WebJobs.Host.Executors
private readonly IConnectionStringProvider _ambientConnectionStringProvider;
private readonly IStorageCredentialsValidator _storageCredentialsValidator;
private readonly IStorageAccountParser _storageAccountParser;
private readonly IServiceProvider _services;
private readonly ConcurrentDictionary<string, Task<IStorageAccount>> _accounts = new ConcurrentDictionary<string, Task<IStorageAccount>>();
private readonly ConcurrentDictionary<string, IStorageAccount> _accounts = new ConcurrentDictionary<string, IStorageAccount>();
private IStorageAccount _dashboardAccount;
private bool _dashboardAccountSet;
private IStorageAccount _storageAccount;
private bool _storageAccountSet;
/// <summary>
/// Constructs a new instance.
/// </summary>
/// <param name="services">The <see cref="IServiceProvider"/> to use.</param>
public DefaultStorageAccountProvider(IServiceProvider services)
: this(services, AmbientConnectionStringProvider.Instance, new StorageAccountParser(), new DefaultStorageCredentialsValidator())
{
}
/// <summary>
/// Initializes a new instance of the class, using a single Microsoft Azure
/// Storage connection string for both reading and writing data as well as logging.
/// </summary>
/// <param name="services">The <see cref="IServiceProvider"/> to use.</param>
/// <param name="dashboardAndStorageConnectionString">
/// The Azure Storage connection string for accessing data and logging.
/// </param>
public DefaultStorageAccountProvider(IServiceProvider services, string dashboardAndStorageConnectionString)
: this(services)
{
StorageConnectionString = dashboardAndStorageConnectionString;
DashboardAccount = StorageAccount;
}
internal DefaultStorageAccountProvider(IServiceProvider services, IConnectionStringProvider ambientConnectionStringProvider,
public DefaultStorageAccountProvider(IConnectionStringProvider ambientConnectionStringProvider,
IStorageAccountParser storageAccountParser, IStorageCredentialsValidator storageCredentialsValidator)
{
if (services == null)
{
throw new ArgumentNullException("services");
}
if (ambientConnectionStringProvider == null)
{
throw new ArgumentNullException("ambientConnectionStringProvider");
}
if (storageAccountParser == null)
{
throw new ArgumentNullException("storageAccountParser");
}
if (storageCredentialsValidator == null)
{
throw new ArgumentNullException("storageCredentialsValidator");
}
_services = services;
_ambientConnectionStringProvider = ambientConnectionStringProvider;
_storageCredentialsValidator = storageCredentialsValidator;
_storageAccountParser = storageAccountParser;
_ambientConnectionStringProvider = ambientConnectionStringProvider ?? throw new ArgumentNullException(nameof(ambientConnectionStringProvider));
_storageCredentialsValidator = storageCredentialsValidator ?? throw new ArgumentNullException(nameof(storageCredentialsValidator));
_storageAccountParser = storageAccountParser ?? throw new ArgumentNullException(nameof(storageAccountParser));
}
/// <summary>Gets or sets the Azure Storage connection string used for logging and diagnostics.</summary>
@ -199,8 +152,17 @@ namespace Microsoft.Azure.WebJobs.Host.Executors
return account;
}
public Task<IStorageAccount> TryGetAccountAsync(string connectionStringName, CancellationToken cancellationToken) =>
_accounts.GetOrAdd(connectionStringName, s => CreateAndValidateAccountAsync(s, cancellationToken));
public async Task<IStorageAccount> TryGetAccountAsync(string connectionStringName, CancellationToken cancellationToken)
{
IStorageAccount account;
if (!_accounts.TryGetValue(connectionStringName, out account))
{
// in rare cases createAndValidateAccountAsync could be called multiple times for the same account
account = await CreateAndValidateAccountAsync(connectionStringName, cancellationToken);
_accounts.AddOrUpdate(connectionStringName, (cs) => account, (cs, a) => account);
}
return account;
}
private IStorageAccount ParseAccount(string connectionStringName)
{
@ -210,7 +172,7 @@ namespace Microsoft.Azure.WebJobs.Host.Executors
private IStorageAccount ParseAccount(string connectionStringName, string connectionString)
{
return _storageAccountParser.ParseAccount(connectionString, connectionStringName, _services);
return _storageAccountParser.ParseAccount(connectionString, connectionStringName);
}
}
}

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

@ -19,10 +19,9 @@ namespace Microsoft.Azure.WebJobs.Host.Executors
internal class DynamicHostIdProvider : IHostIdProvider
{
private readonly IStorageAccountProvider _storageAccountProvider;
private readonly Func<IFunctionIndexProvider> _getFunctionIndexProvider;
private readonly IFunctionIndexProvider _getFunctionIndexProvider;
public DynamicHostIdProvider(IStorageAccountProvider storageAccountProvider,
Func<IFunctionIndexProvider> getFunctionIndexProvider)
public DynamicHostIdProvider(IStorageAccountProvider storageAccountProvider, IFunctionIndexProvider getFunctionIndexProvider)
{
if (storageAccountProvider == null)
{
@ -53,7 +52,7 @@ namespace Microsoft.Azure.WebJobs.Host.Executors
"connection string.", exception);
}
IFunctionIndex index = await _getFunctionIndexProvider.Invoke().GetAsync(cancellationToken);
IFunctionIndex index = await _getFunctionIndexProvider.GetAsync(cancellationToken);
IEnumerable<MethodInfo> indexedMethods = index.ReadAllMethods();
string sharedHostName = GetSharedHostName(indexedMethods, account);

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

@ -1,6 +1,8 @@
// Copyright (c) .NET Foundation. All rights reserved.
// Licensed under the MIT License. See License.txt in the project root for license information.
using Microsoft.Extensions.Options;
using System;
using System.Threading;
using System.Threading.Tasks;
@ -10,9 +12,19 @@ namespace Microsoft.Azure.WebJobs.Host.Executors
{
private readonly string _hostId;
public FixedHostIdProvider(IOptions<JobHostOptions> options)
: this(options.Value?.HostId)
{
}
public FixedHostIdProvider(string hostId)
{
_hostId = hostId;
if (_hostId == null)
{
_hostId = Guid.NewGuid().ToString("N");
}
}
public Task<string> GetHostIdAsync(CancellationToken cancellationToken)

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

@ -24,30 +24,41 @@ namespace Microsoft.Azure.WebJobs.Host.Executors
internal class FunctionExecutor : IFunctionExecutor
{
private readonly IFunctionInstanceLogger _functionInstanceLogger;
private readonly IFunctionOutputLogger _functionOutputLogger;
private readonly IFunctionOutputLoggerProvider _functionOutputLoggerProvider;
private readonly IWebJobsExceptionHandler _exceptionHandler;
private readonly IAsyncCollector<FunctionInstanceLogEntry> _functionEventCollector;
private readonly ILoggerFactory _loggerFactory;
private readonly ILogger _resultsLogger;
private readonly IEnumerable<IFunctionFilter> _globalFunctionFilters;
private IFunctionOutputLogger _functionOutputLogger;
private HostOutputMessage _hostOutputMessage;
public FunctionExecutor(IFunctionInstanceLogger functionInstanceLogger, IFunctionOutputLogger functionOutputLogger,
public FunctionExecutor(IFunctionInstanceLogger functionInstanceLogger, IFunctionOutputLoggerProvider functionOutputLoggerProvider,
IWebJobsExceptionHandler exceptionHandler,
IAsyncCollector<FunctionInstanceLogEntry> functionEventCollector = null,
IAsyncCollector<FunctionInstanceLogEntry> functionEventCollector,
ILoggerFactory loggerFactory = null,
IEnumerable<IFunctionFilter> globalFunctionFilters = null)
{
_functionInstanceLogger = functionInstanceLogger ?? throw new ArgumentNullException(nameof(functionInstanceLogger));
_functionOutputLogger = functionOutputLogger ?? throw new ArgumentNullException(nameof(functionOutputLogger));
_functionOutputLoggerProvider = functionOutputLoggerProvider;
_exceptionHandler = exceptionHandler ?? throw new ArgumentNullException(nameof(exceptionHandler));
_functionEventCollector = functionEventCollector;
_functionEventCollector = functionEventCollector ?? throw new ArgumentNullException(nameof(functionEventCollector));
_loggerFactory = loggerFactory;
_resultsLogger = _loggerFactory?.CreateLogger(LogCategories.Results);
_globalFunctionFilters = globalFunctionFilters ?? Enumerable.Empty<IFunctionFilter>();
}
private async Task<IFunctionOutputLogger> GetFunctionOutputLogger(CancellationToken cancellationToken)
{
if (_functionOutputLogger == null)
{
_functionOutputLogger = await _functionOutputLoggerProvider.GetAsync(cancellationToken);
}
return _functionOutputLogger;
}
public HostOutputMessage HostOutputMessage
{
get { return _hostOutputMessage; }
@ -188,7 +199,8 @@ namespace Microsoft.Azure.WebJobs.Host.Executors
ITaskSeriesTimer updateOutputLogTimer = null;
TextWriter functionOutputTextWriter = null;
outputDefinition = await _functionOutputLogger.CreateAsync(instance, cancellationToken);
IFunctionOutputLogger outputLogger = await GetFunctionOutputLogger(cancellationToken);
outputDefinition = await outputLogger.CreateAsync(instance, cancellationToken);
outputLog = outputDefinition.CreateOutput();
functionOutputTextWriter = outputLog.Output;
updateOutputLogTimer = StartOutputTimer(outputLog.UpdateCommand, _exceptionHandler);
@ -220,11 +232,8 @@ namespace Microsoft.Azure.WebJobs.Host.Executors
{
startedMessageId = await LogFunctionStartedAsync(message, outputDefinition, parameterHelper, cancellationToken);
if (_functionEventCollector != null)
{
// Log started
await NotifyPostBindAsync(instanceLogEntry, message.Arguments);
}
// Log started
await NotifyPostBindAsync(instanceLogEntry, message.Arguments);
try
{
@ -678,11 +687,9 @@ namespace Microsoft.Azure.WebJobs.Host.Executors
};
Debug.Assert(fastItem.IsStart);
if (_functionEventCollector != null)
{
// Log pre-bind event.
await _functionEventCollector.AddAsync(fastItem);
}
// Log pre-bind event.
await _functionEventCollector.AddAsync(fastItem);
return fastItem;
}
@ -693,10 +700,6 @@ namespace Microsoft.Azure.WebJobs.Host.Executors
fastItem.Arguments = arguments;
Debug.Assert(fastItem.IsPostBind);
if (_functionEventCollector == null)
{
return Task.CompletedTask;
}
return _functionEventCollector.AddAsync(fastItem);
}
@ -725,10 +728,6 @@ namespace Microsoft.Azure.WebJobs.Host.Executors
intanceLogEntry.ErrorDetails = ex.Message;
}
if (_functionEventCollector == null)
{
return Task.CompletedTask;
}
return _functionEventCollector.AddAsync(intanceLogEntry);
}
@ -873,7 +872,7 @@ namespace Microsoft.Azure.WebJobs.Host.Executors
{
foreach (KeyValuePair<string, IValueProvider> parameter in _parameters)
{
arguments.Add(parameter.Key, parameter.Value.ToInvokeString());
arguments.Add(parameter.Key, parameter.Value?.ToInvokeString() ?? "null");
}
}

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

@ -8,7 +8,7 @@ using Microsoft.Azure.WebJobs.Host.Protocols;
namespace Microsoft.Azure.WebJobs.Host.Executors
{
internal class FunctionInstanceFactoryContext
public class FunctionInstanceFactoryContext
{
public Guid Id { get; set; }
public Guid? ParentId { get; set; }

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

@ -7,7 +7,7 @@ using Microsoft.Azure.WebJobs.Host.Bindings;
namespace Microsoft.Azure.WebJobs.Host.Executors
{
internal interface IBindingSource
public interface IBindingSource
{
Task<IReadOnlyDictionary<string, IValueProvider>> BindAsync(ValueBindingContext context);
}

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

@ -5,7 +5,7 @@ using System;
namespace Microsoft.Azure.WebJobs.Host.Executors
{
internal interface IDelayedException
public interface IDelayedException
{
Exception Exception { get; }
void Throw();

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

@ -6,7 +6,7 @@ using System.Threading.Tasks;
namespace Microsoft.Azure.WebJobs.Host.Executors
{
internal interface IFunctionExecutor
public interface IFunctionExecutor
{
Task<IDelayedException> TryExecuteAsync(IFunctionInstance instance, CancellationToken cancellationToken);
}

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

@ -6,7 +6,7 @@ using Microsoft.Azure.WebJobs.Host.Protocols;
namespace Microsoft.Azure.WebJobs.Host.Executors
{
internal interface IFunctionInstance
public interface IFunctionInstance
{
Guid Id { get; }

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

@ -7,7 +7,7 @@ using Microsoft.Azure.WebJobs.Host.Protocols;
namespace Microsoft.Azure.WebJobs.Host.Executors
{
internal interface IFunctionInstanceFactory
public interface IFunctionInstanceFactory
{
IFunctionInstance Create(FunctionInstanceFactoryContext context);
}

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

@ -6,7 +6,7 @@ using System.Threading.Tasks;
namespace Microsoft.Azure.WebJobs.Host.Executors
{
internal interface IFunctionInvoker
public interface IFunctionInvoker
{
IReadOnlyList<string> ParameterNames { get; }

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

@ -0,0 +1,16 @@
// Copyright (c) .NET Foundation. All rights reserved.
// Licensed under the MIT License. See License.txt in the project root for license information.
using System;
using System.Collections.Generic;
using System.Text;
using System.Threading;
using System.Threading.Tasks;
namespace Microsoft.Azure.WebJobs.Host.Executors
{
public interface IJobHostContextFactory
{
Task<JobHostContext> Create(CancellationToken shutdownToken, CancellationToken cancellationToken);
}
}

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

@ -1,13 +1,12 @@
// Copyright (c) .NET Foundation. All rights reserved.
// Licensed under the MIT License. See License.txt in the project root for license information.
using System;
using Microsoft.Azure.WebJobs.Host.Storage;
namespace Microsoft.Azure.WebJobs.Host.Executors
{
internal interface IStorageAccountParser
public interface IStorageAccountParser
{
IStorageAccount ParseAccount(string connectionString, string connectionStringName, IServiceProvider services);
IStorageAccount ParseAccount(string connectionString, string connectionStringName);
}
}

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

@ -7,8 +7,14 @@ using Microsoft.Azure.WebJobs.Host.Storage;
namespace Microsoft.Azure.WebJobs.Host.Executors
{
internal interface IStorageAccountProvider
public interface IStorageAccountProvider
{
Task<IStorageAccount> TryGetAccountAsync(string connectionStringName, CancellationToken cancellationToken);
string StorageConnectionString { get; }
string DashboardConnectionString { get; }
string InternalSasStorage { get; }
}
}

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

@ -1,574 +0,0 @@
// Copyright (c) .NET Foundation. All rights reserved.
// Licensed under the MIT License. See License.txt in the project root for license information.
using System;
using System.Collections.Generic;
using System.Diagnostics.CodeAnalysis;
using System.IO;
using System.Linq;
using System.Reflection;
using System.Text;
using System.Threading;
using System.Threading.Tasks;
using Microsoft.Azure.WebJobs.Host.Bindings;
using Microsoft.Azure.WebJobs.Host.Blobs;
using Microsoft.Azure.WebJobs.Host.Config;
using Microsoft.Azure.WebJobs.Host.Dispatch;
using Microsoft.Azure.WebJobs.Host.Indexers;
using Microsoft.Azure.WebJobs.Host.Listeners;
using Microsoft.Azure.WebJobs.Host.Loggers;
using Microsoft.Azure.WebJobs.Host.Protocols;
using Microsoft.Azure.WebJobs.Host.Queues;
using Microsoft.Azure.WebJobs.Host.Queues.Bindings;
using Microsoft.Azure.WebJobs.Host.Queues.Listeners;
using Microsoft.Azure.WebJobs.Host.Storage;
using Microsoft.Azure.WebJobs.Host.Storage.Blob;
using Microsoft.Azure.WebJobs.Host.Storage.Queue;
using Microsoft.Azure.WebJobs.Host.Tables;
using Microsoft.Azure.WebJobs.Host.Timers;
using Microsoft.Azure.WebJobs.Host.Triggers;
using Microsoft.Azure.WebJobs.Logging;
using Microsoft.Extensions.Logging;
using Microsoft.Azure.WebJobs.Host.Blobs.Triggers;
namespace Microsoft.Azure.WebJobs.Host.Executors
{
internal static class JobHostConfigurationExtensions
{
// Static initialization. Returns a service provider with some new services initialized.
// The new services:
// - can retrieve static config like binders and converters; but the listeners haven't yet started.
// - can be flowed into the runtime initialization to get a JobHost spun up and running.
// This is just static initialization and should not need to make any network calls,
// and so this method should not need to be async.
// This can be called multiple times on a config, which is why it returns a new ServiceProviderWrapper
// instead of modifying the config.
public static ServiceProviderWrapper CreateStaticServices(this JobHostConfiguration config)
{
var services = new ServiceProviderWrapper(config);
var nameResolver = services.GetService<INameResolver>();
IWebJobsExceptionHandler exceptionHandler = services.GetService<IWebJobsExceptionHandler>();
IQueueConfiguration queueConfiguration = services.GetService<IQueueConfiguration>();
var blobsConfiguration = config.Blobs;
IStorageAccountProvider storageAccountProvider = services.GetService<IStorageAccountProvider>();
IBindingProvider bindingProvider = services.GetService<IBindingProvider>();
SingletonManager singletonManager = services.GetService<SingletonManager>();
IHostIdProvider hostIdProvider = services.GetService<IHostIdProvider>();
var hostId = config.HostId;
if (hostId != null)
{
hostIdProvider = new FixedHostIdProvider(hostId);
}
// Need a deferred getter since the IFunctionIndexProvider service isn't created until later.
Func<IFunctionIndexProvider> deferredGetter = () => services.GetService<IFunctionIndexProvider>();
if (hostIdProvider == null)
{
hostIdProvider = new DynamicHostIdProvider(storageAccountProvider, deferredGetter);
}
services.AddService<IHostIdProvider>(hostIdProvider);
AzureStorageDeploymentValidator.Validate();
IExtensionTypeLocator extensionTypeLocator = services.GetService<IExtensionTypeLocator>();
if (extensionTypeLocator == null)
{
extensionTypeLocator = new ExtensionTypeLocator(services.GetService<ITypeLocator>());
services.AddService<IExtensionTypeLocator>(extensionTypeLocator);
}
ContextAccessor<IMessageEnqueuedWatcher> messageEnqueuedWatcherAccessor = new ContextAccessor<IMessageEnqueuedWatcher>();
services.AddService(messageEnqueuedWatcherAccessor);
ContextAccessor<IBlobWrittenWatcher> blobWrittenWatcherAccessor = new ContextAccessor<IBlobWrittenWatcher>();
services.AddService(blobWrittenWatcherAccessor);
ISharedContextProvider sharedContextProvider = new SharedContextProvider();
// Add built-in extensions
var metadataProvider = new JobHostMetadataProvider(deferredGetter);
metadataProvider.AddAttributesFromAssembly(typeof(TableAttribute).Assembly);
var converterManager = (ConverterManager)config.ConverterManager;
var exts = config.GetExtensions();
bool builtinsAdded = exts.GetExtensions<IExtensionConfigProvider>().OfType<TableExtension>().Any();
if (!builtinsAdded)
{
AddStreamConverters(extensionTypeLocator, converterManager);
config.AddExtension(new TableExtension());
config.AddExtension(new QueueExtension());
config.AddExtension(new Blobs.Bindings.BlobExtensionConfig());
config.AddExtension(new BlobTriggerExtensionConfig());
}
ExtensionConfigContext context = new ExtensionConfigContext
{
Config = config,
PerHostServices = services
};
InvokeExtensionConfigProviders(context);
// After this point, all user configuration has been set.
if (singletonManager == null)
{
var logger = config.LoggerFactory?.CreateLogger(LogCategories.Singleton);
IDistributedLockManager lockManager = services.GetService<IDistributedLockManager>();
if (lockManager == null)
{
var sas = config.InternalStorageConfiguration;
if (sas != null && sas.InternalContainer != null)
{
lockManager = new BlobLeaseDistributedLockManager.SasContainer(
sas.InternalContainer,
logger);
}
else
{
lockManager = new BlobLeaseDistributedLockManager.DedicatedStorage(
storageAccountProvider,
logger);
}
services.AddService<IDistributedLockManager>(lockManager);
}
singletonManager = new SingletonManager(
lockManager,
config.Singleton,
exceptionHandler,
config.LoggerFactory,
hostIdProvider,
services.GetService<INameResolver>());
services.AddService<SingletonManager>(singletonManager);
}
IExtensionRegistry extensions = services.GetExtensions();
services.AddService<SharedQueueHandler>(new SharedQueueHandler(storageAccountProvider, hostIdProvider, exceptionHandler,
config.LoggerFactory, queueConfiguration, sharedContextProvider, messageEnqueuedWatcherAccessor));
ITriggerBindingProvider triggerBindingProvider = DefaultTriggerBindingProvider.Create(nameResolver,
storageAccountProvider, extensionTypeLocator, hostIdProvider, queueConfiguration, blobsConfiguration, exceptionHandler,
messageEnqueuedWatcherAccessor, blobWrittenWatcherAccessor, sharedContextProvider, extensions, singletonManager, config.LoggerFactory);
services.AddService<ITriggerBindingProvider>(triggerBindingProvider);
if (bindingProvider == null)
{
bindingProvider = DefaultBindingProvider.Create(nameResolver, config.LoggerFactory, storageAccountProvider, extensionTypeLocator, blobWrittenWatcherAccessor, extensions);
services.AddService<IBindingProvider>(bindingProvider);
}
metadataProvider.Initialize(bindingProvider, converterManager, exts);
services.AddService<IJobHostMetadataProvider>(metadataProvider);
return services;
}
// Do the full runtime intitialization. This includes static initialization.
// This mainly means:
// - indexing the functions
// - spinning up the listeners (so connecting to the services)
public static async Task<JobHostContext> CreateJobHostContextAsync(
this JobHostConfiguration config,
ServiceProviderWrapper services, // Results from first phase
JobHost host,
CancellationToken shutdownToken,
CancellationToken cancellationToken)
{
FunctionExecutor functionExecutor = services.GetService<FunctionExecutor>();
IFunctionIndexProvider functionIndexProvider = services.GetService<IFunctionIndexProvider>();
ITriggerBindingProvider triggerBindingProvider = services.GetService<ITriggerBindingProvider>();
IBindingProvider bindingProvider = services.GetService<IBindingProvider>();
SingletonManager singletonManager = services.GetService<SingletonManager>();
IJobActivator activator = services.GetService<IJobActivator>();
IHostIdProvider hostIdProvider = services.GetService<IHostIdProvider>();
INameResolver nameResolver = services.GetService<INameResolver>();
IExtensionRegistry extensions = services.GetExtensions();
IStorageAccountProvider storageAccountProvider = services.GetService<IStorageAccountProvider>();
ILoggerFactory loggerFactory = services.GetService<ILoggerFactory>();
IFunctionResultAggregatorFactory aggregatorFactory = services.GetService<IFunctionResultAggregatorFactory>();
IAsyncCollector<FunctionInstanceLogEntry> functionEventCollector = null;
SharedQueueHandler hostSharedQueue = services.GetService<SharedQueueHandler>();
// Create the aggregator if all the pieces are configured
IAsyncCollector<FunctionInstanceLogEntry> aggregator = null;
if (loggerFactory != null && aggregatorFactory != null && config.Aggregator.IsEnabled)
{
aggregator = aggregatorFactory.Create(config.Aggregator.BatchSize, config.Aggregator.FlushTimeout, loggerFactory);
}
IQueueConfiguration queueConfiguration = services.GetService<IQueueConfiguration>();
var blobsConfiguration = config.Blobs;
IAsyncCollector<FunctionInstanceLogEntry> registeredFunctionEventCollector = services.GetService<IAsyncCollector<FunctionInstanceLogEntry>>();
if (registeredFunctionEventCollector != null && aggregator != null)
{
// If there are both an aggregator and a registered FunctionEventCollector, wrap them in a composite
functionEventCollector = new CompositeFunctionEventCollector(new[] { registeredFunctionEventCollector, aggregator });
}
else
{
// Otherwise, take whichever one is null (or use null if both are)
functionEventCollector = aggregator ?? registeredFunctionEventCollector;
}
IWebJobsExceptionHandler exceptionHandler = services.GetService<IWebJobsExceptionHandler>();
if (exceptionHandler != null)
{
exceptionHandler.Initialize(host);
}
bool hasFastTableHook = services.GetService<IAsyncCollector<FunctionInstanceLogEntry>>() != null;
bool noDashboardStorage = config.DashboardConnectionString == null;
// Only testing will override these interfaces.
IHostInstanceLoggerProvider hostInstanceLoggerProvider = services.GetService<IHostInstanceLoggerProvider>();
IFunctionInstanceLoggerProvider functionInstanceLoggerProvider = services.GetService<IFunctionInstanceLoggerProvider>();
IFunctionOutputLoggerProvider functionOutputLoggerProvider = services.GetService<IFunctionOutputLoggerProvider>();
if (hostInstanceLoggerProvider == null && functionInstanceLoggerProvider == null && functionOutputLoggerProvider == null)
{
if (hasFastTableHook && noDashboardStorage)
{
var loggerProvider = new FastTableLoggerProvider(loggerFactory);
hostInstanceLoggerProvider = loggerProvider;
functionInstanceLoggerProvider = loggerProvider;
functionOutputLoggerProvider = loggerProvider;
}
else
{
var loggerProvider = new DefaultLoggerProvider(storageAccountProvider, loggerFactory);
hostInstanceLoggerProvider = loggerProvider;
functionInstanceLoggerProvider = loggerProvider;
functionOutputLoggerProvider = loggerProvider;
}
}
using (CancellationTokenSource combinedCancellationSource = CancellationTokenSource.CreateLinkedTokenSource(cancellationToken, shutdownToken))
{
CancellationToken combinedCancellationToken = combinedCancellationSource.Token;
await WriteSiteExtensionManifestAsync(combinedCancellationToken);
IStorageAccount dashboardAccount = await storageAccountProvider.GetDashboardAccountAsync(combinedCancellationToken);
IHostInstanceLogger hostInstanceLogger = await hostInstanceLoggerProvider.GetAsync(combinedCancellationToken);
IFunctionInstanceLogger functionInstanceLogger = await functionInstanceLoggerProvider.GetAsync(combinedCancellationToken);
IFunctionOutputLogger functionOutputLogger = await functionOutputLoggerProvider.GetAsync(combinedCancellationToken);
loggerFactory.AddProvider(new FunctionOutputLoggerProvider());
if (functionExecutor == null)
{
var extensionRegistry = config.GetService<IExtensionRegistry>();
var globalFunctionFilters = extensionRegistry.GetFunctionFilters();
functionExecutor = new FunctionExecutor(functionInstanceLogger, functionOutputLogger, exceptionHandler, functionEventCollector, loggerFactory, globalFunctionFilters);
services.AddService(functionExecutor);
}
if (functionIndexProvider == null)
{
var defaultTimeout = config.FunctionTimeout?.ToAttribute();
functionIndexProvider = new FunctionIndexProvider(
services.GetService<ITypeLocator>(),
triggerBindingProvider,
bindingProvider,
activator,
functionExecutor,
extensions,
singletonManager,
loggerFactory,
hostSharedQueue,
defaultTimeout,
config.AllowPartialHostStartup);
// Important to set this so that the func we passed to DynamicHostIdProvider can pick it up.
services.AddService<IFunctionIndexProvider>(functionIndexProvider);
}
IFunctionIndex functions = await functionIndexProvider.GetAsync(combinedCancellationToken);
IListenerFactory functionsListenerFactory = new HostListenerFactory(functions.ReadAll(), singletonManager, activator, nameResolver, loggerFactory, config.AllowPartialHostStartup);
IFunctionExecutor hostCallExecutor;
IListener listener;
HostOutputMessage hostOutputMessage;
string hostId = await hostIdProvider.GetHostIdAsync(cancellationToken);
if (string.Compare(config.HostId, hostId, StringComparison.OrdinalIgnoreCase) != 0)
{
// if this isn't a static host ID, provide the HostId on the config
// so it is accessible
config.HostId = hostId;
}
if (dashboardAccount == null)
{
hostCallExecutor = new ShutdownFunctionExecutor(shutdownToken, functionExecutor);
IListener factoryListener = new ListenerFactoryListener(functionsListenerFactory, hostSharedQueue);
IListener shutdownListener = new ShutdownListener(shutdownToken, factoryListener);
listener = shutdownListener;
hostOutputMessage = new DataOnlyHostOutputMessage();
}
else
{
string sharedQueueName = HostQueueNames.GetHostQueueName(hostId);
IStorageQueueClient dashboardQueueClient = dashboardAccount.CreateQueueClient();
IStorageQueue sharedQueue = dashboardQueueClient.GetQueueReference(sharedQueueName);
IListenerFactory sharedQueueListenerFactory = new HostMessageListenerFactory(sharedQueue,
queueConfiguration, exceptionHandler, loggerFactory, functions,
functionInstanceLogger, functionExecutor);
Guid hostInstanceId = Guid.NewGuid();
string instanceQueueName = HostQueueNames.GetHostQueueName(hostInstanceId.ToString("N"));
IStorageQueue instanceQueue = dashboardQueueClient.GetQueueReference(instanceQueueName);
IListenerFactory instanceQueueListenerFactory = new HostMessageListenerFactory(instanceQueue,
queueConfiguration, exceptionHandler, loggerFactory, functions,
functionInstanceLogger, functionExecutor);
HeartbeatDescriptor heartbeatDescriptor = new HeartbeatDescriptor
{
SharedContainerName = HostContainerNames.Hosts,
SharedDirectoryName = HostDirectoryNames.Heartbeats + "/" + hostId,
InstanceBlobName = hostInstanceId.ToString("N"),
ExpirationInSeconds = (int)HeartbeatIntervals.ExpirationInterval.TotalSeconds
};
IStorageBlockBlob blob = dashboardAccount.CreateBlobClient()
.GetContainerReference(heartbeatDescriptor.SharedContainerName)
.GetBlockBlobReference(heartbeatDescriptor.SharedDirectoryName + "/" + heartbeatDescriptor.InstanceBlobName);
IRecurrentCommand heartbeatCommand = new UpdateHostHeartbeatCommand(new HeartbeatCommand(blob));
IEnumerable<MethodInfo> indexedMethods = functions.ReadAllMethods();
Assembly hostAssembly = GetHostAssembly(indexedMethods);
string displayName = hostAssembly != null ? AssemblyNameCache.GetName(hostAssembly).Name : "Unknown";
hostOutputMessage = new DataOnlyHostOutputMessage
{
HostInstanceId = hostInstanceId,
HostDisplayName = displayName,
SharedQueueName = sharedQueueName,
InstanceQueueName = instanceQueueName,
Heartbeat = heartbeatDescriptor,
WebJobRunIdentifier = WebJobRunIdentifier.Current
};
hostCallExecutor = CreateHostCallExecutor(instanceQueueListenerFactory, heartbeatCommand,
exceptionHandler, shutdownToken, functionExecutor);
IListenerFactory hostListenerFactory = new CompositeListenerFactory(functionsListenerFactory,
sharedQueueListenerFactory, instanceQueueListenerFactory);
listener = CreateHostListener(hostListenerFactory, hostSharedQueue, heartbeatCommand, exceptionHandler, shutdownToken);
// Publish this to Azure logging account so that a web dashboard can see it.
await LogHostStartedAsync(functions, hostOutputMessage, hostInstanceLogger, combinedCancellationToken);
}
functionExecutor.HostOutputMessage = hostOutputMessage;
IEnumerable<FunctionDescriptor> descriptors = functions.ReadAllDescriptors();
int descriptorsCount = descriptors.Count();
ILogger startupLogger = loggerFactory?.CreateLogger(LogCategories.Startup);
if (config.UsingDevelopmentSettings)
{
string msg = "Development settings applied";
startupLogger?.LogDebug(msg);
}
if (descriptorsCount == 0)
{
string msg = string.Format("No job functions found. Try making your job classes and methods public. {0}",
Constants.ExtensionInitializationMessage);
startupLogger?.LogWarning(msg);
}
else
{
StringBuilder functionsTrace = new StringBuilder();
functionsTrace.AppendLine("Found the following functions:");
foreach (FunctionDescriptor descriptor in descriptors)
{
functionsTrace.AppendLine(descriptor.FullName);
}
string msg = functionsTrace.ToString();
startupLogger?.LogInformation(msg);
}
return new JobHostContext(
functions,
hostCallExecutor,
listener,
functionEventCollector,
loggerFactory);
}
}
private static void InvokeExtensionConfigProviders(ExtensionConfigContext context)
{
IExtensionRegistry extensions = context.Config.GetExtensions();
IEnumerable<IExtensionConfigProvider> configProviders = extensions.GetExtensions(typeof(IExtensionConfigProvider)).Cast<IExtensionConfigProvider>();
foreach (IExtensionConfigProvider configProvider in configProviders)
{
context.Current = configProvider;
configProvider.Initialize(context);
}
context.ApplyRules();
}
private static IFunctionExecutor CreateHostCallExecutor(IListenerFactory instanceQueueListenerFactory,
IRecurrentCommand heartbeatCommand, IWebJobsExceptionHandler exceptionHandler,
CancellationToken shutdownToken, IFunctionExecutor innerExecutor)
{
IFunctionExecutor heartbeatExecutor = new HeartbeatFunctionExecutor(heartbeatCommand,
exceptionHandler, innerExecutor);
IFunctionExecutor abortListenerExecutor = new AbortListenerFunctionExecutor(instanceQueueListenerFactory, heartbeatExecutor);
IFunctionExecutor shutdownFunctionExecutor = new ShutdownFunctionExecutor(shutdownToken, abortListenerExecutor);
return shutdownFunctionExecutor;
}
[SuppressMessage("Microsoft.Reliability", "CA2000:Dispose objects before losing scope")]
private static IListener CreateHostListener(IListenerFactory allFunctionsListenerFactory, SharedQueueHandler sharedQueue,
IRecurrentCommand heartbeatCommand, IWebJobsExceptionHandler exceptionHandler,
CancellationToken shutdownToken)
{
IListener factoryListener = new ListenerFactoryListener(allFunctionsListenerFactory, sharedQueue);
IListener heartbeatListener = new HeartbeatListener(heartbeatCommand, exceptionHandler, factoryListener);
IListener shutdownListener = new ShutdownListener(shutdownToken, heartbeatListener);
return shutdownListener;
}
private static Assembly GetHostAssembly(IEnumerable<MethodInfo> methods)
{
// 1. Try to get the assembly name from the first method.
MethodInfo firstMethod = methods.FirstOrDefault();
if (firstMethod != null)
{
return firstMethod.DeclaringType.Assembly;
}
// 2. If there are no function definitions, try to use the entry assembly.
Assembly entryAssembly = Assembly.GetEntryAssembly();
if (entryAssembly != null)
{
return entryAssembly;
}
// 3. If there's no entry assembly either, we don't have anything to use.
return null;
}
private static Task LogHostStartedAsync(IFunctionIndex functionIndex, HostOutputMessage hostOutputMessage,
IHostInstanceLogger logger, CancellationToken cancellationToken)
{
IEnumerable<FunctionDescriptor> functions = functionIndex.ReadAllDescriptors();
HostStartedMessage message = new HostStartedMessage
{
HostInstanceId = hostOutputMessage.HostInstanceId,
HostDisplayName = hostOutputMessage.HostDisplayName,
SharedQueueName = hostOutputMessage.SharedQueueName,
InstanceQueueName = hostOutputMessage.InstanceQueueName,
Heartbeat = hostOutputMessage.Heartbeat,
WebJobRunIdentifier = hostOutputMessage.WebJobRunIdentifier,
Functions = functions
};
return logger.LogHostStartedAsync(message, cancellationToken);
}
// When running in Azure Web Sites, write out a manifest file. This manifest file is read by
// the Kudu site extension to provide custom behaviors for SDK jobs
private static async Task WriteSiteExtensionManifestAsync(CancellationToken cancellationToken)
{
string jobDataPath = Environment.GetEnvironmentVariable(WebSitesKnownKeyNames.JobDataPath);
if (jobDataPath == null)
{
// we're not in Azure Web Sites, bye bye.
return;
}
const string Filename = "WebJobsSdk.marker";
var path = Path.Combine(jobDataPath, Filename);
const int DefaultBufferSize = 4096;
try
{
using (Stream stream = new FileStream(path, FileMode.OpenOrCreate, FileAccess.Write, FileShare.None, DefaultBufferSize, useAsync: true))
using (TextWriter writer = new StreamWriter(stream))
{
// content is not really important, this would help debugging though
cancellationToken.ThrowIfCancellationRequested();
await writer.WriteAsync(DateTime.UtcNow.ToString("s") + "Z");
await writer.FlushAsync();
}
}
catch (Exception ex)
{
if (ex is UnauthorizedAccessException || ex is IOException)
{
// simultaneous access error or an error caused by some other issue
// ignore it and skip marker creation
}
else
{
throw;
}
}
}
#region Backwards compat shim for ExtensionLocator
// We can remove this when we fix https://github.com/Azure/azure-webjobs-sdk/issues/995
// create IConverterManager adapters to any legacy ICloudBlobStreamBinder<T>.
static void AddStreamConverters(IExtensionTypeLocator extensionTypeLocator, ConverterManager cm)
{
if (extensionTypeLocator == null)
{
return;
}
foreach (var type in extensionTypeLocator.GetCloudBlobStreamBinderTypes())
{
var instance = Activator.CreateInstance(type);
var bindingType = Blobs.CloudBlobStreamObjectBinder.GetBindingValueType(type);
var method = typeof(JobHostConfigurationExtensions).GetMethod("AddAdapter", BindingFlags.Static | BindingFlags.NonPublic);
method = method.MakeGenericMethod(bindingType);
method.Invoke(null, new object[] { cm, instance });
}
}
static void AddAdapter<T>(ConverterManager cm, ICloudBlobStreamBinder<T> x)
{
cm.AddExactConverter<Stream, T>(stream => x.ReadFromStreamAsync(stream, CancellationToken.None).Result);
cm.AddExactConverter<ApplyConversion<T, Stream>, object>(pair =>
{
T value = pair.Value;
Stream stream = pair.Existing;
x.WriteToStreamAsync(value, stream, CancellationToken.None).Wait();
return null;
});
}
#endregion
private class DataOnlyHostOutputMessage : HostOutputMessage
{
internal override void AddMetadata(IDictionary<string, string> metadata)
{
throw new NotSupportedException();
}
}
}
}

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

@ -11,12 +11,12 @@ namespace Microsoft.Azure.WebJobs.Host.Executors
{
// JobHostContext are the fields that a JobHost needs to operate at runtime.
// This is created from a JobHostConfiguration.
internal sealed class JobHostContext : IDisposable
public sealed class JobHostContext : IDisposable
{
private readonly IFunctionIndexLookup _functionLookup;
private readonly IFunctionExecutor _executor;
private readonly IListener _listener;
private readonly IAsyncCollector<FunctionInstanceLogEntry> _functionEventCollector; // optional
private readonly IAsyncCollector<FunctionInstanceLogEntry> _eventCollector;
private readonly ILoggerFactory _loggerFactory;
private bool _disposed;
@ -24,13 +24,13 @@ namespace Microsoft.Azure.WebJobs.Host.Executors
public JobHostContext(IFunctionIndexLookup functionLookup,
IFunctionExecutor executor,
IListener listener,
IAsyncCollector<FunctionInstanceLogEntry> functionEventCollector = null,
IAsyncCollector<FunctionInstanceLogEntry> eventCollector,
ILoggerFactory loggerFactory = null)
{
_functionLookup = functionLookup;
_executor = executor;
_listener = listener;
_functionEventCollector = functionEventCollector;
_eventCollector = eventCollector;
_loggerFactory = loggerFactory;
}
@ -61,12 +61,12 @@ namespace Microsoft.Azure.WebJobs.Host.Executors
}
}
public IAsyncCollector<FunctionInstanceLogEntry> FunctionEventCollector
public IAsyncCollector<FunctionInstanceLogEntry> EventCollector
{
get
{
ThrowIfDisposed();
return _functionEventCollector;
return _eventCollector;
}
}

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

@ -0,0 +1,383 @@
// Copyright (c) .NET Foundation. All rights reserved.
// Licensed under the MIT License. See License.txt in the project root for license information.
using System;
using System.Collections.Generic;
using System.Diagnostics.CodeAnalysis;
using System.IO;
using System.Linq;
using System.Reflection;
using System.Text;
using System.Threading;
using System.Threading.Tasks;
using Microsoft.Azure.WebJobs.Host.Config;
using Microsoft.Azure.WebJobs.Host.Dispatch;
using Microsoft.Azure.WebJobs.Host.Indexers;
using Microsoft.Azure.WebJobs.Host.Listeners;
using Microsoft.Azure.WebJobs.Host.Loggers;
using Microsoft.Azure.WebJobs.Host.Protocols;
using Microsoft.Azure.WebJobs.Host.Queues.Listeners;
using Microsoft.Azure.WebJobs.Host.Storage;
using Microsoft.Azure.WebJobs.Host.Storage.Blob;
using Microsoft.Azure.WebJobs.Host.Storage.Queue;
using Microsoft.Azure.WebJobs.Host.Timers;
using Microsoft.Azure.WebJobs.Host.Triggers;
using Microsoft.Azure.WebJobs.Logging;
using Microsoft.Extensions.Logging;
using Microsoft.Extensions.Options;
namespace Microsoft.Azure.WebJobs.Host.Executors
{
internal class JobHostContextFactory : IJobHostContextFactory
{
private readonly IFunctionExecutor _functionExecutor;
private readonly IFunctionIndexProvider _functionIndexProvider;
private readonly ITriggerBindingProvider _triggerBindingProvider;
private readonly SingletonManager _singletonManager;
private readonly IJobActivator _activator;
private readonly IHostIdProvider _hostIdProvider;
private readonly INameResolver _nameResolver;
private readonly IExtensionRegistry _extensions;
private readonly IExtensionTypeLocator _extensionTypeLocator;
private readonly IStorageAccountProvider _storageAccountProvider;
private readonly ILoggerFactory _loggerFactory;
private readonly IOptions<JobHostQueuesOptions> _queueConfiguration;
private readonly IWebJobsExceptionHandler _exceptionHandler;
private readonly SharedQueueHandler _sharedQueueHandler;
private readonly IOptions<JobHostOptions> _jobHostOptions;
private readonly IOptions<JobHostBlobsOptions> _blobsConfiguration;
private readonly IHostInstanceLogger _hostInstanceLogger;
private readonly IFunctionInstanceLogger _functionInstanceLogger;
private readonly IFunctionOutputLogger _functionOutputLogger;
private readonly IConverterManager _converterManager;
private readonly IAsyncCollector<FunctionInstanceLogEntry> _eventCollector;
public JobHostContextFactory(IFunctionExecutor functionExecutor,
IFunctionIndexProvider functionIndexProvider,
ITriggerBindingProvider triggerBindingProvider,
SingletonManager singletonManager,
IJobActivator activator,
IHostIdProvider hostIdProvider,
INameResolver nameResolver,
IExtensionRegistry extensions,
IExtensionTypeLocator extensionTypeLocator,
IStorageAccountProvider storageAccountProvider,
ILoggerFactory loggerFactory,
IWebJobsExceptionHandler exceptionHandler,
SharedQueueHandler sharedQueueHandler,
IOptions<JobHostOptions> jobHostOptions,
IOptions<JobHostQueuesOptions> queueOptions,
IOptions<JobHostBlobsOptions> blobsConfiguration,
IHostInstanceLogger hostInstanceLogger,
IFunctionInstanceLogger functionInstanceLogger,
IFunctionOutputLogger functionOutputLogger,
IConverterManager converterManager,
IAsyncCollector<FunctionInstanceLogEntry> eventCollector)
{
_functionExecutor = functionExecutor;
_functionIndexProvider = functionIndexProvider;
_triggerBindingProvider = triggerBindingProvider;
_singletonManager = singletonManager;
_activator = activator;
_hostIdProvider = hostIdProvider;
_nameResolver = nameResolver;
_extensions = extensions;
_extensionTypeLocator = extensionTypeLocator;
_storageAccountProvider = storageAccountProvider;
_loggerFactory = loggerFactory;
_queueConfiguration = queueOptions;
_exceptionHandler = exceptionHandler;
_sharedQueueHandler = sharedQueueHandler;
_jobHostOptions = jobHostOptions;
_blobsConfiguration = blobsConfiguration;
_hostInstanceLogger = hostInstanceLogger;
_functionInstanceLogger = functionInstanceLogger;
_functionOutputLogger = functionOutputLogger;
_converterManager = converterManager;
_eventCollector = eventCollector;
}
public async Task<JobHostContext> Create(CancellationToken shutdownToken, CancellationToken cancellationToken)
{
using (CancellationTokenSource combinedCancellationSource = CancellationTokenSource.CreateLinkedTokenSource(cancellationToken, shutdownToken))
{
CancellationToken combinedCancellationToken = combinedCancellationSource.Token;
AddStreamConverters(_extensionTypeLocator, _converterManager);
await WriteSiteExtensionManifestAsync(combinedCancellationToken);
IStorageAccount dashboardAccount = await _storageAccountProvider.GetDashboardAccountAsync(combinedCancellationToken);
// TODO: FACAVAL: Chat with Brettsam, this should probably be moved out of here.
_loggerFactory.AddProvider(new FunctionOutputLoggerProvider());
IFunctionIndex functions = await _functionIndexProvider.GetAsync(combinedCancellationToken);
IListenerFactory functionsListenerFactory = new HostListenerFactory(functions.ReadAll(), _singletonManager, _activator, _nameResolver, _loggerFactory);
IFunctionExecutor hostCallExecutor;
IListener listener;
HostOutputMessage hostOutputMessage;
string hostId = await _hostIdProvider.GetHostIdAsync(cancellationToken);
if (string.Compare(_jobHostOptions.Value.HostId, hostId, StringComparison.OrdinalIgnoreCase) != 0)
{
// if this isn't a static host ID, provide the HostId on the config
// so it is accessible
_jobHostOptions.Value.HostId = hostId;
}
if (dashboardAccount == null)
{
hostCallExecutor = new ShutdownFunctionExecutor(shutdownToken, _functionExecutor);
IListener factoryListener = new ListenerFactoryListener(functionsListenerFactory, _sharedQueueHandler);
IListener shutdownListener = new ShutdownListener(shutdownToken, factoryListener);
listener = shutdownListener;
hostOutputMessage = new DataOnlyHostOutputMessage();
}
else
{
string sharedQueueName = HostQueueNames.GetHostQueueName(hostId);
IStorageQueueClient dashboardQueueClient = dashboardAccount.CreateQueueClient();
IStorageQueue sharedQueue = dashboardQueueClient.GetQueueReference(sharedQueueName);
IListenerFactory sharedQueueListenerFactory = new HostMessageListenerFactory(sharedQueue,
_queueConfiguration.Value, _exceptionHandler, _loggerFactory, functions,
_functionInstanceLogger, _functionExecutor);
Guid hostInstanceId = Guid.NewGuid();
string instanceQueueName = HostQueueNames.GetHostQueueName(hostInstanceId.ToString("N"));
IStorageQueue instanceQueue = dashboardQueueClient.GetQueueReference(instanceQueueName);
IListenerFactory instanceQueueListenerFactory = new HostMessageListenerFactory(instanceQueue,
_queueConfiguration.Value, _exceptionHandler, _loggerFactory, functions,
_functionInstanceLogger, _functionExecutor);
HeartbeatDescriptor heartbeatDescriptor = new HeartbeatDescriptor
{
SharedContainerName = HostContainerNames.Hosts,
SharedDirectoryName = HostDirectoryNames.Heartbeats + "/" + hostId,
InstanceBlobName = hostInstanceId.ToString("N"),
ExpirationInSeconds = (int)HeartbeatIntervals.ExpirationInterval.TotalSeconds
};
IStorageBlockBlob blob = dashboardAccount.CreateBlobClient()
.GetContainerReference(heartbeatDescriptor.SharedContainerName)
.GetBlockBlobReference(heartbeatDescriptor.SharedDirectoryName + "/" + heartbeatDescriptor.InstanceBlobName);
IRecurrentCommand heartbeatCommand = new UpdateHostHeartbeatCommand(new HeartbeatCommand(blob));
IEnumerable<MethodInfo> indexedMethods = functions.ReadAllMethods();
Assembly hostAssembly = GetHostAssembly(indexedMethods);
string displayName = hostAssembly != null ? AssemblyNameCache.GetName(hostAssembly).Name : "Unknown";
hostOutputMessage = new DataOnlyHostOutputMessage
{
HostInstanceId = hostInstanceId,
HostDisplayName = displayName,
SharedQueueName = sharedQueueName,
InstanceQueueName = instanceQueueName,
Heartbeat = heartbeatDescriptor,
WebJobRunIdentifier = WebJobRunIdentifier.Current
};
hostCallExecutor = CreateHostCallExecutor(instanceQueueListenerFactory, heartbeatCommand,
_exceptionHandler, shutdownToken, _functionExecutor);
IListenerFactory hostListenerFactory = new CompositeListenerFactory(functionsListenerFactory,
sharedQueueListenerFactory, instanceQueueListenerFactory);
listener = CreateHostListener(hostListenerFactory, _sharedQueueHandler, heartbeatCommand, _exceptionHandler, shutdownToken);
// Publish this to Azure logging account so that a web dashboard can see it.
await LogHostStartedAsync(functions, hostOutputMessage, _hostInstanceLogger, combinedCancellationToken);
}
if (_functionExecutor is FunctionExecutor executor)
{
executor.HostOutputMessage = hostOutputMessage;
}
IEnumerable<FunctionDescriptor> descriptors = functions.ReadAllDescriptors();
int descriptorsCount = descriptors.Count();
ILogger startupLogger = _loggerFactory?.CreateLogger(LogCategories.Startup);
if (_jobHostOptions.Value.UsingDevelopmentSettings)
{
string msg = "Development settings applied";
startupLogger?.LogDebug(msg);
}
if (descriptorsCount == 0)
{
string msg = string.Format("No job functions found. Try making your job classes and methods public. {0}",
Constants.ExtensionInitializationMessage);
startupLogger?.LogWarning(msg);
}
else
{
StringBuilder functionsTrace = new StringBuilder();
functionsTrace.AppendLine("Found the following functions:");
foreach (FunctionDescriptor descriptor in descriptors)
{
functionsTrace.AppendLine(descriptor.FullName);
}
string msg = functionsTrace.ToString();
startupLogger?.LogInformation(msg);
}
return new JobHostContext(
functions,
hostCallExecutor,
listener,
_eventCollector,
_loggerFactory);
}
}
[SuppressMessage("Microsoft.Reliability", "CA2000:Dispose objects before losing scope")]
private static IListener CreateHostListener(IListenerFactory allFunctionsListenerFactory, SharedQueueHandler sharedQueue,
IRecurrentCommand heartbeatCommand, IWebJobsExceptionHandler exceptionHandler, CancellationToken shutdownToken)
{
IListener factoryListener = new ListenerFactoryListener(allFunctionsListenerFactory, sharedQueue);
IListener heartbeatListener = new HeartbeatListener(heartbeatCommand, exceptionHandler, factoryListener);
IListener shutdownListener = new ShutdownListener(shutdownToken, heartbeatListener);
return shutdownListener;
}
private static Task LogHostStartedAsync(IFunctionIndex functionIndex, HostOutputMessage hostOutputMessage,
IHostInstanceLogger logger, CancellationToken cancellationToken)
{
IEnumerable<FunctionDescriptor> functions = functionIndex.ReadAllDescriptors();
HostStartedMessage message = new HostStartedMessage
{
HostInstanceId = hostOutputMessage.HostInstanceId,
HostDisplayName = hostOutputMessage.HostDisplayName,
SharedQueueName = hostOutputMessage.SharedQueueName,
InstanceQueueName = hostOutputMessage.InstanceQueueName,
Heartbeat = hostOutputMessage.Heartbeat,
WebJobRunIdentifier = hostOutputMessage.WebJobRunIdentifier,
Functions = functions
};
return logger.LogHostStartedAsync(message, cancellationToken);
}
private static Assembly GetHostAssembly(IEnumerable<MethodInfo> methods)
{
// 1. Try to get the assembly name from the first method.
MethodInfo firstMethod = methods.FirstOrDefault();
if (firstMethod != null)
{
return firstMethod.DeclaringType.Assembly;
}
// 2. If there are no function definitions, try to use the entry assembly.
Assembly entryAssembly = Assembly.GetEntryAssembly();
if (entryAssembly != null)
{
return entryAssembly;
}
// 3. If there's no entry assembly either, we don't have anything to use.
return null;
}
private static async Task WriteSiteExtensionManifestAsync(CancellationToken cancellationToken)
{
string jobDataPath = Environment.GetEnvironmentVariable(WebSitesKnownKeyNames.JobDataPath);
if (jobDataPath == null)
{
// we're not in Azure Web Sites, bye bye.
return;
}
const string Filename = "WebJobsSdk.marker";
var path = Path.Combine(jobDataPath, Filename);
const int DefaultBufferSize = 4096;
try
{
using (Stream stream = new FileStream(path, FileMode.OpenOrCreate, FileAccess.Write, FileShare.None, DefaultBufferSize, useAsync: true))
using (TextWriter writer = new StreamWriter(stream))
{
// content is not really important, this would help debugging though
cancellationToken.ThrowIfCancellationRequested();
await writer.WriteAsync(DateTime.UtcNow.ToString("s") + "Z");
await writer.FlushAsync();
}
}
catch (Exception ex)
{
if (ex is UnauthorizedAccessException || ex is IOException)
{
// simultaneous access error or an error caused by some other issue
// ignore it and skip marker creation
}
else
{
throw;
}
}
}
private static IFunctionExecutor CreateHostCallExecutor(IListenerFactory instanceQueueListenerFactory,
IRecurrentCommand heartbeatCommand, IWebJobsExceptionHandler exceptionHandler,
CancellationToken shutdownToken, IFunctionExecutor innerExecutor)
{
IFunctionExecutor heartbeatExecutor = new HeartbeatFunctionExecutor(heartbeatCommand,
exceptionHandler, innerExecutor);
IFunctionExecutor abortListenerExecutor = new AbortListenerFunctionExecutor(instanceQueueListenerFactory, heartbeatExecutor);
IFunctionExecutor shutdownFunctionExecutor = new ShutdownFunctionExecutor(shutdownToken, abortListenerExecutor);
return shutdownFunctionExecutor;
}
#region Backwards compat shim for ExtensionLocator
// We can remove this when we fix https://github.com/Azure/azure-webjobs-sdk/issues/995
// create IConverterManager adapters to any legacy ICloudBlobStreamBinder<T>.
private static void AddStreamConverters(IExtensionTypeLocator extensionTypeLocator, IConverterManager cm)
{
if (extensionTypeLocator == null)
{
return;
}
foreach (var type in extensionTypeLocator.GetCloudBlobStreamBinderTypes())
{
var instance = Activator.CreateInstance(type);
var bindingType = Blobs.CloudBlobStreamObjectBinder.GetBindingValueType(type);
var method = typeof(JobHostContextFactory).GetMethod("AddAdapter", BindingFlags.Static | BindingFlags.NonPublic);
method = method.MakeGenericMethod(bindingType);
method.Invoke(null, new object[] { cm, instance });
}
}
private static void AddAdapter<T>(ConverterManager cm, ICloudBlobStreamBinder<T> x)
{
cm.AddExactConverter<Stream, T>(stream => x.ReadFromStreamAsync(stream, CancellationToken.None).Result);
cm.AddExactConverter<ApplyConversion<T, Stream>, object>(pair =>
{
T value = pair.Value;
Stream stream = pair.Existing;
x.WriteToStreamAsync(value, stream, CancellationToken.None).Wait();
return null;
});
}
#endregion
private class DataOnlyHostOutputMessage : HostOutputMessage
{
internal override void AddMetadata(IDictionary<string, string> metadata)
{
throw new NotSupportedException();
}
}
}
}

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

@ -16,15 +16,21 @@ namespace Microsoft.Azure.WebJobs.Host.Executors
/// </summary>
internal sealed class StorageAccountParser : IStorageAccountParser
{
private readonly StorageClientFactory _storageClientFactory;
public StorageAccountParser(StorageClientFactory storageClientFactory)
{
_storageClientFactory = storageClientFactory;
}
/// <summary>
/// Throwing version of parse account API. It calls TryParseAccount internally, analyzes returned result,
/// and throws an exception with formatted message in case of error.
/// </summary>
/// <param name="connectionString">A Storage account connection string as retrieved from the config</param>
/// <param name="connectionStringName">Friendly connection string name used to format error message</param>
/// <param name="services">The <see cref="IServiceProvider"/> to use.</param>
/// <returns>An instance of <see cref="StorageAccount"/> associated with the given connection string</returns>
public IStorageAccount ParseAccount(string connectionString, string connectionStringName, IServiceProvider services)
public IStorageAccount ParseAccount(string connectionString, string connectionStringName)
{
CloudStorageAccount account;
StorageAccountParseResult result = TryParseAccount(connectionString, out account);
@ -35,7 +41,7 @@ namespace Microsoft.Azure.WebJobs.Host.Executors
throw new InvalidOperationException(message);
}
return new StorageAccount(account, services);
return new StorageAccount(account, _storageClientFactory);
}
/// <summary>

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

@ -0,0 +1,10 @@
// Copyright (c) .NET Foundation. All rights reserved.
// Licensed under the MIT License. See License.txt in the project root for license information.
namespace Microsoft.Azure.WebJobs.Host
{
public interface IJobHostMetadataProviderFactory
{
IJobHostMetadataProvider Create();
}
}

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

@ -26,34 +26,35 @@ namespace Microsoft.Azure.WebJobs.Host
private IBindingProvider _root;
private readonly Func<IFunctionIndexProvider> _getFunctionIndexProvider;
private readonly IFunctionIndexProvider _functionIndexProvider;
private readonly IExtensionRegistry _extensionRegistry;
private readonly IBindingProvider _bindingProvider;
private readonly IConverterManager _converter;
public JobHostMetadataProvider(Func<IFunctionIndexProvider> getFunctionIndexProvider)
public JobHostMetadataProvider(IFunctionIndexProvider functionIndexProvider, IExtensionRegistry extensionRegistry, IBindingProvider bindingProvider, IConverterManager converter)
{
if (getFunctionIndexProvider == null)
{
throw new ArgumentNullException("getFunctionIndexProvider");
}
_getFunctionIndexProvider = getFunctionIndexProvider;
_functionIndexProvider = functionIndexProvider;
_extensionRegistry = extensionRegistry;
_bindingProvider = bindingProvider;
_converter = converter;
}
internal void Initialize(IBindingProvider bindingProvider, ConverterManager converter, IExtensionRegistry extensionRegistry)
internal void Initialize()
{
foreach (var extension in extensionRegistry.GetExtensions<IExtensionConfigProvider>())
foreach (var extension in _extensionRegistry.GetExtensions<IExtensionConfigProvider>())
{
this.AddExtension(extension);
}
this._root = bindingProvider;
this._root = _bindingProvider;
// Populate assembly resolution from converters.
if (converter != null)
if (_converter != null)
{
converter.AddAssemblies((type) => this.AddAssembly(type));
// _converter.AddAssemblies((type) => AddAssembly(type));
}
AddTypesFromGraph(bindingProvider as IBindingRuleProvider);
AddTypesFromGraph(_bindingProvider as IBindingRuleProvider);
}
// Resolve an assembly from the given name.
@ -328,19 +329,17 @@ namespace Microsoft.Azure.WebJobs.Host
public FunctionMetadata GetFunctionMetadata(string functionName)
{
FunctionMetadata result = null;
var provider = _getFunctionIndexProvider.Invoke();
if (provider != null)
var index = _functionIndexProvider.GetAsync(CancellationToken.None).GetAwaiter().GetResult();
var functionDefinition = index.LookupByName(functionName);
if (functionDefinition != null)
{
var index = provider.GetAsync(CancellationToken.None).GetAwaiter().GetResult();
var functionDefinition = index.LookupByName(functionName);
if (functionDefinition != null)
result = new FunctionMetadata()
{
result = new FunctionMetadata()
{
IsDisabled = functionDefinition.Descriptor.IsDisabled
};
}
IsDisabled = functionDefinition.Descriptor.IsDisabled
};
}
return result;
}
}

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

@ -0,0 +1,31 @@
// Copyright (c) .NET Foundation. All rights reserved.
// Licensed under the MIT License. See License.txt in the project root for license information.
using Microsoft.Azure.WebJobs.Host.Bindings;
using Microsoft.Azure.WebJobs.Host.Indexers;
namespace Microsoft.Azure.WebJobs.Host
{
internal class JobHostMetadataProviderFactory : IJobHostMetadataProviderFactory
{
private readonly IFunctionIndexProvider _functionIndexProvider;
private readonly IExtensionRegistry _extensionRegistry;
private readonly IBindingProvider _bindingProvider;
private readonly IConverterManager _converterManager;
public JobHostMetadataProviderFactory(IFunctionIndexProvider functionIndexProvider, IExtensionRegistry extensionRegistry, IBindingProvider bindingProvider, IConverterManager converterManager)
{
_functionIndexProvider = functionIndexProvider;
_extensionRegistry = extensionRegistry;
_bindingProvider = bindingProvider;
_converterManager = converterManager;
}
public IJobHostMetadataProvider Create()
{
var provider = new JobHostMetadataProvider(_functionIndexProvider, _extensionRegistry, _bindingProvider, _converterManager);
provider.Initialize();
return provider;
}
}
}

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

@ -0,0 +1,21 @@
// Copyright (c) .NET Foundation. All rights reserved.
// Licensed under the MIT License. See License.txt in the project root for license information.
using System;
using System.Collections.Generic;
using System.Text;
using Microsoft.Extensions.Hosting;
using Microsoft.Extensions.Configuration;
namespace Microsoft.Azure.WebJobs.Hosting
{
public class JobHostBuilder
{
public static IHostBuilder CreateDefault(Action<JobHostOptions> configure)
{
configure = configure ?? new Action<JobHostOptions>(o => { });
return new HostBuilder()
.ConfigureWebJobsHost(configure);
}
}
}

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

@ -0,0 +1,36 @@
using Microsoft.Extensions.Configuration;
using Microsoft.Extensions.Hosting;
using Microsoft.Extensions.Logging;
using Microsoft.Extensions.Options;
using System;
using System.Collections.Generic;
using System.Text;
using System.Threading;
using System.Threading.Tasks;
namespace Microsoft.Azure.WebJobs.Hosting
{
public class JobHostService : IHostedService
{
private readonly ILogger<JobHostService> _logger;
private readonly IJobHost _jobHost;
public JobHostService(IJobHost jobhost, ILogger<JobHostService> logger)
{
_logger = logger ?? throw new ArgumentNullException(nameof(logger));
_jobHost = jobhost;
}
public Task StartAsync(CancellationToken cancellationToken)
{
_logger.LogInformation("Starting JobHost");
return _jobHost.StartAsync(cancellationToken);
}
public Task StopAsync(CancellationToken cancellationToken)
{
_logger.LogInformation("Stopping JobHost");
return _jobHost.StopAsync();
}
}
}

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

@ -0,0 +1,62 @@
// Copyright (c) .NET Foundation. All rights reserved.
// Licensed under the MIT License. See License.txt in the project root for license information.
using System;
using Microsoft.Azure.WebJobs;
using Microsoft.Azure.WebJobs.Host.Config;
using Microsoft.Azure.WebJobs.Hosting;
using Microsoft.Extensions.Configuration;
using Microsoft.Extensions.DependencyInjection;
using Microsoft.Extensions.DependencyInjection.Extensions;
namespace Microsoft.Extensions.Hosting
{
public static class WebJobsHostExtensions
{
public static IHostBuilder ConfigureWebJobsHost(this IHostBuilder builder)
{
return builder.ConfigureWebJobsHost(o => { });
}
public static IHostBuilder ConfigureWebJobsHost(this IHostBuilder builder, Action<JobHostOptions> configure)
{
builder.ConfigureAppConfiguration(config =>
{
config.AddJsonFile("appsettings.json", optional: true);
config.AddEnvironmentVariables();
});
builder.ConfigureServices((context, services) =>
{
services.Configure<JobHostOptions>(context.Configuration);
services.AddWebJobs(configure);
services.AddSingleton<IHostedService, JobHostService>();
});
return builder;
}
public static IHostBuilder AddExtension<TExtension>(this IHostBuilder builder)
where TExtension : class, IExtensionConfigProvider
{
builder.ConfigureServices(services =>
{
services.TryAddEnumerable(ServiceDescriptor.Singleton<IExtensionConfigProvider, TExtension>());
});
return builder;
}
public static IHostBuilder AddExtension(this IHostBuilder builder, IExtensionConfigProvider instance)
{
builder.ConfigureServices(services =>
{
services.TryAddEnumerable(ServiceDescriptor.Singleton<IExtensionConfigProvider>(instance));
});
return builder;
}
}
}

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

@ -0,0 +1,157 @@
// Copyright (c) .NET Foundation. All rights reserved.
// Licensed under the MIT License. See License.txt in the project root for license information.
using System;
using System.Threading;
using Microsoft.Azure.WebJobs.Host;
using Microsoft.Azure.WebJobs.Host.Bindings;
using Microsoft.Azure.WebJobs.Host.Blobs;
using Microsoft.Azure.WebJobs.Host.Blobs.Bindings;
using Microsoft.Azure.WebJobs.Host.Blobs.Triggers;
using Microsoft.Azure.WebJobs.Host.Config;
using Microsoft.Azure.WebJobs.Host.Configuration;
using Microsoft.Azure.WebJobs.Host.Dispatch;
using Microsoft.Azure.WebJobs.Host.Executors;
using Microsoft.Azure.WebJobs.Host.Indexers;
using Microsoft.Azure.WebJobs.Host.Listeners;
using Microsoft.Azure.WebJobs.Host.Loggers;
using Microsoft.Azure.WebJobs.Host.Queues;
using Microsoft.Azure.WebJobs.Host.Queues.Bindings;
using Microsoft.Azure.WebJobs.Host.Tables;
using Microsoft.Azure.WebJobs.Host.Timers;
using Microsoft.Azure.WebJobs.Host.Triggers;
using Microsoft.Azure.WebJobs.Logging;
using Microsoft.Extensions.DependencyInjection;
using Microsoft.Extensions.DependencyInjection.Extensions;
using Microsoft.Extensions.Hosting;
using Microsoft.Extensions.Options;
namespace Microsoft.Azure.WebJobs
{
/// <summary>
/// Extension methods for setting up WebJobs services in a <see cref="IServiceCollection" />.
/// </summary>
public static class WebJobsServiceCollectionExtensions
{
/// <summary>
/// Adds the WebJobs services to the provided <see cref="IServiceCollection"/>.
/// </summary>
/// <param name="services"></param>
/// <returns></returns>
public static IServiceCollection AddWebJobs(this IServiceCollection services, Action<JobHostOptions> configure)
{
if (services == null)
{
throw new ArgumentNullException(nameof(services));
}
services.Configure(configure);
// A LOT of the service registrations below need to be cleaned up
// maintaining some of the existing dependencies and model we previously had,
// but this should be reviewed as it can be improved.
services.TryAddSingleton<IExtensionRegistryFactory, DefaultExtensionRegistryFactory>();
services.TryAddSingleton<IExtensionRegistry>(p => p.GetRequiredService<IExtensionRegistryFactory>().Create());
// Type conversion
services.TryAddSingleton<ITypeLocator, DefaultTypeLocator>();
services.TryAddSingleton<IConverterManager, ConverterManager>();
services.TryAddSingleton<IFunctionIndexProvider, FunctionIndexProvider>();
services.TryAddSingleton<SingletonManager>();
services.TryAddSingleton<SharedQueueHandler>();
services.TryAddSingleton<IFunctionExecutor, FunctionExecutor>();
services.TryAddSingleton<IJobHostContextFactory, JobHostContextFactory>();
services.TryAddSingleton<IBindingProviderFactory, DefaultBindingProvider>();
services.TryAddSingleton<IBindingProvider>(p => p.GetRequiredService<IBindingProviderFactory>().Create());
services.TryAddSingleton<ISharedContextProvider, SharedContextProvider>();
services.TryAddSingleton<IContextSetter<IMessageEnqueuedWatcher>>((p) => new ContextAccessor<IMessageEnqueuedWatcher>());
services.TryAddSingleton<IContextSetter<IBlobWrittenWatcher>>((p) => new ContextAccessor<IBlobWrittenWatcher>());
services.TryAddSingleton((p) => p.GetService<IContextSetter<IMessageEnqueuedWatcher>>() as IContextGetter<IMessageEnqueuedWatcher>);
services.TryAddSingleton((p) => p.GetService<IContextSetter<IBlobWrittenWatcher>>() as IContextGetter<IBlobWrittenWatcher>);
services.TryAddSingleton<IDistributedLockManagerFactory, DefaultDistributedLockManagerFactory>();
services.TryAddSingleton<IDistributedLockManager>(p => p.GetRequiredService<IDistributedLockManagerFactory>().Create());
services.TryAddSingleton<IJobHostMetadataProviderFactory, JobHostMetadataProviderFactory>();
services.TryAddSingleton<IJobHostMetadataProvider>(p => p.GetService<IJobHostMetadataProviderFactory>().Create());
services.TryAddSingleton<IExtensionTypeLocator, ExtensionTypeLocator>();
services.AddWebJobsLogging();
// TODO: FACAVAL FIX THIS - Right now, We're only registering the FixedIdProvider
// need to register the dynamic ID provider and verify if the logic in it can be improved (and have the storage dependency removed)
services.TryAddSingleton<IHostIdProvider, FixedHostIdProvider>();
services.TryAddSingleton<DefaultTriggerBindingFactory>();
services.TryAddSingleton<ITriggerBindingProvider>(p => p.GetRequiredService<DefaultTriggerBindingFactory>().Create());
// Exception handler
services.TryAddSingleton<IWebJobsExceptionHandlerFactory, DefaultWebJobsExceptionHandlerFactory>();
services.TryAddSingleton<IWebJobsExceptionHandler>(p => p.GetRequiredService<IWebJobsExceptionHandlerFactory>().Create(p.GetRequiredService<IHost>()));
services.TryAddSingleton<IStorageAccountProvider, DefaultStorageAccountProvider>();
services.TryAddSingleton<IConnectionStringProvider, AmbientConnectionStringProvider>();
services.TryAddSingleton<IStorageAccountParser, StorageAccountParser>();
services.TryAddSingleton<IStorageCredentialsValidator, DefaultStorageCredentialsValidator>();
services.TryAddSingleton<StorageClientFactory>();
services.TryAddSingleton<INameResolver, DefaultNameResolver>();
services.TryAddSingleton<IJobActivator, DefaultJobActivator>();
// Event collector
services.TryAddEnumerable(ServiceDescriptor.Singleton<IEventCollectorProvider, FunctionResultAggregatorProvider>());
services.TryAddSingleton<IEventCollectorFactory, EventCollectorFactory>();
services.TryAddSingleton<IAsyncCollector<FunctionInstanceLogEntry>>(p => p.GetRequiredService<IEventCollectorFactory>().Create());
// Options setup
services.TryAddEnumerable(ServiceDescriptor.Transient<IConfigureOptions<JobHostInternalStorageOptions>, JobHostInternalStorageOptionsSetup>());
services.RegisterBuiltInBindings();
// Core host services
services.TryAddSingleton<IJobHost, JobHost>();
return services;
}
private static IServiceCollection AddWebJobsLogging(this IServiceCollection services)
{
// Logging related services (lots of them...)
services.TryAddSingleton<LoggerProviderFactory>();
services.TryAddSingleton<IFunctionOutputLoggerProvider>(p => p.GetRequiredService<LoggerProviderFactory>().GetLoggerProvider<IFunctionOutputLoggerProvider>());
services.TryAddSingleton<IFunctionOutputLogger>(p => p.GetRequiredService<IFunctionOutputLoggerProvider>().GetAsync(CancellationToken.None).GetAwaiter().GetResult());
services.TryAddSingleton<IFunctionInstanceLoggerProvider>(p => p.GetRequiredService<LoggerProviderFactory>().GetLoggerProvider<IFunctionInstanceLoggerProvider>());
services.TryAddSingleton<IFunctionInstanceLogger>(p => p.GetRequiredService<IFunctionInstanceLoggerProvider>().GetAsync(CancellationToken.None).GetAwaiter().GetResult());
services.TryAddSingleton<IHostInstanceLoggerProvider>(p => p.GetRequiredService<LoggerProviderFactory>().GetLoggerProvider<IHostInstanceLoggerProvider>());
services.TryAddSingleton<IHostInstanceLogger>(p => p.GetRequiredService<IHostInstanceLoggerProvider>().GetAsync(CancellationToken.None).GetAwaiter().GetResult());
return services;
}
/// <summary>
/// Adds the following bindings: <see cref="Tables.TableExtension"/>, <see cref="Queues.Bindings.QueueExtension"/>,
/// <see cref="Blobs.Bindings.BlobExtensionConfig"/> and <see cref="Blobs.Triggers.BlobTriggerExtensionConfig"/>.
/// </summary>
/// <param name="services"></param>
/// <returns></returns>
public static IServiceCollection RegisterBuiltInBindings(this IServiceCollection services)
{
if (services == null)
{
throw new ArgumentNullException(nameof(services));
}
services.TryAddEnumerable(ServiceDescriptor.Singleton<IExtensionConfigProvider, TableExtension>());
services.TryAddEnumerable(ServiceDescriptor.Singleton<IExtensionConfigProvider, QueueExtension>());
services.TryAddEnumerable(ServiceDescriptor.Singleton<IExtensionConfigProvider, BlobExtensionConfig>());
services.TryAddEnumerable(ServiceDescriptor.Singleton<IExtensionConfigProvider, BlobTriggerExtensionConfig>());
return services;
}
}
}

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

@ -1,14 +1,13 @@
// Copyright (c) .NET Foundation. All rights reserved.
// Licensed under the MIT License. See License.txt in the project root for license information.
using System.Collections.Generic;
namespace Microsoft.Azure.WebJobs.Host
{
/// <summary>
/// Defines an interface for providing connection strings.
/// </summary>
internal interface IConnectionStringProvider
// TODO: We should be able to remove this and rely strictly on IConfiguration.
public interface IConnectionStringProvider
{
/// <summary>
/// Get the connection string for the specified name.

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

@ -0,0 +1,10 @@
// Copyright (c) .NET Foundation. All rights reserved.
// Licensed under the MIT License. See License.txt in the project root for license information.
namespace Microsoft.Azure.WebJobs.Host
{
public interface IExtensionRegistryFactory
{
IExtensionRegistry Create();
}
}

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

@ -0,0 +1,16 @@
// Copyright (c) .NET Foundation. All rights reserved.
// Licensed under the MIT License. See License.txt in the project root for license information.
using System;
using System.Threading;
using System.Threading.Tasks;
namespace Microsoft.Azure.WebJobs
{
public interface IJobHost
{
Task StartAsync(CancellationToken cancellationToken);
Task StopAsync();
}
}

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

@ -13,33 +13,44 @@ using Microsoft.Extensions.Logging;
namespace Microsoft.Azure.WebJobs.Host.Indexers
{
internal static class DefaultBindingProvider
internal sealed class DefaultBindingProvider : IBindingProviderFactory
{
public static IBindingProvider Create(
private readonly INameResolver _nameResolver;
private readonly ILoggerFactory _loggerFactory;
private readonly IStorageAccountProvider _storageAccountProvider;
private readonly IExtensionRegistry _extensions;
public DefaultBindingProvider(
INameResolver nameResolver,
ILoggerFactory loggerFactory,
IStorageAccountProvider storageAccountProvider,
IExtensionTypeLocator extensionTypeLocator,
IContextGetter<IBlobWrittenWatcher> blobWrittenWatcherGetter,
IExtensionRegistry extensions)
{
_nameResolver = nameResolver;
_loggerFactory = loggerFactory;
_storageAccountProvider = storageAccountProvider;
_extensions = extensions;
}
public IBindingProvider Create()
{
List<IBindingProvider> innerProviders = new List<IBindingProvider>();
// add any registered extension binding providers
// Queue and Table bindings were added as an extension, so those rules get included here.
foreach (IBindingProvider provider in extensions.GetExtensions(typeof(IBindingProvider)))
foreach (IBindingProvider provider in _extensions.GetExtensions(typeof(IBindingProvider)))
{
innerProviders.Add(provider);
}
innerProviders.Add(new CloudStorageAccountBindingProvider(storageAccountProvider));
innerProviders.Add(new CloudStorageAccountBindingProvider(_storageAccountProvider));
innerProviders.Add(new CancellationTokenBindingProvider());
// The TraceWriter binder handles all remaining TraceWriter/TextWriter parameters. It must come after the
// Blob binding provider; otherwise bindings like Do([Blob("a/b")] TextWriter blob) wouldn't work.
innerProviders.Add(new TraceWriterBindingProvider(loggerFactory));
innerProviders.Add(new TraceWriterBindingProvider(_loggerFactory));
innerProviders.Add(new ILoggerBindingProvider(loggerFactory));
innerProviders.Add(new ILoggerBindingProvider(_loggerFactory));
ContextAccessor<IBindingProvider> bindingProviderAccessor = new ContextAccessor<IBindingProvider>();
innerProviders.Add(new RuntimeBindingProvider(bindingProviderAccessor));

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

@ -0,0 +1,84 @@
// Copyright (c) .NET Foundation. All rights reserved.
// Licensed under the MIT License. See License.txt in the project root for license information.
using System.Collections.Generic;
using Microsoft.Azure.WebJobs.Host.Blobs;
using Microsoft.Azure.WebJobs.Host.Blobs.Triggers;
using Microsoft.Azure.WebJobs.Host.Executors;
using Microsoft.Azure.WebJobs.Host.Listeners;
using Microsoft.Azure.WebJobs.Host.Queues;
using Microsoft.Azure.WebJobs.Host.Queues.Triggers;
using Microsoft.Azure.WebJobs.Host.Timers;
using Microsoft.Azure.WebJobs.Host.Triggers;
using Microsoft.Extensions.Logging;
using Microsoft.Extensions.Options;
namespace Microsoft.Azure.WebJobs.Host.Indexers
{
internal class DefaultTriggerBindingFactory
{
private readonly INameResolver _nameResolver;
private readonly IStorageAccountProvider _storageAccountProvider;
private readonly IExtensionTypeLocator _extensionTypeLocator;
private readonly IHostIdProvider _hostIdProvider;
private readonly IOptions<JobHostQueuesOptions> _queueConfiguration;
private readonly IOptions<JobHostBlobsOptions> _blobsConfiguration;
private readonly IWebJobsExceptionHandler _exceptionHandler;
private readonly IContextSetter<IMessageEnqueuedWatcher> _messageEnqueuedWatcherSetter;
private readonly IContextSetter<IBlobWrittenWatcher> _blobWrittenWatcherSetter;
private readonly ISharedContextProvider _sharedContextProvider;
private readonly IExtensionRegistry _extensions;
private readonly SingletonManager _singletonManager;
private readonly ILoggerFactory _loggerFactory;
public DefaultTriggerBindingFactory(INameResolver nameResolver,
IStorageAccountProvider storageAccountProvider,
IExtensionTypeLocator extensionTypeLocator,
IHostIdProvider hostIdProvider,
IOptions<JobHostQueuesOptions> queueConfiguration,
IOptions<JobHostBlobsOptions> blobsConfiguration,
IWebJobsExceptionHandler exceptionHandler,
IContextSetter<IMessageEnqueuedWatcher> messageEnqueuedWatcherSetter,
IContextSetter<IBlobWrittenWatcher> blobWrittenWatcherSetter,
ISharedContextProvider sharedContextProvider,
IExtensionRegistry extensions,
SingletonManager singletonManager,
ILoggerFactory loggerFactory)
{
_nameResolver = nameResolver ?? throw new System.ArgumentNullException(nameof(nameResolver));
_storageAccountProvider = storageAccountProvider ?? throw new System.ArgumentNullException(nameof(storageAccountProvider));
_extensionTypeLocator = extensionTypeLocator ?? throw new System.ArgumentNullException(nameof(extensionTypeLocator));
_hostIdProvider = hostIdProvider ?? throw new System.ArgumentNullException(nameof(hostIdProvider));
_queueConfiguration = queueConfiguration ?? throw new System.ArgumentNullException(nameof(queueConfiguration));
_blobsConfiguration = blobsConfiguration ?? throw new System.ArgumentNullException(nameof(blobsConfiguration));
_exceptionHandler = exceptionHandler ?? throw new System.ArgumentNullException(nameof(exceptionHandler));
_messageEnqueuedWatcherSetter = messageEnqueuedWatcherSetter ?? throw new System.ArgumentNullException(nameof(messageEnqueuedWatcherSetter));
_blobWrittenWatcherSetter = blobWrittenWatcherSetter ?? throw new System.ArgumentNullException(nameof(blobWrittenWatcherSetter));
_sharedContextProvider = sharedContextProvider ?? throw new System.ArgumentNullException(nameof(sharedContextProvider));
_extensions = extensions ?? throw new System.ArgumentNullException(nameof(extensions));
_singletonManager = singletonManager ?? throw new System.ArgumentNullException(nameof(singletonManager));
_loggerFactory = loggerFactory ?? throw new System.ArgumentNullException(nameof(loggerFactory));
}
public ITriggerBindingProvider Create()
{
var innerProviders = new List<ITriggerBindingProvider>
{
new QueueTriggerAttributeBindingProvider(_nameResolver, _storageAccountProvider,
_queueConfiguration.Value, _exceptionHandler, _messageEnqueuedWatcherSetter,
_sharedContextProvider, _loggerFactory),
new BlobTriggerAttributeBindingProvider(_nameResolver, _storageAccountProvider, _extensionTypeLocator,
_hostIdProvider, _queueConfiguration.Value, _blobsConfiguration.Value, _exceptionHandler, _blobWrittenWatcherSetter,
_messageEnqueuedWatcherSetter, _sharedContextProvider, _singletonManager, _loggerFactory)
};
// add any registered extension binding providers
foreach (ITriggerBindingProvider provider in _extensions.GetExtensions(typeof(ITriggerBindingProvider)))
{
innerProviders.Add(provider);
}
return new CompositeTriggerBindingProvider(innerProviders);
}
}
}

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

@ -1,50 +0,0 @@
// Copyright (c) .NET Foundation. All rights reserved.
// Licensed under the MIT License. See License.txt in the project root for license information.
using System.Collections.Generic;
using Microsoft.Azure.WebJobs.Host.Blobs;
using Microsoft.Azure.WebJobs.Host.Blobs.Triggers;
using Microsoft.Azure.WebJobs.Host.Executors;
using Microsoft.Azure.WebJobs.Host.Listeners;
using Microsoft.Azure.WebJobs.Host.Queues;
using Microsoft.Azure.WebJobs.Host.Queues.Triggers;
using Microsoft.Azure.WebJobs.Host.Timers;
using Microsoft.Azure.WebJobs.Host.Triggers;
using Microsoft.Extensions.Logging;
namespace Microsoft.Azure.WebJobs.Host.Indexers
{
internal static class DefaultTriggerBindingProvider
{
public static ITriggerBindingProvider Create(INameResolver nameResolver,
IStorageAccountProvider storageAccountProvider,
IExtensionTypeLocator extensionTypeLocator,
IHostIdProvider hostIdProvider,
IQueueConfiguration queueConfiguration,
JobHostBlobsConfiguration blobsConfiguration,
IWebJobsExceptionHandler exceptionHandler,
IContextSetter<IMessageEnqueuedWatcher> messageEnqueuedWatcherSetter,
IContextSetter<IBlobWrittenWatcher> blobWrittenWatcherSetter,
ISharedContextProvider sharedContextProvider,
IExtensionRegistry extensions,
SingletonManager singletonManager,
ILoggerFactory loggerFactory)
{
List<ITriggerBindingProvider> innerProviders = new List<ITriggerBindingProvider>();
innerProviders.Add(new QueueTriggerAttributeBindingProvider(nameResolver, storageAccountProvider,
queueConfiguration, exceptionHandler, messageEnqueuedWatcherSetter,
sharedContextProvider, loggerFactory));
innerProviders.Add(new BlobTriggerAttributeBindingProvider(nameResolver, storageAccountProvider, extensionTypeLocator,
hostIdProvider, queueConfiguration, blobsConfiguration, exceptionHandler, blobWrittenWatcherSetter,
messageEnqueuedWatcherSetter, sharedContextProvider, singletonManager, loggerFactory));
// add any registered extension binding providers
foreach (ITriggerBindingProvider provider in extensions.GetExtensions(typeof(ITriggerBindingProvider)))
{
innerProviders.Add(provider);
}
return new CompositeTriggerBindingProvider(innerProviders);
}
}
}

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

@ -6,6 +6,8 @@ using System.Collections.Generic;
using System.IO;
using System.Linq;
using System.Reflection;
using Microsoft.Azure.WebJobs.Logging;
using Microsoft.Extensions.Logging;
namespace Microsoft.Azure.WebJobs.Host.Indexers
{
@ -14,22 +16,13 @@ namespace Microsoft.Azure.WebJobs.Host.Indexers
{
private static readonly string WebJobsAssemblyName = AssemblyNameCache.GetName(typeof(TableAttribute).Assembly).Name;
private readonly TextWriter _log;
private readonly ILogger _logger;
private readonly IExtensionRegistry _extensions;
public DefaultTypeLocator(TextWriter log, IExtensionRegistry extensions)
public DefaultTypeLocator(TextWriter log, IExtensionRegistry extensions, ILoggerFactory loggerFactory)
{
if (log == null)
{
throw new ArgumentNullException("log");
}
if (extensions == null)
{
throw new ArgumentNullException("extensions");
}
_log = log;
_extensions = extensions;
_extensions = extensions ?? throw new ArgumentNullException(nameof(extensions));
_logger = loggerFactory.CreateLogger(LogCategories.Startup);
}
// Helper to filter out assemblies that don't reference the SDK or
@ -42,7 +35,7 @@ namespace Microsoft.Azure.WebJobs.Host.Indexers
return false;
}
AssemblyName[] referencedAssemblyNames = assembly.GetReferencedAssemblies();
AssemblyName[] referencedAssemblyNames = assembly.GetReferencedAssemblies();
foreach (var referencedAssemblyName in referencedAssemblyNames)
{
if (String.Equals(referencedAssemblyName.Name, WebJobsAssemblyName, StringComparison.OrdinalIgnoreCase))
@ -139,8 +132,8 @@ namespace Microsoft.Azure.WebJobs.Host.Indexers
}
catch (Exception ex)
{
_log.WriteLine("Warning: Failed to get types from assembly: {0}", assembly.FullName);
_log.WriteLine("Exception message: {0}", ex.ToString());
_logger.LogInformation("Warning: Failed to get types from assembly: {0}", assembly.FullName);
_logger.LogInformation("Exception message: {0}", ex.ToString());
}
return types;

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

@ -10,6 +10,7 @@ using Microsoft.Azure.WebJobs.Host.Dispatch;
using Microsoft.Azure.WebJobs.Host.Executors;
using Microsoft.Azure.WebJobs.Host.Triggers;
using Microsoft.Extensions.Logging;
using Microsoft.Extensions.Options;
namespace Microsoft.Azure.WebJobs.Host.Indexers
{
@ -17,7 +18,7 @@ namespace Microsoft.Azure.WebJobs.Host.Indexers
{
private readonly ITypeLocator _typeLocator;
private readonly ITriggerBindingProvider _triggerBindingProvider;
private readonly IBindingProvider _bindingProvider;
private readonly IBindingProviderFactory _bindingProviderFactory;
private readonly IJobActivator _activator;
private readonly IFunctionExecutor _executor;
private readonly IExtensionRegistry _extensions;
@ -31,20 +32,20 @@ namespace Microsoft.Azure.WebJobs.Host.Indexers
public FunctionIndexProvider(ITypeLocator typeLocator,
ITriggerBindingProvider triggerBindingProvider,
IBindingProvider bindingProvider,
IBindingProviderFactory bindingProviderFactory,
IJobActivator activator,
IFunctionExecutor executor,
IExtensionRegistry extensions,
SingletonManager singletonManager,
ILoggerFactory loggerFactory,
SharedQueueHandler sharedQueue,
TimeoutAttribute defaultTimeout,
bool allowPartialHostStartup = false)
bool allowPartialHostStartup = false,
IOptions<JobHostFunctionTimeoutOptions> timeoutOptions)
{
_typeLocator = typeLocator ?? throw new ArgumentNullException(nameof(typeLocator));
_triggerBindingProvider = triggerBindingProvider ?? throw new ArgumentNullException(nameof(triggerBindingProvider));
_bindingProvider = bindingProvider ?? throw new ArgumentNullException(nameof(bindingProvider));
_bindingProviderFactory = bindingProviderFactory ?? throw new ArgumentNullException(nameof(bindingProviderFactory));
_activator = activator ?? throw new ArgumentNullException(nameof(activator));
_executor = executor ?? throw new ArgumentNullException(nameof(executor));
_extensions = extensions ?? throw new ArgumentNullException(nameof(extensions));
@ -52,8 +53,7 @@ namespace Microsoft.Azure.WebJobs.Host.Indexers
_sharedQueue = sharedQueue ?? throw new ArgumentNullException(nameof(sharedQueue));
_loggerFactory = loggerFactory;
_defaultTimeout = defaultTimeout;
_allowPartialHostStartup = allowPartialHostStartup;
_defaultTimeout = timeoutOptions.Value.ToAttribute();
}
public async Task<IFunctionIndex> GetAsync(CancellationToken cancellationToken)
@ -69,7 +69,8 @@ namespace Microsoft.Azure.WebJobs.Host.Indexers
private async Task<IFunctionIndex> CreateAsync(CancellationToken cancellationToken)
{
FunctionIndex index = new FunctionIndex();
FunctionIndexer indexer = new FunctionIndexer(_triggerBindingProvider, _bindingProvider, _activator, _executor, _extensions, _singletonManager, _loggerFactory, null, _sharedQueue, allowPartialHostStartup: _allowPartialHostStartup);
IBindingProvider bindingProvider = _bindingProviderFactory.Create();
FunctionIndexer indexer = new FunctionIndexer(_triggerBindingProvider, bindingProvider, _activator, _executor, _extensions, _singletonManager, _loggerFactory, null, _sharedQueue);
IReadOnlyList<Type> types = _typeLocator.GetTypes();
foreach (Type type in types)

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

@ -7,7 +7,7 @@ using Microsoft.Azure.WebJobs.Host.Protocols;
namespace Microsoft.Azure.WebJobs.Host.Indexers
{
internal interface IFunctionDefinition
public interface IFunctionDefinition
{
FunctionDescriptor Descriptor { get; }

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

@ -5,7 +5,7 @@ using System.Reflection;
namespace Microsoft.Azure.WebJobs.Host.Indexers
{
internal interface IFunctionIndexLookup
public interface IFunctionIndexLookup
{
IFunctionDefinition Lookup(string functionId);

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

@ -9,13 +9,13 @@ using System.Reflection;
using System.Threading;
using System.Threading.Tasks;
using Microsoft.Azure.WebJobs.Host;
using Microsoft.Azure.WebJobs.Host.Config;
using Microsoft.Azure.WebJobs.Host.Executors;
using Microsoft.Azure.WebJobs.Host.Indexers;
using Microsoft.Azure.WebJobs.Host.Listeners;
using Microsoft.Azure.WebJobs.Host.Protocols;
using Microsoft.Azure.WebJobs.Logging;
using Microsoft.Extensions.Logging;
using Microsoft.Extensions.Options;
using Microsoft.WindowsAzure.Storage;
namespace Microsoft.Azure.WebJobs
@ -24,14 +24,15 @@ namespace Microsoft.Azure.WebJobs
/// A <see cref="JobHost"/> is the execution container for jobs. Once started, the
/// <see cref="JobHost"/> will manage and run job functions when they are triggered.
/// </summary>
public class JobHost : IDisposable, IJobInvoker
public class JobHost : IJobHost, IDisposable, IJobInvoker
{
private const int StateNotStarted = 0;
private const int StateStarting = 1;
private const int StateStarted = 2;
private const int StateStoppingOrStopped = 3;
private readonly JobHostConfiguration _config;
private readonly JobHostOptions _options;
private readonly IJobHostContextFactory _jobHostContextFactory;
private readonly CancellationTokenSource _shutdownTokenSource;
private readonly WebJobsShutdownWatcher _shutdownWatcher;
private readonly CancellationTokenSource _stoppingTokenSource;
@ -44,29 +45,15 @@ namespace Microsoft.Azure.WebJobs
// Points to a completed task after initialization.
private Task _initializationRunning = null;
// These are services that are accessible without starting the execution container.
// They include the initial set of JobHostConfiguration services as well as
// additional services created.
private ServiceProviderWrapper _services;
private int _state;
private Task _stopTask;
private bool _disposed;
// Common lock to protect fields.
// Common lock to protect fields.
private object _lock = new object();
private ILogger _logger;
/// <summary>
/// Initializes a new instance of the <see cref="JobHost"/> class, using a Microsoft Azure Storage connection
/// string located in the connectionStrings section of the configuration file or in environment variables.
/// </summary>
public JobHost()
: this(new JobHostConfiguration())
{
}
static JobHost()
{
// add webjobs to user agent for all storage calls
@ -82,14 +69,15 @@ namespace Microsoft.Azure.WebJobs
/// Initializes a new instance of the <see cref="JobHost"/> class using the configuration provided.
/// </summary>
/// <param name="configuration">The job host configuration.</param>
public JobHost(JobHostConfiguration configuration)
public JobHost(IOptions<JobHostOptions> options, IJobHostContextFactory jobHostContextFactory)
{
if (configuration == null)
if (options == null)
{
throw new ArgumentNullException("configuration");
throw new ArgumentNullException(nameof(options));
}
_config = configuration;
_options = options.Value;
_jobHostContextFactory = jobHostContextFactory;
_shutdownTokenSource = new CancellationTokenSource();
_shutdownWatcher = WebJobsShutdownWatcher.Create(_shutdownTokenSource);
_stoppingTokenSource = CancellationTokenSource.CreateLinkedTokenSource(_shutdownTokenSource.Token);
@ -174,11 +162,7 @@ namespace Microsoft.Azure.WebJobs
await _listener.StopAsync(cancellationToken);
// Flush remaining logs
var functionEventCollector = _context.FunctionEventCollector;
if (functionEventCollector != null)
{
await functionEventCollector.FlushAsync(cancellationToken);
}
await _context.EventCollector.FlushAsync(cancellationToken);
string msg = "Job host stopped";
_logger?.LogInformation(msg);
@ -422,9 +406,7 @@ namespace Microsoft.Azure.WebJobs
{
try
{
InitializeServices();
var context = await _config.CreateJobHostContextAsync(_services, this, _shutdownTokenSource.Token, cancellationToken);
var context = await _jobHostContextFactory.Create(_shutdownTokenSource.Token, cancellationToken);
// must call this BEFORE setting the results below
// since listener startup is blocking on those members
@ -456,32 +438,5 @@ namespace Microsoft.Azure.WebJobs
protected virtual void OnHostStarted()
{
}
// Ensure the static services are initialized.
// These are derived from the underlying JobHostConfiguration.
// Caller ensures this is single threaded.
private void InitializeServices()
{
if (this._services != null)
{
return; // already Created
}
var services = this._config.CreateStaticServices();
_services = services;
}
/// <summary>
/// Get set of services.
/// </summary>
public IServiceProvider Services
{
get
{
InitializeServices();
return _services;
}
}
}
}

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

@ -1,445 +0,0 @@
// Copyright (c) .NET Foundation. All rights reserved.
// Licensed under the MIT License. See License.txt in the project root for license information.
using System;
using System.Collections.Concurrent;
using System.Diagnostics;
using Microsoft.Azure.WebJobs.Host;
using Microsoft.Azure.WebJobs.Host.Bindings;
using Microsoft.Azure.WebJobs.Host.Config;
using Microsoft.Azure.WebJobs.Host.Executors;
using Microsoft.Azure.WebJobs.Host.Indexers;
using Microsoft.Azure.WebJobs.Host.Loggers;
using Microsoft.Azure.WebJobs.Host.Queues;
using Microsoft.Azure.WebJobs.Host.Timers;
using Microsoft.Azure.WebJobs.Logging;
using Microsoft.Extensions.Logging;
using Newtonsoft.Json.Linq;
using Microsoft.Extensions.Configuration;
using Microsoft.WindowsAzure.Storage.Blob;
namespace Microsoft.Azure.WebJobs
{
/// <summary>
/// Represents the configuration settings for a <see cref="JobHost"/>.
/// </summary>
public sealed class JobHostConfiguration : IServiceProvider
{
private static readonly IConsoleProvider ConsoleProvider = new DefaultConsoleProvider();
private readonly DefaultStorageAccountProvider _storageAccountProvider;
private readonly JobHostQueuesConfiguration _queueConfiguration = new JobHostQueuesConfiguration();
private readonly JobHostBlobsConfiguration _blobsConfiguration = new JobHostBlobsConfiguration();
private readonly ConcurrentDictionary<Type, object> _services = new ConcurrentDictionary<Type, object>();
private string _hostId;
/// <summary>
/// Initializes a new instance of the <see cref="JobHostConfiguration"/> class.
/// </summary>
public JobHostConfiguration()
: this(null, null)
{
}
/// <summary>
/// Initializes a new instance of the <see cref="JobHostConfiguration"/> class, using the
/// specified connection string for both reading and writing data as well as Dashboard logging.
/// </summary>
/// <param name="dashboardAndStorageConnectionString">The Azure Storage connection string to use.
/// </param>
public JobHostConfiguration(string dashboardAndStorageConnectionString)
: this(dashboardAndStorageConnectionString, null)
{
}
/// <summary>
/// Initializes a new instance of the <see cref="JobHostConfiguration"/> class.
/// </summary>
/// <param name="configuration">A configuration object that will be used as the source of application settings.</param>
/// </param>
public JobHostConfiguration(IConfiguration configuration)
: this(null, configuration)
{
}
/// <summary>
/// Initializes a new instance of the <see cref="JobHostConfiguration"/> class, using the
/// specified connection string for both reading and writing data as well as Dashboard logging.
/// </summary>
/// <param name="dashboardAndStorageConnectionString">The Azure Storage connection string to use.
/// <param name="configuration">A configuration object that will be used as the source of application settings.</param>
/// </param>
public JobHostConfiguration(string dashboardAndStorageConnectionString, IConfiguration configuration)
{
if (configuration != null)
{
ConfigurationUtility.SetConfigurationFactory(() => configuration);
}
if (!string.IsNullOrEmpty(dashboardAndStorageConnectionString))
{
_storageAccountProvider = new DefaultStorageAccountProvider(this, dashboardAndStorageConnectionString);
}
else
{
_storageAccountProvider = new DefaultStorageAccountProvider(this);
}
var sasBlobContainer = _storageAccountProvider.InternalSasStorage;
if (sasBlobContainer != null)
{
var uri = new Uri(sasBlobContainer);
var sdkContainer = new CloudBlobContainer(uri);
this.InternalStorageConfiguration = new JobHostInternalStorageConfiguration
{
InternalContainer = sdkContainer
};
}
Singleton = new SingletonConfiguration();
Aggregator = new FunctionResultAggregatorConfiguration();
// add our built in services here
IExtensionRegistry extensions = new DefaultExtensionRegistry();
ITypeLocator typeLocator = new DefaultTypeLocator(ConsoleProvider.Out, extensions);
IConverterManager converterManager = new ConverterManager();
IWebJobsExceptionHandler exceptionHandler = new WebJobsExceptionHandler();
AddService<IQueueConfiguration>(_queueConfiguration);
AddService<IConsoleProvider>(ConsoleProvider);
AddService<ILoggerFactory>(new LoggerFactory());
AddService<IStorageAccountProvider>(_storageAccountProvider);
AddService<IExtensionRegistry>(extensions);
AddService<StorageClientFactory>(new StorageClientFactory());
AddService<INameResolver>(new DefaultNameResolver());
AddService<IJobActivator>(DefaultJobActivator.Instance);
AddService<ITypeLocator>(typeLocator);
AddService<IConverterManager>(converterManager);
AddService<IWebJobsExceptionHandler>(exceptionHandler);
AddService<IFunctionResultAggregatorFactory>(new FunctionResultAggregatorFactory());
string value = ConfigurationUtility.GetSetting(Host.Constants.EnvironmentSettingName);
IsDevelopment = string.Compare(Host.Constants.DevelopmentEnvironmentValue, value, StringComparison.OrdinalIgnoreCase) == 0;
}
/// <summary>
/// Gets or sets a value indicating whether the host should be able to start partially
/// when some functions are in error. The default is false.
/// </summary>
/// <remarks>
/// Normally when a function encounters an indexing error or its listener fails to start
/// the error will propagate and the host will not start. However, with this option set,
/// the host will be allowed to start in "partial" mode:
/// - Functions without errors will run normally
/// - Functions with indexing errors will not be running
/// - Functions listener startup failures will be retried in the background
/// until they start.
/// </remarks>
public bool AllowPartialHostStartup { get; set; }
/// <summary>
/// Gets a value indicating whether the <see cref="JobHost"/> is running in a Development environment.
/// You can use this property in conjunction with <see cref="UseDevelopmentSettings"/> to default
/// configuration settings to values optimized for local development.
/// <Remarks>
/// The environment is determined by the value of the "AzureWebJobsEnv" environment variable. When this
/// is set to "Development", this property will return true.
/// </Remarks>
/// </summary>
public bool IsDevelopment { get; private set; }
/// <summary>
/// Returns true if <see cref="UseDevelopmentSettings"/> has been called on this instance.
/// </summary>
internal bool UsingDevelopmentSettings { get; set; }
/// <summary>Gets or sets the host ID.</summary>
/// <remarks>
/// <para>
/// All host instances that share the same host ID must be homogeneous. For example, they must use the same
/// storage accounts and have the same list of functions. Host instances with the same host ID will scale out
/// and share handling of work such as BlobTrigger and run from dashboard processing and providing a heartbeat
/// to the dashboard indicating that an instance of the host running.
/// </para>
/// <para>
/// If this value is <see langword="null"/> on startup, a host ID will automatically be generated based on the assembly
/// name of the first function, and that host ID will be made available via this property after the host has fully started.
/// </para>
/// <para>
/// If non-homogeneous host instances share the same first function assembly,
/// this property must be set explicitly; otherwise, the host instances will incorrectly try to share work as if
/// they were homogeneous.
/// </para>
/// </remarks>
public string HostId
{
get
{
return _hostId;
}
set
{
if (value != null && !HostIdValidator.IsValid(value))
{
throw new ArgumentException(HostIdValidator.ValidationMessage, "value");
}
_hostId = value;
}
}
/// <summary>Gets or sets the job activator.</summary>
/// <remarks>The job activator creates instances of job classes when calling instance methods.</remarks>
public IJobActivator JobActivator
{
get
{
return GetService<IJobActivator>();
}
set
{
if (value == null)
{
throw new ArgumentNullException("value");
}
AddService<IJobActivator>(value);
}
}
/// <summary>
/// Gets or sets the Azure Storage connection string used for logging and diagnostics.
/// </summary>
public string DashboardConnectionString
{
get { return _storageAccountProvider.DashboardConnectionString; }
set { _storageAccountProvider.DashboardConnectionString = value; }
}
/// <summary>
/// Gets or sets the Azure Storage connection string used for reading and writing data.
/// </summary>
public string StorageConnectionString
{
get { return _storageAccountProvider.StorageConnectionString; }
set { _storageAccountProvider.StorageConnectionString = value; }
}
/// <summary>
/// Gets or sets a the storage configuration that the runtime needs for its internal operations.
/// </summary>
public JobHostInternalStorageConfiguration InternalStorageConfiguration { get; set; }
/// <summary>Gets or sets the type locator.</summary>
public ITypeLocator TypeLocator
{
get
{
return GetService<ITypeLocator>();
}
set
{
if (value == null)
{
throw new ArgumentNullException("value");
}
AddService<ITypeLocator>(value);
}
}
/// <summary>
/// Gets or sets the name resolver used during indexing.
/// </summary>
public INameResolver NameResolver
{
get
{
return GetService<INameResolver>();
}
set
{
if (value == null)
{
throw new ArgumentNullException("value");
}
AddService<INameResolver>(value);
}
}
/// <summary>
/// Get the converter manager, which can be used to register additional conversions for
/// customizing model binding.
/// </summary>
internal IConverterManager ConverterManager
{
get
{
return GetService<IConverterManager>();
}
}
/// <summary>
/// Gets the configuration used by <see cref="QueueTriggerAttribute"/>.
/// </summary>
public JobHostQueuesConfiguration Queues
{
get { return _queueConfiguration; }
}
/// <summary>
/// Gets the configuration used by <see cref="BlobTriggerAttribute"/>.
/// </summary>
public JobHostBlobsConfiguration Blobs
{
get { return _blobsConfiguration; }
}
/// <summary>
/// Gets the configuration used by <see cref="SingletonAttribute"/>.
/// </summary>
public SingletonConfiguration Singleton
{
get;
private set;
}
/// <summary>
/// Gets the configuration used by the logging aggregator.
/// </summary>
public FunctionResultAggregatorConfiguration Aggregator
{
get;
private set;
}
/// <summary>
/// Gets or sets the <see cref="ILoggerFactory"/>.
/// </summary>
public ILoggerFactory LoggerFactory
{
get
{
return GetService<ILoggerFactory>();
}
set
{
AddService<ILoggerFactory>(value);
}
}
/// <summary>
/// get host-level metadata which the extension may read to do configuration.
/// </summary>
[Obsolete("Not ready for public consumption.")]
public JObject HostConfigMetadata { get; set; }
/// <summary>
/// Gets or sets the <see cref="Host.StorageClientFactory"/> that will be used to create
/// Azure Storage clients.
/// </summary>
public StorageClientFactory StorageClientFactory
{
get
{
return GetService<StorageClientFactory>();
}
set
{
if (value == null)
{
throw new ArgumentNullException("value");
}
AddService<StorageClientFactory>(value);
}
}
/// <summary>
/// Default timeout configuration to apply to all functions.
/// If <see cref="TimeoutAttribute"/> is explicitly applied to a function, the values specified by the attribute will take precedence
/// </summary>
public JobHostFunctionTimeoutConfiguration FunctionTimeout { get; set; }
/// <summary>
/// Configures various configuration settings on this <see cref="JobHostConfiguration"/> to
/// optimize for local development.
/// </summary>
public void UseDevelopmentSettings()
{
Queues.MaxPollingInterval = TimeSpan.FromSeconds(2);
Singleton.ListenerLockPeriod = TimeSpan.FromSeconds(15);
UsingDevelopmentSettings = true;
}
/// <summary>Gets the service object of the specified type.</summary>
/// <param name="serviceType">The type of service to get.</param>
/// <returns>
/// A service of the specified type, if one is available; otherwise, <see langword="null"/>.
/// </returns>
public object GetService(Type serviceType)
{
if (serviceType == null)
{
throw new ArgumentNullException("serviceType");
}
object service = null;
_services.TryGetValue(serviceType, out service);
return service;
}
/// <summary>
/// Gets the service object of the specified type.
/// </summary>
/// <typeparam name="TService">The type of service object to get.</typeparam>
/// <returns>A service of the specified type, if one is available; otherwise, <see langword="null"/>.</returns>
public TService GetService<TService>()
{
return (TService)this.GetService(typeof(TService));
}
/// <summary>
/// Adds the specified service instance, replacing any existing service.
/// </summary>
/// <param name="serviceType">The service type</param>
/// <param name="serviceInstance">The service instance</param>
public void AddService(Type serviceType, object serviceInstance)
{
if (serviceType == null)
{
throw new ArgumentNullException("serviceType");
}
if (!serviceType.IsAssignableFrom(serviceInstance.GetType()))
{
throw new ArgumentOutOfRangeException("serviceInstance");
}
_services.AddOrUpdate(serviceType, serviceInstance, (key, existingValue) =>
{
// always replace existing values
return serviceInstance;
});
}
/// <summary>
/// Adds the specified service instance, replacing any existing service.
/// </summary>
/// <typeparam name="TService">The service type</typeparam>
/// <param name="serviceInstance">The service instance</param>
public void AddService<TService>(TService serviceInstance)
{
AddService(typeof(TService), serviceInstance);
}
/// <summary>
/// Add an extension to register new binding attributes and converters.
/// </summary>
/// <param name="extension"></param>
public void AddExtension(IExtensionConfigProvider extension)
{
var exts = this.GetExtensions();
exts.RegisterExtension<IExtensionConfigProvider>(extension);
}
}
}

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

@ -10,7 +10,7 @@ namespace Microsoft.Azure.WebJobs.Host.Listeners
/// Interface defining methods used to create <see cref="IListener"/>s for
/// trigger parameter bindings.
/// </summary>
internal interface IListenerFactory
public interface IListenerFactory
{
/// <summary>
/// Creates a listener.

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

@ -4,9 +4,8 @@
using System.Collections.Generic;
using System.Threading;
using System.Threading.Tasks;
using Microsoft.Azure.WebJobs.Host.Loggers;
namespace Microsoft.Azure.WebJobs.Host.Executors
namespace Microsoft.Azure.WebJobs.Host.Loggers
{
internal class CompositeFunctionEventCollector : IAsyncCollector<FunctionInstanceLogEntry>
{

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

@ -0,0 +1,58 @@
// Copyright (c) .NET Foundation. All rights reserved.
// Licensed under the MIT License. See License.txt in the project root for license information.
using System;
using System.Collections.Generic;
using System.Linq;
using System.Threading;
using System.Threading.Tasks;
using Microsoft.Azure.WebJobs.Host.Loggers;
namespace Microsoft.Azure.WebJobs.Logging
{
internal class EventCollectorFactory : IEventCollectorFactory
{
private readonly IEnumerable<IEventCollectorProvider> _providers;
// This allows us to take all registered providers and wrap them in a composite, if necessary.
public EventCollectorFactory(IEnumerable<IEventCollectorProvider> providers)
{
_providers = providers ?? throw new ArgumentNullException(nameof(providers));
}
public IAsyncCollector<FunctionInstanceLogEntry> Create()
{
var collectors = new List<IAsyncCollector<FunctionInstanceLogEntry>>();
foreach (IEventCollectorProvider provider in _providers)
{
var collector = provider.Create();
if (collector != null)
{
collectors.Add(collector);
}
}
// If we have no providers, return a no-op collector
if (!collectors.Any())
{
return new NullFunctionEventCollector();
}
return new CompositeFunctionEventCollector(collectors.ToArray());
}
private class NullFunctionEventCollector : IAsyncCollector<FunctionInstanceLogEntry>
{
public Task AddAsync(FunctionInstanceLogEntry item, CancellationToken cancellationToken = default(CancellationToken))
{
return Task.CompletedTask;
}
public Task FlushAsync(CancellationToken cancellationToken = default(CancellationToken))
{
return Task.CompletedTask;
}
}
}
}

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

@ -1,12 +1,10 @@
// Copyright (c) .NET Foundation. All rights reserved.
// Licensed under the MIT License. See License.txt in the project root for license information.
using System.IO;
namespace Microsoft.Azure.WebJobs.Host.Loggers
{
internal interface IConsoleProvider
public interface IEventCollectorFactory
{
TextWriter Out { get; }
IAsyncCollector<FunctionInstanceLogEntry> Create();
}
}

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

@ -1,16 +1,10 @@
// Copyright (c) .NET Foundation. All rights reserved.
// Licensed under the MIT License. See License.txt in the project root for license information.
using System;
using System.IO;
namespace Microsoft.Azure.WebJobs.Host.Loggers
{
internal class DefaultConsoleProvider : IConsoleProvider
public interface IEventCollectorProvider
{
public TextWriter Out
{
get { return Console.Out; }
}
IAsyncCollector<FunctionInstanceLogEntry> Create();
}
}

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

@ -1,17 +1,32 @@
// Copyright (c) .NET Foundation. All rights reserved.
// Licensed under the MIT License. See License.txt in the project root for license information.
using System;
using Microsoft.Azure.WebJobs.Host.Loggers;
using Microsoft.Extensions.Logging;
using Microsoft.Extensions.Options;
namespace Microsoft.Azure.WebJobs.Logging
{
internal class FunctionResultAggregatorFactory : IFunctionResultAggregatorFactory
internal class FunctionResultAggregatorProvider : IEventCollectorProvider
{
public IAsyncCollector<FunctionInstanceLogEntry> Create(int batchSize, TimeSpan batchTimeout, ILoggerFactory loggerFactory)
protected readonly FunctionResultAggregatorOptions _options;
private readonly ILoggerFactory _loggerFactory;
public FunctionResultAggregatorProvider(IOptions<FunctionResultAggregatorOptions> options, ILoggerFactory loggerFactory)
{
return new FunctionResultAggregator(batchSize, batchTimeout, loggerFactory);
_options = options.Value;
_loggerFactory = loggerFactory;
}
public virtual IAsyncCollector<FunctionInstanceLogEntry> Create()
{
// If the pieces aren't configured, don't create an aggregator.
if (!_options.IsEnabled)
{
return null;
}
return new FunctionResultAggregator(_options.BatchSize, _options.FlushTimeout, _loggerFactory);
}
}
}

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

@ -9,7 +9,7 @@ namespace Microsoft.Azure.WebJobs.Logging
/// <summary>
/// Configuration options for function result aggregation.
/// </summary>
public class FunctionResultAggregatorConfiguration
public class FunctionResultAggregatorOptions
{
private int _batchSize;
private TimeSpan _flushTimeout;
@ -23,7 +23,7 @@ namespace Microsoft.Azure.WebJobs.Logging
/// <summary>
/// Constructs a new instance.
/// </summary>
public FunctionResultAggregatorConfiguration()
public FunctionResultAggregatorOptions()
{
_batchSize = DefaultBatchSize;
_flushTimeout = DefaultFlushTimeout;

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

@ -1,14 +0,0 @@
// Copyright (c) .NET Foundation. All rights reserved.
// Licensed under the MIT License. See License.txt in the project root for license information.
using System;
using Microsoft.Azure.WebJobs.Host.Loggers;
using Microsoft.Extensions.Logging;
namespace Microsoft.Azure.WebJobs.Logging
{
internal interface IFunctionResultAggregatorFactory
{
IAsyncCollector<FunctionInstanceLogEntry> Create(int batchSize, TimeSpan batchTimeout, ILoggerFactory loggerFactory);
}
}

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

@ -0,0 +1,42 @@
// Copyright (c) .NET Foundation. All rights reserved.
// Licensed under the MIT License. See License.txt in the project root for license information.
using System;
using Microsoft.Azure.WebJobs.Host.Executors;
using Microsoft.Extensions.Logging;
namespace Microsoft.Azure.WebJobs.Host.Loggers
{
internal sealed class LoggerProviderFactory
{
private readonly IStorageAccountProvider _storageAccountProvider;
private readonly ILoggerFactory _loggerFactory;
private readonly bool _hasFastTableHook;
private readonly Lazy<object> _loggerProvider;
public LoggerProviderFactory(IStorageAccountProvider storageAccountProvider,
ILoggerFactory loggerFactory,
IEventCollectorFactory fastLoggerFactory = null)
{
_storageAccountProvider = storageAccountProvider;
_loggerFactory = loggerFactory;
_hasFastTableHook = fastLoggerFactory != null;
_loggerProvider = new Lazy<object>(CreateLoggerProvider);
}
private object CreateLoggerProvider()
{
bool noDashboardStorage = _storageAccountProvider.DashboardConnectionString == null;
if (_hasFastTableHook && noDashboardStorage)
{
return new FastTableLoggerProvider(_loggerFactory);
}
return new DefaultLoggerProvider(_storageAccountProvider, _loggerFactory);
}
public T GetLoggerProvider<T>() where T : class => _loggerProvider.Value as T;
}
}

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

@ -21,8 +21,13 @@ namespace Microsoft.Azure.WebJobs.Host.Queues.Bindings
// This is fundemantentally an IAsyncCollector<IStorageQueueMessage>
internal class QueueExtension : IExtensionConfigProvider
{
public QueueExtension()
private readonly IContextGetter<IMessageEnqueuedWatcher> _contextGetter;
private readonly IStorageAccountProvider _storageAccountProvider;
public QueueExtension(IStorageAccountProvider storageAccountProvider, IContextGetter<IMessageEnqueuedWatcher> contextGetter)
{
_contextGetter = contextGetter;
_storageAccountProvider = storageAccountProvider;
}
public void Initialize(ExtensionConfigContext context)
@ -33,7 +38,7 @@ namespace Microsoft.Azure.WebJobs.Host.Queues.Bindings
}
var config = new PerHostConfig();
config.Initialize(context);
config.Initialize(context, _storageAccountProvider, _contextGetter);
}
// Multiple JobHost objects may share the same JobHostConfiguration.
@ -46,14 +51,15 @@ namespace Microsoft.Azure.WebJobs.Host.Queues.Bindings
// Optimization where a queue output can directly trigger a queue input.
// This is per-host (not per-config)
private ContextAccessor<IMessageEnqueuedWatcher> _messageEnqueuedWatcherGetter;
private IContextGetter<IMessageEnqueuedWatcher> _messageEnqueuedWatcherGetter;
public void Initialize(ExtensionConfigContext context)
public void Initialize(ExtensionConfigContext context, IStorageAccountProvider storageAccountProvider, IContextGetter<IMessageEnqueuedWatcher> contextGetter)
{
_messageEnqueuedWatcherGetter = context.PerHostServices.GetService<ContextAccessor<IMessageEnqueuedWatcher>>();
_accountProvider = context.Config.GetService<IStorageAccountProvider>();
_accountProvider = storageAccountProvider;
_messageEnqueuedWatcherGetter = contextGetter;
context.ApplyConfig(context.Config.Queues, "queues");
// TODO: FACAVAL replace this with queue options. This should no longer be needed.
//context.ApplyConfig(context.Config.Queues, "queues");
// IStorageQueueMessage is the core testing interface
var binding = context.AddBindingRule<QueueAttribute>();

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

@ -6,7 +6,7 @@ using System;
namespace Microsoft.Azure.WebJobs.Host.Queues
{
/// <summary>
/// The default <see cref="IQueueProcessorFactory"/> implementation used by <see cref="JobHostQueuesConfiguration"/>.
/// The default <see cref="IQueueProcessorFactory"/> implementation used by <see cref="JobHostQueuesOptions"/>.
/// </summary>
internal class DefaultQueueProcessorFactory : IQueueProcessorFactory
{

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

@ -1,22 +0,0 @@
// Copyright (c) .NET Foundation. All rights reserved.
// Licensed under the MIT License. See License.txt in the project root for license information.
using System;
namespace Microsoft.Azure.WebJobs.Host.Queues
{
internal interface IQueueConfiguration
{
int BatchSize { get; }
int NewBatchThreshold { get; }
TimeSpan MaxPollingInterval { get; }
int MaxDequeueCount { get; }
TimeSpan VisibilityTimeout { get; }
IQueueProcessorFactory QueueProcessorFactory { get; }
}
}

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

@ -17,7 +17,7 @@ namespace Microsoft.Azure.WebJobs.Host.Queues.Listeners
internal class HostMessageListenerFactory : IListenerFactory
{
private readonly IStorageQueue _queue;
private readonly IQueueConfiguration _queueConfiguration;
private readonly JobHostQueuesOptions _queueConfiguration;
private readonly IWebJobsExceptionHandler _exceptionHandler;
private readonly ILoggerFactory _loggerFactory;
private readonly IFunctionIndexLookup _functionLookup;
@ -25,7 +25,7 @@ namespace Microsoft.Azure.WebJobs.Host.Queues.Listeners
private readonly IFunctionExecutor _executor;
public HostMessageListenerFactory(IStorageQueue queue,
IQueueConfiguration queueConfiguration,
JobHostQueuesOptions queueOptions,
IWebJobsExceptionHandler exceptionHandler,
ILoggerFactory loggerFactory,
IFunctionIndexLookup functionLookup,
@ -33,7 +33,7 @@ namespace Microsoft.Azure.WebJobs.Host.Queues.Listeners
IFunctionExecutor executor)
{
_queue = queue ?? throw new ArgumentNullException(nameof(queue));
_queueConfiguration = queueConfiguration ?? throw new ArgumentNullException(nameof(queueConfiguration));
_queueConfiguration = queueOptions ?? throw new ArgumentNullException(nameof(queueOptions));
_exceptionHandler = exceptionHandler ?? throw new ArgumentNullException(nameof(exceptionHandler));
_functionLookup = functionLookup ?? throw new ArgumentNullException(nameof(functionLookup));
_functionInstanceLogger = functionInstanceLogger ?? throw new ArgumentNullException(nameof(functionInstanceLogger));
@ -57,7 +57,7 @@ namespace Microsoft.Azure.WebJobs.Host.Queues.Listeners
exceptionHandler: _exceptionHandler,
loggerFactory: _loggerFactory,
sharedWatcher: null,
queueConfiguration: _queueConfiguration,
queueOptions: _queueConfiguration,
maxPollingInterval: maxPollingInterval);
return Task.FromResult(listener);

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

@ -28,7 +28,7 @@ namespace Microsoft.Azure.WebJobs.Host.Queues.Listeners
private readonly IMessageEnqueuedWatcher _sharedWatcher;
private readonly List<Task> _processing = new List<Task>();
private readonly object _stopWaitingTaskSourceLock = new object();
private readonly IQueueConfiguration _queueConfiguration;
private readonly JobHostQueuesOptions _queueuOptions;
private readonly QueueProcessor _queueProcessor;
private readonly TimeSpan _visibilityTimeout;
@ -42,21 +42,21 @@ namespace Microsoft.Azure.WebJobs.Host.Queues.Listeners
IWebJobsExceptionHandler exceptionHandler,
ILoggerFactory loggerFactory,
SharedQueueWatcher sharedWatcher,
IQueueConfiguration queueConfiguration,
JobHostQueuesOptions queueOptions,
QueueProcessor queueProcessor = null,
TimeSpan? maxPollingInterval = null)
{
if (queueConfiguration == null)
if (queueOptions == null)
{
throw new ArgumentNullException("queueConfiguration");
}
if (queueConfiguration.BatchSize <= 0)
if (queueOptions.BatchSize <= 0)
{
throw new ArgumentException("BatchSize must be greater than zero.");
}
if (queueConfiguration.MaxDequeueCount <= 0)
if (queueOptions.MaxDequeueCount <= 0)
{
throw new ArgumentException("MaxDequeueCount must be greater than zero.");
}
@ -66,7 +66,7 @@ namespace Microsoft.Azure.WebJobs.Host.Queues.Listeners
_poisonQueue = poisonQueue;
_triggerExecutor = triggerExecutor;
_exceptionHandler = exceptionHandler;
_queueConfiguration = queueConfiguration;
_queueuOptions = queueOptions;
// if the function runs longer than this, the invisibility will be updated
// on a timer periodically for the duration of the function execution
@ -82,7 +82,7 @@ namespace Microsoft.Azure.WebJobs.Host.Queues.Listeners
EventHandler<PoisonMessageEventArgs> poisonMessageEventHandler = _sharedWatcher != null ? OnMessageAddedToPoisonQueue : (EventHandler<PoisonMessageEventArgs>)null;
_queueProcessor = queueProcessor ?? CreateQueueProcessor(
_queue.SdkObject, _poisonQueue != null ? _poisonQueue.SdkObject : null,
loggerFactory, _queueConfiguration, poisonMessageEventHandler);
loggerFactory, _queueuOptions, poisonMessageEventHandler);
TimeSpan maximumInterval = _queueProcessor.MaxPollingInterval;
if (maxPollingInterval.HasValue && maximumInterval > maxPollingInterval.Value)
@ -305,7 +305,7 @@ namespace Microsoft.Azure.WebJobs.Host.Queues.Listeners
}
internal static QueueProcessor CreateQueueProcessor(CloudQueue queue, CloudQueue poisonQueue, ILoggerFactory loggerFactory,
IQueueConfiguration queueConfig, EventHandler<PoisonMessageEventArgs> poisonQueueMessageAddedHandler)
JobHostQueuesOptions queueConfig, EventHandler<PoisonMessageEventArgs> poisonQueueMessageAddedHandler)
{
QueueProcessorFactoryContext context = new QueueProcessorFactoryContext(queue, loggerFactory, queueConfig, poisonQueue);

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

@ -19,7 +19,7 @@ namespace Microsoft.Azure.WebJobs.Host.Queues.Listeners
private readonly IStorageQueue _queue;
private readonly IStorageQueue _poisonQueue;
private readonly IQueueConfiguration _queueConfiguration;
private readonly JobHostQueuesOptions _queueOptions;
private readonly IWebJobsExceptionHandler _exceptionHandler;
private readonly IContextSetter<IMessageEnqueuedWatcher> _messageEnqueuedWatcherSetter;
private readonly ISharedContextProvider _sharedContextProvider;
@ -27,7 +27,7 @@ namespace Microsoft.Azure.WebJobs.Host.Queues.Listeners
private readonly ITriggeredFunctionExecutor _executor;
public QueueListenerFactory(IStorageQueue queue,
IQueueConfiguration queueConfiguration,
JobHostQueuesOptions queueOptions,
IWebJobsExceptionHandler exceptionHandler,
IContextSetter<IMessageEnqueuedWatcher> messageEnqueuedWatcherSetter,
ISharedContextProvider sharedContextProvider,
@ -35,7 +35,7 @@ namespace Microsoft.Azure.WebJobs.Host.Queues.Listeners
ITriggeredFunctionExecutor executor)
{
_queue = queue ?? throw new ArgumentNullException(nameof(queue));
_queueConfiguration = queueConfiguration ?? throw new ArgumentNullException(nameof(queueConfiguration));
_queueOptions = queueOptions ?? throw new ArgumentNullException(nameof(queueOptions));
_exceptionHandler = exceptionHandler ?? throw new ArgumentNullException(nameof(exceptionHandler));
_messageEnqueuedWatcherSetter = messageEnqueuedWatcherSetter ?? throw new ArgumentNullException(nameof(messageEnqueuedWatcherSetter));
_sharedContextProvider = sharedContextProvider ?? throw new ArgumentNullException(nameof(sharedContextProvider));
@ -54,7 +54,7 @@ namespace Microsoft.Azure.WebJobs.Host.Queues.Listeners
new SharedQueueWatcherFactory(_messageEnqueuedWatcherSetter));
IListener listener = new QueueListener(_queue, _poisonQueue, triggerExecutor, _exceptionHandler, _loggerFactory,
sharedWatcher, _queueConfiguration);
sharedWatcher, _queueOptions);
return Task.FromResult(listener);
}

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

@ -19,7 +19,7 @@ namespace Microsoft.Azure.WebJobs.Host.Queues
/// </summary>
/// <remarks>
/// Custom <see cref="QueueProcessor"/> implementations can be registered by implementing
/// a custom <see cref="IQueueProcessorFactory"/> and setting it via <see cref="JobHostQueuesConfiguration.QueueProcessorFactory"/>.
/// a custom <see cref="IQueueProcessorFactory"/> and setting it via <see cref="JobHostQueuesOptions.QueueProcessorFactory"/>.
/// </remarks>
public class QueueProcessor
{

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

@ -21,12 +21,7 @@ namespace Microsoft.Azure.WebJobs.Host.Queues
/// <param name="poisonQueue">The queue to move messages to when unable to process a message after the maximum dequeue count has been exceeded. May be null.</param>
public QueueProcessorFactoryContext(CloudQueue queue, ILoggerFactory loggerFactory, CloudQueue poisonQueue = null)
{
if (queue == null)
{
throw new ArgumentNullException("queue");
}
Queue = queue;
Queue = queue ?? throw new ArgumentNullException("queue");
PoisonQueue = poisonQueue;
Logger = loggerFactory?.CreateLogger(LogCategories.CreateTriggerCategory("Queue"));
}
@ -36,16 +31,16 @@ namespace Microsoft.Azure.WebJobs.Host.Queues
/// </summary>
/// <param name="queue">The <see cref="CloudQueue"/> the <see cref="QueueProcessor"/> will operate on.</param>
/// <param name="loggerFactory">The <see cref="ILoggerFactory"/> to create an <see cref="ILogger"/> from.</param>
/// <param name="queueConfiguration">The queue configuration.</param>
/// <param name="options">The queue configuration.</param>
/// <param name="poisonQueue">The queue to move messages to when unable to process a message after the maximum dequeue count has been exceeded. May be null.</param>
internal QueueProcessorFactoryContext(CloudQueue queue, ILoggerFactory loggerFactory, IQueueConfiguration queueConfiguration, CloudQueue poisonQueue = null)
internal QueueProcessorFactoryContext(CloudQueue queue, ILoggerFactory loggerFactory, JobHostQueuesOptions options, CloudQueue poisonQueue = null)
: this(queue, loggerFactory, poisonQueue)
{
BatchSize = queueConfiguration.BatchSize;
MaxDequeueCount = queueConfiguration.MaxDequeueCount;
NewBatchThreshold = queueConfiguration.NewBatchThreshold;
VisibilityTimeout = queueConfiguration.VisibilityTimeout;
MaxPollingInterval = queueConfiguration.MaxPollingInterval;
BatchSize = options.BatchSize;
MaxDequeueCount = options.MaxDequeueCount;
NewBatchThreshold = options.NewBatchThreshold;
VisibilityTimeout = options.VisibilityTimeout;
MaxPollingInterval = options.MaxPollingInterval;
}
/// <summary>

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

@ -27,7 +27,7 @@ namespace Microsoft.Azure.WebJobs.Host.Queues.Triggers
private readonly INameResolver _nameResolver;
private readonly IStorageAccountProvider _accountProvider;
private readonly IQueueConfiguration _queueConfiguration;
private readonly JobHostQueuesOptions _queueOptions;
private readonly IWebJobsExceptionHandler _exceptionHandler;
private readonly IContextSetter<IMessageEnqueuedWatcher> _messageEnqueuedWatcherSetter;
private readonly ISharedContextProvider _sharedContextProvider;
@ -35,14 +35,14 @@ namespace Microsoft.Azure.WebJobs.Host.Queues.Triggers
public QueueTriggerAttributeBindingProvider(INameResolver nameResolver,
IStorageAccountProvider accountProvider,
IQueueConfiguration queueConfiguration,
JobHostQueuesOptions queueOptions,
IWebJobsExceptionHandler exceptionHandler,
IContextSetter<IMessageEnqueuedWatcher> messageEnqueuedWatcherSetter,
ISharedContextProvider sharedContextProvider,
ILoggerFactory loggerFactory)
{
_accountProvider = accountProvider ?? throw new ArgumentNullException(nameof(accountProvider));
_queueConfiguration = queueConfiguration ?? throw new ArgumentNullException(nameof(queueConfiguration));
_queueOptions = queueOptions ?? throw new ArgumentNullException(nameof(queueOptions));
_exceptionHandler = exceptionHandler ?? throw new ArgumentNullException(nameof(exceptionHandler));
_messageEnqueuedWatcherSetter = messageEnqueuedWatcherSetter ?? throw new ArgumentNullException(nameof(messageEnqueuedWatcherSetter));
_sharedContextProvider = sharedContextProvider ?? throw new ArgumentNullException(nameof(sharedContextProvider));
@ -84,7 +84,7 @@ namespace Microsoft.Azure.WebJobs.Host.Queues.Triggers
IStorageQueue queue = client.GetQueueReference(queueName);
ITriggerBinding binding = new QueueTriggerBinding(parameter.Name, queue, argumentBinding,
_queueConfiguration, _exceptionHandler, _messageEnqueuedWatcherSetter,
_queueOptions, _exceptionHandler, _messageEnqueuedWatcherSetter,
_sharedContextProvider, _loggerFactory);
return binding;
}

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

@ -23,7 +23,7 @@ namespace Microsoft.Azure.WebJobs.Host.Queues.Triggers
private readonly IStorageQueue _queue;
private readonly ITriggerDataArgumentBinding<IStorageQueueMessage> _argumentBinding;
private readonly IReadOnlyDictionary<string, Type> _bindingDataContract;
private readonly IQueueConfiguration _queueConfiguration;
private readonly JobHostQueuesOptions _queueOptions;
private readonly IWebJobsExceptionHandler _exceptionHandler;
private readonly IContextSetter<IMessageEnqueuedWatcher> _messageEnqueuedWatcherSetter;
private readonly ISharedContextProvider _sharedContextProvider;
@ -33,7 +33,7 @@ namespace Microsoft.Azure.WebJobs.Host.Queues.Triggers
public QueueTriggerBinding(string parameterName,
IStorageQueue queue,
ITriggerDataArgumentBinding<IStorageQueueMessage> argumentBinding,
IQueueConfiguration queueConfiguration,
JobHostQueuesOptions queueOptions,
IWebJobsExceptionHandler exceptionHandler,
IContextSetter<IMessageEnqueuedWatcher> messageEnqueuedWatcherSetter,
ISharedContextProvider sharedContextProvider,
@ -42,7 +42,7 @@ namespace Microsoft.Azure.WebJobs.Host.Queues.Triggers
_queue = queue ?? throw new ArgumentNullException(nameof(queue));
_argumentBinding = argumentBinding ?? throw new ArgumentNullException(nameof(argumentBinding));
_bindingDataContract = CreateBindingDataContract(argumentBinding.BindingDataContract);
_queueConfiguration = queueConfiguration ?? throw new ArgumentNullException(nameof(queueConfiguration));
_queueOptions = queueOptions ?? throw new ArgumentNullException(nameof(queueOptions));
_exceptionHandler = exceptionHandler ?? throw new ArgumentNullException(nameof(exceptionHandler));
_messageEnqueuedWatcherSetter = messageEnqueuedWatcherSetter ?? throw new ArgumentNullException(nameof(messageEnqueuedWatcherSetter));
_sharedContextProvider = sharedContextProvider ?? throw new ArgumentNullException(nameof(sharedContextProvider));
@ -123,7 +123,7 @@ namespace Microsoft.Azure.WebJobs.Host.Queues.Triggers
throw new ArgumentNullException("context");
}
var factory = new QueueListenerFactory(_queue, _queueConfiguration, _exceptionHandler,
var factory = new QueueListenerFactory(_queue, _queueOptions, _exceptionHandler,
_messageEnqueuedWatcherSetter, _sharedContextProvider, _loggerFactory, context.Executor);
return factory.CreateAsync(context.CancellationToken);

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

@ -1,41 +0,0 @@
// Copyright (c) .NET Foundation. All rights reserved.
// Licensed under the MIT License. See License.txt in the project root for license information.
using System;
using System.Globalization;
namespace Microsoft.Azure.WebJobs.Host
{
internal static class ServiceProviderExtensions
{
public static IExtensionRegistry GetExtensions(this IServiceProvider serviceProvider)
{
return GetService<IExtensionRegistry>(serviceProvider);
}
public static T GetService<T>(this IServiceProvider serviceProvider) where T : class
{
if (serviceProvider == null)
{
throw new ArgumentNullException("serviceProvider");
}
object service = serviceProvider.GetService(typeof(T));
if (service == null)
{
return null;
}
T typedService = service as T;
if (typedService == null)
{
string message = String.Format(CultureInfo.InvariantCulture,
"The returned service must be assignable to {0}.", typeof(T).Name);
throw new InvalidOperationException(message);
}
return typedService;
}
}
}

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

@ -0,0 +1,42 @@
using Microsoft.Azure.WebJobs.Host.Executors;
using Microsoft.Extensions.Logging;
using Microsoft.Extensions.Options;
using System;
using System.Collections.Generic;
using System.Text;
namespace Microsoft.Azure.WebJobs.Host
{
public class DefaultDistributedLockManagerFactory : IDistributedLockManagerFactory
{
private readonly IOptions<JobHostInternalStorageOptions> _internalStorageOptions;
private readonly IStorageAccountProvider _storageAccountProvider;
private readonly ILoggerFactory _loggerFactory;
public DefaultDistributedLockManagerFactory(IOptions<JobHostInternalStorageOptions> internalStorageOptions,
IStorageAccountProvider storageAccountProvider,
ILoggerFactory loggerFactory)
{
_internalStorageOptions = internalStorageOptions;
_storageAccountProvider = storageAccountProvider;
_loggerFactory = loggerFactory;
}
public IDistributedLockManager Create()
{
ILogger logger = _loggerFactory.CreateLogger<IDistributedLockManager>();
var sas = _internalStorageOptions.Value;
IDistributedLockManager lockManager;
if (sas != null && sas.InternalContainer != null)
{
lockManager = new BlobLeaseDistributedLockManager.SasContainer(sas.InternalContainer, logger);
}
else
{
lockManager = new BlobLeaseDistributedLockManager.DedicatedStorage(_storageAccountProvider, logger);
}
return lockManager;
}
}
}

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

@ -9,7 +9,7 @@ using Microsoft.Azure.WebJobs.Host.Executors;
namespace Microsoft.Azure.WebJobs.Host
{
/// <summary>
/// Manage distributed lock. A lock is specified by (account, lockId). Interface implementations should cooperate with <see cref="SingletonConfiguration"/>
/// Manage distributed lock. A lock is specified by (account, lockId). Interface implementations should cooperate with <see cref="SingletonOptions"/>
/// </summary>
/// <remarks>
/// 1. Account can be null or it may be a storage account name intended for <see cref="IStorageAccountProvider"/>.
@ -42,7 +42,7 @@ namespace Microsoft.Azure.WebJobs.Host
CancellationToken cancellationToken);
/// <summary>
/// Called by the client to renew the lease. The timing internals here are determined by <see cref="SingletonConfiguration"/>
/// Called by the client to renew the lease. The timing internals here are determined by <see cref="SingletonOptions"/>
/// </summary>
/// <param name="lockHandle"></param>
/// <param name="cancellationToken"></param>

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

@ -0,0 +1,14 @@
// Copyright (c) .NET Foundation. All rights reserved.
// Licensed under the MIT License. See License.txt in the project root for license information.
using System;
using System.Collections.Generic;
using System.Text;
namespace Microsoft.Azure.WebJobs.Host
{
public interface IDistributedLockManagerFactory
{
IDistributedLockManager Create();
}
}

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

@ -16,7 +16,7 @@ namespace Microsoft.Azure.WebJobs.Host.Listeners
{
private readonly SingletonAttribute _attribute;
private readonly SingletonManager _singletonManager;
private readonly SingletonConfiguration _singletonConfig;
private readonly SingletonOptions _singletonConfig;
private readonly IListener _innerListener;
private readonly ILogger _logger;
private string _lockId;

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

@ -15,6 +15,7 @@ using Microsoft.Azure.WebJobs.Host.Protocols;
using Microsoft.Azure.WebJobs.Host.Timers;
using Microsoft.Azure.WebJobs.Logging;
using Microsoft.Extensions.Logging;
using Microsoft.Extensions.Options;
namespace Microsoft.Azure.WebJobs.Host
{
@ -24,7 +25,7 @@ namespace Microsoft.Azure.WebJobs.Host
internal class SingletonManager
{
private readonly INameResolver _nameResolver;
private readonly SingletonConfiguration _config;
private readonly SingletonOptions _config;
private readonly ILogger _logger;
private readonly ILoggerFactory _loggerFactory;
private readonly IDistributedLockManager _lockManager;
@ -40,12 +41,12 @@ namespace Microsoft.Azure.WebJobs.Host
{
}
public SingletonManager(IDistributedLockManager lockManager, SingletonConfiguration config, IWebJobsExceptionHandler exceptionHandler,
public SingletonManager(IDistributedLockManager lockManager, IOptions<SingletonOptions> config, IWebJobsExceptionHandler exceptionHandler,
ILoggerFactory loggerFactory, IHostIdProvider hostIdProvider, INameResolver nameResolver = null)
{
_lockManager = lockManager;
_nameResolver = nameResolver;
_config = config;
_config = config.Value;
_loggerFactory = loggerFactory;
_exceptionHandler = exceptionHandler;
_logger = _loggerFactory?.CreateLogger(LogCategories.Singleton);
@ -65,7 +66,7 @@ namespace Microsoft.Azure.WebJobs.Host
}
}
internal virtual SingletonConfiguration Config
internal virtual SingletonOptions Config
{
get
{
@ -144,7 +145,7 @@ namespace Microsoft.Azure.WebJobs.Host
return new TaskSeriesTimer(command, this._exceptionHandler, Task.Delay(normalUpdateInterval));
}
internal static TimeSpan GetLockPeriod(SingletonAttribute attribute, SingletonConfiguration config)
internal static TimeSpan GetLockPeriod(SingletonAttribute attribute, SingletonOptions config)
{
return attribute.Mode == SingletonMode.Listener ?
config.ListenerLockPeriod : config.LockPeriod;
@ -277,7 +278,7 @@ namespace Microsoft.Azure.WebJobs.Host
return singletonAttribute;
}
internal static TimeSpan GetLockAcquisitionTimeout(SingletonAttribute attribute, SingletonConfiguration config)
internal static TimeSpan GetLockAcquisitionTimeout(SingletonAttribute attribute, SingletonOptions config)
{
TimeSpan acquisitionTimeout = attribute.LockAcquisitionTimeout != -1
? TimeSpan.FromSeconds(attribute.LockAcquisitionTimeout)

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

@ -11,7 +11,7 @@ namespace Microsoft.Azure.WebJobs.Host
/// The configuration needs to cooperate with the SDK's registered <see cref="IDistributedLockManager"/>.
/// For example, this configuration determines the refresh frequently for calls on <see cref="IDistributedLockManager.RenewAsync(IDistributedLock, CancellationToken)"/>.
/// </summary>
public sealed class SingletonConfiguration
public sealed class SingletonOptions
{
// These are the min/max values supported by Azure Storage
private static readonly TimeSpan MinimumLeasePeriod = TimeSpan.FromSeconds(15);
@ -26,7 +26,7 @@ namespace Microsoft.Azure.WebJobs.Host
/// <summary>
/// Constructs a new instance.
/// </summary>
public SingletonConfiguration()
public SingletonOptions()
{
_lockPeriod = MinimumLeasePeriod;
_listenerLockPeriod = MaximumLeasePeriod;

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

@ -13,7 +13,7 @@ namespace Microsoft.Azure.WebJobs.Host.Storage
/// <summary>
/// Defines a storage account.
/// </summary>
internal interface IStorageAccount
public interface IStorageAccount
{
/// <summary>Gets the primary endpoint for the blob service.</summary>
Uri BlobEndpoint { get; }

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

@ -16,26 +16,17 @@ namespace Microsoft.Azure.WebJobs.Host.Storage
internal class StorageAccount : IStorageAccount
{
private readonly CloudStorageAccount _sdkAccount;
private readonly IServiceProvider _services;
private readonly StorageClientFactory _storageClientFactory;
/// <summary>
/// Initializes a new instance of the <see cref="StorageAccount"/> class.
/// </summary>
/// <param name="sdkAccount">The underlying SDK cloud storage account.</param>
/// <param name="services">The <see cref="IServiceProvider"/> to use.</param>
public StorageAccount(CloudStorageAccount sdkAccount, IServiceProvider services)
public StorageAccount(CloudStorageAccount sdkAccount, StorageClientFactory storageClientFactory)
{
if (sdkAccount == null)
{
throw new ArgumentNullException("sdkAccount");
}
if (services == null)
{
throw new ArgumentNullException("services");
}
_sdkAccount = sdkAccount;
_services = services;
_sdkAccount = sdkAccount ?? throw new ArgumentNullException(nameof(sdkAccount));
_storageClientFactory = storageClientFactory ?? throw new ArgumentNullException(nameof(storageClientFactory));
Type = StorageAccountType.GeneralPurpose;
}
@ -60,13 +51,7 @@ namespace Microsoft.Azure.WebJobs.Host.Storage
/// <inheritdoc />
public StorageAccountType Type { get; set; }
private StorageClientFactory ClientFactory
{
get
{
return (StorageClientFactory)_services.GetService(typeof(StorageClientFactory));
}
}
private StorageClientFactory ClientFactory => _storageClientFactory;
/// <inheritdoc />
public IStorageBlobClient CreateBlobClient(StorageClientFactoryContext context = null)

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

@ -3,7 +3,7 @@
namespace Microsoft.Azure.WebJobs.Host.Storage
{
internal enum StorageAccountType
public enum StorageAccountType
{
// Supports blob/table/queue.
// Can be used as primary or secondary storage.

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

@ -22,23 +22,27 @@ namespace Microsoft.Azure.WebJobs.Host.Tables
internal class TableExtension : IExtensionConfigProvider
{
private IStorageAccountProvider _accountProvider;
private readonly INameResolver _nameResolver;
// Property names on TableAttribute
private const string RowKeyProperty = nameof(TableAttribute.RowKey);
private const string PartitionKeyProperty = nameof(TableAttribute.PartitionKey);
public TableExtension(IStorageAccountProvider accountProvider, INameResolver nameResolver)
{
_accountProvider = accountProvider;
_nameResolver = nameResolver;
}
public void Initialize(ExtensionConfigContext context)
{
if (context == null)
{
throw new ArgumentNullException(nameof(context));
}
var config = context.Config;
_accountProvider = config.GetService<IStorageAccountProvider>();
var nameResolver = config.NameResolver;
// rules for single entity.
var original = new TableAttributeBindingProvider(nameResolver, _accountProvider);
var original = new TableAttributeBindingProvider(_nameResolver, _accountProvider);
var builder = new JObjectBuilder(this);

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

@ -0,0 +1,12 @@
using Microsoft.Extensions.Hosting;
using System;
using System.Collections.Generic;
using System.Text;
namespace Microsoft.Azure.WebJobs.Host.Timers
{
public class DefaultWebJobsExceptionHandlerFactory : IWebJobsExceptionHandlerFactory
{
public IWebJobsExceptionHandler Create(IHost jobHost) => new WebJobsExceptionHandler(jobHost);
}
}

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

@ -12,12 +12,6 @@ namespace Microsoft.Azure.WebJobs.Host.Timers
/// </summary>
public interface IWebJobsExceptionHandler
{
/// <summary>
/// Called during host creation. Allows a reference to the host.
/// </summary>
/// <param name="host">The JobHost.</param>
void Initialize(JobHost host);
/// <summary>
/// Called when a timeout occurs.
/// </summary>

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

@ -0,0 +1,15 @@
// Copyright (c) .NET Foundation. All rights reserved.
// Licensed under the MIT License. See License.txt in the project root for license information.
using Microsoft.Extensions.Hosting;
using System;
using System.Collections.Generic;
using System.Text;
namespace Microsoft.Azure.WebJobs.Host.Timers
{
public interface IWebJobsExceptionHandlerFactory
{
IWebJobsExceptionHandler Create(IHost jobHost);
}
}

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

@ -1,6 +1,7 @@
// Copyright (c) .NET Foundation. All rights reserved.
// Licensed under the MIT License. See License.txt in the project root for license information.
using Microsoft.Extensions.Hosting;
using System;
using System.Diagnostics;
using System.Runtime.ExceptionServices;
@ -14,16 +15,10 @@ namespace Microsoft.Azure.WebJobs.Host.Timers
/// </summary>
public class WebJobsExceptionHandler : IWebJobsExceptionHandler
{
private JobHost _host;
private IHost _host;
/// <inheritdoc />
public void Initialize(JobHost host)
public WebJobsExceptionHandler(IHost host)
{
if (host == null)
{
throw new ArgumentNullException("host");
}
_host = host;
}
@ -41,11 +36,12 @@ namespace Microsoft.Azure.WebJobs.Host.Timers
}
finally
{
_host.Dispose();
// TODO: FACAVAL: This shouldn't be here! - review
(_host as IDisposable)?.Dispose();
}
await Task.Delay(timeoutGracePeriod);
await this.OnUnhandledExceptionAsync(exceptionInfo);
await OnUnhandledExceptionAsync(exceptionInfo);
}
/// <inheritdoc />

Некоторые файлы не были показаны из-за слишком большого количества измененных файлов Показать больше