From da1932ed49a948e70e87045b8444cbf84947d845 Mon Sep 17 00:00:00 2001 From: Marc Gravell Date: Mon, 4 Apr 2022 14:47:38 +0100 Subject: [PATCH 01/17] remove RX as the primary pipe for gRPC communications remove RX as the primary pipe for gRPC communications --- .../Channel/GrpcWorkerChannel.cs | 319 +++++++++++++++--- .../Channel/GrpcWorkerChannelFactory.cs | 2 + .../Eventing/GrpcEventExtensions.cs | 67 ++++ .../Eventing/IGrpcEventManager.cs | 17 + .../Extensions/InboundGrpcEventExtensions.cs | 22 -- .../Server/FunctionRpcService.cs | 113 ++++--- .../WebJobs.Script.Grpc.csproj | 1 + .../Eventing/IScriptEventManager.cs | 6 + .../Eventing/ScriptEventManager.cs | 35 +- .../Workers/Rpc/RpcWorkerConstants.cs | 1 + .../WorkerConcurrencyManagerEndToEndTests.cs | 10 + .../TestScriptEventManager.cs | 22 ++ .../DefaultDependencyValidatorTests.cs | 22 ++ .../InboundGrpcEventExtensionsTests.cs | 2 +- .../Workers/Rpc/GrpcWorkerChannelTests.cs | 153 ++++++--- .../Rpc/RpcWorkerConfigFactoryTests.cs | 6 + .../Workers/Rpc/TestFunctionRpcService.cs | 89 ++++- 17 files changed, 703 insertions(+), 184 deletions(-) create mode 100644 src/WebJobs.Script.Grpc/Eventing/GrpcEventExtensions.cs create mode 100644 src/WebJobs.Script.Grpc/Eventing/IGrpcEventManager.cs delete mode 100644 src/WebJobs.Script.Grpc/Extensions/InboundGrpcEventExtensions.cs diff --git a/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs b/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs index 8b832fe7a5..12a37f322b 100644 --- a/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs +++ b/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs @@ -10,6 +10,7 @@ using System.Reactive.Linq; using System.Text; using System.Threading; +using System.Threading.Channels; using System.Threading.Tasks; using System.Threading.Tasks.Dataflow; using Microsoft.Azure.WebJobs.Logging; @@ -25,7 +26,7 @@ using Microsoft.Azure.WebJobs.Script.Workers.SharedMemoryDataTransfer; using Microsoft.Extensions.Logging; using Microsoft.Extensions.Options; -using Newtonsoft.Json.Linq; + using static Microsoft.Azure.WebJobs.Script.Grpc.Messages.RpcLog.Types; using FunctionMetadata = Microsoft.Azure.WebJobs.Script.Description.FunctionMetadata; using MsgType = Microsoft.Azure.WebJobs.Script.Grpc.Messages.StreamingMessage.ContentOneofCase; @@ -43,6 +44,11 @@ internal class GrpcWorkerChannel : IRpcWorkerChannel, IDisposable private readonly ISharedMemoryManager _sharedMemoryManager; private readonly List _workerStatusLatencyHistory = new List(); private readonly IOptions _workerConcurrencyOptions; + private readonly WaitCallback _processInbound; + private readonly object _syncLock = new object(); + private readonly Dictionary> _pendingActions = new (); + private readonly ChannelWriter _outbound; + private readonly ChannelReader _inbound; private IDisposable _functionLoadRequestResponseEvent; private bool _disposed; @@ -55,10 +61,8 @@ internal class GrpcWorkerChannel : IRpcWorkerChannel, IDisposable private ConcurrentDictionary _executingInvocations = new ConcurrentDictionary(); private IDictionary> _functionInputBuffers = new ConcurrentDictionary>(); private ConcurrentDictionary> _workerStatusRequests = new ConcurrentDictionary>(); - private IObservable _inboundWorkerEvents; private List _inputLinks = new List(); private List _eventSubscriptions = new List(); - private IDisposable _startSubscription; private IDisposable _startLatencyMetric; private IEnumerable _functions; private GrpcCapabilities _workerCapabilities; @@ -71,7 +75,6 @@ internal class GrpcWorkerChannel : IRpcWorkerChannel, IDisposable private TimeSpan _functionLoadTimeout = TimeSpan.FromMinutes(10); private bool _isSharedMemoryDataTransferEnabled; - private object _syncLock = new object(); private System.Timers.Timer _timer; internal GrpcWorkerChannel( @@ -99,31 +102,22 @@ internal GrpcWorkerChannel( _applicationHostOptions = applicationHostOptions; _sharedMemoryManager = sharedMemoryManager; _workerConcurrencyOptions = workerConcurrencyOptions; + _processInbound = state => ProcessItem((InboundGrpcEvent)state); _workerCapabilities = new GrpcCapabilities(_workerChannelLogger); - _inboundWorkerEvents = _eventManager.OfType() - .Where(msg => msg.WorkerId == _workerId); - - _eventSubscriptions.Add(_inboundWorkerEvents - .Where(msg => msg.IsMessageOfType(MsgType.RpcLog) && !msg.IsLogOfCategory(RpcLogCategory.System)) - .Subscribe(Log)); - - _eventSubscriptions.Add(_inboundWorkerEvents - .Where(msg => msg.IsMessageOfType(MsgType.RpcLog) && msg.IsLogOfCategory(RpcLogCategory.System)) - .Subscribe(SystemLog)); + if (eventManager.TryGetGrpcChannels(workerId, out var inbound, out var outbound)) + { + _outbound = outbound.Writer; + _inbound = inbound.Reader; + _ = ProcessInbound(); + } _eventSubscriptions.Add(_eventManager.OfType() .Where(msg => _workerConfig.Description.Extensions.Contains(Path.GetExtension(msg.FileChangeArguments.FullPath))) .Throttle(TimeSpan.FromMilliseconds(300)) // debounce .Subscribe(msg => _eventManager.Publish(new HostRestartEvent()))); - _eventSubscriptions.Add(_inboundWorkerEvents.Where(msg => msg.MessageType == MsgType.InvocationResponse) - .Subscribe(async (msg) => await InvokeResponse(msg.Message.InvocationResponse))); - - _inboundWorkerEvents.Where(msg => msg.MessageType == MsgType.WorkerStatusResponse) - .Subscribe((msg) => ReceiveWorkerStatusResponse(msg.Message.RequestId, msg.Message.WorkerStatusResponse)); - _startLatencyMetric = metricsLogger?.LatencyEvent(string.Format(MetricEventNames.WorkerInitializeLatency, workerConfig.Description.Language, attemptCount)); _state = RpcWorkerChannelState.Default; @@ -137,6 +131,128 @@ internal GrpcWorkerChannel( internal RpcWorkerConfig Config => _workerConfig; + private void ProcessItem(InboundGrpcEvent msg) + { + // note this is a QUWI entry-point + try + { + switch (msg.MessageType) + { + case MsgType.RpcLog: + switch (msg.Message.RpcLog.LogCategory) + { + case RpcLogCategory.System: + SystemLog(msg); + break; + default: + Log(msg); + break; + } + break; + case MsgType.WorkerStatusResponse: + ReceiveWorkerStatusResponse(msg.Message.RequestId, msg.Message.WorkerStatusResponse); + break; + case MsgType.InvocationResponse: + _ = InvokeResponse(msg.Message.InvocationResponse); + break; + default: + OnNext(msg); + break; + } + } + catch + { + // TODO: log? + } + } + + private void OnNext(InboundGrpcEvent message) + { + Queue queue; + lock (_pendingActions) + { + if (!_pendingActions.TryGetValue(message.MessageType, out queue)) + { + return; // nothing to do + } + } + PendingItem next; + lock (queue) + { + do + { + if (!queue.TryDequeue(out next)) + { + return; // nothing to do + } + } + while (next.IsComplete); + } + next.SetResult(message); + } + + private void OnNext(MsgType messageType, TimeSpan timeout, int count, Action callback, Action faultHandler) + { + Queue queue; + lock (_pendingActions) + { + if (!_pendingActions.TryGetValue(messageType, out queue)) + { + queue = new Queue(); + _pendingActions.Add(messageType, queue); + } + } + + lock (queue) + { + // while we have the lock, discard any dead items (to prevent unbounded growth on stall) + while (queue.TryPeek(out var next) && next.IsComplete) + { + queue.Dequeue(); + } + for (int i = 0; i < count; i++) + { + var newItem = i == count - 1 && timeout != TimeSpan.Zero + ? new PendingItem(callback, faultHandler, timeout) + : new PendingItem(callback, faultHandler); + queue.Enqueue(newItem); + } + } + } + + private async Task ProcessInbound() + { + try + { + await Task.Yield(); // free up the caller + bool debug = _workerChannelLogger.IsEnabled(LogLevel.Debug); + if (debug) + { + _workerChannelLogger.LogDebug("[channel] processing reader loop for worker {0}:", _workerId); + } + while (await _inbound.WaitToReadAsync()) + { + while (_inbound.TryRead(out var msg)) + { + if (debug) + { + _workerChannelLogger.LogDebug("[channel] received {0}: {1}", msg.WorkerId, msg.MessageType); + } + ThreadPool.QueueUserWorkItem(_processInbound, msg); + } + } + } + catch (Exception ex) + { + _workerChannelLogger.LogError(ex, "Error processing inbound messages"); + } + finally + { + // we're not listening any more! shut down the channels + _eventManager.RemoveGrpcChannels(_workerId); + } + } + public bool IsChannelReadyForInvocations() { return !_disposing && !_disposed && _state.HasFlag(RpcWorkerChannelState.InvocationBuffersInitialized | RpcWorkerChannelState.Initialized); @@ -144,10 +260,7 @@ public bool IsChannelReadyForInvocations() public async Task StartWorkerProcessAsync(CancellationToken cancellationToken) { - _startSubscription = _inboundWorkerEvents.Where(msg => msg.MessageType == MsgType.StartStream) - .Timeout(_workerConfig.CountOptions.ProcessStartupTimeout) - .Take(1) - .Subscribe(SendWorkerInitRequest, HandleWorkerStartStreamError); + OnNext(MsgType.StartStream, _workerConfig.CountOptions.ProcessStartupTimeout, 1, SendWorkerInitRequest, HandleWorkerStartStreamError); _workerChannelLogger.LogDebug("Initiating Worker Process start up"); await _rpcWorkerProcess.StartProcessAsync(); @@ -174,7 +287,7 @@ public async Task GetWorkerStatusAsync() var tcs = new TaskCompletionSource(); if (_workerStatusRequests.TryAdd(message.RequestId, tcs)) { - SendStreamingMessage(message); + await SendStreamingMessageAsync(message); await tcs.Task; var elapsed = sw.GetElapsedTime(); workerStatus.Latency = elapsed; @@ -195,10 +308,7 @@ public async Task GetWorkerStatusAsync() internal void SendWorkerInitRequest(GrpcEvent startEvent) { _workerChannelLogger.LogDebug("Worker Process started. Received StartStream message"); - _inboundWorkerEvents.Where(msg => msg.MessageType == MsgType.WorkerInitResponse) - .Timeout(_workerConfig.CountOptions.InitializationTimeout) - .Take(1) - .Subscribe(WorkerInitResponse, HandleWorkerInitError); + OnNext(MsgType.WorkerInitResponse, _workerConfig.CountOptions.InitializationTimeout, 1, WorkerInitResponse, HandleWorkerInitError); WorkerInitRequest initRequest = GetWorkerInitRequest(); @@ -217,6 +327,11 @@ internal void SendWorkerInitRequest(GrpcEvent startEvent) initRequest.Capabilities.Add(RpcWorkerConstants.FunctionDataCache, "true"); } + // advertise that we support multiple streams, and hint at a number; with this flag, we allow + // clients to connect multiple back-hauls *with the same workerid*, and rely on the internal + // plumbing to make sure we don't process everything N times + initRequest.Capabilities.Add(RpcWorkerConstants.MultiStream, "10"); // TODO: make this configurable + SendStreamingMessage(new StreamingMessage { WorkerInitRequest = initRequest @@ -287,26 +402,18 @@ public void SendFunctionLoadRequests(ManagedDependencyOptions managedDependencyO { if (_functions != null) { + var count = _functions.Count(); if (functionTimeout.HasValue) { _functionLoadTimeout = functionTimeout.Value > _functionLoadTimeout ? functionTimeout.Value : _functionLoadTimeout; - _eventSubscriptions.Add(_inboundWorkerEvents.Where(msg => msg.MessageType == MsgType.FunctionLoadResponse) - .Timeout(_functionLoadTimeout) - .Take(_functions.Count()) - .Subscribe((msg) => LoadResponse(msg.Message.FunctionLoadResponse), HandleWorkerFunctionLoadError)); - _eventSubscriptions.Add(_inboundWorkerEvents.Where(msg => msg.MessageType == MsgType.FunctionLoadResponseCollection) - .Timeout(_functionLoadTimeout) - .Take(_functions.Count()) - .Subscribe((msg) => LoadResponse(msg.Message.FunctionLoadResponseCollection), HandleWorkerFunctionLoadError)); + OnNext(MsgType.FunctionLoadResponse, _functionLoadTimeout, count, msg => LoadResponse(msg.Message.FunctionLoadResponse), HandleWorkerFunctionLoadError); + OnNext(MsgType.FunctionLoadResponseCollection, _functionLoadTimeout, count, msg => LoadResponse(msg.Message.FunctionLoadResponseCollection), HandleWorkerFunctionLoadError); } else { - _eventSubscriptions.Add(_inboundWorkerEvents.Where(msg => msg.MessageType == MsgType.FunctionLoadResponse) - .Subscribe((msg) => LoadResponse(msg.Message.FunctionLoadResponse), HandleWorkerFunctionLoadError)); - - _eventSubscriptions.Add(_inboundWorkerEvents.Where(msg => msg.MessageType == MsgType.FunctionLoadResponseCollection) - .Subscribe((msg) => LoadResponse(msg.Message.FunctionLoadResponseCollection), HandleWorkerFunctionLoadError)); + OnNext(MsgType.FunctionLoadResponse, TimeSpan.Zero, count, msg => LoadResponse(msg.Message.FunctionLoadResponse), HandleWorkerFunctionLoadError); + OnNext(MsgType.FunctionLoadResponseCollection, TimeSpan.Zero, count, msg => LoadResponse(msg.Message.FunctionLoadResponseCollection), HandleWorkerFunctionLoadError); } // Load Request is also sent for disabled function as it is invocable using the portal and admin endpoints @@ -362,11 +469,8 @@ public Task SendFunctionEnvironmentReloadRequest() _workerChannelLogger.LogDebug("Sending FunctionEnvironmentReloadRequest to WorkerProcess with Pid: '{0}'", _rpcWorkerProcess.Id); IDisposable latencyEvent = _metricsLogger.LatencyEvent(MetricEventNames.SpecializationEnvironmentReloadRequestResponse); - _eventSubscriptions - .Add(_inboundWorkerEvents.Where(msg => msg.MessageType == MsgType.FunctionEnvironmentReloadResponse) - .Timeout(_workerConfig.CountOptions.EnvironmentReloadTimeout) - .Take(1) - .Subscribe((msg) => FunctionEnvironmentReloadResponse(msg.Message.FunctionEnvironmentReloadResponse, latencyEvent), HandleWorkerEnvReloadError)); + OnNext(MsgType.FunctionEnvironmentReloadResponse, _workerConfig.CountOptions.EnvironmentReloadTimeout, 1, + msg => FunctionEnvironmentReloadResponse(msg.Message.FunctionEnvironmentReloadResponse, latencyEvent), HandleWorkerEnvReloadError); IDictionary processEnv = Environment.GetEnvironmentVariables(); @@ -507,7 +611,7 @@ internal async Task SendInvocationRequest(ScriptInvocationContext context) var invocationRequest = await context.ToRpcInvocationRequest(_workerChannelLogger, _workerCapabilities, _isSharedMemoryDataTransferEnabled, _sharedMemoryManager); _executingInvocations.TryAdd(invocationRequest.InvocationId, context); - SendStreamingMessage(new StreamingMessage + await SendStreamingMessageAsync(new StreamingMessage { InvocationRequest = invocationRequest }); @@ -527,10 +631,8 @@ public Task> GetFunctionMetadata() internal Task> SendFunctionMetadataRequest() { - _eventSubscriptions.Add(_inboundWorkerEvents.Where(msg => msg.MessageType == MsgType.FunctionMetadataResponse) - .Timeout(_functionLoadTimeout) - .Take(1) - .Subscribe((msg) => ProcessFunctionMetadataResponses(msg.Message.FunctionMetadataResponse), HandleWorkerMetadataRequestError)); + OnNext(MsgType.FunctionMetadataResponse, _functionLoadTimeout, 1, + msg => ProcessFunctionMetadataResponses(msg.Message.FunctionMetadataResponse), HandleWorkerMetadataRequestError); _workerChannelLogger.LogDebug("Sending WorkerMetadataRequest to {language} worker with worker ID {workerID}", _runtime, _workerId); @@ -840,9 +942,46 @@ internal void HandleWorkerMetadataRequestError(Exception exc) _eventManager.Publish(new WorkerErrorEvent(_runtime, Id, exc)); } + private ValueTask SendStreamingMessageAsync(StreamingMessage msg) + { + if (_outbound is null) + { + return default; + } + var evt = new OutboundGrpcEvent(_workerId, msg); + return _outbound.TryWrite(evt) ? default : _outbound.WriteAsync(evt); + } + private void SendStreamingMessage(StreamingMessage msg) { - _eventManager.Publish(new OutboundGrpcEvent(_workerId, msg)); + if (_outbound is null) + { + return; + } + var evt = new OutboundGrpcEvent(_workerId, msg); + if (!_outbound.TryWrite(evt)) + { + var pending = _outbound.WriteAsync(evt); + if (pending.IsCompleted) + { + pending.GetAwaiter().GetResult(); // ensure observed, for IVTS reasons + } + else + { + _ = ObserveEventually(pending); + } + } + static async Task ObserveEventually(ValueTask valueTask) + { + try + { + await valueTask; + } + catch + { + // log somewhere? + } + } } internal void ReceiveWorkerStatusResponse(string requestId, WorkerStatusResponse response) @@ -860,7 +999,6 @@ protected virtual void Dispose(bool disposing) if (disposing) { _startLatencyMetric?.Dispose(); - _startSubscription?.Dispose(); _workerInitTask?.TrySetCanceled(); _timer?.Dispose(); @@ -876,6 +1014,9 @@ protected virtual void Dispose(bool disposing) { sub.Dispose(); } + + // shut down the channels + _eventManager.RemoveGrpcChannels(_workerId); } _disposed = true; } @@ -1015,5 +1156,75 @@ private void AddSample(List samples, T sample) samples.Add(sample); } } + + private sealed class PendingItem + { + private readonly Action _callback; + private readonly Action _faultHandler; + private CancellationTokenRegistration _ctr; + private int _state; + + public PendingItem(Action callback, Action faultHandler) + { + _callback = callback; + _faultHandler = faultHandler; + } + + public PendingItem(Action callback, Action faultHandler, TimeSpan timeout) + : this(callback, faultHandler) + { + var cts = new CancellationTokenSource(); + cts.CancelAfter(timeout); + _ctr = cts.Token.Register(static state => ((PendingItem)state).OnTimeout(), this); + } + + public bool IsComplete => Volatile.Read(ref _state) != 0; + + private bool MakeComplete() => Interlocked.CompareExchange(ref _state, 1, 0) == 0; + + public void SetResult(InboundGrpcEvent message) + { + _ctr.Dispose(); + _ctr = default; + if (MakeComplete() && _callback != null) + { + try + { + _callback.Invoke(message); + } + catch (Exception fault) + { + try + { + _faultHandler?.Invoke(fault); + } + catch + { + } + } + } + } + + private void OnTimeout() + { + if (MakeComplete() && _faultHandler != null) + { + try + { + throw new TimeoutException(); + } + catch (Exception timeout) + { + try + { + _faultHandler(timeout); + } + catch + { + } + } + } + } + } } } \ No newline at end of file diff --git a/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannelFactory.cs b/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannelFactory.cs index ea17b97913..09ed942576 100644 --- a/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannelFactory.cs +++ b/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannelFactory.cs @@ -7,6 +7,7 @@ using System.Reactive.Linq; using Microsoft.Azure.WebJobs.Script.Diagnostics; using Microsoft.Azure.WebJobs.Script.Eventing; +using Microsoft.Azure.WebJobs.Script.Grpc.Eventing; using Microsoft.Azure.WebJobs.Script.Workers; using Microsoft.Azure.WebJobs.Script.Workers.Rpc; using Microsoft.Azure.WebJobs.Script.Workers.SharedMemoryDataTransfer; @@ -47,6 +48,7 @@ public IRpcWorkerChannel Create(string scriptRootPath, string runtime, IMetricsL throw new InvalidOperationException($"WorkerCofig for runtime: {runtime} not found"); } string workerId = Guid.NewGuid().ToString(); + _eventManager.AddGrpcChannels(workerId); // prepare the inbound/outbound dedicated channels ILogger workerLogger = _loggerFactory.CreateLogger($"Worker.LanguageWorkerChannel.{runtime}.{workerId}"); IWorkerProcess rpcWorkerProcess = _rpcWorkerProcessFactory.Create(workerId, runtime, scriptRootPath, languageWorkerConfig); return new GrpcWorkerChannel( diff --git a/src/WebJobs.Script.Grpc/Eventing/GrpcEventExtensions.cs b/src/WebJobs.Script.Grpc/Eventing/GrpcEventExtensions.cs new file mode 100644 index 0000000000..15cbee10c5 --- /dev/null +++ b/src/WebJobs.Script.Grpc/Eventing/GrpcEventExtensions.cs @@ -0,0 +1,67 @@ +// 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.Channels; +using Microsoft.Azure.WebJobs.Script.Eventing; + +namespace Microsoft.Azure.WebJobs.Script.Grpc.Eventing; + +internal static class GrpcEventExtensions +{ + // flow here is: + // 1) external request is proxied to the the GrpcWorkerChannel via one of the many Send* APIs, which writes + // to outbound-writer; this means we can have concurrent writes to outbound + // 2) if an out-of-process function is connected, a FunctionRpcService-EventStream will consume + // from outbound-reader (we'll allow for the multi-stream possibility, hence concurrent), and push it via gRPC + // 3) when the out-of-process function provides a response to FunctionRpcService-EventStream, it is written to + // inbound-writer (note we will allow for multi-stream possibility) + // 4) the GrpcWorkerChannel has a single dedicated consumer of inbound-reader, which it then marries to + // in-flight operations + private static readonly UnboundedChannelOptions InboundOptions = new UnboundedChannelOptions + { + SingleReader = true, // see 4 + SingleWriter = false, // see 3 + AllowSynchronousContinuations = false, + }; + + private static readonly UnboundedChannelOptions OutboundOptions = new UnboundedChannelOptions + { + SingleReader = false, // see 2 + SingleWriter = false, // see 1 + AllowSynchronousContinuations = false, + }; + + public static void AddGrpcChannels(this IScriptEventManager manager, string workerId) + { + var inbound = Channel.CreateUnbounded(InboundOptions); + if (manager.TryAddWorkerState(workerId, inbound)) + { + var outbound = Channel.CreateUnbounded(OutboundOptions); + if (manager.TryAddWorkerState(workerId, outbound)) + { + return; // successfully added both + } + // we added the inbound but not the outbound; revert + manager.TryRemoveWorkerState(workerId, out inbound); + } + // this is not anticipated, so don't panic abount the allocs above + throw new ArgumentException("Duplicate worker id: " + workerId, nameof(workerId)); + } + + public static bool TryGetGrpcChannels(this IScriptEventManager manager, string workerId, out Channel inbound, out Channel outbound) + => manager.TryGetWorkerState(workerId, out inbound) & manager.TryGetWorkerState(workerId, out outbound); + + public static void RemoveGrpcChannels(this IScriptEventManager manager, string workerId) + { + // remove any channels, and shut them down + if (manager.TryGetWorkerState>(workerId, out var inbound)) + { + inbound.Writer.TryComplete(); + } + if (manager.TryGetWorkerState>(workerId, out var outbound)) + { + outbound.Writer.TryComplete(); + } + } +} diff --git a/src/WebJobs.Script.Grpc/Eventing/IGrpcEventManager.cs b/src/WebJobs.Script.Grpc/Eventing/IGrpcEventManager.cs new file mode 100644 index 0000000000..9cafcb4fae --- /dev/null +++ b/src/WebJobs.Script.Grpc/Eventing/IGrpcEventManager.cs @@ -0,0 +1,17 @@ +// Copyright (c) .NET Foundation. All rights reserved. +// Licensed under the MIT License. See License.txt in the project root for license information. + +//using System.Threading.Channels; +//using Microsoft.Azure.WebJobs.Script.Eventing; + +//namespace Microsoft.Azure.WebJobs.Script.Grpc.Eventing; + +///// A script event manager with dedicated support for gRPC channels +//public interface IGrpcEventManager : IScriptEventManager +//{ +// /// Prepares the specified workerId for usage (unexpected inbound worker ids will be rejected) +// void AddWorker(string workerId); + +// /// Obtains the gRPC channels for the specified workerId, if it has been prepared +// bool TryGetGrpcChannels(string workerId, out Channel inbound, out Channel outbound); +//} diff --git a/src/WebJobs.Script.Grpc/Extensions/InboundGrpcEventExtensions.cs b/src/WebJobs.Script.Grpc/Extensions/InboundGrpcEventExtensions.cs deleted file mode 100644 index a1ae865b08..0000000000 --- a/src/WebJobs.Script.Grpc/Extensions/InboundGrpcEventExtensions.cs +++ /dev/null @@ -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 Microsoft.Azure.WebJobs.Script.Grpc.Eventing; -using static Microsoft.Azure.WebJobs.Script.Grpc.Messages.RpcLog.Types; -using MessageType = Microsoft.Azure.WebJobs.Script.Grpc.Messages.StreamingMessage.ContentOneofCase; - -namespace Microsoft.Azure.WebJobs.Script.Grpc -{ - public static class InboundGrpcEventExtensions - { - public static bool IsMessageOfType(this InboundGrpcEvent inboundEvent, MessageType typeToCheck) - { - return inboundEvent.MessageType.Equals(typeToCheck); - } - - public static bool IsLogOfCategory(this InboundGrpcEvent inboundEvent, RpcLogCategory categoryToCheck) - { - return inboundEvent.Message.RpcLog.LogCategory.Equals(categoryToCheck); - } - } -} diff --git a/src/WebJobs.Script.Grpc/Server/FunctionRpcService.cs b/src/WebJobs.Script.Grpc/Server/FunctionRpcService.cs index fd9bcfde0f..e69c21aad5 100644 --- a/src/WebJobs.Script.Grpc/Server/FunctionRpcService.cs +++ b/src/WebJobs.Script.Grpc/Server/FunctionRpcService.cs @@ -2,10 +2,9 @@ // Licensed under the MIT License. See License.txt in the project root for license information. using System; -using System.Collections.Generic; -using System.Reactive.Concurrency; using System.Reactive.Linq; using System.Threading; +using System.Threading.Channels; using System.Threading.Tasks; using Grpc.Core; using Microsoft.Azure.WebJobs.Script.Eventing; @@ -21,7 +20,6 @@ namespace Microsoft.Azure.WebJobs.Script.Grpc // TODO: move to WebJobs.Script.Grpc package and provide event stream abstraction internal class FunctionRpcService : FunctionRpc.FunctionRpcBase { - private readonly SemaphoreSlim _writeLock = new SemaphoreSlim(1, 1); private readonly IScriptEventManager _eventManager; private readonly ILogger _logger; @@ -33,64 +31,48 @@ public FunctionRpcService(IScriptEventManager eventManager, ILogger requestStream, IServerStreamWriter responseStream, ServerCallContext context) { - var cancelSource = new TaskCompletionSource(); - IDictionary outboundEventSubscriptions = new Dictionary(); - + var cancelSource = new TaskCompletionSource(TaskCreationOptions.RunContinuationsAsynchronously); + var cts = CancellationTokenSource.CreateLinkedTokenSource(context.CancellationToken); + CancellationTokenRegistration ctr = cts.Token.Register(static state => ((TaskCompletionSource)state).TrySetResult(false), cancelSource); try { - context.CancellationToken.Register(() => cancelSource.TrySetResult(false)); - Func> messageAvailable = async () => + static Task> MoveNextAsync(IAsyncStreamReader requestStream, TaskCompletionSource cancelSource) { // GRPC does not accept cancellation tokens for individual reads, hence wrapper var requestTask = requestStream.MoveNext(CancellationToken.None); - var completed = await Task.WhenAny(cancelSource.Task, requestTask); - return completed.Result; - }; + return Task.WhenAny(cancelSource.Task, requestTask); + } - if (await messageAvailable()) + if (await await MoveNextAsync(requestStream, cancelSource)) { - string workerId = requestStream.Current.StartStream.WorkerId; - _logger.LogDebug("Established RPC channel. WorkerId: {workerId}", workerId); - outboundEventSubscriptions.Add(workerId, _eventManager.OfType() - .Where(evt => evt.WorkerId == workerId) - .ObserveOn(NewThreadScheduler.Default) - .Subscribe(async evt => + var currentMessage = requestStream.Current; + // expect first operation (and only the first; we don't support re-registration) to be StartStream + if (currentMessage.ContentCase == MsgType.StartStream) + { + var workerId = currentMessage.StartStream?.WorkerId; + if (!string.IsNullOrEmpty(workerId) && _eventManager.TryGetGrpcChannels(workerId, out var inbound, out var outbound)) { - try - { - if (evt.MessageType == MsgType.InvocationRequest) - { - _logger.LogTrace("Writing invocation request invocationId: {invocationId} to workerId: {workerId}", evt.Message.InvocationRequest.InvocationId, workerId); - } + // send work + _ = PushFromOutboundToGrpc(workerId, responseStream, outbound.Reader, cts.Token); - try + // this loop is "pull from gRPC and push to inbound" + do + { + currentMessage = requestStream.Current; + if (currentMessage.ContentCase == MsgType.InvocationResponse && !string.IsNullOrEmpty(currentMessage.InvocationResponse?.InvocationId)) { - // WriteAsync only allows one pending write at a time, so we - // serialize access to the stream for each subscription - await _writeLock.WaitAsync(); - await responseStream.WriteAsync(evt.Message); + _logger.LogTrace("Received invocation response for invocationId: {invocationId} from workerId: {workerId}", currentMessage.InvocationResponse.InvocationId, workerId); } - finally + var newInbound = new InboundGrpcEvent(workerId, currentMessage); + if (!inbound.Writer.TryWrite(newInbound)) { - _writeLock.Release(); + await inbound.Writer.WriteAsync(newInbound); } + currentMessage = null; // allow old messages to be collected while we wait } - catch (Exception subscribeEventEx) - { - _logger.LogError(subscribeEventEx, "Error writing message type {messageType} to workerId: {workerId}", evt.MessageType, workerId); - } - })); - - do - { - var currentMessage = requestStream.Current; - if (currentMessage.InvocationResponse != null && !string.IsNullOrEmpty(currentMessage.InvocationResponse.InvocationId)) - { - _logger.LogTrace("Received invocation response for invocationId: {invocationId} from workerId: {workerId}", currentMessage.InvocationResponse.InvocationId, workerId); + while (await await MoveNextAsync(requestStream, cancelSource)); } - _eventManager.Publish(new InboundGrpcEvent(workerId, currentMessage)); } - while (await messageAvailable()); } } catch (Exception rpcException) @@ -101,14 +83,47 @@ public override async Task EventStream(IAsyncStreamReader requ } finally { - foreach (var sub in outboundEventSubscriptions) - { - sub.Value?.Dispose(); - } + cts.Cancel(); + ctr.Dispose(); // ensure cancellationSource task completes cancelSource.TrySetResult(false); } } + + private async Task PushFromOutboundToGrpc(string workerId, IServerStreamWriter responseStream, ChannelReader source, CancellationToken cancellationToken) + { + try + { + _logger.LogDebug("Established RPC channel. WorkerId: {workerId}", workerId); + await Task.Yield(); // free up the caller + while (await source.WaitToReadAsync(cancellationToken)) + { + while (source.TryRead(out var evt)) + { + if (evt.MessageType == MsgType.InvocationRequest) + { + _logger.LogTrace("Writing invocation request invocationId: {invocationId} to workerId: {workerId}", evt.Message.InvocationRequest.InvocationId, workerId); + } + try + { + await responseStream.WriteAsync(evt.Message); + } + catch (Exception subscribeEventEx) + { + _logger.LogError(subscribeEventEx, "Error writing message type {messageType} to workerId: {workerId}", evt.MessageType, workerId); + } + } + } + } + catch (OperationCanceledException oce) when (oce.CancellationToken == cancellationToken) + { + // that's fine; normaly exit through cancellation + } + catch (Exception ex) + { + _logger.LogError(ex, "Error pushing from outbound to gRPC"); + } + } } } diff --git a/src/WebJobs.Script.Grpc/WebJobs.Script.Grpc.csproj b/src/WebJobs.Script.Grpc/WebJobs.Script.Grpc.csproj index fdab414a04..1c26ded690 100644 --- a/src/WebJobs.Script.Grpc/WebJobs.Script.Grpc.csproj +++ b/src/WebJobs.Script.Grpc/WebJobs.Script.Grpc.csproj @@ -30,6 +30,7 @@ all + diff --git a/src/WebJobs.Script/Eventing/IScriptEventManager.cs b/src/WebJobs.Script/Eventing/IScriptEventManager.cs index 889d508e96..8c3c8badb0 100644 --- a/src/WebJobs.Script/Eventing/IScriptEventManager.cs +++ b/src/WebJobs.Script/Eventing/IScriptEventManager.cs @@ -8,5 +8,11 @@ namespace Microsoft.Azure.WebJobs.Script.Eventing public interface IScriptEventManager : IObservable { void Publish(ScriptEvent scriptEvent); + + bool TryAddWorkerState(string workerId, T state); + + bool TryGetWorkerState(string workerId, out T state); + + bool TryRemoveWorkerState(string workerId, out T state); } } diff --git a/src/WebJobs.Script/Eventing/ScriptEventManager.cs b/src/WebJobs.Script/Eventing/ScriptEventManager.cs index 91c29ff40f..f3d1e360ce 100644 --- a/src/WebJobs.Script/Eventing/ScriptEventManager.cs +++ b/src/WebJobs.Script/Eventing/ScriptEventManager.cs @@ -2,13 +2,16 @@ // Licensed under the MIT License. See License.txt in the project root for license information. using System; +using System.Collections.Concurrent; using System.Reactive.Subjects; namespace Microsoft.Azure.WebJobs.Script.Eventing { - public sealed class ScriptEventManager : IScriptEventManager, IDisposable + public class ScriptEventManager : IScriptEventManager, IDisposable { private readonly Subject _subject = new Subject(); + private readonly ConcurrentDictionary<(string, Type), object> _workerState = new (); + private bool _disposed = false; public void Publish(ScriptEvent scriptEvent) @@ -47,5 +50,35 @@ private void Dispose(bool disposing) } public void Dispose() => Dispose(true); + + bool IScriptEventManager.TryAddWorkerState(string workerId, T state) + { + var key = (workerId, typeof(T)); + return _workerState.TryAdd(key, state); + } + + bool IScriptEventManager.TryGetWorkerState(string workerId, out T state) + { + var key = (workerId, typeof(T)); + if (_workerState.TryGetValue(key, out var tmp) && tmp is T typed) + { + state = typed; + return true; + } + state = default; + return false; + } + + bool IScriptEventManager.TryRemoveWorkerState(string workerId, out T state) + { + var key = (workerId, typeof(T)); + if (_workerState.TryRemove(key, out var tmp) && tmp is T typed) + { + state = typed; + return true; + } + state = default; + return false; + } } } diff --git a/src/WebJobs.Script/Workers/Rpc/RpcWorkerConstants.cs b/src/WebJobs.Script/Workers/Rpc/RpcWorkerConstants.cs index 5a65d66ab7..2fd104eba8 100644 --- a/src/WebJobs.Script/Workers/Rpc/RpcWorkerConstants.cs +++ b/src/WebJobs.Script/Workers/Rpc/RpcWorkerConstants.cs @@ -51,6 +51,7 @@ public static class RpcWorkerConstants // Host Capabilites public const string V2Compatable = "V2Compatable"; + public const string MultiStream = nameof(MultiStream); // dotnet executable file path components public const string DotNetExecutableName = "dotnet"; diff --git a/test/WebJobs.Script.Tests.Integration/Rpc/WorkerConcurrencyManagerEndToEndTests.cs b/test/WebJobs.Script.Tests.Integration/Rpc/WorkerConcurrencyManagerEndToEndTests.cs index d1053d24fe..00b5b759d3 100644 --- a/test/WebJobs.Script.Tests.Integration/Rpc/WorkerConcurrencyManagerEndToEndTests.cs +++ b/test/WebJobs.Script.Tests.Integration/Rpc/WorkerConcurrencyManagerEndToEndTests.cs @@ -1,6 +1,7 @@ using System; using System.Collections.Generic; using System.Linq; +using System.Threading.Channels; using System.Threading.Tasks; using Microsoft.Azure.WebJobs.Script.Eventing; using Microsoft.Azure.WebJobs.Script.Workers.Rpc; @@ -73,6 +74,15 @@ public IDisposable Subscribe(IObserver observer) } public void Dispose() => ((IDisposable)_scriptEventManager).Dispose(); + + public bool TryAddWorkerState(string workerId, T state) + => _scriptEventManager.TryAddWorkerState(workerId, state); + + public bool TryGetWorkerState(string workerId, out T state) + => _scriptEventManager.TryGetWorkerState(workerId, out state); + + public bool TryRemoveWorkerState(string workerId, out T state) + => _scriptEventManager.TryRemoveWorkerState(workerId, out state); } } } diff --git a/test/WebJobs.Script.Tests.Shared/TestScriptEventManager.cs b/test/WebJobs.Script.Tests.Shared/TestScriptEventManager.cs index 62bddceae6..d92da6cc3c 100644 --- a/test/WebJobs.Script.Tests.Shared/TestScriptEventManager.cs +++ b/test/WebJobs.Script.Tests.Shared/TestScriptEventManager.cs @@ -2,6 +2,7 @@ // Licensed under the MIT License. See License.txt in the project root for license information. using System; +using System.Threading.Channels; using Microsoft.Azure.WebJobs.Script.Eventing; namespace Microsoft.Azure.WebJobs.Script.Tests @@ -16,5 +17,26 @@ public IDisposable Subscribe(IObserver observer) { return null; } + + public bool TryGetDedicatedChannelFor(string workerId, out Channel channel) where T : ScriptEvent + { + channel = default; + return false; + } + + bool IScriptEventManager.TryAddWorkerState(string workerId, T state) + => false; + + bool IScriptEventManager.TryGetWorkerState(string workerId, out T state) + { + state = default; + return false; + } + + bool IScriptEventManager.TryRemoveWorkerState(string workerId, out T state) + { + state = default; + return false; + } } } diff --git a/test/WebJobs.Script.Tests/Configuration/DefaultDependencyValidatorTests.cs b/test/WebJobs.Script.Tests/Configuration/DefaultDependencyValidatorTests.cs index bcb6657413..8c87300597 100644 --- a/test/WebJobs.Script.Tests/Configuration/DefaultDependencyValidatorTests.cs +++ b/test/WebJobs.Script.Tests/Configuration/DefaultDependencyValidatorTests.cs @@ -6,6 +6,7 @@ using System.IO; using System.Linq; using System.Threading; +using System.Threading.Channels; using System.Threading.Tasks; using Microsoft.AspNetCore.Hosting; using Microsoft.Azure.WebJobs.Script.Diagnostics; @@ -151,6 +152,27 @@ public IDisposable Subscribe(IObserver observer) { return null; } + + public bool TryGetDedicatedChannelFor(string workerId, out Channel channel) where T : ScriptEvent + { + channel = null; + return false; + } + + bool IScriptEventManager.TryAddWorkerState(string workerId, T state) + => false; + + bool IScriptEventManager.TryGetWorkerState(string workerId, out T state) + { + state = default; + return false; + } + + bool IScriptEventManager.TryRemoveWorkerState(string workerId, out T state) + { + state = default; + return false; + } } private class MyMetricsLogger : IMetricsLogger diff --git a/test/WebJobs.Script.Tests/Extensions/InboundGrpcEventExtensionsTests.cs b/test/WebJobs.Script.Tests/Extensions/InboundGrpcEventExtensionsTests.cs index 9d63c730d8..c3baf8d6df 100644 --- a/test/WebJobs.Script.Tests/Extensions/InboundGrpcEventExtensionsTests.cs +++ b/test/WebJobs.Script.Tests/Extensions/InboundGrpcEventExtensionsTests.cs @@ -25,7 +25,7 @@ public void TestLogCategories(RpcLogCategory categoryToTest) } }); - Assert.True(inboundEvent.IsLogOfCategory(categoryToTest)); + Assert.True(inboundEvent.Message.RpcLog.LogCategory == categoryToTest); } } } diff --git a/test/WebJobs.Script.Tests/Workers/Rpc/GrpcWorkerChannelTests.cs b/test/WebJobs.Script.Tests/Workers/Rpc/GrpcWorkerChannelTests.cs index 65ce8bc6c2..4e122b6d25 100644 --- a/test/WebJobs.Script.Tests/Workers/Rpc/GrpcWorkerChannelTests.cs +++ b/test/WebJobs.Script.Tests/Workers/Rpc/GrpcWorkerChannelTests.cs @@ -23,6 +23,7 @@ using Microsoft.Extensions.Options; using Moq; using Xunit; +using Xunit.Abstractions; namespace Microsoft.Azure.WebJobs.Script.Tests.Workers.Rpc { @@ -51,10 +52,12 @@ public class GrpcWorkerChannelTests : IDisposable private readonly ISharedMemoryManager _sharedMemoryManager; private readonly IFunctionDataCache _functionDataCache; private readonly IOptions _workerConcurrencyOptions; + private readonly ITestOutputHelper _testOutput; private GrpcWorkerChannel _workerChannel; - public GrpcWorkerChannelTests() + public GrpcWorkerChannelTests(ITestOutputHelper testOutput) { + _testOutput = testOutput; _logger = new TestLogger("FunctionDispatcherTests"); _testFunctionRpcService = new TestFunctionRpcService(_eventManager, _workerId, _logger, _expectedLogMsg); _testWorkerConfig = TestHelpers.GetTestWorkerConfigs().FirstOrDefault(); @@ -105,6 +108,20 @@ public GrpcWorkerChannelTests() _workerConcurrencyOptions); } + private void ShowOutput(string message) + => _testOutput?.WriteLine(message); + + private void ShowOutput(IList messages) + { + if (_testOutput is not null && messages is not null) + { + foreach (var msg in messages) + { + _testOutput.WriteLine(msg.FormattedMessage); + } + } + } + public void Dispose() { _sharedMemoryManager.Dispose(); @@ -139,15 +156,23 @@ public async Task StartWorkerProcessAsync_Invoked_SetupFunctionBuffers_Verify_Re [Fact] public async Task DisposingChannel_NotReadyForInvocation() { - var initTask = _workerChannel.StartWorkerProcessAsync(CancellationToken.None); - _testFunctionRpcService.PublishStartStreamEvent(_workerId); - _testFunctionRpcService.PublishWorkerInitResponseEvent(); - await initTask; - Assert.False(_workerChannel.IsChannelReadyForInvocations()); - _workerChannel.SetupFunctionInvocationBuffers(GetTestFunctionsList("node")); - Assert.True(_workerChannel.IsChannelReadyForInvocations()); - _workerChannel.Dispose(); - Assert.False(_workerChannel.IsChannelReadyForInvocations()); + try + { + var initTask = _workerChannel.StartWorkerProcessAsync(CancellationToken.None); + _testFunctionRpcService.PublishStartStreamEvent(_workerId); + _testFunctionRpcService.PublishWorkerInitResponseEvent(); + await initTask; + Assert.False(_workerChannel.IsChannelReadyForInvocations()); + _workerChannel.SetupFunctionInvocationBuffers(GetTestFunctionsList("node")); + Assert.True(_workerChannel.IsChannelReadyForInvocations()); + _workerChannel.Dispose(); + Assert.False(_workerChannel.IsChannelReadyForInvocations()); + } + finally + { + var traces = _logger.GetLogMessages(); + ShowOutput(traces); + } } [Fact] @@ -203,7 +228,7 @@ public async Task StartWorkerProcessAsync_WorkerProcess_Throws() } [Fact] - public void SendWorkerInitRequest_PublishesOutboundEvent() + public async Task SendWorkerInitRequest_PublishesOutboundEvent() { StartStream startStream = new StartStream() { @@ -216,6 +241,7 @@ public void SendWorkerInitRequest_PublishesOutboundEvent() GrpcEvent rpcEvent = new GrpcEvent(_workerId, startStreamMessage); _workerChannel.SendWorkerInitRequest(rpcEvent); _testFunctionRpcService.PublishWorkerInitResponseEvent(); + await Task.Delay(500); var traces = _logger.GetLogMessages(); Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, _expectedLogMsg))); } @@ -233,7 +259,7 @@ public void WorkerInitRequest_Expected() } [Fact] - public void SendWorkerInitRequest_PublishesOutboundEvent_V2Compatable() + public async Task SendWorkerInitRequest_PublishesOutboundEvent_V2Compatable() { _testEnvironment.SetEnvironmentVariable(EnvironmentSettingNames.FunctionsV2CompatibilityModeKey, "true"); StartStream startStream = new StartStream() @@ -247,6 +273,7 @@ public void SendWorkerInitRequest_PublishesOutboundEvent_V2Compatable() GrpcEvent rpcEvent = new GrpcEvent(_workerId, startStreamMessage); _workerChannel.SendWorkerInitRequest(rpcEvent); _testFunctionRpcService.PublishWorkerInitResponseEvent(); + await Task.Delay(500); var traces = _logger.GetLogMessages(); Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, _expectedLogMsg))); Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, "Worker and host running in V2 compatibility mode"))); @@ -257,9 +284,10 @@ public void SendWorkerInitRequest_PublishesOutboundEvent_V2Compatable() [InlineData(RpcLog.Types.Level.Error, RpcLog.Types.Level.Error)] [InlineData(RpcLog.Types.Level.Warning, RpcLog.Types.Level.Warning)] [InlineData(RpcLog.Types.Level.Trace, RpcLog.Types.Level.Information)] - public void SendSystemLogMessage_PublishesSystemLogMessage(RpcLog.Types.Level levelToTest, RpcLog.Types.Level expectedLogLevel) + public async Task SendSystemLogMessage_PublishesSystemLogMessage(RpcLog.Types.Level levelToTest, RpcLog.Types.Level expectedLogLevel) { _testFunctionRpcService.PublishSystemLogEvent(levelToTest); + await Task.Delay(500); var traces = _logger.GetLogMessages(); Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, _expectedSystemLogMessage) && m.Level.ToString().Equals(expectedLogLevel.ToString()))); } @@ -269,6 +297,7 @@ public async Task SendInvocationRequest_PublishesOutboundEvent() { ScriptInvocationContext scriptInvocationContext = GetTestScriptInvocationContext(Guid.NewGuid(), null); await _workerChannel.SendInvocationRequest(scriptInvocationContext); + await Task.Delay(500); var traces = _logger.GetLogMessages(); Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, _expectedLogMsg))); } @@ -293,6 +322,7 @@ public async Task SendInvocationRequest_InputsTransferredOverSharedMemory() // Send invocation which will be using RpcSharedMemory for the inputs ScriptInvocationContext scriptInvocationContext = GetTestScriptInvocationContextWithSharedMemoryInputs(Guid.NewGuid(), null); await _workerChannel.SendInvocationRequest(scriptInvocationContext); + await Task.Delay(500); var traces = _logger.GetLogMessages(); Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, _expectedLogMsg))); } @@ -347,11 +377,12 @@ public async Task InFlight_Functions_FailedWithException() } [Fact] - public void SendLoadRequests_PublishesOutboundEvents() + public async Task SendLoadRequests_PublishesOutboundEvents() { _metricsLogger.ClearCollections(); _workerChannel.SetupFunctionInvocationBuffers(GetTestFunctionsList("node")); _workerChannel.SendFunctionLoadRequests(null, TimeSpan.FromMinutes(5)); + await Task.Delay(500); var traces = _logger.GetLogMessages(); var functionLoadLogs = traces.Where(m => string.Equals(m.FormattedMessage, _expectedLogMsg)); AreExpectedMetricsGenerated(); @@ -359,7 +390,7 @@ public void SendLoadRequests_PublishesOutboundEvents() } [Fact] - public void SendLoadRequestCollection_PublishesOutboundEvents() + public async Task SendLoadRequestCollection_PublishesOutboundEvents() { StartStream startStream = new StartStream() { @@ -376,7 +407,9 @@ public void SendLoadRequestCollection_PublishesOutboundEvents() IEnumerable functionMetadata = GetTestFunctionsList("node"); _workerChannel.SetupFunctionInvocationBuffers(functionMetadata); _workerChannel.SendFunctionLoadRequests(null, TimeSpan.FromMinutes(5)); + await Task.Delay(500); var traces = _logger.GetLogMessages(); + ShowOutput(traces); var functionLoadLogs = traces.Where(m => string.Equals(m.FormattedMessage, _expectedLogMsg)); AreExpectedMetricsGenerated(); Assert.True(functionLoadLogs.Count() == 2); @@ -384,7 +417,7 @@ public void SendLoadRequestCollection_PublishesOutboundEvents() } [Fact] - public void SendLoadRequests_PublishesOutboundEvents_OrdersDisabled() + public async Task SendLoadRequests_PublishesOutboundEvents_OrdersDisabled() { var funcName = "ADisabledFunc"; var functions = GetTestFunctionsList_WithDisabled("node", funcName); @@ -394,7 +427,9 @@ public void SendLoadRequests_PublishesOutboundEvents_OrdersDisabled() _workerChannel.SetupFunctionInvocationBuffers(functions); _workerChannel.SendFunctionLoadRequests(null, TimeSpan.FromMinutes(5)); + await Task.Delay(500); var traces = _logger.GetLogMessages(); + ShowOutput(traces); var functionLoadLogs = traces.Where(m => m.FormattedMessage?.Contains(_expectedLoadMsgPartial) ?? false); var t = functionLoadLogs.Last().FormattedMessage; @@ -405,26 +440,41 @@ public void SendLoadRequests_PublishesOutboundEvents_OrdersDisabled() } [Fact] - public void SendLoadRequests_DoesNotTimeout_FunctionTimeoutNotSet() + public async Task SendLoadRequests_DoesNotTimeout_FunctionTimeoutNotSet() { var funcName = "ADisabledFunc"; var functions = GetTestFunctionsList_WithDisabled("node", funcName); _workerChannel.SetupFunctionInvocationBuffers(functions); _workerChannel.SendFunctionLoadRequests(null, null); + await Task.Delay(500); var traces = _logger.GetLogMessages(); + ShowOutput(traces); var errorLogs = traces.Where(m => m.Level == LogLevel.Error); Assert.Empty(errorLogs); } [Fact] - public void SendSendFunctionEnvironmentReloadRequest_PublishesOutboundEvents() + public async Task SendSendFunctionEnvironmentReloadRequest_PublishesOutboundEvents() { - Environment.SetEnvironmentVariable("TestNull", null); - Environment.SetEnvironmentVariable("TestEmpty", string.Empty); - Environment.SetEnvironmentVariable("TestValid", "TestValue"); - _workerChannel.SendFunctionEnvironmentReloadRequest(); - _testFunctionRpcService.PublishFunctionEnvironmentReloadResponseEvent(); + try + { + Environment.SetEnvironmentVariable("TestNull", null); + Environment.SetEnvironmentVariable("TestEmpty", string.Empty); + Environment.SetEnvironmentVariable("TestValid", "TestValue"); + var pending = _workerChannel.SendFunctionEnvironmentReloadRequest(); + _testFunctionRpcService.PublishFunctionEnvironmentReloadResponseEvent(); + await Task.Delay(500); + await pending; // this can timeout + } + catch + { + // show what we got even if we fail + var tmp = _logger.GetLogMessages(); + ShowOutput(tmp); + throw; + } var traces = _logger.GetLogMessages(); + ShowOutput(traces); var functionLoadLogs = traces.Where(m => string.Equals(m.FormattedMessage, "Sending FunctionEnvironmentReloadRequest to WorkerProcess with Pid: '910'")); Assert.True(functionLoadLogs.Count() == 1); } @@ -469,28 +519,31 @@ public void SendFunctionEnvironmentReloadRequest_WithDirectory() } [Fact] - public void ReceivesInboundEvent_InvocationResponse() + public async Task ReceivesInboundEvent_InvocationResponse() { _testFunctionRpcService.PublishInvocationResponseEvent(); + await Task.Delay(500); var traces = _logger.GetLogMessages(); Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, "InvocationResponse received for invocation id: 'TestInvocationId'"))); } [Fact] - public void ReceivesInboundEvent_FunctionLoadResponse() + public async Task ReceivesInboundEvent_FunctionLoadResponse() { var functionMetadatas = GetTestFunctionsList("node"); _workerChannel.SetupFunctionInvocationBuffers(functionMetadatas); _workerChannel.SendFunctionLoadRequests(null, TimeSpan.FromMinutes(5)); _testFunctionRpcService.PublishFunctionLoadResponseEvent("TestFunctionId1"); + await Task.Delay(500); var traces = _logger.GetLogMessages(); - Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, "Setting up FunctionInvocationBuffer for function: 'js1' with functionId: 'TestFunctionId1'"))); - Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, "Setting up FunctionInvocationBuffer for function: 'js2' with functionId: 'TestFunctionId2'"))); - Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, "Received FunctionLoadResponse for function: 'js1' with functionId: 'TestFunctionId1'."))); + ShowOutput(traces); + Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, "Setting up FunctionInvocationBuffer for function: 'js1' with functionId: 'TestFunctionId1'")), "FunctionInvocationBuffer TestFunctionId1"); + Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, "Setting up FunctionInvocationBuffer for function: 'js2' with functionId: 'TestFunctionId2'")), "FunctionInvocationBuffer TestFunctionId2"); + Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, "Received FunctionLoadResponse for function: 'js1' with functionId: 'TestFunctionId1'.")), "FunctionLoadResponse TestFunctionId1"); } [Fact] - public void ReceivesInboundEvent_Failed_FunctionLoadResponses() + public async Task ReceivesInboundEvent_Failed_FunctionLoadResponses() { var functionMetadatas = GetTestFunctionsList("node"); _workerChannel.SetupFunctionInvocationBuffers(functionMetadatas); @@ -498,15 +551,17 @@ public void ReceivesInboundEvent_Failed_FunctionLoadResponses() _testFunctionRpcService.PublishFunctionLoadResponsesEvent( new List() { "TestFunctionId1", "TestFunctionId2" }, new StatusResult() { Status = StatusResult.Types.Status.Failure }); + await Task.Delay(500); var traces = _logger.GetLogMessages(); - Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, "Setting up FunctionInvocationBuffer for function: 'js1' with functionId: 'TestFunctionId1'"))); - Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, "Setting up FunctionInvocationBuffer for function: 'js2' with functionId: 'TestFunctionId2'"))); - Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, "Worker failed to load function: 'js1' with function id: 'TestFunctionId1'."))); - Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, "Worker failed to load function: 'js2' with function id: 'TestFunctionId2'."))); + ShowOutput(traces); + Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, "Setting up FunctionInvocationBuffer for function: 'js1' with functionId: 'TestFunctionId1'")), "FunctionInvocationBuffer TestFunctionId1"); + Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, "Setting up FunctionInvocationBuffer for function: 'js2' with functionId: 'TestFunctionId2'")), "FunctionInvocationBuffer TestFunctionId2"); + Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, "Worker failed to load function: 'js1' with function id: 'TestFunctionId1'.")), "failed to load TestFunctionId1"); + Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, "Worker failed to load function: 'js2' with function id: 'TestFunctionId2'.")), "failed to load TestFunctionId2"); } [Fact] - public void ReceivesInboundEvent_FunctionLoadResponses() + public async Task ReceivesInboundEvent_FunctionLoadResponses() { var functionMetadatas = GetTestFunctionsList("node"); _workerChannel.SetupFunctionInvocationBuffers(functionMetadatas); @@ -514,7 +569,9 @@ public void ReceivesInboundEvent_FunctionLoadResponses() _testFunctionRpcService.PublishFunctionLoadResponsesEvent( new List() { "TestFunctionId1", "TestFunctionId2" }, new StatusResult() { Status = StatusResult.Types.Status.Success }); + await Task.Delay(500); var traces = _logger.GetLogMessages(); + ShowOutput(traces); Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, "Setting up FunctionInvocationBuffer for function: 'js1' with functionId: 'TestFunctionId1'"))); Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, "Setting up FunctionInvocationBuffer for function: 'js2' with functionId: 'TestFunctionId2'"))); Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, string.Format("Received FunctionLoadResponseCollection with number of functions: '{0}'.", functionMetadatas.ToList().Count)))); @@ -523,68 +580,80 @@ public void ReceivesInboundEvent_FunctionLoadResponses() } [Fact] - public void ReceivesInboundEvent_Successful_FunctionMetadataResponse() + public async Task ReceivesInboundEvent_Successful_FunctionMetadataResponse() { var functionMetadata = GetTestFunctionsList("python"); var functions = _workerChannel.GetFunctionMetadata(); var functionId = "id123"; _testFunctionRpcService.PublishWorkerMetadataResponse("TestFunctionId1", functionId, functionMetadata, true); + await Task.Delay(500); var traces = _logger.GetLogMessages(); + ShowOutput(traces); Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, $"Received the worker function metadata response from worker {_workerChannel.Id}"))); } [Fact] - public void ReceivesInboundEvent_Successful_FunctionMetadataResponse_UseDefaultMetadataIndexing_True() + public async Task ReceivesInboundEvent_Successful_FunctionMetadataResponse_UseDefaultMetadataIndexing_True() { var functionMetadata = GetTestFunctionsList("python"); var functions = _workerChannel.GetFunctionMetadata(); var functionId = "id123"; _testFunctionRpcService.PublishWorkerMetadataResponse("TestFunctionId1", functionId, functionMetadata, true, useDefaultMetadataIndexing: true); + await Task.Delay(500); var traces = _logger.GetLogMessages(); + ShowOutput(traces); Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, $"Received the worker function metadata response from worker {_workerChannel.Id}"))); } [Fact] - public void ReceivesInboundEvent_Successful_FunctionMetadataResponse_UseDefaultMetadataIndexing_False() + public async Task ReceivesInboundEvent_Successful_FunctionMetadataResponse_UseDefaultMetadataIndexing_False() { var functionMetadata = GetTestFunctionsList("python"); var functions = _workerChannel.GetFunctionMetadata(); var functionId = "id123"; _testFunctionRpcService.PublishWorkerMetadataResponse("TestFunctionId1", functionId, functionMetadata, true, useDefaultMetadataIndexing: false); + await Task.Delay(500); var traces = _logger.GetLogMessages(); + ShowOutput(traces); Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, $"Received the worker function metadata response from worker {_workerChannel.Id}"))); } [Fact] - public void ReceivesInboundEvent_Failed_UseDefaultMetadataIndexing_True_HostIndexing() + public async Task ReceivesInboundEvent_Failed_UseDefaultMetadataIndexing_True_HostIndexing() { var functionMetadata = GetTestFunctionsList("python"); var functions = _workerChannel.GetFunctionMetadata(); var functionId = "id123"; _testFunctionRpcService.PublishWorkerMetadataResponse("TestFunctionId1", functionId, functionMetadata, false, useDefaultMetadataIndexing: true); + await Task.Delay(500); var traces = _logger.GetLogMessages(); + ShowOutput(traces); Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, $"Received the worker function metadata response from worker {_workerChannel.Id}"))); } [Fact] - public void ReceivesInboundEvent_Failed_UseDefaultMetadataIndexing_False_WorkerIndexing() + public async Task ReceivesInboundEvent_Failed_UseDefaultMetadataIndexing_False_WorkerIndexing() { var functionMetadata = GetTestFunctionsList("python"); var functions = _workerChannel.GetFunctionMetadata(); var functionId = "id123"; _testFunctionRpcService.PublishWorkerMetadataResponse("TestFunctionId1", functionId, functionMetadata, false, useDefaultMetadataIndexing: false); + await Task.Delay(500); var traces = _logger.GetLogMessages(); + ShowOutput(traces); Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, $"Worker failed to index function {functionId}"))); } [Fact] - public void ReceivesInboundEvent_Failed_FunctionMetadataResponse() + public async Task ReceivesInboundEvent_Failed_FunctionMetadataResponse() { var functionMetadata = GetTestFunctionsList("python"); var functions = _workerChannel.GetFunctionMetadata(); var functionId = "id123"; _testFunctionRpcService.PublishWorkerMetadataResponse("TestFunctionId1", functionId, functionMetadata, false); + await Task.Delay(500); var traces = _logger.GetLogMessages(); + ShowOutput(traces); Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, $"Worker failed to index function {functionId}"))); } @@ -607,10 +676,10 @@ public void FunctionLoadRequest_IsExpected() /// Verify that shared memory data transfer is enabled if all required settings are set. /// [Fact] - public void SharedMemoryDataTransferSetting_VerifyEnabled() + public async Task SharedMemoryDataTransferSetting_VerifyEnabled() { EnableSharedMemoryDataTransfer(); - + await Task.Delay(500); Assert.True(_workerChannel.IsSharedMemoryDataTransferEnabled()); } diff --git a/test/WebJobs.Script.Tests/Workers/Rpc/RpcWorkerConfigFactoryTests.cs b/test/WebJobs.Script.Tests/Workers/Rpc/RpcWorkerConfigFactoryTests.cs index baea4e1087..517d6deac0 100644 --- a/test/WebJobs.Script.Tests/Workers/Rpc/RpcWorkerConfigFactoryTests.cs +++ b/test/WebJobs.Script.Tests/Workers/Rpc/RpcWorkerConfigFactoryTests.cs @@ -92,6 +92,12 @@ public void LanguageWorker_WorkersDir_NotSet() [Fact] public void JavaPath_FromEnvVars() { + var javaHome = Environment.GetEnvironmentVariable("JAVA_HOME"); + if (string.IsNullOrWhiteSpace(javaHome)) + { + // if the var doesn't exist, set something temporary to make it at least work + Environment.SetEnvironmentVariable("JAVA_HOME", Path.GetTempPath()); + } var configBuilder = ScriptSettingsManager.CreateDefaultConfigurationBuilder(); var config = configBuilder.Build(); var scriptSettingsManager = new ScriptSettingsManager(config); diff --git a/test/WebJobs.Script.Tests/Workers/Rpc/TestFunctionRpcService.cs b/test/WebJobs.Script.Tests/Workers/Rpc/TestFunctionRpcService.cs index 7ddf91527b..16da2f51cb 100644 --- a/test/WebJobs.Script.Tests/Workers/Rpc/TestFunctionRpcService.cs +++ b/test/WebJobs.Script.Tests/Workers/Rpc/TestFunctionRpcService.cs @@ -6,6 +6,7 @@ using System.Linq; using System.Reactive.Linq; using System.Runtime.InteropServices; +using System.Threading.Channels; using System.Threading.Tasks; using Microsoft.Azure.WebJobs.Script.Description; using Microsoft.Azure.WebJobs.Script.Diagnostics; @@ -30,18 +31,76 @@ public class TestFunctionRpcService private ILogger _logger; private string _workerId; private IDictionary _outboundEventSubscriptions = new Dictionary(); + private ChannelWriter _inboundWriter; public TestFunctionRpcService(IScriptEventManager eventManager, string workerId, TestLogger logger, string expectedLogMsg = "") { _eventManager = eventManager; _logger = logger; _workerId = workerId; - _outboundEventSubscriptions.Add(workerId, _eventManager.OfType() - .Where(evt => evt.WorkerId == workerId) - .Subscribe(evt => - { - _logger.LogInformation(expectedLogMsg); - })); + if (eventManager.TryGetGrpcChannels(workerId, out var inbound, out var outbound)) + { + _ = ListenAsync(outbound.Reader, expectedLogMsg); + _inboundWriter = inbound.Writer; + } + } + + private async Task ListenAsync(ChannelReader source, string expectedLogMsg) + { + await Task.Yield(); // free up caller + try + { + while (await source.WaitToReadAsync()) + { + while (source.TryRead(out var evt)) + { + _logger.LogDebug("[service] received {0}, {1}", evt.WorkerId, evt.MessageType); + _logger.LogInformation(expectedLogMsg); + } + } + } + catch + { + } + } + + private ValueTask WriteAsync(StreamingMessage message) + => _inboundWriter is null ? default + : _inboundWriter.WriteAsync(new InboundGrpcEvent(_workerId, message)); + + private void Write(StreamingMessage message) + { + if (_inboundWriter is null) + { + _logger.LogDebug("[service] no writer for {0}, {1}", _workerId, message.ContentCase); + return; + } + var evt = new InboundGrpcEvent(_workerId, message); + _logger.LogDebug("[service] sending {0}, {1}", evt.WorkerId, evt.MessageType); + if (_inboundWriter.TryWrite(evt)) + { + return; + } + var vt = _inboundWriter.WriteAsync(evt); + if (vt.IsCompleted) + { + vt.GetAwaiter().GetResult(); + } + else + { + _ = ObserveEventually(vt); + } + static async Task ObserveEventually(ValueTask valueTask) + { + try + { + await valueTask; + } + catch + { + // log somewhere? + } + } } public void PublishFunctionLoadResponseEvent(string functionId) @@ -59,7 +118,7 @@ public void PublishFunctionLoadResponseEvent(string functionId) { FunctionLoadResponse = functionLoadResponse }; - _eventManager.Publish(new InboundGrpcEvent(_workerId, responseMessage)); + Write(responseMessage); } public void PublishFunctionLoadResponsesEvent(List functionIds, StatusResult statusResult) @@ -81,7 +140,7 @@ public void PublishFunctionLoadResponsesEvent(List functionIds, StatusRe { FunctionLoadResponseCollection = functionLoadResponseCollection }; - _eventManager.Publish(new InboundGrpcEvent(_workerId, responseMessage)); + Write(responseMessage); } public void PublishFunctionEnvironmentReloadResponseEvent() @@ -91,7 +150,7 @@ public void PublishFunctionEnvironmentReloadResponseEvent() { FunctionEnvironmentReloadResponse = relaodEnvResponse }; - _eventManager.Publish(new InboundGrpcEvent(_workerId, responseMessage)); + Write(responseMessage); } public void PublishWorkerInitResponseEvent(IDictionary capabilities = null) @@ -116,7 +175,7 @@ public void PublishWorkerInitResponseEvent(IDictionary capabilit WorkerInitResponse = initResponse }; - _eventManager.Publish(new InboundGrpcEvent(_workerId, responseMessage)); + Write(responseMessage); } public void PublishWorkerInitResponseEventWithSharedMemoryDataTransferCapability() @@ -133,7 +192,7 @@ public void PublishWorkerInitResponseEventWithSharedMemoryDataTransferCapability { WorkerInitResponse = initResponse }; - _eventManager.Publish(new InboundGrpcEvent(_workerId, responseMessage)); + Write(responseMessage); } public void PublishSystemLogEvent(RpcLog.Types.Level inputLevel) @@ -149,7 +208,7 @@ public void PublishSystemLogEvent(RpcLog.Types.Level inputLevel) { RpcLog = rpcLog }; - _eventManager.Publish(new InboundGrpcEvent(_workerId, logMessage)); + Write(logMessage); } public static FunctionEnvironmentReloadResponse GetTestFunctionEnvReloadResponse() @@ -180,7 +239,7 @@ public void PublishInvocationResponseEvent() { InvocationResponse = invocationResponse }; - _eventManager.Publish(new InboundGrpcEvent(_workerId, responseMessage)); + Write(responseMessage); } public void PublishStartStreamEvent(string workerId) @@ -197,7 +256,7 @@ public void PublishStartStreamEvent(string workerId) { StartStream = startStream }; - _eventManager.Publish(new InboundGrpcEvent(_workerId, responseMessage)); + Write(responseMessage); } public void PublishWorkerMetadataResponse(string workerId, string functionId, IEnumerable functionMetadata, bool successful, bool useDefaultMetadataIndexing = false) @@ -232,7 +291,7 @@ public void PublishWorkerMetadataResponse(string workerId, string functionId, IE { FunctionMetadataResponse = overallResponse }; - _eventManager.Publish(new InboundGrpcEvent(_workerId, responseMessage)); + Write(responseMessage); } } } \ No newline at end of file From ea4aee8fe9d7dc99feb9196ddef731be2b7d44fb Mon Sep 17 00:00:00 2001 From: Marc Gravell Date: Tue, 5 Apr 2022 12:28:04 +0100 Subject: [PATCH 02/17] fixup test failures 1: GrpcWorkerChannelTests span up duplicate channels 2: suppress write failures (async void) --- .../Channel/GrpcWorkerChannel.cs | 11 ++++- .../Workers/Rpc/GrpcWorkerChannelTests.cs | 45 +++++++++++++++++++ .../Workers/Rpc/TestFunctionRpcService.cs | 17 ++++--- 3 files changed, 66 insertions(+), 7 deletions(-) diff --git a/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs b/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs index 12a37f322b..df844a1798 100644 --- a/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs +++ b/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs @@ -964,7 +964,14 @@ private void SendStreamingMessage(StreamingMessage msg) var pending = _outbound.WriteAsync(evt); if (pending.IsCompleted) { - pending.GetAwaiter().GetResult(); // ensure observed, for IVTS reasons + try + { + pending.GetAwaiter().GetResult(); // ensure observed, for IVTS reasons + } + catch + { + // suppress failure + } } else { @@ -979,7 +986,7 @@ static async Task ObserveEventually(ValueTask valueTask) } catch { - // log somewhere? + // nowehere to log } } } diff --git a/test/WebJobs.Script.Tests/Workers/Rpc/GrpcWorkerChannelTests.cs b/test/WebJobs.Script.Tests/Workers/Rpc/GrpcWorkerChannelTests.cs index 4e122b6d25..f3806d5c5e 100644 --- a/test/WebJobs.Script.Tests/Workers/Rpc/GrpcWorkerChannelTests.cs +++ b/test/WebJobs.Script.Tests/Workers/Rpc/GrpcWorkerChannelTests.cs @@ -57,6 +57,7 @@ public class GrpcWorkerChannelTests : IDisposable public GrpcWorkerChannelTests(ITestOutputHelper testOutput) { + _eventManager.AddGrpcChannels(_workerId); _testOutput = testOutput; _logger = new TestLogger("FunctionDispatcherTests"); _testFunctionRpcService = new TestFunctionRpcService(_eventManager, _workerId, _logger, _expectedLogMsg); @@ -92,7 +93,10 @@ public GrpcWorkerChannelTests(ITestOutputHelper testOutput) HasParentScope = true }; _hostOptionsMonitor = TestHelpers.CreateOptionsMonitor(hostOptions); + } + private void CreateDefaultWorkerChannel() + { _workerChannel = new GrpcWorkerChannel( _workerId, _eventManager, @@ -130,6 +134,7 @@ public void Dispose() [Fact] public async Task StartWorkerProcessAsync_ThrowsTaskCanceledException_IfDisposed() { + CreateDefaultWorkerChannel(); var initTask = _workerChannel.StartWorkerProcessAsync(CancellationToken.None); _workerChannel.Dispose(); _testFunctionRpcService.PublishStartStreamEvent(_workerId); @@ -143,6 +148,7 @@ await Assert.ThrowsAsync(async () => [Fact] public async Task StartWorkerProcessAsync_Invoked_SetupFunctionBuffers_Verify_ReadyForInvocation() { + CreateDefaultWorkerChannel(); var initTask = _workerChannel.StartWorkerProcessAsync(CancellationToken.None); _testFunctionRpcService.PublishStartStreamEvent(_workerId); _testFunctionRpcService.PublishWorkerInitResponseEvent(); @@ -156,6 +162,7 @@ public async Task StartWorkerProcessAsync_Invoked_SetupFunctionBuffers_Verify_Re [Fact] public async Task DisposingChannel_NotReadyForInvocation() { + CreateDefaultWorkerChannel(); try { var initTask = _workerChannel.StartWorkerProcessAsync(CancellationToken.None); @@ -178,6 +185,7 @@ public async Task DisposingChannel_NotReadyForInvocation() [Fact] public void SetupFunctionBuffers_Verify_ReadyForInvocation_Returns_False() { + CreateDefaultWorkerChannel(); Assert.False(_workerChannel.IsChannelReadyForInvocations()); _workerChannel.SetupFunctionInvocationBuffers(GetTestFunctionsList("node")); Assert.False(_workerChannel.IsChannelReadyForInvocations()); @@ -186,6 +194,7 @@ public void SetupFunctionBuffers_Verify_ReadyForInvocation_Returns_False() [Fact] public async Task StartWorkerProcessAsync_TimesOut() { + CreateDefaultWorkerChannel(); var initTask = _workerChannel.StartWorkerProcessAsync(CancellationToken.None); await Assert.ThrowsAsync(async () => await initTask); } @@ -193,6 +202,7 @@ public async Task StartWorkerProcessAsync_TimesOut() [Fact] public async Task SendEnvironmentReloadRequest_Generates_ExpectedMetrics() { + CreateDefaultWorkerChannel(); _metricsLogger.ClearCollections(); Task waitForMetricsTask = Task.Factory.StartNew(() => { @@ -208,6 +218,7 @@ public async Task SendEnvironmentReloadRequest_Generates_ExpectedMetrics() [Fact] public async Task StartWorkerProcessAsync_WorkerProcess_Throws() { + // note: uses custom worker channel Mock mockrpcWorkerProcessThatThrows = new Mock(); mockrpcWorkerProcessThatThrows.Setup(m => m.StartProcessAsync()).Throws(); @@ -230,6 +241,7 @@ public async Task StartWorkerProcessAsync_WorkerProcess_Throws() [Fact] public async Task SendWorkerInitRequest_PublishesOutboundEvent() { + CreateDefaultWorkerChannel(); StartStream startStream = new StartStream() { WorkerId = _workerId @@ -249,6 +261,7 @@ public async Task SendWorkerInitRequest_PublishesOutboundEvent() [Fact] public void WorkerInitRequest_Expected() { + CreateDefaultWorkerChannel(); WorkerInitRequest initRequest = _workerChannel.GetWorkerInitRequest(); Assert.NotNull(initRequest.WorkerDirectory); Assert.NotNull(initRequest.FunctionAppDirectory); @@ -261,6 +274,7 @@ public void WorkerInitRequest_Expected() [Fact] public async Task SendWorkerInitRequest_PublishesOutboundEvent_V2Compatable() { + CreateDefaultWorkerChannel(); _testEnvironment.SetEnvironmentVariable(EnvironmentSettingNames.FunctionsV2CompatibilityModeKey, "true"); StartStream startStream = new StartStream() { @@ -286,6 +300,7 @@ public async Task SendWorkerInitRequest_PublishesOutboundEvent_V2Compatable() [InlineData(RpcLog.Types.Level.Trace, RpcLog.Types.Level.Information)] public async Task SendSystemLogMessage_PublishesSystemLogMessage(RpcLog.Types.Level levelToTest, RpcLog.Types.Level expectedLogLevel) { + CreateDefaultWorkerChannel(); _testFunctionRpcService.PublishSystemLogEvent(levelToTest); await Task.Delay(500); var traces = _logger.GetLogMessages(); @@ -295,6 +310,7 @@ public async Task SendSystemLogMessage_PublishesSystemLogMessage(RpcLog.Types.Le [Fact] public async Task SendInvocationRequest_PublishesOutboundEvent() { + CreateDefaultWorkerChannel(); ScriptInvocationContext scriptInvocationContext = GetTestScriptInvocationContext(Guid.NewGuid(), null); await _workerChannel.SendInvocationRequest(scriptInvocationContext); await Task.Delay(500); @@ -305,6 +321,7 @@ public async Task SendInvocationRequest_PublishesOutboundEvent() [Fact] public async Task SendInvocationRequest_IsInExecutingInvocation() { + CreateDefaultWorkerChannel(); ScriptInvocationContext scriptInvocationContext = GetTestScriptInvocationContext(Guid.NewGuid(), null); await _workerChannel.SendInvocationRequest(scriptInvocationContext); Assert.True(_workerChannel.IsExecutingInvocation(scriptInvocationContext.ExecutionContext.InvocationId.ToString())); @@ -317,6 +334,7 @@ public async Task SendInvocationRequest_IsInExecutingInvocation() [Fact] public async Task SendInvocationRequest_InputsTransferredOverSharedMemory() { + CreateDefaultWorkerChannel(); EnableSharedMemoryDataTransfer(); // Send invocation which will be using RpcSharedMemory for the inputs @@ -330,6 +348,7 @@ public async Task SendInvocationRequest_InputsTransferredOverSharedMemory() [Fact] public async Task Drain_Verify() { + // note: uses custom worker channel var resultSource = new TaskCompletionSource(); Guid invocationId = Guid.NewGuid(); GrpcWorkerChannel channel = new GrpcWorkerChannel( @@ -365,6 +384,7 @@ await channel.InvokeResponse(new InvocationResponse [Fact] public async Task InFlight_Functions_FailedWithException() { + CreateDefaultWorkerChannel(); var resultSource = new TaskCompletionSource(); ScriptInvocationContext scriptInvocationContext = GetTestScriptInvocationContext(Guid.NewGuid(), resultSource); await _workerChannel.SendInvocationRequest(scriptInvocationContext); @@ -379,6 +399,7 @@ public async Task InFlight_Functions_FailedWithException() [Fact] public async Task SendLoadRequests_PublishesOutboundEvents() { + CreateDefaultWorkerChannel(); _metricsLogger.ClearCollections(); _workerChannel.SetupFunctionInvocationBuffers(GetTestFunctionsList("node")); _workerChannel.SendFunctionLoadRequests(null, TimeSpan.FromMinutes(5)); @@ -392,6 +413,7 @@ public async Task SendLoadRequests_PublishesOutboundEvents() [Fact] public async Task SendLoadRequestCollection_PublishesOutboundEvents() { + CreateDefaultWorkerChannel(); StartStream startStream = new StartStream() { WorkerId = _workerId @@ -419,6 +441,7 @@ public async Task SendLoadRequestCollection_PublishesOutboundEvents() [Fact] public async Task SendLoadRequests_PublishesOutboundEvents_OrdersDisabled() { + CreateDefaultWorkerChannel(); var funcName = "ADisabledFunc"; var functions = GetTestFunctionsList_WithDisabled("node", funcName); @@ -442,6 +465,7 @@ public async Task SendLoadRequests_PublishesOutboundEvents_OrdersDisabled() [Fact] public async Task SendLoadRequests_DoesNotTimeout_FunctionTimeoutNotSet() { + CreateDefaultWorkerChannel(); var funcName = "ADisabledFunc"; var functions = GetTestFunctionsList_WithDisabled("node", funcName); _workerChannel.SetupFunctionInvocationBuffers(functions); @@ -456,6 +480,7 @@ public async Task SendLoadRequests_DoesNotTimeout_FunctionTimeoutNotSet() [Fact] public async Task SendSendFunctionEnvironmentReloadRequest_PublishesOutboundEvents() { + CreateDefaultWorkerChannel(); try { Environment.SetEnvironmentVariable("TestNull", null); @@ -482,6 +507,7 @@ public async Task SendSendFunctionEnvironmentReloadRequest_PublishesOutboundEven [Fact] public async Task SendSendFunctionEnvironmentReloadRequest_ThrowsTimeout() { + CreateDefaultWorkerChannel(); var reloadTask = _workerChannel.SendFunctionEnvironmentReloadRequest(); await Assert.ThrowsAsync(async () => await reloadTask); } @@ -489,6 +515,7 @@ public async Task SendSendFunctionEnvironmentReloadRequest_ThrowsTimeout() [Fact] public void SendFunctionEnvironmentReloadRequest_SanitizedEnvironmentVariables() { + CreateDefaultWorkerChannel(); var environmentVariables = new Dictionary() { { "TestNull", null }, @@ -508,6 +535,7 @@ public void SendFunctionEnvironmentReloadRequest_SanitizedEnvironmentVariables() [Fact] public void SendFunctionEnvironmentReloadRequest_WithDirectory() { + CreateDefaultWorkerChannel(); var environmentVariables = new Dictionary() { { "TestValid", "TestValue" } @@ -521,6 +549,7 @@ public void SendFunctionEnvironmentReloadRequest_WithDirectory() [Fact] public async Task ReceivesInboundEvent_InvocationResponse() { + CreateDefaultWorkerChannel(); _testFunctionRpcService.PublishInvocationResponseEvent(); await Task.Delay(500); var traces = _logger.GetLogMessages(); @@ -530,6 +559,7 @@ public async Task ReceivesInboundEvent_InvocationResponse() [Fact] public async Task ReceivesInboundEvent_FunctionLoadResponse() { + CreateDefaultWorkerChannel(); var functionMetadatas = GetTestFunctionsList("node"); _workerChannel.SetupFunctionInvocationBuffers(functionMetadatas); _workerChannel.SendFunctionLoadRequests(null, TimeSpan.FromMinutes(5)); @@ -545,6 +575,7 @@ public async Task ReceivesInboundEvent_FunctionLoadResponse() [Fact] public async Task ReceivesInboundEvent_Failed_FunctionLoadResponses() { + CreateDefaultWorkerChannel(); var functionMetadatas = GetTestFunctionsList("node"); _workerChannel.SetupFunctionInvocationBuffers(functionMetadatas); _workerChannel.SendFunctionLoadRequests(null, TimeSpan.FromMinutes(1)); @@ -563,6 +594,7 @@ public async Task ReceivesInboundEvent_Failed_FunctionLoadResponses() [Fact] public async Task ReceivesInboundEvent_FunctionLoadResponses() { + CreateDefaultWorkerChannel(); var functionMetadatas = GetTestFunctionsList("node"); _workerChannel.SetupFunctionInvocationBuffers(functionMetadatas); _workerChannel.SendFunctionLoadRequests(null, TimeSpan.FromMinutes(1)); @@ -582,6 +614,7 @@ public async Task ReceivesInboundEvent_FunctionLoadResponses() [Fact] public async Task ReceivesInboundEvent_Successful_FunctionMetadataResponse() { + CreateDefaultWorkerChannel(); var functionMetadata = GetTestFunctionsList("python"); var functions = _workerChannel.GetFunctionMetadata(); var functionId = "id123"; @@ -595,6 +628,7 @@ public async Task ReceivesInboundEvent_Successful_FunctionMetadataResponse() [Fact] public async Task ReceivesInboundEvent_Successful_FunctionMetadataResponse_UseDefaultMetadataIndexing_True() { + CreateDefaultWorkerChannel(); var functionMetadata = GetTestFunctionsList("python"); var functions = _workerChannel.GetFunctionMetadata(); var functionId = "id123"; @@ -608,6 +642,7 @@ public async Task ReceivesInboundEvent_Successful_FunctionMetadataResponse_UseDe [Fact] public async Task ReceivesInboundEvent_Successful_FunctionMetadataResponse_UseDefaultMetadataIndexing_False() { + CreateDefaultWorkerChannel(); var functionMetadata = GetTestFunctionsList("python"); var functions = _workerChannel.GetFunctionMetadata(); var functionId = "id123"; @@ -621,6 +656,7 @@ public async Task ReceivesInboundEvent_Successful_FunctionMetadataResponse_UseDe [Fact] public async Task ReceivesInboundEvent_Failed_UseDefaultMetadataIndexing_True_HostIndexing() { + CreateDefaultWorkerChannel(); var functionMetadata = GetTestFunctionsList("python"); var functions = _workerChannel.GetFunctionMetadata(); var functionId = "id123"; @@ -634,6 +670,7 @@ public async Task ReceivesInboundEvent_Failed_UseDefaultMetadataIndexing_True_Ho [Fact] public async Task ReceivesInboundEvent_Failed_UseDefaultMetadataIndexing_False_WorkerIndexing() { + CreateDefaultWorkerChannel(); var functionMetadata = GetTestFunctionsList("python"); var functions = _workerChannel.GetFunctionMetadata(); var functionId = "id123"; @@ -647,6 +684,7 @@ public async Task ReceivesInboundEvent_Failed_UseDefaultMetadataIndexing_False_W [Fact] public async Task ReceivesInboundEvent_Failed_FunctionMetadataResponse() { + CreateDefaultWorkerChannel(); var functionMetadata = GetTestFunctionsList("python"); var functions = _workerChannel.GetFunctionMetadata(); var functionId = "id123"; @@ -660,6 +698,7 @@ public async Task ReceivesInboundEvent_Failed_FunctionMetadataResponse() [Fact] public void FunctionLoadRequest_IsExpected() { + CreateDefaultWorkerChannel(); FunctionMetadata metadata = new FunctionMetadata() { Language = "node", @@ -678,6 +717,7 @@ public void FunctionLoadRequest_IsExpected() [Fact] public async Task SharedMemoryDataTransferSetting_VerifyEnabled() { + CreateDefaultWorkerChannel(); EnableSharedMemoryDataTransfer(); await Task.Delay(500); Assert.True(_workerChannel.IsSharedMemoryDataTransferEnabled()); @@ -689,6 +729,7 @@ public async Task SharedMemoryDataTransferSetting_VerifyEnabled() [Fact] public void SharedMemoryDataTransferSetting_VerifyDisabled() { + CreateDefaultWorkerChannel(); Assert.False(_workerChannel.IsSharedMemoryDataTransferEnabled()); } @@ -699,6 +740,7 @@ public void SharedMemoryDataTransferSetting_VerifyDisabled() [Fact] public void SharedMemoryDataTransferSetting_VerifyDisabledIfWorkerCapabilityAbsent() { + CreateDefaultWorkerChannel(); // Enable shared memory data transfer in the environment _testEnvironment.SetEnvironmentVariable(RpcWorkerConstants.FunctionsWorkerSharedMemoryDataTransferEnabledSettingName, "1"); @@ -727,6 +769,7 @@ public void SharedMemoryDataTransferSetting_VerifyDisabledIfWorkerCapabilityAbse [Fact] public void SharedMemoryDataTransferSetting_VerifyDisabledIfEnvironmentVariableAbsent() { + CreateDefaultWorkerChannel(); // Enable shared memory data transfer capability in the worker IDictionary capabilities = new Dictionary() { @@ -754,6 +797,7 @@ public void SharedMemoryDataTransferSetting_VerifyDisabledIfEnvironmentVariableA [Fact] public async Task GetLatencies_StartsTimer_WhenDynamicConcurrencyEnabled() { + // note: uses custom worker channel RpcWorkerConfig config = new RpcWorkerConfig() { Description = new RpcWorkerDescription() @@ -792,6 +836,7 @@ await TestHelpers.Await(() => [Fact] public async Task GetLatencies_DoesNot_StartTimer_WhenDynamicConcurrencyDisabled() { + // note: uses custom worker channels RpcWorkerConfig config = new RpcWorkerConfig() { Description = new RpcWorkerDescription() diff --git a/test/WebJobs.Script.Tests/Workers/Rpc/TestFunctionRpcService.cs b/test/WebJobs.Script.Tests/Workers/Rpc/TestFunctionRpcService.cs index 16da2f51cb..5b28e2c1a7 100644 --- a/test/WebJobs.Script.Tests/Workers/Rpc/TestFunctionRpcService.cs +++ b/test/WebJobs.Script.Tests/Workers/Rpc/TestFunctionRpcService.cs @@ -84,21 +84,28 @@ private void Write(StreamingMessage message) var vt = _inboundWriter.WriteAsync(evt); if (vt.IsCompleted) { - vt.GetAwaiter().GetResult(); + try + { + vt.GetAwaiter().GetResult(); + } + catch (Exception ex) + { + _logger.LogError(ex.Message); + } } else { - _ = ObserveEventually(vt); + _ = ObserveEventually(vt, _logger); } - static async Task ObserveEventually(ValueTask valueTask) + static async Task ObserveEventually(ValueTask valueTask, ILogger logger) { try { await valueTask; } - catch + catch (Exception ex) { - // log somewhere? + logger.LogError(ex.Message); } } } From 37396125b1171162015903136927d0fbd53c7e3b Mon Sep 17 00:00:00 2001 From: Marc Gravell Date: Wed, 6 Apr 2022 12:59:45 +0100 Subject: [PATCH 03/17] tweak the worker channel process flow to ensure it is testable; fixup TestFunctionRpcService and GrpcWorkerChannelTests to match --- .../Channel/GrpcWorkerChannel.cs | 10 +- .../WebJobs.Script.Tests.Shared/TestLogger.cs | 8 +- .../Workers/Rpc/GrpcWorkerChannelTests.cs | 238 +++++++----------- .../Workers/Rpc/TestFunctionRpcService.cs | 60 +++-- 4 files changed, 148 insertions(+), 168 deletions(-) diff --git a/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs b/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs index df844a1798..de5197d998 100644 --- a/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs +++ b/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs @@ -110,7 +110,7 @@ internal GrpcWorkerChannel( { _outbound = outbound.Writer; _inbound = inbound.Reader; - _ = ProcessInbound(); + // note: we don't start the read loop until StartWorkerProcessAsync is called } _eventSubscriptions.Add(_eventManager.OfType() @@ -261,6 +261,8 @@ public bool IsChannelReadyForInvocations() public async Task StartWorkerProcessAsync(CancellationToken cancellationToken) { OnNext(MsgType.StartStream, _workerConfig.CountOptions.ProcessStartupTimeout, 1, SendWorkerInitRequest, HandleWorkerStartStreamError); + // note: it is important that the ^^^ StartStream is in place *before* we start process the loop, otherwise we get a race condition + _ = ProcessInbound(); _workerChannelLogger.LogDebug("Initiating Worker Process start up"); await _rpcWorkerProcess.StartProcessAsync(); @@ -371,7 +373,7 @@ internal void WorkerInitResponse(GrpcEvent initEvent) if (_initMessage.Result.IsFailure(out Exception exc)) { HandleWorkerInitError(exc); - _workerInitTask.SetResult(false); + _workerInitTask.TrySetResult(false); return; } _state = _state | RpcWorkerChannelState.Initialized; @@ -384,7 +386,7 @@ internal void WorkerInitResponse(GrpcEvent initEvent) ScriptHost.IsFunctionDataCacheEnabled = false; } - _workerInitTask.SetResult(true); + _workerInitTask.TrySetResult(true); } public void SetupFunctionInvocationBuffers(IEnumerable functions) @@ -924,7 +926,7 @@ internal void HandleWorkerFunctionLoadError(Exception exc) private void PublishWorkerErrorEvent(Exception exc) { - _workerInitTask.SetException(exc); + _workerInitTask.TrySetException(exc); if (_disposing || _disposed) { return; diff --git a/test/WebJobs.Script.Tests.Shared/TestLogger.cs b/test/WebJobs.Script.Tests.Shared/TestLogger.cs index dbb6669f03..e5b3af70ec 100644 --- a/test/WebJobs.Script.Tests.Shared/TestLogger.cs +++ b/test/WebJobs.Script.Tests.Shared/TestLogger.cs @@ -6,6 +6,7 @@ using System.Diagnostics; using System.Linq; using Microsoft.Extensions.Logging; +using Xunit.Abstractions; namespace Microsoft.Azure.WebJobs.Script.Tests { @@ -14,11 +15,13 @@ public class TestLogger : ILogger { private readonly object _syncLock = new object(); private readonly IExternalScopeProvider _scopeProvider; - private IList _logMessages = new List(); + private readonly IList _logMessages = new List(); + private readonly ITestOutputHelper _testOutput; // optionally write direct to the test output - public TestLogger(string category) + public TestLogger(string category, ITestOutputHelper testOutput = null) : this(category, new LoggerExternalScopeProvider()) { + _testOutput = testOutput; } public TestLogger(string category, IExternalScopeProvider scopeProvider) @@ -72,6 +75,7 @@ public void Log(LogLevel logLevel, EventId eventId, TState state, Except { _logMessages.Add(logMessage); } + _testOutput?.WriteLine($"{logLevel}: {formatter(state, exception)}"); } } diff --git a/test/WebJobs.Script.Tests/Workers/Rpc/GrpcWorkerChannelTests.cs b/test/WebJobs.Script.Tests/Workers/Rpc/GrpcWorkerChannelTests.cs index f3806d5c5e..4146c03405 100644 --- a/test/WebJobs.Script.Tests/Workers/Rpc/GrpcWorkerChannelTests.cs +++ b/test/WebJobs.Script.Tests/Workers/Rpc/GrpcWorkerChannelTests.cs @@ -59,7 +59,7 @@ public GrpcWorkerChannelTests(ITestOutputHelper testOutput) { _eventManager.AddGrpcChannels(_workerId); _testOutput = testOutput; - _logger = new TestLogger("FunctionDispatcherTests"); + _logger = new TestLogger("FunctionDispatcherTests", testOutput); _testFunctionRpcService = new TestFunctionRpcService(_eventManager, _workerId, _logger, _expectedLogMsg); _testWorkerConfig = TestHelpers.GetTestWorkerConfigs().FirstOrDefault(); _testWorkerConfig.CountOptions.ProcessStartupTimeout = TimeSpan.FromSeconds(5); @@ -95,7 +95,7 @@ public GrpcWorkerChannelTests(ITestOutputHelper testOutput) _hostOptionsMonitor = TestHelpers.CreateOptionsMonitor(hostOptions); } - private void CreateDefaultWorkerChannel() + private Task CreateDefaultWorkerChannel(bool autoStart = true, IDictionary capabilities = null) { _workerChannel = new GrpcWorkerChannel( _workerId, @@ -110,6 +110,17 @@ private void CreateDefaultWorkerChannel() _sharedMemoryManager, _functionDataCache, _workerConcurrencyOptions); + if (autoStart) + { + // for most tests, we want things to be responsive to inbound messages + _testFunctionRpcService.OnMessage(StreamingMessage.ContentOneofCase.WorkerInitRequest, + () => _testFunctionRpcService.PublishWorkerInitResponseEvent(capabilities)); + return _workerChannel.StartWorkerProcessAsync(CancellationToken.None); + } + else + { + return Task.CompletedTask; + } } private void ShowOutput(string message) @@ -134,11 +145,9 @@ public void Dispose() [Fact] public async Task StartWorkerProcessAsync_ThrowsTaskCanceledException_IfDisposed() { - CreateDefaultWorkerChannel(); - var initTask = _workerChannel.StartWorkerProcessAsync(CancellationToken.None); + var initTask = CreateDefaultWorkerChannel(); + _workerChannel.Dispose(); - _testFunctionRpcService.PublishStartStreamEvent(_workerId); - _testFunctionRpcService.PublishWorkerInitResponseEvent(); await Assert.ThrowsAsync(async () => { await initTask; @@ -148,11 +157,7 @@ await Assert.ThrowsAsync(async () => [Fact] public async Task StartWorkerProcessAsync_Invoked_SetupFunctionBuffers_Verify_ReadyForInvocation() { - CreateDefaultWorkerChannel(); - var initTask = _workerChannel.StartWorkerProcessAsync(CancellationToken.None); - _testFunctionRpcService.PublishStartStreamEvent(_workerId); - _testFunctionRpcService.PublishWorkerInitResponseEvent(); - await initTask; + await CreateDefaultWorkerChannel(); _mockrpcWorkerProcess.Verify(m => m.StartProcessAsync(), Times.Once); Assert.False(_workerChannel.IsChannelReadyForInvocations()); _workerChannel.SetupFunctionInvocationBuffers(GetTestFunctionsList("node")); @@ -162,13 +167,9 @@ public async Task StartWorkerProcessAsync_Invoked_SetupFunctionBuffers_Verify_Re [Fact] public async Task DisposingChannel_NotReadyForInvocation() { - CreateDefaultWorkerChannel(); try { - var initTask = _workerChannel.StartWorkerProcessAsync(CancellationToken.None); - _testFunctionRpcService.PublishStartStreamEvent(_workerId); - _testFunctionRpcService.PublishWorkerInitResponseEvent(); - await initTask; + await CreateDefaultWorkerChannel(); Assert.False(_workerChannel.IsChannelReadyForInvocations()); _workerChannel.SetupFunctionInvocationBuffers(GetTestFunctionsList("node")); Assert.True(_workerChannel.IsChannelReadyForInvocations()); @@ -194,7 +195,7 @@ public void SetupFunctionBuffers_Verify_ReadyForInvocation_Returns_False() [Fact] public async Task StartWorkerProcessAsync_TimesOut() { - CreateDefaultWorkerChannel(); + await CreateDefaultWorkerChannel(autoStart: false); // suppress for timeout var initTask = _workerChannel.StartWorkerProcessAsync(CancellationToken.None); await Assert.ThrowsAsync(async () => await initTask); } @@ -202,7 +203,7 @@ public async Task StartWorkerProcessAsync_TimesOut() [Fact] public async Task SendEnvironmentReloadRequest_Generates_ExpectedMetrics() { - CreateDefaultWorkerChannel(); + await CreateDefaultWorkerChannel(); _metricsLogger.ClearCollections(); Task waitForMetricsTask = Task.Factory.StartNew(() => { @@ -241,7 +242,7 @@ public async Task StartWorkerProcessAsync_WorkerProcess_Throws() [Fact] public async Task SendWorkerInitRequest_PublishesOutboundEvent() { - CreateDefaultWorkerChannel(); + await CreateDefaultWorkerChannel(autoStart: false); // we'll do it manually here StartStream startStream = new StartStream() { WorkerId = _workerId @@ -251,8 +252,8 @@ public async Task SendWorkerInitRequest_PublishesOutboundEvent() StartStream = startStream }; GrpcEvent rpcEvent = new GrpcEvent(_workerId, startStreamMessage); + _testFunctionRpcService.AutoReply(StreamingMessage.ContentOneofCase.WorkerInitRequest); _workerChannel.SendWorkerInitRequest(rpcEvent); - _testFunctionRpcService.PublishWorkerInitResponseEvent(); await Task.Delay(500); var traces = _logger.GetLogMessages(); Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, _expectedLogMsg))); @@ -261,7 +262,7 @@ public async Task SendWorkerInitRequest_PublishesOutboundEvent() [Fact] public void WorkerInitRequest_Expected() { - CreateDefaultWorkerChannel(); + CreateDefaultWorkerChannel(autoStart: false); // doesn't actually need to run; just not be null WorkerInitRequest initRequest = _workerChannel.GetWorkerInitRequest(); Assert.NotNull(initRequest.WorkerDirectory); Assert.NotNull(initRequest.FunctionAppDirectory); @@ -274,19 +275,8 @@ public void WorkerInitRequest_Expected() [Fact] public async Task SendWorkerInitRequest_PublishesOutboundEvent_V2Compatable() { - CreateDefaultWorkerChannel(); _testEnvironment.SetEnvironmentVariable(EnvironmentSettingNames.FunctionsV2CompatibilityModeKey, "true"); - StartStream startStream = new StartStream() - { - WorkerId = _workerId - }; - StreamingMessage startStreamMessage = new StreamingMessage() - { - StartStream = startStream - }; - GrpcEvent rpcEvent = new GrpcEvent(_workerId, startStreamMessage); - _workerChannel.SendWorkerInitRequest(rpcEvent); - _testFunctionRpcService.PublishWorkerInitResponseEvent(); + await CreateDefaultWorkerChannel(); await Task.Delay(500); var traces = _logger.GetLogMessages(); Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, _expectedLogMsg))); @@ -300,7 +290,7 @@ public async Task SendWorkerInitRequest_PublishesOutboundEvent_V2Compatable() [InlineData(RpcLog.Types.Level.Trace, RpcLog.Types.Level.Information)] public async Task SendSystemLogMessage_PublishesSystemLogMessage(RpcLog.Types.Level levelToTest, RpcLog.Types.Level expectedLogLevel) { - CreateDefaultWorkerChannel(); + await CreateDefaultWorkerChannel(); _testFunctionRpcService.PublishSystemLogEvent(levelToTest); await Task.Delay(500); var traces = _logger.GetLogMessages(); @@ -310,7 +300,7 @@ public async Task SendSystemLogMessage_PublishesSystemLogMessage(RpcLog.Types.Le [Fact] public async Task SendInvocationRequest_PublishesOutboundEvent() { - CreateDefaultWorkerChannel(); + await CreateDefaultWorkerChannel(); ScriptInvocationContext scriptInvocationContext = GetTestScriptInvocationContext(Guid.NewGuid(), null); await _workerChannel.SendInvocationRequest(scriptInvocationContext); await Task.Delay(500); @@ -321,7 +311,7 @@ public async Task SendInvocationRequest_PublishesOutboundEvent() [Fact] public async Task SendInvocationRequest_IsInExecutingInvocation() { - CreateDefaultWorkerChannel(); + await CreateDefaultWorkerChannel(); ScriptInvocationContext scriptInvocationContext = GetTestScriptInvocationContext(Guid.NewGuid(), null); await _workerChannel.SendInvocationRequest(scriptInvocationContext); Assert.True(_workerChannel.IsExecutingInvocation(scriptInvocationContext.ExecutionContext.InvocationId.ToString())); @@ -334,8 +324,7 @@ public async Task SendInvocationRequest_IsInExecutingInvocation() [Fact] public async Task SendInvocationRequest_InputsTransferredOverSharedMemory() { - CreateDefaultWorkerChannel(); - EnableSharedMemoryDataTransfer(); + await CreateSharedMemoryEnabledWorkerChannel(); // Send invocation which will be using RpcSharedMemory for the inputs ScriptInvocationContext scriptInvocationContext = GetTestScriptInvocationContextWithSharedMemoryInputs(Guid.NewGuid(), null); @@ -384,7 +373,7 @@ await channel.InvokeResponse(new InvocationResponse [Fact] public async Task InFlight_Functions_FailedWithException() { - CreateDefaultWorkerChannel(); + await CreateDefaultWorkerChannel(); var resultSource = new TaskCompletionSource(); ScriptInvocationContext scriptInvocationContext = GetTestScriptInvocationContext(Guid.NewGuid(), resultSource); await _workerChannel.SendInvocationRequest(scriptInvocationContext); @@ -399,7 +388,7 @@ public async Task InFlight_Functions_FailedWithException() [Fact] public async Task SendLoadRequests_PublishesOutboundEvents() { - CreateDefaultWorkerChannel(); + await CreateDefaultWorkerChannel(); _metricsLogger.ClearCollections(); _workerChannel.SetupFunctionInvocationBuffers(GetTestFunctionsList("node")); _workerChannel.SendFunctionLoadRequests(null, TimeSpan.FromMinutes(5)); @@ -407,24 +396,14 @@ public async Task SendLoadRequests_PublishesOutboundEvents() var traces = _logger.GetLogMessages(); var functionLoadLogs = traces.Where(m => string.Equals(m.FormattedMessage, _expectedLogMsg)); AreExpectedMetricsGenerated(); - Assert.True(functionLoadLogs.Count() == 2); + Assert.Equal(3, functionLoadLogs.Count()); // one WorkInitRequest, two FunctionLoadRequest } [Fact] public async Task SendLoadRequestCollection_PublishesOutboundEvents() { - CreateDefaultWorkerChannel(); - StartStream startStream = new StartStream() - { - WorkerId = _workerId - }; - StreamingMessage startStreamMessage = new StreamingMessage() - { - StartStream = startStream - }; - GrpcEvent rpcEvent = new GrpcEvent(_workerId, startStreamMessage); - _workerChannel.SendWorkerInitRequest(rpcEvent); - _testFunctionRpcService.PublishWorkerInitResponseEvent(new Dictionary() { { RpcWorkerConstants.AcceptsListOfFunctionLoadRequests, "true" } }); + await CreateDefaultWorkerChannel(capabilities: new Dictionary() { { RpcWorkerConstants.AcceptsListOfFunctionLoadRequests, "true" } }); + _metricsLogger.ClearCollections(); IEnumerable functionMetadata = GetTestFunctionsList("node"); _workerChannel.SetupFunctionInvocationBuffers(functionMetadata); @@ -434,14 +413,14 @@ public async Task SendLoadRequestCollection_PublishesOutboundEvents() ShowOutput(traces); var functionLoadLogs = traces.Where(m => string.Equals(m.FormattedMessage, _expectedLogMsg)); AreExpectedMetricsGenerated(); - Assert.True(functionLoadLogs.Count() == 2); + Assert.Equal(2, functionLoadLogs.Count()); Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, string.Format("Sending FunctionLoadRequestCollection with number of functions:'{0}'", functionMetadata.ToList().Count)))); } [Fact] public async Task SendLoadRequests_PublishesOutboundEvents_OrdersDisabled() { - CreateDefaultWorkerChannel(); + await CreateDefaultWorkerChannel(); var funcName = "ADisabledFunc"; var functions = GetTestFunctionsList_WithDisabled("node", funcName); @@ -459,13 +438,13 @@ public async Task SendLoadRequests_PublishesOutboundEvents_OrdersDisabled() // Make sure that disabled func shows up last Assert.True(functionLoadLogs.Last().FormattedMessage.Contains(funcName)); Assert.False(functionLoadLogs.First().FormattedMessage.Contains(funcName)); - Assert.True(functionLoadLogs.Count() == 3); + Assert.Equal(3, functionLoadLogs.Count()); } [Fact] public async Task SendLoadRequests_DoesNotTimeout_FunctionTimeoutNotSet() { - CreateDefaultWorkerChannel(); + await CreateDefaultWorkerChannel(); var funcName = "ADisabledFunc"; var functions = GetTestFunctionsList_WithDisabled("node", funcName); _workerChannel.SetupFunctionInvocationBuffers(functions); @@ -480,14 +459,14 @@ public async Task SendLoadRequests_DoesNotTimeout_FunctionTimeoutNotSet() [Fact] public async Task SendSendFunctionEnvironmentReloadRequest_PublishesOutboundEvents() { - CreateDefaultWorkerChannel(); + await CreateDefaultWorkerChannel(); try { Environment.SetEnvironmentVariable("TestNull", null); Environment.SetEnvironmentVariable("TestEmpty", string.Empty); Environment.SetEnvironmentVariable("TestValid", "TestValue"); + _testFunctionRpcService.AutoReply(StreamingMessage.ContentOneofCase.FunctionEnvironmentReloadRequest); var pending = _workerChannel.SendFunctionEnvironmentReloadRequest(); - _testFunctionRpcService.PublishFunctionEnvironmentReloadResponseEvent(); await Task.Delay(500); await pending; // this can timeout } @@ -501,13 +480,13 @@ public async Task SendSendFunctionEnvironmentReloadRequest_PublishesOutboundEven var traces = _logger.GetLogMessages(); ShowOutput(traces); var functionLoadLogs = traces.Where(m => string.Equals(m.FormattedMessage, "Sending FunctionEnvironmentReloadRequest to WorkerProcess with Pid: '910'")); - Assert.True(functionLoadLogs.Count() == 1); + Assert.Equal(1, functionLoadLogs.Count()); } [Fact] public async Task SendSendFunctionEnvironmentReloadRequest_ThrowsTimeout() { - CreateDefaultWorkerChannel(); + await CreateDefaultWorkerChannel(); var reloadTask = _workerChannel.SendFunctionEnvironmentReloadRequest(); await Assert.ThrowsAsync(async () => await reloadTask); } @@ -549,7 +528,7 @@ public void SendFunctionEnvironmentReloadRequest_WithDirectory() [Fact] public async Task ReceivesInboundEvent_InvocationResponse() { - CreateDefaultWorkerChannel(); + await CreateDefaultWorkerChannel(); _testFunctionRpcService.PublishInvocationResponseEvent(); await Task.Delay(500); var traces = _logger.GetLogMessages(); @@ -559,11 +538,13 @@ public async Task ReceivesInboundEvent_InvocationResponse() [Fact] public async Task ReceivesInboundEvent_FunctionLoadResponse() { - CreateDefaultWorkerChannel(); + await CreateDefaultWorkerChannel(); var functionMetadatas = GetTestFunctionsList("node"); _workerChannel.SetupFunctionInvocationBuffers(functionMetadatas); + _testFunctionRpcService.OnMessage(StreamingMessage.ContentOneofCase.FunctionLoadRequest, + () => _testFunctionRpcService.PublishFunctionLoadResponseEvent("TestFunctionId1")); _workerChannel.SendFunctionLoadRequests(null, TimeSpan.FromMinutes(5)); - _testFunctionRpcService.PublishFunctionLoadResponseEvent("TestFunctionId1"); + await Task.Delay(500); var traces = _logger.GetLogMessages(); ShowOutput(traces); @@ -575,13 +556,16 @@ public async Task ReceivesInboundEvent_FunctionLoadResponse() [Fact] public async Task ReceivesInboundEvent_Failed_FunctionLoadResponses() { - CreateDefaultWorkerChannel(); + await CreateDefaultWorkerChannel(); var functionMetadatas = GetTestFunctionsList("node"); _workerChannel.SetupFunctionInvocationBuffers(functionMetadatas); - _workerChannel.SendFunctionLoadRequests(null, TimeSpan.FromMinutes(1)); - _testFunctionRpcService.PublishFunctionLoadResponsesEvent( + + _testFunctionRpcService.OnMessage(StreamingMessage.ContentOneofCase.FunctionLoadRequest, + () => _testFunctionRpcService.PublishFunctionLoadResponsesEvent( new List() { "TestFunctionId1", "TestFunctionId2" }, - new StatusResult() { Status = StatusResult.Types.Status.Failure }); + new StatusResult() { Status = StatusResult.Types.Status.Failure })); + _workerChannel.SendFunctionLoadRequests(null, TimeSpan.FromMinutes(1)); + await Task.Delay(500); var traces = _logger.GetLogMessages(); ShowOutput(traces); @@ -594,7 +578,7 @@ public async Task ReceivesInboundEvent_Failed_FunctionLoadResponses() [Fact] public async Task ReceivesInboundEvent_FunctionLoadResponses() { - CreateDefaultWorkerChannel(); + await CreateDefaultWorkerChannel(); var functionMetadatas = GetTestFunctionsList("node"); _workerChannel.SetupFunctionInvocationBuffers(functionMetadatas); _workerChannel.SendFunctionLoadRequests(null, TimeSpan.FromMinutes(1)); @@ -614,11 +598,13 @@ public async Task ReceivesInboundEvent_FunctionLoadResponses() [Fact] public async Task ReceivesInboundEvent_Successful_FunctionMetadataResponse() { - CreateDefaultWorkerChannel(); + await CreateDefaultWorkerChannel(); var functionMetadata = GetTestFunctionsList("python"); - var functions = _workerChannel.GetFunctionMetadata(); var functionId = "id123"; - _testFunctionRpcService.PublishWorkerMetadataResponse("TestFunctionId1", functionId, functionMetadata, true); + _testFunctionRpcService.OnMessage(StreamingMessage.ContentOneofCase.FunctionsMetadataRequest, + () => _testFunctionRpcService.PublishWorkerMetadataResponse(functionId, functionMetadata, true)); + var functions = _workerChannel.GetFunctionMetadata(); + await Task.Delay(500); var traces = _logger.GetLogMessages(); ShowOutput(traces); @@ -628,11 +614,13 @@ public async Task ReceivesInboundEvent_Successful_FunctionMetadataResponse() [Fact] public async Task ReceivesInboundEvent_Successful_FunctionMetadataResponse_UseDefaultMetadataIndexing_True() { - CreateDefaultWorkerChannel(); + await CreateDefaultWorkerChannel(); var functionMetadata = GetTestFunctionsList("python"); - var functions = _workerChannel.GetFunctionMetadata(); var functionId = "id123"; - _testFunctionRpcService.PublishWorkerMetadataResponse("TestFunctionId1", functionId, functionMetadata, true, useDefaultMetadataIndexing: true); + _testFunctionRpcService.OnMessage(StreamingMessage.ContentOneofCase.FunctionsMetadataRequest, + () => _testFunctionRpcService.PublishWorkerMetadataResponse(functionId, functionMetadata, true, useDefaultMetadataIndexing: true)); + var functions = _workerChannel.GetFunctionMetadata(); + await Task.Delay(500); var traces = _logger.GetLogMessages(); ShowOutput(traces); @@ -642,11 +630,13 @@ public async Task ReceivesInboundEvent_Successful_FunctionMetadataResponse_UseDe [Fact] public async Task ReceivesInboundEvent_Successful_FunctionMetadataResponse_UseDefaultMetadataIndexing_False() { - CreateDefaultWorkerChannel(); + await CreateDefaultWorkerChannel(); var functionMetadata = GetTestFunctionsList("python"); - var functions = _workerChannel.GetFunctionMetadata(); var functionId = "id123"; - _testFunctionRpcService.PublishWorkerMetadataResponse("TestFunctionId1", functionId, functionMetadata, true, useDefaultMetadataIndexing: false); + _testFunctionRpcService.OnMessage(StreamingMessage.ContentOneofCase.FunctionsMetadataRequest, + () => _testFunctionRpcService.PublishWorkerMetadataResponse(functionId, functionMetadata, true, useDefaultMetadataIndexing: false)); + var functions = _workerChannel.GetFunctionMetadata(); + await Task.Delay(500); var traces = _logger.GetLogMessages(); ShowOutput(traces); @@ -656,11 +646,12 @@ public async Task ReceivesInboundEvent_Successful_FunctionMetadataResponse_UseDe [Fact] public async Task ReceivesInboundEvent_Failed_UseDefaultMetadataIndexing_True_HostIndexing() { - CreateDefaultWorkerChannel(); + await CreateDefaultWorkerChannel(); var functionMetadata = GetTestFunctionsList("python"); - var functions = _workerChannel.GetFunctionMetadata(); var functionId = "id123"; - _testFunctionRpcService.PublishWorkerMetadataResponse("TestFunctionId1", functionId, functionMetadata, false, useDefaultMetadataIndexing: true); + _testFunctionRpcService.OnMessage(StreamingMessage.ContentOneofCase.FunctionsMetadataRequest, + () => _testFunctionRpcService.PublishWorkerMetadataResponse(functionId, functionMetadata, false, useDefaultMetadataIndexing: true)); + var functions = _workerChannel.GetFunctionMetadata(); await Task.Delay(500); var traces = _logger.GetLogMessages(); ShowOutput(traces); @@ -670,11 +661,12 @@ public async Task ReceivesInboundEvent_Failed_UseDefaultMetadataIndexing_True_Ho [Fact] public async Task ReceivesInboundEvent_Failed_UseDefaultMetadataIndexing_False_WorkerIndexing() { - CreateDefaultWorkerChannel(); + await CreateDefaultWorkerChannel(); var functionMetadata = GetTestFunctionsList("python"); - var functions = _workerChannel.GetFunctionMetadata(); var functionId = "id123"; - _testFunctionRpcService.PublishWorkerMetadataResponse("TestFunctionId1", functionId, functionMetadata, false, useDefaultMetadataIndexing: false); + _testFunctionRpcService.OnMessage(StreamingMessage.ContentOneofCase.FunctionsMetadataRequest, + () => _testFunctionRpcService.PublishWorkerMetadataResponse(functionId, functionMetadata, false, useDefaultMetadataIndexing: false)); + var functions = _workerChannel.GetFunctionMetadata(); await Task.Delay(500); var traces = _logger.GetLogMessages(); ShowOutput(traces); @@ -684,11 +676,12 @@ public async Task ReceivesInboundEvent_Failed_UseDefaultMetadataIndexing_False_W [Fact] public async Task ReceivesInboundEvent_Failed_FunctionMetadataResponse() { - CreateDefaultWorkerChannel(); + await CreateDefaultWorkerChannel(); + var functionId = "id123"; var functionMetadata = GetTestFunctionsList("python"); + _testFunctionRpcService.OnMessage(StreamingMessage.ContentOneofCase.FunctionsMetadataRequest, + () => _testFunctionRpcService.PublishWorkerMetadataResponse(functionId, functionMetadata, false)); var functions = _workerChannel.GetFunctionMetadata(); - var functionId = "id123"; - _testFunctionRpcService.PublishWorkerMetadataResponse("TestFunctionId1", functionId, functionMetadata, false); await Task.Delay(500); var traces = _logger.GetLogMessages(); ShowOutput(traces); @@ -717,10 +710,9 @@ public void FunctionLoadRequest_IsExpected() [Fact] public async Task SharedMemoryDataTransferSetting_VerifyEnabled() { - CreateDefaultWorkerChannel(); - EnableSharedMemoryDataTransfer(); + await CreateSharedMemoryEnabledWorkerChannel(); await Task.Delay(500); - Assert.True(_workerChannel.IsSharedMemoryDataTransferEnabled()); + Assert.True(_workerChannel.IsSharedMemoryDataTransferEnabled(), "shared memory should be enabled"); } /// @@ -740,24 +732,9 @@ public void SharedMemoryDataTransferSetting_VerifyDisabled() [Fact] public void SharedMemoryDataTransferSetting_VerifyDisabledIfWorkerCapabilityAbsent() { - CreateDefaultWorkerChannel(); // Enable shared memory data transfer in the environment _testEnvironment.SetEnvironmentVariable(RpcWorkerConstants.FunctionsWorkerSharedMemoryDataTransferEnabledSettingName, "1"); - - StartStream startStream = new StartStream() - { - WorkerId = _workerId - }; - - StreamingMessage startStreamMessage = new StreamingMessage() - { - StartStream = startStream - }; - - // Send worker init request and enable the capabilities - GrpcEvent rpcEvent = new GrpcEvent(_workerId, startStreamMessage); - _workerChannel.SendWorkerInitRequest(rpcEvent); - _testFunctionRpcService.PublishWorkerInitResponseEvent(); + CreateDefaultWorkerChannel(); Assert.False(_workerChannel.IsSharedMemoryDataTransferEnabled()); } @@ -769,28 +746,7 @@ public void SharedMemoryDataTransferSetting_VerifyDisabledIfWorkerCapabilityAbse [Fact] public void SharedMemoryDataTransferSetting_VerifyDisabledIfEnvironmentVariableAbsent() { - CreateDefaultWorkerChannel(); - // Enable shared memory data transfer capability in the worker - IDictionary capabilities = new Dictionary() - { - { RpcWorkerConstants.SharedMemoryDataTransfer, "1" } - }; - - StartStream startStream = new StartStream() - { - WorkerId = _workerId - }; - - StreamingMessage startStreamMessage = new StreamingMessage() - { - StartStream = startStream - }; - - // Send worker init request and enable the capabilities - GrpcEvent rpcEvent = new GrpcEvent(_workerId, startStreamMessage); - _workerChannel.SendWorkerInitRequest(rpcEvent); - _testFunctionRpcService.PublishWorkerInitResponseEvent(capabilities); - + CreateSharedMemoryEnabledWorkerChannel(setEnvironmentVariable: false); Assert.False(_workerChannel.IsSharedMemoryDataTransferEnabled()); } @@ -864,7 +820,7 @@ public async Task GetLatencies_DoesNot_StartTimer_WhenDynamicConcurrencyDisabled IEnumerable latencyHistory = workerChannel.GetLatencies(); - Assert.True(latencyHistory.Count() == 0); + Assert.Equal(0, latencyHistory.Count()); } private IEnumerable GetTestFunctionsList(string runtime) @@ -970,31 +926,21 @@ private bool AreExpectedMetricsGenerated() return _metricsLogger.EventsBegan.Contains(MetricEventNames.FunctionLoadRequestResponse); } - private void EnableSharedMemoryDataTransfer() + private Task CreateSharedMemoryEnabledWorkerChannel(bool setEnvironmentVariable = true) { - // Enable shared memory data transfer in the environment - _testEnvironment.SetEnvironmentVariable(RpcWorkerConstants.FunctionsWorkerSharedMemoryDataTransferEnabledSettingName, "1"); + if (setEnvironmentVariable) + { + // Enable shared memory data transfer in the environment + _testEnvironment.SetEnvironmentVariable(RpcWorkerConstants.FunctionsWorkerSharedMemoryDataTransferEnabledSettingName, "1"); + } // Enable shared memory data transfer capability in the worker IDictionary capabilities = new Dictionary() { { RpcWorkerConstants.SharedMemoryDataTransfer, "1" } }; - - StartStream startStream = new StartStream() - { - WorkerId = _workerId - }; - - StreamingMessage startStreamMessage = new StreamingMessage() - { - StartStream = startStream - }; - // Send worker init request and enable the capabilities - GrpcEvent rpcEvent = new GrpcEvent(_workerId, startStreamMessage); - _workerChannel.SendWorkerInitRequest(rpcEvent); - _testFunctionRpcService.PublishWorkerInitResponseEvent(capabilities); + return CreateDefaultWorkerChannel(capabilities: capabilities); } } } diff --git a/test/WebJobs.Script.Tests/Workers/Rpc/TestFunctionRpcService.cs b/test/WebJobs.Script.Tests/Workers/Rpc/TestFunctionRpcService.cs index 5b28e2c1a7..3099ab89ec 100644 --- a/test/WebJobs.Script.Tests/Workers/Rpc/TestFunctionRpcService.cs +++ b/test/WebJobs.Script.Tests/Workers/Rpc/TestFunctionRpcService.cs @@ -2,26 +2,16 @@ // 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.Linq; -using System.Reactive.Linq; -using System.Runtime.InteropServices; using System.Threading.Channels; using System.Threading.Tasks; using Microsoft.Azure.WebJobs.Script.Description; using Microsoft.Azure.WebJobs.Script.Diagnostics; using Microsoft.Azure.WebJobs.Script.Eventing; -using Microsoft.Azure.WebJobs.Script.Grpc; using Microsoft.Azure.WebJobs.Script.Grpc.Eventing; using Microsoft.Azure.WebJobs.Script.Grpc.Messages; -using Microsoft.Azure.WebJobs.Script.Workers; -using Microsoft.Azure.WebJobs.Script.Workers.Rpc; -using Microsoft.Azure.WebJobs.Script.Workers.SharedMemoryDataTransfer; using Microsoft.Extensions.Logging; -using Microsoft.Extensions.Logging.Abstractions; -using Microsoft.Extensions.Options; -using Moq; -using Xunit; namespace Microsoft.Azure.WebJobs.Script.Tests.Workers.Rpc { @@ -32,6 +22,7 @@ public class TestFunctionRpcService private string _workerId; private IDictionary _outboundEventSubscriptions = new Dictionary(); private ChannelWriter _inboundWriter; + private ConcurrentDictionary _handlers = new ConcurrentDictionary(); public TestFunctionRpcService(IScriptEventManager eventManager, string workerId, TestLogger logger, string expectedLogMsg = "") { @@ -42,6 +33,41 @@ public TestFunctionRpcService(IScriptEventManager eventManager, string workerId, { _ = ListenAsync(outbound.Reader, expectedLogMsg); _inboundWriter = inbound.Writer; + + PublishStartStreamEvent(); // simulate the start-stream immediately + } + } + + public void OnMessage(StreamingMessage.ContentOneofCase messageType, Action callback) + => _handlers.AddOrUpdate(messageType, callback, (messageType, oldValue) => oldValue + callback); + + public void AutoReply(StreamingMessage.ContentOneofCase messageType) + { + // apply standard default responses + Action callback = messageType switch + { + StreamingMessage.ContentOneofCase.FunctionEnvironmentReloadRequest => PublishFunctionEnvironmentReloadResponseEvent, + _ => null, + }; + if (callback is not null) + { + OnMessage(messageType, callback); + } + } + + private void OnMessage(StreamingMessage.ContentOneofCase messageType) + { + if (_handlers.TryRemove(messageType, out var action)) + { + try + { + _logger.LogDebug("[service] invoking auto-reply for {0}, {1}: {2}", _workerId, messageType, action?.Method?.Name); + action?.Invoke(); + } + catch (Exception ex) + { + _logger.LogError(ex.Message); + } } } @@ -56,6 +82,8 @@ private async Task ListenAsync(ChannelReader source, string e { _logger.LogDebug("[service] received {0}, {1}", evt.WorkerId, evt.MessageType); _logger.LogInformation(expectedLogMsg); + + OnMessage(evt.MessageType); } } } @@ -150,7 +178,7 @@ public void PublishFunctionLoadResponsesEvent(List functionIds, StatusRe Write(responseMessage); } - public void PublishFunctionEnvironmentReloadResponseEvent() + private void PublishFunctionEnvironmentReloadResponseEvent() { FunctionEnvironmentReloadResponse relaodEnvResponse = GetTestFunctionEnvReloadResponse(); StreamingMessage responseMessage = new StreamingMessage() @@ -185,7 +213,7 @@ public void PublishWorkerInitResponseEvent(IDictionary capabilit Write(responseMessage); } - public void PublishWorkerInitResponseEventWithSharedMemoryDataTransferCapability() + private void PublishWorkerInitResponseEventWithSharedMemoryDataTransferCapability() { StatusResult statusResult = new StatusResult() { @@ -249,7 +277,7 @@ public void PublishInvocationResponseEvent() Write(responseMessage); } - public void PublishStartStreamEvent(string workerId) + private void PublishStartStreamEvent() { StatusResult statusResult = new StatusResult() { @@ -257,7 +285,7 @@ public void PublishStartStreamEvent(string workerId) }; StartStream startStream = new StartStream() { - WorkerId = workerId + WorkerId = _workerId }; StreamingMessage responseMessage = new StreamingMessage() { @@ -266,7 +294,7 @@ public void PublishStartStreamEvent(string workerId) Write(responseMessage); } - public void PublishWorkerMetadataResponse(string workerId, string functionId, IEnumerable functionMetadata, bool successful, bool useDefaultMetadataIndexing = false) + public void PublishWorkerMetadataResponse(string functionId, IEnumerable functionMetadata, bool successful, bool useDefaultMetadataIndexing = false) { StatusResult statusResult = new StatusResult(); if (successful) From 0daaa10cfcca285b41c9a1d6eae1885b857437ef Mon Sep 17 00:00:00 2001 From: Marc Gravell Date: Thu, 5 May 2022 16:13:45 +0100 Subject: [PATCH 04/17] merge removal of PublishFunctionLoadResponsesEvent work --- .../Workers/Rpc/GrpcWorkerChannelTests.cs | 42 ------------------- 1 file changed, 42 deletions(-) diff --git a/test/WebJobs.Script.Tests/Workers/Rpc/GrpcWorkerChannelTests.cs b/test/WebJobs.Script.Tests/Workers/Rpc/GrpcWorkerChannelTests.cs index 9ac2808f49..f1725343de 100644 --- a/test/WebJobs.Script.Tests/Workers/Rpc/GrpcWorkerChannelTests.cs +++ b/test/WebJobs.Script.Tests/Workers/Rpc/GrpcWorkerChannelTests.cs @@ -554,48 +554,6 @@ public async Task ReceivesInboundEvent_FunctionLoadResponse() Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, "Received FunctionLoadResponse for function: 'js1' with functionId: 'TestFunctionId1'.")), "FunctionLoadResponse TestFunctionId1"); } - [Fact] - public async Task ReceivesInboundEvent_Failed_FunctionLoadResponses() - { - await CreateDefaultWorkerChannel(); - var functionMetadatas = GetTestFunctionsList("node"); - _workerChannel.SetupFunctionInvocationBuffers(functionMetadatas); - - _testFunctionRpcService.OnMessage(StreamingMessage.ContentOneofCase.FunctionLoadRequest, - () => _testFunctionRpcService.PublishFunctionLoadResponsesEvent( - new List() { "TestFunctionId1", "TestFunctionId2" }, - new StatusResult() { Status = StatusResult.Types.Status.Failure })); - _workerChannel.SendFunctionLoadRequests(null, TimeSpan.FromMinutes(1)); - - await Task.Delay(500); - var traces = _logger.GetLogMessages(); - ShowOutput(traces); - Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, "Setting up FunctionInvocationBuffer for function: 'js1' with functionId: 'TestFunctionId1'")), "FunctionInvocationBuffer TestFunctionId1"); - Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, "Setting up FunctionInvocationBuffer for function: 'js2' with functionId: 'TestFunctionId2'")), "FunctionInvocationBuffer TestFunctionId2"); - Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, "Worker failed to load function: 'js1' with function id: 'TestFunctionId1'.")), "failed to load TestFunctionId1"); - Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, "Worker failed to load function: 'js2' with function id: 'TestFunctionId2'.")), "failed to load TestFunctionId2"); - } - - [Fact] - public async Task ReceivesInboundEvent_FunctionLoadResponses() - { - await CreateDefaultWorkerChannel(); - var functionMetadatas = GetTestFunctionsList("node"); - _workerChannel.SetupFunctionInvocationBuffers(functionMetadatas); - _workerChannel.SendFunctionLoadRequests(null, TimeSpan.FromMinutes(1)); - _testFunctionRpcService.PublishFunctionLoadResponsesEvent( - new List() { "TestFunctionId1", "TestFunctionId2" }, - new StatusResult() { Status = StatusResult.Types.Status.Success }); - await Task.Delay(500); - var traces = _logger.GetLogMessages(); - ShowOutput(traces); - Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, "Setting up FunctionInvocationBuffer for function: 'js1' with functionId: 'TestFunctionId1'"))); - Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, "Setting up FunctionInvocationBuffer for function: 'js2' with functionId: 'TestFunctionId2'"))); - Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, string.Format("Received FunctionLoadResponseCollection with number of functions: '{0}'.", functionMetadatas.ToList().Count)))); - Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, "Received FunctionLoadResponse for function: 'js1' with functionId: 'TestFunctionId1'."))); - Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, "Received FunctionLoadResponse for function: 'js2' with functionId: 'TestFunctionId2'."))); - } - [Fact] public async Task ReceivesInboundEvent_Successful_FunctionMetadataResponse() { From 60e01c87a74630f500ddbd02ffc52d7d320a5225 Mon Sep 17 00:00:00 2001 From: Marc Gravell Date: Mon, 20 Jun 2022 14:19:30 +0100 Subject: [PATCH 05/17] Update src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs Co-authored-by: Lilian Kasem --- src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs b/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs index 3db5c7b62e..ed63b12666 100644 --- a/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs +++ b/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs @@ -212,7 +212,7 @@ private void OnNext(MsgType messageType, TimeSpan timeout, int count, Action Date: Mon, 20 Jun 2022 14:19:41 +0100 Subject: [PATCH 06/17] Update src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs Co-authored-by: Lilian Kasem --- src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs b/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs index ed63b12666..e05a7dc12c 100644 --- a/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs +++ b/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs @@ -981,7 +981,7 @@ static async Task ObserveEventually(ValueTask valueTask) } catch { - // nowehere to log + // no where to log } } } From 29540a2982c69115f5890e8b828558f1e164520c Mon Sep 17 00:00:00 2001 From: Marc Gravell Date: Mon, 20 Jun 2022 14:25:38 +0100 Subject: [PATCH 07/17] nits and renames from PR review feedback --- .../Channel/GrpcWorkerChannel.cs | 33 ++++++++----------- .../Eventing/IGrpcEventManager.cs | 17 ---------- 2 files changed, 14 insertions(+), 36 deletions(-) delete mode 100644 src/WebJobs.Script.Grpc/Eventing/IGrpcEventManager.cs diff --git a/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs b/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs index 3db5c7b62e..c23afab075 100644 --- a/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs +++ b/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs @@ -133,21 +133,16 @@ internal GrpcWorkerChannel( private void ProcessItem(InboundGrpcEvent msg) { - // note this is a QUWI entry-point + // note this method is a thread-pool (QueueUserWorkItem) entry-point try { switch (msg.MessageType) { + case MsgType.RpcLog when msg.Message.RpcLog.LogCategory == RpcLogCategory.System: + SystemLog(msg); + break; case MsgType.RpcLog: - switch (msg.Message.RpcLog.LogCategory) - { - case RpcLogCategory.System: - SystemLog(msg); - break; - default: - Log(msg); - break; - } + Log(msg); break; case MsgType.WorkerStatusResponse: ReceiveWorkerStatusResponse(msg.Message.RequestId, msg.Message.WorkerStatusResponse); @@ -156,7 +151,7 @@ private void ProcessItem(InboundGrpcEvent msg) _ = InvokeResponse(msg.Message.InvocationResponse); break; default: - OnNext(msg); + ProcessRegisteredGrpcCallbacks(msg); break; } } @@ -166,7 +161,7 @@ private void ProcessItem(InboundGrpcEvent msg) } } - private void OnNext(InboundGrpcEvent message) + private void ProcessRegisteredGrpcCallbacks(InboundGrpcEvent message) { Queue queue; lock (_pendingActions) @@ -191,7 +186,7 @@ private void OnNext(InboundGrpcEvent message) next.SetResult(message); } - private void OnNext(MsgType messageType, TimeSpan timeout, int count, Action callback, Action faultHandler) + private void RegisterCallbackForNextGrpcMessage(MsgType messageType, TimeSpan timeout, int count, Action callback, Action faultHandler) { Queue queue; lock (_pendingActions) @@ -260,7 +255,7 @@ public bool IsChannelReadyForInvocations() public async Task StartWorkerProcessAsync(CancellationToken cancellationToken) { - OnNext(MsgType.StartStream, _workerConfig.CountOptions.ProcessStartupTimeout, 1, SendWorkerInitRequest, HandleWorkerStartStreamError); + RegisterCallbackForNextGrpcMessage(MsgType.StartStream, _workerConfig.CountOptions.ProcessStartupTimeout, 1, SendWorkerInitRequest, HandleWorkerStartStreamError); // note: it is important that the ^^^ StartStream is in place *before* we start process the loop, otherwise we get a race condition _ = ProcessInbound(); @@ -310,7 +305,7 @@ public async Task GetWorkerStatusAsync() internal void SendWorkerInitRequest(GrpcEvent startEvent) { _workerChannelLogger.LogDebug("Worker Process started. Received StartStream message"); - OnNext(MsgType.WorkerInitResponse, _workerConfig.CountOptions.InitializationTimeout, 1, WorkerInitResponse, HandleWorkerInitError); + RegisterCallbackForNextGrpcMessage(MsgType.WorkerInitResponse, _workerConfig.CountOptions.InitializationTimeout, 1, WorkerInitResponse, HandleWorkerInitError); WorkerInitRequest initRequest = GetWorkerInitRequest(); @@ -409,11 +404,11 @@ public void SendFunctionLoadRequests(ManagedDependencyOptions managedDependencyO { _functionLoadTimeout = functionTimeout.Value > _functionLoadTimeout ? functionTimeout.Value : _functionLoadTimeout; - OnNext(MsgType.FunctionLoadResponse, _functionLoadTimeout, count, msg => LoadResponse(msg.Message.FunctionLoadResponse), HandleWorkerFunctionLoadError); + RegisterCallbackForNextGrpcMessage(MsgType.FunctionLoadResponse, _functionLoadTimeout, count, msg => LoadResponse(msg.Message.FunctionLoadResponse), HandleWorkerFunctionLoadError); } else { - OnNext(MsgType.FunctionLoadResponse, TimeSpan.Zero, count, msg => LoadResponse(msg.Message.FunctionLoadResponse), HandleWorkerFunctionLoadError); + RegisterCallbackForNextGrpcMessage(MsgType.FunctionLoadResponse, TimeSpan.Zero, count, msg => LoadResponse(msg.Message.FunctionLoadResponse), HandleWorkerFunctionLoadError); } // Load Request is also sent for disabled function as it is invocable using the portal and admin endpoints @@ -469,7 +464,7 @@ public Task SendFunctionEnvironmentReloadRequest() _workerChannelLogger.LogDebug("Sending FunctionEnvironmentReloadRequest to WorkerProcess with Pid: '{0}'", _rpcWorkerProcess.Id); IDisposable latencyEvent = _metricsLogger.LatencyEvent(MetricEventNames.SpecializationEnvironmentReloadRequestResponse); - OnNext(MsgType.FunctionEnvironmentReloadResponse, _workerConfig.CountOptions.EnvironmentReloadTimeout, 1, + RegisterCallbackForNextGrpcMessage(MsgType.FunctionEnvironmentReloadResponse, _workerConfig.CountOptions.EnvironmentReloadTimeout, 1, msg => FunctionEnvironmentReloadResponse(msg.Message.FunctionEnvironmentReloadResponse, latencyEvent), HandleWorkerEnvReloadError); IDictionary processEnv = Environment.GetEnvironmentVariables(); @@ -621,7 +616,7 @@ public Task> GetFunctionMetadata() internal Task> SendFunctionMetadataRequest() { - OnNext(MsgType.FunctionMetadataResponse, _functionLoadTimeout, 1, + RegisterCallbackForNextGrpcMessage(MsgType.FunctionMetadataResponse, _functionLoadTimeout, 1, msg => ProcessFunctionMetadataResponses(msg.Message.FunctionMetadataResponse), HandleWorkerMetadataRequestError); _workerChannelLogger.LogDebug("Sending WorkerMetadataRequest to {language} worker with worker ID {workerID}", _runtime, _workerId); diff --git a/src/WebJobs.Script.Grpc/Eventing/IGrpcEventManager.cs b/src/WebJobs.Script.Grpc/Eventing/IGrpcEventManager.cs deleted file mode 100644 index 9cafcb4fae..0000000000 --- a/src/WebJobs.Script.Grpc/Eventing/IGrpcEventManager.cs +++ /dev/null @@ -1,17 +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.Threading.Channels; -//using Microsoft.Azure.WebJobs.Script.Eventing; - -//namespace Microsoft.Azure.WebJobs.Script.Grpc.Eventing; - -///// A script event manager with dedicated support for gRPC channels -//public interface IGrpcEventManager : IScriptEventManager -//{ -// /// Prepares the specified workerId for usage (unexpected inbound worker ids will be rejected) -// void AddWorker(string workerId); - -// /// Obtains the gRPC channels for the specified workerId, if it has been prepared -// bool TryGetGrpcChannels(string workerId, out Channel inbound, out Channel outbound); -//} From 255c7bbc75c9fa830335fc95991f79269c14efdc Mon Sep 17 00:00:00 2001 From: Marc Gravell Date: Tue, 5 Jul 2022 15:33:48 +0100 Subject: [PATCH 08/17] fix tests on script-invocation context settings - was mock; now we simply check what actually arrives at the server --- .../Workers/Rpc/GrpcWorkerChannelTests.cs | 119 +++++++++--------- .../Workers/Rpc/TestFunctionRpcService.cs | 18 +-- 2 files changed, 67 insertions(+), 70 deletions(-) diff --git a/test/WebJobs.Script.Tests/Workers/Rpc/GrpcWorkerChannelTests.cs b/test/WebJobs.Script.Tests/Workers/Rpc/GrpcWorkerChannelTests.cs index 9cbcece11c..1917d62f06 100644 --- a/test/WebJobs.Script.Tests/Workers/Rpc/GrpcWorkerChannelTests.cs +++ b/test/WebJobs.Script.Tests/Workers/Rpc/GrpcWorkerChannelTests.cs @@ -39,7 +39,6 @@ public class GrpcWorkerChannelTests : IDisposable private readonly string _workerId = "testWorkerId"; private readonly string _scriptRootPath = "c:\testdir"; private readonly IScriptEventManager _eventManager = new ScriptEventManager(); - private readonly Mock _eventManagerMock = new Mock(); private readonly TestMetricsLogger _metricsLogger = new TestMetricsLogger(); private readonly Mock _mockConsoleLogger = new Mock(); private readonly Mock _mockFunctionRpcService = new Mock(); @@ -57,7 +56,6 @@ public class GrpcWorkerChannelTests : IDisposable private readonly IOptions _workerConcurrencyOptions; private readonly ITestOutputHelper _testOutput; private GrpcWorkerChannel _workerChannel; - private GrpcWorkerChannel _workerChannelwithMockEventManager; public GrpcWorkerChannelTests(ITestOutputHelper testOutput) { @@ -102,28 +100,9 @@ public GrpcWorkerChannelTests(ITestOutputHelper testOutput) _testEnvironment.SetEnvironmentVariable("APPLICATIONINSIGHTS_ENABLE_AGENT", "true"); } - private Task CreateDefaultWorkerChannel(bool autoStart = true, IDictionary capabilities = null, bool mockEventManager = false) + private Task CreateDefaultWorkerChannel(bool autoStart = true, IDictionary capabilities = null) { - if (mockEventManager) - { - _eventManagerMock.Setup(proxy => proxy.Publish(It.IsAny())).Verifiable(); - _workerChannelwithMockEventManager = new GrpcWorkerChannel( - _workerId, - _eventManagerMock.Object, - _testWorkerConfig, - _mockrpcWorkerProcess.Object, - _logger, - _metricsLogger, - 0, - _testEnvironment, - _hostOptionsMonitor, - _sharedMemoryManager, - _functionDataCache, - _workerConcurrencyOptions); - } - else - { - _workerChannel = new GrpcWorkerChannel( + _workerChannel = new GrpcWorkerChannel( _workerId, _eventManager, _testWorkerConfig, @@ -136,14 +115,13 @@ private Task CreateDefaultWorkerChannel(bool autoStart = true, IDictionary _testFunctionRpcService.PublishWorkerInitResponseEvent(capabilities)); - return (_workerChannel ?? _workerChannelwithMockEventManager).StartWorkerProcessAsync(CancellationToken.None); + _ => _testFunctionRpcService.PublishWorkerInitResponseEvent(capabilities)); + return _workerChannel.StartWorkerProcessAsync(CancellationToken.None); } else { @@ -582,7 +560,7 @@ public async Task ReceivesInboundEvent_FunctionLoadResponse() var functionMetadatas = GetTestFunctionsList("node"); _workerChannel.SetupFunctionInvocationBuffers(functionMetadatas); _testFunctionRpcService.OnMessage(StreamingMessage.ContentOneofCase.FunctionLoadRequest, - () => _testFunctionRpcService.PublishFunctionLoadResponseEvent("TestFunctionId1")); + _ => _testFunctionRpcService.PublishFunctionLoadResponseEvent("TestFunctionId1")); _workerChannel.SendFunctionLoadRequests(null, TimeSpan.FromMinutes(5)); await Task.Delay(500); @@ -597,11 +575,11 @@ public async Task ReceivesInboundEvent_FunctionLoadResponse() [Fact] public async Task ReceivesInboundEvent_Failed_FunctionLoadResponses() { - await CreateDefaultWorkerChannel(); IDictionary capabilities = new Dictionary() { { RpcWorkerConstants.SupportsLoadResponseCollection, "1" } }; + await CreateDefaultWorkerChannel(capabilities: capabilities); StartStream startStream = new StartStream() { @@ -615,7 +593,6 @@ public async Task ReceivesInboundEvent_Failed_FunctionLoadResponses() GrpcEvent rpcEvent = new GrpcEvent(_workerId, startStreamMessage); _workerChannel.SendWorkerInitRequest(rpcEvent); - _testFunctionRpcService.PublishWorkerInitResponseEvent(capabilities); var functionMetadatas = GetTestFunctionsList("node"); _workerChannel.SetupFunctionInvocationBuffers(functionMetadatas); @@ -635,11 +612,11 @@ public async Task ReceivesInboundEvent_Failed_FunctionLoadResponses() [Fact] public async Task ReceivesInboundEvent_FunctionLoadResponses() { - await CreateDefaultWorkerChannel(); IDictionary capabilities = new Dictionary() { { RpcWorkerConstants.SupportsLoadResponseCollection, "1" } }; + await CreateDefaultWorkerChannel(capabilities: capabilities); StartStream startStream = new StartStream() { @@ -653,7 +630,6 @@ public async Task ReceivesInboundEvent_FunctionLoadResponses() GrpcEvent rpcEvent = new GrpcEvent(_workerId, startStreamMessage); _workerChannel.SendWorkerInitRequest(rpcEvent); - _testFunctionRpcService.PublishWorkerInitResponseEvent(capabilities); var functionMetadatas = GetTestFunctionsList("node"); _workerChannel.SetupFunctionInvocationBuffers(functionMetadatas); @@ -678,7 +654,7 @@ public async Task ReceivesInboundEvent_Successful_FunctionMetadataResponse() var functionMetadata = GetTestFunctionsList("python"); var functionId = "id123"; _testFunctionRpcService.OnMessage(StreamingMessage.ContentOneofCase.FunctionsMetadataRequest, - () => _testFunctionRpcService.PublishWorkerMetadataResponse(_workerId, functionId, functionMetadata, true)); + _ => _testFunctionRpcService.PublishWorkerMetadataResponse(_workerId, functionId, functionMetadata, true)); var functions = _workerChannel.GetFunctionMetadata(); await Task.Delay(500); @@ -694,7 +670,7 @@ public async Task ReceivesInboundEvent_Successful_FunctionMetadataResponse_UseDe var functionMetadata = GetTestFunctionsList("python"); var functionId = "id123"; _testFunctionRpcService.OnMessage(StreamingMessage.ContentOneofCase.FunctionsMetadataRequest, - () => _testFunctionRpcService.PublishWorkerMetadataResponse(_workerId, functionId, functionMetadata, true, useDefaultMetadataIndexing: true)); + _ => _testFunctionRpcService.PublishWorkerMetadataResponse(_workerId, functionId, functionMetadata, true, useDefaultMetadataIndexing: true)); var functions = _workerChannel.GetFunctionMetadata(); await Task.Delay(500); @@ -710,7 +686,7 @@ public async Task ReceivesInboundEvent_Successful_FunctionMetadataResponse_UseDe var functionMetadata = GetTestFunctionsList("python"); var functionId = "id123"; _testFunctionRpcService.OnMessage(StreamingMessage.ContentOneofCase.FunctionsMetadataRequest, - () => _testFunctionRpcService.PublishWorkerMetadataResponse(_workerId, functionId, functionMetadata, true, useDefaultMetadataIndexing: false)); + _ => _testFunctionRpcService.PublishWorkerMetadataResponse(_workerId, functionId, functionMetadata, true, useDefaultMetadataIndexing: false)); var functions = _workerChannel.GetFunctionMetadata(); await Task.Delay(500); @@ -726,7 +702,7 @@ public async Task ReceivesInboundEvent_Failed_UseDefaultMetadataIndexing_True_Ho var functionMetadata = GetTestFunctionsList("python"); var functionId = "id123"; _testFunctionRpcService.OnMessage(StreamingMessage.ContentOneofCase.FunctionsMetadataRequest, - () => _testFunctionRpcService.PublishWorkerMetadataResponse(_workerId, functionId, functionMetadata, false, useDefaultMetadataIndexing: true)); + _ => _testFunctionRpcService.PublishWorkerMetadataResponse(_workerId, functionId, functionMetadata, false, useDefaultMetadataIndexing: true)); var functions = _workerChannel.GetFunctionMetadata(); await Task.Delay(500); var traces = _logger.GetLogMessages(); @@ -741,7 +717,7 @@ public async Task ReceivesInboundEvent_Failed_UseDefaultMetadataIndexing_False_W var functionMetadata = GetTestFunctionsList("python"); var functionId = "id123"; _testFunctionRpcService.OnMessage(StreamingMessage.ContentOneofCase.FunctionsMetadataRequest, - () => _testFunctionRpcService.PublishWorkerMetadataResponse(_workerId, functionId, functionMetadata, false, useDefaultMetadataIndexing: false)); + _ => _testFunctionRpcService.PublishWorkerMetadataResponse(_workerId, functionId, functionMetadata, false, useDefaultMetadataIndexing: false)); var functions = _workerChannel.GetFunctionMetadata(); await Task.Delay(500); var traces = _logger.GetLogMessages(); @@ -756,7 +732,7 @@ public async Task ReceivesInboundEvent_Failed_FunctionMetadataResponse() var functionId = "id123"; var functionMetadata = GetTestFunctionsList("python"); _testFunctionRpcService.OnMessage(StreamingMessage.ContentOneofCase.FunctionsMetadataRequest, - () => _testFunctionRpcService.PublishWorkerMetadataResponse(_workerId, functionId, functionMetadata, false)); + _ => _testFunctionRpcService.PublishWorkerMetadataResponse(_workerId, functionId, functionMetadata, false)); var functions = _workerChannel.GetFunctionMetadata(); await Task.Delay(500); var traces = _logger.GetLogMessages(); @@ -769,7 +745,7 @@ public async Task ReceivesInboundEvent_Failed_OverallFunctionMetadataResponse() { await CreateDefaultWorkerChannel(); _testFunctionRpcService.OnMessage(StreamingMessage.ContentOneofCase.FunctionsMetadataRequest, - () => _testFunctionRpcService.PublishWorkerMetadataResponse("TestFunctionId1", null, null, false, false, false)); + _ => _testFunctionRpcService.PublishWorkerMetadataResponse("TestFunctionId1", null, null, false, false, false)); var functions = _workerChannel.GetFunctionMetadata(); await Task.Delay(500); var traces = _logger.GetLogMessages(); @@ -914,53 +890,74 @@ public async Task GetLatencies_DoesNot_StartTimer_WhenDynamicConcurrencyDisabled [Fact] public async Task SendInvocationRequest_ValidateTraceContext() { - await CreateDefaultWorkerChannel(mockEventManager: true); + await CreateDefaultWorkerChannel(); ScriptInvocationContext scriptInvocationContext = GetTestScriptInvocationContext(Guid.NewGuid(), null); - await _workerChannelwithMockEventManager.SendInvocationRequest(scriptInvocationContext); + + await _workerChannel.SendInvocationRequest(scriptInvocationContext); + + RpcTraceContext ctx = null; + _testFunctionRpcService.OnMessage(StreamingMessage.ContentOneofCase.InvocationRequest, evt => + { + ctx = evt.Message.InvocationRequest.TraceContext; + }); + await Task.Delay(500); + + Assert.NotNull(ctx); + var attribs = ctx.Attributes; + Assert.NotNull(attribs); + if (_testEnvironment.IsApplicationInsightsAgentEnabled()) { - _eventManagerMock.Verify(proxy => proxy.Publish(It.Is( - grpcEvent => grpcEvent.Message.InvocationRequest.TraceContext.Attributes.ContainsKey(ScriptConstants.LogPropertyProcessIdKey) - && grpcEvent.Message.InvocationRequest.TraceContext.Attributes.ContainsKey(ScriptConstants.LogPropertyHostInstanceIdKey) - && grpcEvent.Message.InvocationRequest.TraceContext.Attributes.ContainsKey(LogConstants.CategoryNameKey) - && grpcEvent.Message.InvocationRequest.TraceContext.Attributes[LogConstants.CategoryNameKey].Equals("testcat1") - && grpcEvent.Message.InvocationRequest.TraceContext.Attributes.Count == 3))); + _testOutput.WriteLine("Checking ENABLED app-insights fields..."); + Assert.True(attribs.ContainsKey(ScriptConstants.LogPropertyProcessIdKey), "ScriptConstants.LogPropertyProcessIdKey"); + Assert.True(attribs.ContainsKey(ScriptConstants.LogPropertyHostInstanceIdKey), "ScriptConstants.LogPropertyHostInstanceIdKey"); + Assert.True(attribs.TryGetValue(LogConstants.CategoryNameKey, out var catKey), "LogConstants.CategoryNameKey"); + Assert.Equal(catKey, "testcat1"); + Assert.Equal(3, attribs.Count); } else { - _eventManagerMock.Verify(proxy => proxy.Publish(It.Is( - grpcEvent => !grpcEvent.Message.InvocationRequest.TraceContext.Attributes.ContainsKey(ScriptConstants.LogPropertyProcessIdKey) - && !grpcEvent.Message.InvocationRequest.TraceContext.Attributes.ContainsKey(ScriptConstants.LogPropertyHostInstanceIdKey) - && !grpcEvent.Message.InvocationRequest.TraceContext.Attributes.ContainsKey(LogConstants.CategoryNameKey)))); + _testOutput.WriteLine("Checking DISABLED app-insights fields..."); + Assert.False(attribs.ContainsKey(ScriptConstants.LogPropertyProcessIdKey), "ScriptConstants.LogPropertyProcessIdKey"); + Assert.False(attribs.ContainsKey(ScriptConstants.LogPropertyHostInstanceIdKey), "ScriptConstants.LogPropertyHostInstanceIdKey"); + Assert.False(attribs.ContainsKey(LogConstants.CategoryNameKey), "LogConstants.CategoryNameKey"); + Assert.Equal(0, attribs.Count); } } [Fact] public async Task SendInvocationRequest_ValidateTraceContext_SessionId() { - await CreateDefaultWorkerChannel(mockEventManager: true); + await CreateDefaultWorkerChannel(); string sessionId = "sessionId1234"; Activity activity = new Activity("testActivity"); activity.AddBaggage(ScriptConstants.LiveLogsSessionAIKey, sessionId); activity.Start(); ScriptInvocationContext scriptInvocationContext = GetTestScriptInvocationContext(Guid.NewGuid(), null); - await _workerChannelwithMockEventManager.SendInvocationRequest(scriptInvocationContext); + + OutboundGrpcEvent grpcEvent = null; + _testFunctionRpcService.OnMessage(StreamingMessage.ContentOneofCase.InvocationRequest, evt => + { + grpcEvent = evt; + }); + await _workerChannel.SendInvocationRequest(scriptInvocationContext); + await Task.Delay(500); + + Assert.NotNull(grpcEvent); activity.Stop(); - _eventManagerMock.Verify(p => p.Publish(It.Is(grpcEvent => ValidateInvocationRequest(grpcEvent, sessionId)))); - } + var attribs = grpcEvent.Message.InvocationRequest.TraceContext.Attributes; - private bool ValidateInvocationRequest(OutboundGrpcEvent grpcEvent, string sessionId) - { if (_testEnvironment.IsApplicationInsightsAgentEnabled()) { - return grpcEvent.Message.InvocationRequest.TraceContext.Attributes[ScriptConstants.LiveLogsSessionAIKey].Equals(sessionId) - && grpcEvent.Message.InvocationRequest.TraceContext.Attributes.ContainsKey(LogConstants.CategoryNameKey) - && grpcEvent.Message.InvocationRequest.TraceContext.Attributes[LogConstants.CategoryNameKey].Equals("testcat1") - && grpcEvent.Message.InvocationRequest.TraceContext.Attributes.Count == 4; + Assert.True(attribs.TryGetValue(ScriptConstants.LiveLogsSessionAIKey, out var aiKey), "ScriptConstants.LiveLogsSessionAIKey"); + Assert.Equal(sessionId, aiKey); + Assert.True(attribs.TryGetValue(LogConstants.CategoryNameKey, out var catKey), "LogConstants.CategoryNameKey"); + Assert.Equal("testcat1", catKey); + Assert.Equal(4, attribs.Count); } else { - return !grpcEvent.Message.InvocationRequest.TraceContext.Attributes.ContainsKey(LogConstants.CategoryNameKey); + Assert.False(attribs.ContainsKey(LogConstants.CategoryNameKey), "LogConstants.CategoryNameKey"); } } diff --git a/test/WebJobs.Script.Tests/Workers/Rpc/TestFunctionRpcService.cs b/test/WebJobs.Script.Tests/Workers/Rpc/TestFunctionRpcService.cs index 5766a4203a..080e7d7b8d 100644 --- a/test/WebJobs.Script.Tests/Workers/Rpc/TestFunctionRpcService.cs +++ b/test/WebJobs.Script.Tests/Workers/Rpc/TestFunctionRpcService.cs @@ -21,7 +21,7 @@ public class TestFunctionRpcService private string _workerId; private IDictionary _outboundEventSubscriptions = new Dictionary(); private ChannelWriter _inboundWriter; - private ConcurrentDictionary _handlers = new ConcurrentDictionary(); + private ConcurrentDictionary> _handlers = new ConcurrentDictionary>(); public TestFunctionRpcService(IScriptEventManager eventManager, string workerId, TestLogger logger, string expectedLogMsg = "") { @@ -36,15 +36,15 @@ public TestFunctionRpcService(IScriptEventManager eventManager, string workerId, } } - public void OnMessage(StreamingMessage.ContentOneofCase messageType, Action callback) + public void OnMessage(StreamingMessage.ContentOneofCase messageType, Action callback) => _handlers.AddOrUpdate(messageType, callback, (messageType, oldValue) => oldValue + callback); public void AutoReply(StreamingMessage.ContentOneofCase messageType) { // apply standard default responses - Action callback = messageType switch + Action callback = messageType switch { - StreamingMessage.ContentOneofCase.FunctionEnvironmentReloadRequest => PublishFunctionEnvironmentReloadResponseEvent, + StreamingMessage.ContentOneofCase.FunctionEnvironmentReloadRequest => _ => PublishFunctionEnvironmentReloadResponseEvent(), _ => null, }; if (callback is not null) @@ -53,14 +53,14 @@ public void AutoReply(StreamingMessage.ContentOneofCase messageType) } } - private void OnMessage(StreamingMessage.ContentOneofCase messageType) + private void OnMessage(OutboundGrpcEvent message) { - if (_handlers.TryRemove(messageType, out var action)) + if (_handlers.TryRemove(message.MessageType, out var action)) { try { - _logger.LogDebug("[service] invoking auto-reply for {0}, {1}: {2}", _workerId, messageType, action?.Method?.Name); - action?.Invoke(); + _logger.LogDebug("[service] invoking auto-reply for {0}, {1}: {2}", _workerId, message.MessageType, action?.Method?.Name); + action?.Invoke(message); } catch (Exception ex) { @@ -81,7 +81,7 @@ private async Task ListenAsync(ChannelReader source, string e _logger.LogDebug("[service] received {0}, {1}", evt.WorkerId, evt.MessageType); _logger.LogInformation(expectedLogMsg); - OnMessage(evt.MessageType); + OnMessage(evt); } } } From a4fde1c01cd86b4723b3280410a2adceb1a18aff Mon Sep 17 00:00:00 2001 From: Nick Craver Date: Tue, 26 Jul 2022 20:29:29 -0400 Subject: [PATCH 09/17] Fix latest dev tests --- .../Workers/Rpc/GrpcWorkerChannelTests.cs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/test/WebJobs.Script.Tests/Workers/Rpc/GrpcWorkerChannelTests.cs b/test/WebJobs.Script.Tests/Workers/Rpc/GrpcWorkerChannelTests.cs index 1e1705cc3b..5ca845eba0 100644 --- a/test/WebJobs.Script.Tests/Workers/Rpc/GrpcWorkerChannelTests.cs +++ b/test/WebJobs.Script.Tests/Workers/Rpc/GrpcWorkerChannelTests.cs @@ -163,7 +163,7 @@ await Assert.ThrowsAsync(async () => [Fact] public void WorkerChannel_Dispose_With_WorkerTerminateCapability() { - var initTask = _workerChannel.StartWorkerProcessAsync(CancellationToken.None); + var initTask = CreateDefaultWorkerChannel(); IDictionary capabilities = new Dictionary() { @@ -194,7 +194,7 @@ public void WorkerChannel_Dispose_With_WorkerTerminateCapability() [Fact] public void WorkerChannel_Dispose_Without_WorkerTerminateCapability() { - var initTask = _workerChannel.StartWorkerProcessAsync(CancellationToken.None); + var initTask = CreateDefaultWorkerChannel(); _workerChannel.Dispose(); var traces = _logger.GetLogMessages(); From 2842743f6801dc8c192440b1dbbf4be37b9c335e Mon Sep 17 00:00:00 2001 From: Nick Craver Date: Tue, 26 Jul 2022 21:26:13 -0400 Subject: [PATCH 10/17] Fix new capabilities test to use channel properly --- .../Workers/Rpc/GrpcWorkerChannelTests.cs | 17 +++++++---------- 1 file changed, 7 insertions(+), 10 deletions(-) diff --git a/test/WebJobs.Script.Tests/Workers/Rpc/GrpcWorkerChannelTests.cs b/test/WebJobs.Script.Tests/Workers/Rpc/GrpcWorkerChannelTests.cs index 5ca845eba0..21c0168229 100644 --- a/test/WebJobs.Script.Tests/Workers/Rpc/GrpcWorkerChannelTests.cs +++ b/test/WebJobs.Script.Tests/Workers/Rpc/GrpcWorkerChannelTests.cs @@ -161,14 +161,9 @@ await Assert.ThrowsAsync(async () => } [Fact] - public void WorkerChannel_Dispose_With_WorkerTerminateCapability() + public async Task WorkerChannel_Dispose_With_WorkerTerminateCapability() { - var initTask = CreateDefaultWorkerChannel(); - - IDictionary capabilities = new Dictionary() - { - { RpcWorkerConstants.HandlesWorkerTerminateMessage, "1" } - }; + await CreateDefaultWorkerChannel(capabilities: new Dictionary() { { RpcWorkerConstants.HandlesWorkerTerminateMessage, "1" } }); StartStream startStream = new StartStream() { @@ -182,8 +177,10 @@ public void WorkerChannel_Dispose_With_WorkerTerminateCapability() // Send worker init request and enable the capabilities GrpcEvent rpcEvent = new GrpcEvent(_workerId, startStreamMessage); + _testFunctionRpcService.AutoReply(StreamingMessage.ContentOneofCase.WorkerInitRequest); _workerChannel.SendWorkerInitRequest(rpcEvent); - _testFunctionRpcService.PublishWorkerInitResponseEvent(capabilities); + + await Task.Delay(500); _workerChannel.Dispose(); var traces = _logger.GetLogMessages(); @@ -192,9 +189,9 @@ public void WorkerChannel_Dispose_With_WorkerTerminateCapability() } [Fact] - public void WorkerChannel_Dispose_Without_WorkerTerminateCapability() + public async Task WorkerChannel_Dispose_Without_WorkerTerminateCapability() { - var initTask = CreateDefaultWorkerChannel(); + await CreateDefaultWorkerChannel(); _workerChannel.Dispose(); var traces = _logger.GetLogMessages(); From 4a1b0c63b0ba8f1a9587bee96b65f5bf2c1a53b5 Mon Sep 17 00:00:00 2001 From: Nick Craver Date: Tue, 26 Jul 2022 22:28:28 -0400 Subject: [PATCH 11/17] Tidy up/PR comments --- src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs | 2 +- .../Rpc/WorkerConcurrencyManagerEndToEndTests.cs | 2 -- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs b/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs index a6cf8ec2ea..59c452f79e 100644 --- a/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs +++ b/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs @@ -109,7 +109,7 @@ internal GrpcWorkerChannel( _workerCapabilities = new GrpcCapabilities(_workerChannelLogger); - if (eventManager.TryGetGrpcChannels(workerId, out var inbound, out var outbound)) + if (_eventManager.TryGetGrpcChannels(workerId, out var inbound, out var outbound)) { _outbound = outbound.Writer; _inbound = inbound.Reader; diff --git a/test/WebJobs.Script.Tests.Integration/Rpc/WorkerConcurrencyManagerEndToEndTests.cs b/test/WebJobs.Script.Tests.Integration/Rpc/WorkerConcurrencyManagerEndToEndTests.cs index 00b5b759d3..47ac2e2215 100644 --- a/test/WebJobs.Script.Tests.Integration/Rpc/WorkerConcurrencyManagerEndToEndTests.cs +++ b/test/WebJobs.Script.Tests.Integration/Rpc/WorkerConcurrencyManagerEndToEndTests.cs @@ -1,7 +1,6 @@ using System; using System.Collections.Generic; using System.Linq; -using System.Threading.Channels; using System.Threading.Tasks; using Microsoft.Azure.WebJobs.Script.Eventing; using Microsoft.Azure.WebJobs.Script.Workers.Rpc; @@ -46,7 +45,6 @@ internal class TestScriptEventManager : IScriptEventManager, IDisposable { private readonly IScriptEventManager _scriptEventManager; - public TestScriptEventManager() { _scriptEventManager = new ScriptEventManager(); From 914b7d68ec5ea2b44afca5fa6fa0e5f4a7095952 Mon Sep 17 00:00:00 2001 From: Nick Craver Date: Wed, 27 Jul 2022 12:25:00 -0400 Subject: [PATCH 12/17] Fix WorkerStatus_NewWorkerAdded by properly simulating latency in the new system Here we're simulating latency in the outbound pipe was it was intended to when everything was posting through the script manager through RX. Also makes the latency non-static and more test friendly. --- .../Eventing/GrpcEventExtensions.cs | 4 +- .../WorkerConcurrencyManagerEndToEndTests.cs | 63 ++++++++++++++----- .../ScriptHostEndToEndTestFixture.cs | 8 ++- 3 files changed, 55 insertions(+), 20 deletions(-) diff --git a/src/WebJobs.Script.Grpc/Eventing/GrpcEventExtensions.cs b/src/WebJobs.Script.Grpc/Eventing/GrpcEventExtensions.cs index 15cbee10c5..3a7a587bad 100644 --- a/src/WebJobs.Script.Grpc/Eventing/GrpcEventExtensions.cs +++ b/src/WebJobs.Script.Grpc/Eventing/GrpcEventExtensions.cs @@ -18,14 +18,14 @@ internal static class GrpcEventExtensions // inbound-writer (note we will allow for multi-stream possibility) // 4) the GrpcWorkerChannel has a single dedicated consumer of inbound-reader, which it then marries to // in-flight operations - private static readonly UnboundedChannelOptions InboundOptions = new UnboundedChannelOptions + internal static readonly UnboundedChannelOptions InboundOptions = new UnboundedChannelOptions { SingleReader = true, // see 4 SingleWriter = false, // see 3 AllowSynchronousContinuations = false, }; - private static readonly UnboundedChannelOptions OutboundOptions = new UnboundedChannelOptions + internal static readonly UnboundedChannelOptions OutboundOptions = new UnboundedChannelOptions { SingleReader = false, // see 2 SingleWriter = false, // see 1 diff --git a/test/WebJobs.Script.Tests.Integration/Rpc/WorkerConcurrencyManagerEndToEndTests.cs b/test/WebJobs.Script.Tests.Integration/Rpc/WorkerConcurrencyManagerEndToEndTests.cs index 47ac2e2215..9647ca4820 100644 --- a/test/WebJobs.Script.Tests.Integration/Rpc/WorkerConcurrencyManagerEndToEndTests.cs +++ b/test/WebJobs.Script.Tests.Integration/Rpc/WorkerConcurrencyManagerEndToEndTests.cs @@ -1,8 +1,11 @@ using System; using System.Collections.Generic; using System.Linq; +using System.Threading; +using System.Threading.Channels; using System.Threading.Tasks; using Microsoft.Azure.WebJobs.Script.Eventing; +using Microsoft.Azure.WebJobs.Script.Grpc.Eventing; using Microsoft.Azure.WebJobs.Script.Workers.Rpc; using Xunit; @@ -22,8 +25,7 @@ public async Task WorkerStatus_NewWorkerAdded() { RpcFunctionInvocationDispatcher fd = null; IEnumerable channels = null; - // Latency > 1s - TestScriptEventManager.WaitBeforePublish = TimeSpan.FromSeconds(2); + await TestHelpers.Await(async () => { fd = Fixture.JobHost.FunctionDispatcher as RpcFunctionInvocationDispatcher; @@ -34,9 +36,11 @@ await TestHelpers.Await(async () => public class TestFixture : ScriptHostEndToEndTestFixture { + // Latency > 1s public TestFixture() : base(@"TestScripts\Node", "node", RpcWorkerConstants.NodeLanguageWorkerName, startHost: true, functions: new[] { "HttpTrigger" }, - addWorkerConcurrency: true) + addWorkerConcurrency: true, + addWorkerDelay: TimeSpan.FromSeconds(2)) { } } @@ -44,18 +48,16 @@ public TestFixture() : base(@"TestScripts\Node", "node", RpcWorkerConstants.Node internal class TestScriptEventManager : IScriptEventManager, IDisposable { private readonly IScriptEventManager _scriptEventManager; + private readonly TimeSpan _delay; - public TestScriptEventManager() + public TestScriptEventManager(TimeSpan delay) { _scriptEventManager = new ScriptEventManager(); + _delay = delay; } - public static TimeSpan WaitBeforePublish; - - public async void Publish(ScriptEvent scriptEvent) + public void Publish(ScriptEvent scriptEvent) { - // Emulate long worker status latency - await Task.Delay(WaitBeforePublish); try { _scriptEventManager.Publish(scriptEvent); @@ -66,21 +68,52 @@ public async void Publish(ScriptEvent scriptEvent) } } - public IDisposable Subscribe(IObserver observer) - { - return _scriptEventManager.Subscribe(observer); - } + public IDisposable Subscribe(IObserver observer) => _scriptEventManager.Subscribe(observer); public void Dispose() => ((IDisposable)_scriptEventManager).Dispose(); - public bool TryAddWorkerState(string workerId, T state) - => _scriptEventManager.TryAddWorkerState(workerId, state); + { + // Swap for a channel that imposes a delay into the pipe + if (typeof(T) == typeof(Channel) && _delay > TimeSpan.Zero) + { + state = (T)(object)(new DelayedOutboundChannel(_delay)); + } + return _scriptEventManager.TryAddWorkerState(workerId, state); + } public bool TryGetWorkerState(string workerId, out T state) => _scriptEventManager.TryGetWorkerState(workerId, out state); public bool TryRemoveWorkerState(string workerId, out T state) => _scriptEventManager.TryRemoveWorkerState(workerId, out state); + + + public class DelayedOutboundChannel : Channel + { + public DelayedOutboundChannel(TimeSpan delay) + { + var toWrap = Channel.CreateUnbounded(GrpcEventExtensions.OutboundOptions); + Reader = toWrap.Reader; + Writer = new DelayedChannelWriter(toWrap.Writer, delay); + } + } + + public class DelayedChannelWriter : ChannelWriter + { + private readonly TimeSpan _delay; + private readonly ChannelWriter _inner; + + public DelayedChannelWriter(ChannelWriter toWrap, TimeSpan delay) => (_inner, _delay) = (toWrap, delay); + + public override bool TryWrite(T item) => false; // Always fail, so we bounce to WriteAsync + public override ValueTask WaitToWriteAsync(CancellationToken cancellationToken = default) => _inner.WaitToWriteAsync(cancellationToken); + + public override async ValueTask WriteAsync(T item, CancellationToken cancellationToken = default) + { + await Task.Delay(_delay, cancellationToken); + await _inner.WriteAsync(item, cancellationToken); + } + } } } } diff --git a/test/WebJobs.Script.Tests.Integration/ScriptHostEndToEnd/ScriptHostEndToEndTestFixture.cs b/test/WebJobs.Script.Tests.Integration/ScriptHostEndToEnd/ScriptHostEndToEndTestFixture.cs index 5fcc5b6204..1b15d8d5dd 100644 --- a/test/WebJobs.Script.Tests.Integration/ScriptHostEndToEnd/ScriptHostEndToEndTestFixture.cs +++ b/test/WebJobs.Script.Tests.Integration/ScriptHostEndToEnd/ScriptHostEndToEndTestFixture.cs @@ -40,9 +40,10 @@ public abstract class ScriptHostEndToEndTestFixture : IAsyncLifetime private readonly ICollection _functions; private readonly string _functionsWorkerLanguage; private readonly bool _addWorkerConcurrency; + private readonly TimeSpan? _addWorkerDelay; protected ScriptHostEndToEndTestFixture(string rootPath, string testId, string functionsWorkerLanguage, - bool startHost = true, ICollection functions = null, bool addWorkerConcurrency = false) + bool startHost = true, ICollection functions = null, bool addWorkerConcurrency = false, TimeSpan? addWorkerDelay = null) { _settingsManager = ScriptSettingsManager.Instance; FixtureId = testId; @@ -56,6 +57,7 @@ protected ScriptHostEndToEndTestFixture(string rootPath, string testId, string f _functions = functions; _functionsWorkerLanguage = functionsWorkerLanguage; _addWorkerConcurrency = addWorkerConcurrency; + _addWorkerDelay = addWorkerDelay; } public TestLoggerProvider LoggerProvider { get; } @@ -155,9 +157,9 @@ public async Task InitializeAsync() services.AddSingleton(); } services.AddSingleton(); - if (_addWorkerConcurrency) + if (_addWorkerConcurrency && _addWorkerDelay > TimeSpan.Zero) { - services.AddSingleton(); + services.AddSingleton(new WorkerConcurrencyManagerEndToEndTests.TestScriptEventManager(_addWorkerDelay.Value)); } ConfigureServices(services); From 4764d1c16eeecbfa0090971ddae780cd702af1c8 Mon Sep 17 00:00:00 2001 From: Nick Craver Date: Tue, 30 Aug 2022 09:08:20 -0400 Subject: [PATCH 13/17] Address PR comments Unsure if this covers #8604 completely or not - hope so! --- src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs b/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs index b8dc045af9..48cb7b1a75 100644 --- a/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs +++ b/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs @@ -159,9 +159,9 @@ private void ProcessItem(InboundGrpcEvent msg) break; } } - catch + catch (Exception ex) { - // TODO: log? + _workerChannelLogger.LogError(ex, "Error processing InboundGrpcEvent: " + ex.Message); } } @@ -1006,7 +1006,7 @@ private void SendStreamingMessage(StreamingMessage msg) { try { - pending.GetAwaiter().GetResult(); // ensure observed, for IVTS reasons + pending.GetAwaiter().GetResult(); // ensure observed to ensure the IValueTaskSource completed/result is consumed } catch { From f0233b0e3ba947cb86da1bdbff917345d622c541 Mon Sep 17 00:00:00 2001 From: Nick Craver Date: Tue, 30 Aug 2022 09:08:36 -0400 Subject: [PATCH 14/17] Test fix for SendInvocationRequest_SignalCancellation_WithCapability_SendsInvocationCancelRequest --- test/WebJobs.Script.Tests/Workers/Rpc/GrpcWorkerChannelTests.cs | 1 + 1 file changed, 1 insertion(+) diff --git a/test/WebJobs.Script.Tests/Workers/Rpc/GrpcWorkerChannelTests.cs b/test/WebJobs.Script.Tests/Workers/Rpc/GrpcWorkerChannelTests.cs index adbfcff31c..475901c870 100644 --- a/test/WebJobs.Script.Tests/Workers/Rpc/GrpcWorkerChannelTests.cs +++ b/test/WebJobs.Script.Tests/Workers/Rpc/GrpcWorkerChannelTests.cs @@ -402,6 +402,7 @@ public async Task SendInvocationRequest_SignalCancellation_WithCapability_SendsI break; } } + await Task.Delay(500); var traces = _logger.GetLogMessages(); Assert.True(traces.Any(m => string.Equals(m.FormattedMessage, expectedCancellationLog))); From 8f6ca1a368c2c7e906540c8bea7ef27ac1ad027d Mon Sep 17 00:00:00 2001 From: Nick Craver Date: Tue, 30 Aug 2022 09:45:10 -0400 Subject: [PATCH 15/17] Fix PR comments - ensure _outbound and _inbound are never null --- .../Channel/GrpcWorkerChannel.cs | 19 +++++++------------ 1 file changed, 7 insertions(+), 12 deletions(-) diff --git a/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs b/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs index 48cb7b1a75..e2ece7c52a 100644 --- a/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs +++ b/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs @@ -34,6 +34,7 @@ using FunctionMetadata = Microsoft.Azure.WebJobs.Script.Description.FunctionMetadata; using MsgType = Microsoft.Azure.WebJobs.Script.Grpc.Messages.StreamingMessage.ContentOneofCase; using ParameterBindingType = Microsoft.Azure.WebJobs.Script.Grpc.Messages.ParameterBinding.RpcDataOneofCase; +using RpcException = Microsoft.Azure.WebJobs.Script.Workers.Rpc.RpcException; namespace Microsoft.Azure.WebJobs.Script.Grpc { @@ -110,13 +111,15 @@ internal GrpcWorkerChannel( _workerCapabilities = new GrpcCapabilities(_workerChannelLogger); - if (_eventManager.TryGetGrpcChannels(workerId, out var inbound, out var outbound)) + if (!_eventManager.TryGetGrpcChannels(workerId, out var inbound, out var outbound)) { - _outbound = outbound.Writer; - _inbound = inbound.Reader; - // note: we don't start the read loop until StartWorkerProcessAsync is called + throw new InvalidOperationException("Could not get gRPC channels for worker ID: " + workerId); } + _outbound = outbound.Writer; + _inbound = inbound.Reader; + // note: we don't start the read loop until StartWorkerProcessAsync is called + _eventSubscriptions.Add(_eventManager.OfType() .Where(msg => _workerConfig.Description.Extensions.Contains(Path.GetExtension(msg.FileChangeArguments.FullPath))) .Throttle(TimeSpan.FromMilliseconds(300)) // debounce @@ -984,20 +987,12 @@ internal void HandleWorkerMetadataRequestError(Exception exc) private ValueTask SendStreamingMessageAsync(StreamingMessage msg) { - if (_outbound is null) - { - return default; - } var evt = new OutboundGrpcEvent(_workerId, msg); return _outbound.TryWrite(evt) ? default : _outbound.WriteAsync(evt); } private void SendStreamingMessage(StreamingMessage msg) { - if (_outbound is null) - { - return; - } var evt = new OutboundGrpcEvent(_workerId, msg); if (!_outbound.TryWrite(evt)) { From 6b06dc95fa1820486b424aa3f2158bdc8dc143fe Mon Sep 17 00:00:00 2001 From: Nick Craver Date: Tue, 30 Aug 2022 09:45:25 -0400 Subject: [PATCH 16/17] Remove cruft --- src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs | 1 - 1 file changed, 1 deletion(-) diff --git a/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs b/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs index e2ece7c52a..348018e4ad 100644 --- a/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs +++ b/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs @@ -34,7 +34,6 @@ using FunctionMetadata = Microsoft.Azure.WebJobs.Script.Description.FunctionMetadata; using MsgType = Microsoft.Azure.WebJobs.Script.Grpc.Messages.StreamingMessage.ContentOneofCase; using ParameterBindingType = Microsoft.Azure.WebJobs.Script.Grpc.Messages.ParameterBinding.RpcDataOneofCase; -using RpcException = Microsoft.Azure.WebJobs.Script.Workers.Rpc.RpcException; namespace Microsoft.Azure.WebJobs.Script.Grpc { From 6f9f11ec59a6645b2d4aa37fc1ec6590aaa4ca7b Mon Sep 17 00:00:00 2001 From: Nick Craver Date: Fri, 2 Sep 2022 11:55:42 -0400 Subject: [PATCH 17/17] Capture ObjectDisposedException in disposal race (occasional issue in tests) --- src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs b/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs index bc0e632b7e..46769bd2f5 100644 --- a/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs +++ b/src/WebJobs.Script.Grpc/Channel/GrpcWorkerChannel.cs @@ -1209,7 +1209,14 @@ internal async void OnTimer(object sender, System.Timers.ElapsedEventArgs e) // Don't allow background execptions to escape // E.g. when a rpc channel is shutting down we can process exceptions } - _timer.Start(); + try + { + _timer.Start(); + } + catch (ObjectDisposedException) + { + // Specifically ignore this race - we're exiting and that's okay + } } private void AddSample(List samples, T sample)