diff --git a/src/DurableTask.Netherite/Connections/ConnectionInfoExtensions.cs b/src/DurableTask.Netherite/Connections/ConnectionInfoExtensions.cs index 8510ea80..9500a673 100644 --- a/src/DurableTask.Netherite/Connections/ConnectionInfoExtensions.cs +++ b/src/DurableTask.Netherite/Connections/ConnectionInfoExtensions.cs @@ -14,13 +14,12 @@ namespace DurableTask.Netherite using System.Security.Cryptography; using System.Web; using DurableTask.Netherite.EventHubsTransport; - using Microsoft.Azure.EventHubs; using Azure.Core; using System.Runtime.CompilerServices; - using Microsoft.Azure.EventHubs.Processor; using Newtonsoft.Json.Serialization; using DurableTask.Netherite.Faster; using Azure.Storage.Blobs; + using Azure.Messaging.EventHubs; /// /// Utilities for constructing various SDK objects from a connection information. @@ -106,28 +105,25 @@ public static Azure.Storage.Blobs.BlobServiceClient GetAzureStorageV12BlobServic } } - /// - /// Creates an Event Hub client for the given connection info. + /// Creates an Event Hub connection. /// /// The connection info. /// The event hub name. /// - public static EventHubClient CreateEventHubClient(this ConnectionInfo connectionInfo, string eventHub) + public static EventHubConnection CreateEventHubConnection(this ConnectionInfo connectionInfo, string eventHub) { if (connectionInfo.ConnectionString != null) { - var connectionStringBuilder = new EventHubsConnectionStringBuilder(connectionInfo.ConnectionString) - { - EntityPath = eventHub - }; - return EventHubClient.CreateFromConnectionString(connectionStringBuilder.ToString()); + return new Azure.Messaging.EventHubs.EventHubConnection(connectionInfo.ConnectionString, eventHub); } else { - Uri uri = new Uri($"sb://{connectionInfo.HostName}"); - var tokenProvider = new EventHubsTokenProvider(connectionInfo); - return EventHubClient.CreateWithTokenProvider(uri, eventHub, tokenProvider); + return new Azure.Messaging.EventHubs.EventHubConnection( + fullyQualifiedNamespace: connectionInfo.HostName, + eventHubName: eventHub, + credential: connectionInfo.TokenCredential, + connectionOptions: null); } } @@ -135,67 +131,19 @@ public static EventHubClient CreateEventHubClient(this ConnectionInfo connection /// Creates an event processor host for the given connection info. /// /// The connection info. - /// The host name. - /// The event hub name. - /// The consumer group name. - /// A connection info for the checkpoint storage. - /// The name of the lease container. - /// A prefix for storing the blobs. + /// The constructor arguments. /// An event processor host. - public static async Task GetEventProcessorHostAsync( - this ConnectionInfo connectionInfo, - string hostName, - string eventHubPath, - string consumerGroupName, - ConnectionInfo checkpointStorage, - string leaseContainerName, - string storageBlobPrefix) - { + public static EventProcessorHost CreateEventProcessorHost( + this ConnectionInfo connectionInfo, + EventProcessorHost.ConstructorArguments args) + { if (connectionInfo.ConnectionString != null) { - return new EventProcessorHost( - hostName, - eventHubPath, - consumerGroupName, - connectionInfo.ConnectionString, - checkpointStorage.ConnectionString, - leaseContainerName, - storageBlobPrefix); + return new EventProcessorHost(args, connectionInfo.ConnectionString); } else { - var storageAccount = await checkpointStorage.GetAzureStorageV11AccountAsync(); - return new EventProcessorHost( - new Uri($"sb://{connectionInfo.HostName}"), - eventHubPath, - consumerGroupName, - (ITokenProvider) (new EventHubsTokenProvider(connectionInfo)), - storageAccount, - leaseContainerName, - storageBlobPrefix); - } - } - - class EventHubsTokenProvider : Microsoft.Azure.EventHubs.ITokenProvider - { - readonly ConnectionInfo info; - - public EventHubsTokenProvider(ConnectionInfo info) - { - this.info = info; - } - - static TimeSpan NextRefresh(AccessToken token) - { - DateTimeOffset now = DateTimeOffset.UtcNow; - return token.ExpiresOn - now - TimeSpan.FromMinutes(1); // refresh it a bit early. - } - - async Task ITokenProvider.GetTokenAsync(string appliesTo, TimeSpan timeout) - { - TokenRequestContext request = new(this.info.Scopes); - AccessToken accessToken = await this.info.TokenCredential.GetTokenAsync(request, CancellationToken.None); - return new JsonSecurityToken(accessToken.Token, appliesTo); + return new EventProcessorHost(args, connectionInfo.HostName, connectionInfo.TokenCredential); } } diff --git a/src/DurableTask.Netherite/DurableTask.Netherite.csproj b/src/DurableTask.Netherite/DurableTask.Netherite.csproj index c85268ad..076bb810 100644 --- a/src/DurableTask.Netherite/DurableTask.Netherite.csproj +++ b/src/DurableTask.Netherite/DurableTask.Netherite.csproj @@ -49,10 +49,10 @@ - + - - + + diff --git a/src/DurableTask.Netherite/OrchestrationService/TransportConnectionString.cs b/src/DurableTask.Netherite/OrchestrationService/TransportConnectionString.cs index bd91654a..1fc23c96 100644 --- a/src/DurableTask.Netherite/OrchestrationService/TransportConnectionString.cs +++ b/src/DurableTask.Netherite/OrchestrationService/TransportConnectionString.cs @@ -7,7 +7,7 @@ namespace DurableTask.Netherite using System.Collections.Generic; using System.Runtime; using System.Text; - using Microsoft.Azure.EventHubs; + using Azure.Messaging.EventHubs; /// /// Encapsulates how the transport connection string setting is interpreted. @@ -63,8 +63,8 @@ public static string EventHubsNamespaceName(string transportConnectionString) { try { - var builder = new EventHubsConnectionStringBuilder(transportConnectionString); - var host = builder.Endpoint.Host; + var properties = EventHubsConnectionStringProperties.Parse(transportConnectionString); + var host = properties.Endpoint.Host; return host.Substring(0, host.IndexOf('.')); } catch(Exception e) diff --git a/src/DurableTask.Netherite/StorageLayer/Faster/AzureBlobs/AzureStorageDevice.cs b/src/DurableTask.Netherite/StorageLayer/Faster/AzureBlobs/AzureStorageDevice.cs index 4e64868b..73e29d05 100644 --- a/src/DurableTask.Netherite/StorageLayer/Faster/AzureBlobs/AzureStorageDevice.cs +++ b/src/DurableTask.Netherite/StorageLayer/Faster/AzureBlobs/AzureStorageDevice.cs @@ -82,12 +82,12 @@ public AzureStorageDevice(string blobName, BlobUtilsV12.BlobDirectory blockBlobD this.pageBlobDirectory = pageBlobDirectory; this.blobName = blobName; this.PartitionErrorHandler = blobManager.PartitionErrorHandler; - this.PartitionErrorHandler.Token.Register(this.CancelAllRequests); this.BlobManager = blobManager; this.underLease = underLease; this.hangCheckTimer = new Timer(this.DetectHangs, null, 0, 20000); this.singleWriterSemaphore = underLease ? new SemaphoreSlim(1) : null; this.limit = TimeSpan.FromSeconds(90); + this.PartitionErrorHandler.Token.Register(this.CancelAllRequests); } /// @@ -106,7 +106,7 @@ public async Task StartAsync() var prefix = $"{this.blockBlobDirectory}{this.blobName}."; string continuationToken = null; - IEnumerable pageResults = null; + List pageResults = null; do { @@ -124,25 +124,15 @@ await this.BlobManager.PerformWithRetriesAsync( { var client = this.pageBlobDirectory.Client.WithRetries; - var enumerator = client.GetBlobsAsync( + Azure.AsyncPageable pageable = client.GetBlobsAsync( prefix: prefix, - cancellationToken: this.PartitionErrorHandler.Token) - .AsPages(continuationToken, 100) - .GetAsyncEnumerator(cancellationToken: this.PartitionErrorHandler.Token); + cancellationToken: this.PartitionErrorHandler.Token); - if (await enumerator.MoveNextAsync()) - { - var page = enumerator.Current; - pageResults = page.Values; - continuationToken = page.ContinuationToken; - return page.Values.Count; // not accurate, in terms of bytes, but still useful for tracing purposes - } - else - { - pageResults = Enumerable.Empty(); - continuationToken = null; - return 0; - }; + IAsyncEnumerable> pages = pageable.AsPages(continuationToken, 100); + Azure.Page firstPage = await pages.FirstAsync(); + pageResults = firstPage.Values.ToList(); + continuationToken = firstPage.ContinuationToken; + return pageResults.Count; // not accurate, in terms of bytes, but still useful for tracing purposes }); foreach (var item in pageResults) diff --git a/src/DurableTask.Netherite/TransportLayer/EventHubs/BlobBatchReceiver.cs b/src/DurableTask.Netherite/TransportLayer/EventHubs/BlobBatchReceiver.cs index 2161006c..50e247e5 100644 --- a/src/DurableTask.Netherite/TransportLayer/EventHubs/BlobBatchReceiver.cs +++ b/src/DurableTask.Netherite/TransportLayer/EventHubs/BlobBatchReceiver.cs @@ -11,12 +11,11 @@ namespace DurableTask.Netherite.EventHubsTransport using System.Text; using System.Threading; using System.Threading.Tasks; + using Azure.Messaging.EventHubs; using Azure.Storage.Blobs; using Azure.Storage.Blobs.Models; using Azure.Storage.Blobs.Specialized; using DurableTask.Netherite.Faster; - using Microsoft.Azure.EventHubs; - using Microsoft.Azure.Storage; using Microsoft.Extensions.Azure; using Microsoft.Extensions.Logging; @@ -53,7 +52,7 @@ public BlobBatchReceiver(string traceContext, EventHubsTraceHelper traceHelper, foreach (var eventData in hubMessages) { - var seqno = eventData.SystemProperties.SequenceNumber; + var seqno = eventData.SequenceNumber; if (nextPacketToReceive != null) { @@ -75,11 +74,11 @@ public BlobBatchReceiver(string traceContext, EventHubsTraceHelper traceHelper, try { - Packet.Deserialize(eventData.Body, out evt, out blobReference, guid); + Packet.Deserialize(eventData.EventBody.ToStream(), out evt, out blobReference, guid); } catch (Exception) { - this.traceHelper.LogError("{context} could not deserialize packet #{seqno} ({size} bytes)", this.traceContext, seqno, eventData.Body.Count); + this.traceHelper.LogError("{context} could not deserialize packet #{seqno} ({size} bytes)", this.traceContext, seqno, eventData.EventBody.ToMemory().Length); throw; } @@ -87,7 +86,7 @@ public BlobBatchReceiver(string traceContext, EventHubsTraceHelper traceHelper, { if (evt == null) { - this.lowestTraceLevel?.LogTrace("{context} ignored packet #{seqno} ({size} bytes) because its guid does not match taskhub/client", this.traceContext, seqno, eventData.Body.Count); + this.lowestTraceLevel?.LogTrace("{context} ignored packet #{seqno} ({size} bytes) because its guid does not match taskhub/client", this.traceContext, seqno, eventData.EventBody.ToMemory().Length); ignoredPacketCount++; } else diff --git a/src/DurableTask.Netherite/TransportLayer/EventHubs/BlobBatchSender.cs b/src/DurableTask.Netherite/TransportLayer/EventHubs/BlobBatchSender.cs index b7023bc7..f572e631 100644 --- a/src/DurableTask.Netherite/TransportLayer/EventHubs/BlobBatchSender.cs +++ b/src/DurableTask.Netherite/TransportLayer/EventHubs/BlobBatchSender.cs @@ -11,12 +11,12 @@ namespace DurableTask.Netherite.EventHubsTransport using System.Text; using System.Threading; using System.Threading.Tasks; + using Azure.Messaging.EventHubs; using Azure.Storage.Blobs; using Azure.Storage.Blobs.Models; using Azure.Storage.Blobs.Specialized; using DurableTask.Netherite.Abstractions; using DurableTask.Netherite.Faster; - using Microsoft.Azure.EventHubs; using Microsoft.Extensions.Logging; class BlobBatchSender diff --git a/src/DurableTask.Netherite/TransportLayer/EventHubs/EventHubsClientSender.cs b/src/DurableTask.Netherite/TransportLayer/EventHubs/EventHubsClientSender.cs index 069ed3df..dc99319d 100644 --- a/src/DurableTask.Netherite/TransportLayer/EventHubs/EventHubsClientSender.cs +++ b/src/DurableTask.Netherite/TransportLayer/EventHubs/EventHubsClientSender.cs @@ -3,13 +3,8 @@ namespace DurableTask.Netherite.EventHubsTransport { - using DurableTask.Core.Common; - using Microsoft.Azure.EventHubs; - using Microsoft.Extensions.Logging; + using Azure.Messaging.EventHubs; using System; - using System.Collections.Generic; - using System.Diagnostics; - using System.IO; using System.Linq; using System.Threading; using System.Threading.Tasks; @@ -19,12 +14,12 @@ class EventHubsClientSender readonly EventHubsSender[] channels; int roundRobin; - public EventHubsClientSender(TransportAbstraction.IHost host, Guid clientId, PartitionSender[] senders, CancellationToken shutdownToken, EventHubsTraceHelper traceHelper, NetheriteOrchestrationServiceSettings settings) + public EventHubsClientSender(TransportAbstraction.IHost host, Guid clientId, (EventHubConnection connection, string partitionId)[] partitions, CancellationToken shutdownToken, EventHubsTraceHelper traceHelper, NetheriteOrchestrationServiceSettings settings) { - this.channels = new Netherite.EventHubsTransport.EventHubsSender[senders.Length]; - for (int i = 0; i < senders.Length; i++) + this.channels = new Netherite.EventHubsTransport.EventHubsSender[partitions.Length]; + for (int i = 0; i < partitions.Length; i++) { - this.channels[i] = new EventHubsSender(host, clientId.ToByteArray(), senders[i], shutdownToken, traceHelper, settings); + this.channels[i] = new EventHubsSender(host, clientId.ToByteArray(), partitions[i].connection, partitions[i].partitionId, shutdownToken, traceHelper, settings); } } @@ -44,7 +39,7 @@ public void Submit(ClientEvent toSend) public Task WaitForShutdownAsync() { - return Task.WhenAll(this.channels.Select(sender => sender.WaitForShutdownAsync())); + return Task.WhenAll(this.channels.Select(sender => sender.WaitForShutdownAsync()).ToList()); } } } diff --git a/src/DurableTask.Netherite/TransportLayer/EventHubs/EventHubsConnections.cs b/src/DurableTask.Netherite/TransportLayer/EventHubs/EventHubsConnections.cs index 32598e5d..0851ff7d 100644 --- a/src/DurableTask.Netherite/TransportLayer/EventHubs/EventHubsConnections.cs +++ b/src/DurableTask.Netherite/TransportLayer/EventHubs/EventHubsConnections.cs @@ -9,8 +9,12 @@ namespace DurableTask.Netherite.EventHubsTransport using System.Linq; using System.Threading; using System.Threading.Tasks; + using Azure.Messaging.EventHubs; + using Azure.Messaging.EventHubs.Consumer; + using Azure.Messaging.EventHubs.Primitives; + using Azure.Messaging.EventHubs.Producer; using DurableTask.Netherite.Abstractions; - using Microsoft.Azure.EventHubs; + using Microsoft.Extensions.Azure; using Microsoft.Extensions.Logging; class EventHubsConnections @@ -21,12 +25,12 @@ class EventHubsConnections readonly string loadMonitorHub; readonly CancellationToken shutdownToken; - EventHubClient partitionClient; - List clientClients; - EventHubClient loadMonitorClient; + EventHubConnection partitionConnection; + List clientConnection; + EventHubConnection loadMonitorConnection; - readonly List<(EventHubClient client, string id)> partitionPartitions = new List<(EventHubClient client, string id)>(); - readonly List<(EventHubClient client, string id)> clientPartitions = new List<(EventHubClient client, string id)>(); + readonly List<(EventHubConnection connection, string id)> partitionPartitions = new List<(EventHubConnection connection, string id)>(); + readonly List<(EventHubConnection connection, string id)> clientPartitions = new List<(EventHubConnection connection, string id)>(); public const int NumClientChannels = 2; @@ -78,9 +82,9 @@ async Task StopClientClients() { await Task.WhenAll(this._clientSenders.Values.Select(sender => sender.WaitForShutdownAsync()).ToList()); - if (this.clientClients != null) + if (this.clientConnection != null) { - await Task.WhenAll(this.clientClients.Select(client => client.CloseAsync()).ToList()); + await Task.WhenAll(this.clientConnection.Select(client => client.CloseAsync()).ToList()); } } @@ -88,9 +92,9 @@ async Task StopPartitionClients() { await Task.WhenAll(this._partitionSenders.Values.Select(sender => sender.WaitForShutdownAsync()).ToList()); - if (this.partitionClient != null) + if (this.partitionConnection != null) { - await this.partitionClient.CloseAsync(); + await this.partitionConnection.CloseAsync(); } } @@ -100,7 +104,7 @@ async Task StopLoadMonitorClients() if (this.loadMonitorHub != null) { - await this.loadMonitorClient.CloseAsync(); + await this.loadMonitorConnection.CloseAsync(); } } @@ -127,20 +131,23 @@ internal async Task DeletePartitions() } public async Task EnsurePartitionsAsync(int partitionCount, int retries = EventHubCreationRetries) - { - var client = this.connectionInfo.CreateEventHubClient(this.partitionHub); + { try { - var runtimeInformation = await client.GetRuntimeInformationAsync(); + var connection = this.connectionInfo.CreateEventHubConnection(this.partitionHub); + + await using var client = new EventHubProducerClient(connection); + + EventHubProperties eventHubProperties = await client.GetEventHubPropertiesAsync(); - if (runtimeInformation.PartitionCount == partitionCount) + if (eventHubProperties.PartitionIds.Length == partitionCount) { // we were successful. Record information and create a flat list of partition partitions - this.partitionClient = client; - this.CreationTimestamp = runtimeInformation.CreatedAt; + this.partitionConnection = connection; + this.CreationTimestamp = eventHubProperties.CreatedOn.UtcDateTime; for (int i = 0; i < partitionCount; i++) { - this.partitionPartitions.Add((client, i.ToString())); + this.partitionPartitions.Add((connection, i.ToString())); } return; } @@ -152,7 +159,7 @@ public async Task EnsurePartitionsAsync(int partitionCount, int retries = EventH await Task.Delay(TimeSpan.FromSeconds(10)); } } - catch (Microsoft.Azure.EventHubs.MessagingEntityNotFoundException) when (retries > 0) + catch (Azure.Messaging.EventHubs.EventHubsException e) when (retries > 0 && e.Reason == EventHubsException.FailureReason.ResourceNotFound) { await this.EnsureEventHubExistsAsync(this.partitionHub, partitionCount); } @@ -171,14 +178,14 @@ public async Task EnsurePartitionsAsync(int partitionCount, int retries = EventH async Task EnsureClientsAsync() { - var clientTasks = new List>(); + var clientTasks = new List>(); for (int i = 0; i < this.clientHubs.Count(); i++) { clientTasks.Add(this.EnsureClientAsync(i)); } await Task.WhenAll(clientTasks); - this.clientClients = clientTasks.Select(t => t.Result.Item1).ToList(); + this.clientConnection = clientTasks.Select(t => t.Result.Item1).ToList(); var clientInfos = clientTasks.Select(t => t.Result.Item2).ToList(); // create a flat list of client partitions @@ -186,20 +193,21 @@ async Task EnsureClientsAsync() { foreach (var id in clientTasks[i].Result.Item2.PartitionIds) { - this.clientPartitions.Add((this.clientClients[i], id)); + this.clientPartitions.Add((this.clientConnection[i], id)); } } } - async Task<(EventHubClient, EventHubRuntimeInformation)> EnsureClientAsync(int i, int retries = EventHubCreationRetries) + async Task<(EventHubConnection, EventHubProperties)> EnsureClientAsync(int i, int retries = EventHubCreationRetries) { try { - var client = this.connectionInfo.CreateEventHubClient(this.clientHubs[i]); - var runtimeInformation = await client.GetRuntimeInformationAsync(); - return (client, runtimeInformation); + var connection = this.connectionInfo.CreateEventHubConnection(this.clientHubs[i]); + await using var client = new EventHubProducerClient(connection); + var runtimeInformation = await client.GetEventHubPropertiesAsync(); + return (connection, runtimeInformation); } - catch (Microsoft.Azure.EventHubs.MessagingEntityNotFoundException) when (retries > 0) + catch (Azure.Messaging.EventHubs.EventHubsException e) when (retries > 0 && e.Reason == EventHubsException.FailureReason.ResourceNotFound) { await this.EnsureEventHubExistsAsync(this.clientHubs[i], 32); } @@ -213,11 +221,13 @@ async Task EnsureLoadMonitorAsync(int retries = EventHubCreationRetries) // create loadmonitor client try { - this.loadMonitorClient = this.connectionInfo.CreateEventHubClient(this.loadMonitorHub); - var runtimeInformation = await this.loadMonitorClient.GetRuntimeInformationAsync(); + var connection = this.connectionInfo.CreateEventHubConnection(this.loadMonitorHub); + await using var client = new EventHubProducerClient(connection); + var runtimeInformation = await client.GetEventHubPropertiesAsync(); + this.loadMonitorConnection = connection; return; } - catch (Microsoft.Azure.EventHubs.MessagingEntityNotFoundException) when (retries > 0) + catch (Azure.Messaging.EventHubs.EventHubsException e) when (retries > 0 && e.Reason == EventHubsException.FailureReason.ResourceNotFound) { await this.EnsureEventHubExistsAsync(this.loadMonitorHub, 1); } @@ -227,15 +237,16 @@ async Task EnsureLoadMonitorAsync(int retries = EventHubCreationRetries) public static async Task> GetQueuePositionsAsync(ConnectionInfo connectionInfo, string partitionHub) { - var client = connectionInfo.CreateEventHubClient(partitionHub); try { - var runtimeInformation = await client.GetRuntimeInformationAsync(); - var infoTasks = runtimeInformation.PartitionIds.Select(id => client.GetPartitionRuntimeInformationAsync(id)).ToList(); + var connection = connectionInfo.CreateEventHubConnection(partitionHub); + var client = new EventHubProducerClient(connection); + var partitions = await client.GetPartitionIdsAsync(); + var infoTasks = partitions.Select(id => client.GetPartitionPropertiesAsync(id)).ToList(); await Task.WhenAll(infoTasks); return infoTasks.Select(t => t.Result.LastEnqueuedSequenceNumber + 1).ToList(); } - catch (Microsoft.Azure.EventHubs.MessagingEntityNotFoundException) + catch (Azure.Messaging.EventHubs.EventHubsException e) when (e.Reason == EventHubsException.FailureReason.ResourceNotFound) { return null; } @@ -244,11 +255,15 @@ public static async Task> GetQueuePositionsAsync(ConnectionInfo conne // This is to be used when EventProcessorHost is not used. public PartitionReceiver CreatePartitionReceiver(int partitionId, string consumerGroupName, long nextPacketToReceive) { - (EventHubClient client, string id) = this.partitionPartitions[partitionId]; - // To create a receiver we need to give it the last! packet number and not the next to receive - var eventPosition = EventPosition.FromSequenceNumber(nextPacketToReceive - 1); - var partitionReceiver = client.CreateReceiver(consumerGroupName, id, eventPosition); - this.TraceHelper.LogDebug("Created Partition {partitionId} PartitionReceiver {receiver} from {clientId} to read at {position}", partitionId, partitionReceiver.ClientId, client.ClientId, nextPacketToReceive); + (EventHubConnection connection, string id) = this.partitionPartitions[partitionId]; + + var partitionReceiver = new PartitionReceiver( + consumerGroupName, + partitionId.ToString(), + EventPosition.FromSequenceNumber(nextPacketToReceive - 1), + connection); + + this.TraceHelper.LogDebug("Created Partition {partitionId} PartitionReceiver {receiver} to read at {position}", partitionId, partitionReceiver.Identifier, nextPacketToReceive); return partitionReceiver; } @@ -258,35 +273,32 @@ public PartitionReceiver[] CreateClientReceivers(Guid clientId, string consumerG for (int index = 0; index < EventHubsConnections.NumClientChannels; index++) { int clientBucket = this.GetClientBucket(clientId, index); - (EventHubClient client, string id) = this.clientPartitions[clientBucket]; - var clientReceiver = client.CreateReceiver(consumerGroupName, id, EventPosition.FromEnd()); - this.TraceHelper.LogDebug("Created Client {clientId} PartitionReceiver {receiver} from {clientId}", clientId, clientReceiver.ClientId, client.ClientId); + (EventHubConnection connection, string partitionId) = this.clientPartitions[clientBucket]; + + var clientReceiver = new PartitionReceiver( + consumerGroupName, + partitionId.ToString(), + EventPosition.Latest, + connection); + + this.TraceHelper.LogDebug("Created Client {clientId} PartitionReceiver {receiver}", clientId, clientReceiver.Identifier); partitionReceivers[index] = clientReceiver; } return partitionReceivers; } - public PartitionReceiver CreateLoadMonitorReceiver(string consumerGroupName) - { - var loadMonitorReceiver = this.loadMonitorClient.CreateReceiver(consumerGroupName, "0", EventPosition.FromEnqueuedTime(DateTime.UtcNow - TimeSpan.FromSeconds(10))); - this.TraceHelper.LogDebug("Created LoadMonitor PartitionReceiver {receiver} from {clientId}", loadMonitorReceiver.ClientId, this.loadMonitorClient.ClientId); - return loadMonitorReceiver; - } - - public EventHubsSender GetPartitionSender(int partitionId, byte[] taskHubGuid, NetheriteOrchestrationServiceSettings settings) { return this._partitionSenders.GetOrAdd(partitionId, (key) => { - (EventHubClient client, string id) = this.partitionPartitions[partitionId]; - var partitionSender = client.CreatePartitionSender(id); + (EventHubConnection connection, string id) = this.partitionPartitions[partitionId]; var sender = new EventHubsSender( this.Host, taskHubGuid, - partitionSender, + connection, + id, this.shutdownToken, this.TraceHelper, settings); - this.TraceHelper.LogDebug("Created PartitionSender {sender} from {clientId}", partitionSender.ClientId, client.ClientId); return sender; }); } @@ -295,18 +307,16 @@ public EventHubsClientSender GetClientSender(Guid clientId, NetheriteOrchestrati { return this._clientSenders.GetOrAdd(clientId, (key) => { - var partitionSenders = new PartitionSender[NumClientChannels]; + var partitions = new (EventHubConnection connection, string partitionId)[NumClientChannels]; for (int index = 0; index < NumClientChannels; index++) { int clientBucket = this.GetClientBucket(clientId, index); - (EventHubClient client, string id) = this.clientPartitions[clientBucket]; - partitionSenders[index] = client.CreatePartitionSender(id); - this.TraceHelper.LogDebug("Created ClientSender {sender} from {clientId}", partitionSenders[index].ClientId, client.ClientId); + partitions[index] = this.clientPartitions[clientBucket]; } var sender = new EventHubsClientSender( this.Host, clientId, - partitionSenders, + partitions, this.shutdownToken, this.TraceHelper, settings); @@ -317,15 +327,13 @@ public EventHubsClientSender GetClientSender(Guid clientId, NetheriteOrchestrati public LoadMonitorSender GetLoadMonitorSender(byte[] taskHubGuid) { return this._loadMonitorSenders.GetOrAdd(0, (key) => - { - var loadMonitorSender = this.loadMonitorClient.CreatePartitionSender("0"); + { var sender = new LoadMonitorSender( this.Host, taskHubGuid, - loadMonitorSender, + this.loadMonitorConnection, this.shutdownToken, this.TraceHelper); - this.TraceHelper.LogDebug("Created LoadMonitorSender {sender} from {clientId}", loadMonitorSender.ClientId, this.loadMonitorClient.ClientId); return sender; }); } diff --git a/src/DurableTask.Netherite/TransportLayer/EventHubs/EventHubsPartitionManager.cs b/src/DurableTask.Netherite/TransportLayer/EventHubs/EventHubsPartitionManager.cs index 8bc58ffc..ad19ecc1 100644 --- a/src/DurableTask.Netherite/TransportLayer/EventHubs/EventHubsPartitionManager.cs +++ b/src/DurableTask.Netherite/TransportLayer/EventHubs/EventHubsPartitionManager.cs @@ -3,11 +3,13 @@ namespace DurableTask.Netherite.EventHubsTransport { + using Azure.Messaging.EventHubs; + using Azure.Messaging.EventHubs.Primitives; + using Azure.Messaging.EventHubs.Processor; + using Azure.Storage.Blobs; using DurableTask.Core.Common; using DurableTask.Netherite.Abstractions; - using Microsoft.Azure.EventHubs; - using Microsoft.Azure.EventHubs.Processor; - using Microsoft.Azure.Storage.Blob; + //using Microsoft.Azure.Storage.Blob; using Microsoft.Extensions.Logging; using System; using System.Collections.Generic; @@ -26,7 +28,7 @@ class EventHubsPartitionManager : IPartitionManager readonly TaskhubParameters parameters; readonly NetheriteOrchestrationServiceSettings settings; readonly EventHubsTraceHelper traceHelper; - readonly CloudBlobContainer cloudBlobContainer; + readonly BlobContainerClient cloudBlobContainer; readonly string pathPrefix; readonly EventHubsTransport transport; readonly CancellationToken shutdownToken; @@ -40,7 +42,7 @@ public EventHubsPartitionManager( TaskhubParameters parameters, NetheriteOrchestrationServiceSettings settings, EventHubsTraceHelper traceHelper, - CloudBlobContainer cloudBlobContainer, + BlobContainerClient cloudBlobContainer, string pathPrefix, EventHubsTransport transport, CancellationToken shutdownToken) @@ -65,7 +67,7 @@ public Task StartHostingAsync() public Task StopHostingAsync() { - return Task.WhenAll(this.eventProcessorHost.UnregisterEventProcessorAsync(), this.loadMonitorHost.UnregisterEventProcessorAsync()); + return Task.WhenAll(this.StopHostingPartitionsAsync(), this.StopHostingLoadMonitorAsync()); } async Task StartHostingPartitionsAsync() @@ -74,23 +76,17 @@ async Task StartHostingPartitionsAsync() string formattedCreationDate = this.connections.CreationTimestamp.ToString("o").Replace("/", "-"); - this.eventProcessorHost = await this.settings.EventHubsConnection.GetEventProcessorHostAsync( - Guid.NewGuid().ToString(), - EventHubsTransport.PartitionHub, - EventHubsTransport.PartitionConsumerGroup, - this.settings.BlobStorageConnection, - this.cloudBlobContainer.Name, - $"{this.pathPrefix}eh-checkpoints/{(EventHubsTransport.PartitionHub)}/{formattedCreationDate}"); - - var processorOptions = new EventProcessorOptions() + this.eventProcessorHost = this.settings.EventHubsConnection.CreateEventProcessorHost(new EventProcessorHost.ConstructorArguments() { - MaxBatchSize = 300, - PrefetchCount = 500, - }; + EventHubName = EventHubsTransport.PartitionHub, + ClientOptions = new Azure.Messaging.EventHubs.EventProcessorClientOptions() { PrefetchCount = 500 }, + ConsumerGroup = EventHubsTransport.PartitionConsumerGroup, + BlobContainerClient = this.cloudBlobContainer, + Factory = new PartitionEventProcessorFactory(this), + TraceHelper = this.traceHelper, + }); - await this.eventProcessorHost.RegisterEventProcessorFactoryAsync( - new PartitionEventProcessorFactory(this), - processorOptions); + await this.eventProcessorHost.StartProcessingAsync(this.shutdownToken).ConfigureAwait(false); this.traceHelper.LogInformation($"EventHubsTransport started EventProcessorHost"); } @@ -99,29 +95,40 @@ async Task StartHostingLoadMonitorAsync() { this.traceHelper.LogInformation("EventHubsTransport is registering LoadMonitorHost"); - this.loadMonitorHost = await this.settings.EventHubsConnection.GetEventProcessorHostAsync( - Guid.NewGuid().ToString(), - EventHubsTransport.LoadMonitorHub, - EventHubsTransport.LoadMonitorConsumerGroup, - this.settings.BlobStorageConnection, - this.cloudBlobContainer.Name, - $"{this.pathPrefix}eh-checkpoints/{EventHubsTransport.LoadMonitorHub}"); - - var processorOptions = new EventProcessorOptions() + this.loadMonitorHost = this.settings.EventHubsConnection.CreateEventProcessorHost(new EventProcessorHost.ConstructorArguments() { - InitialOffsetProvider = (s) => EventPosition.FromEnqueuedTime(DateTime.UtcNow - TimeSpan.FromSeconds(30)), - MaxBatchSize = 500, - PrefetchCount = 500, - }; + EventHubName = EventHubsTransport.LoadMonitorHub, + ClientOptions = new Azure.Messaging.EventHubs.EventProcessorClientOptions() { PrefetchCount = 500 }, + ConsumerGroup = EventHubsTransport.LoadMonitorConsumerGroup, + BlobContainerClient = this.cloudBlobContainer, + Factory = new LoadMonitorEventProcessorFactory(this), + TraceHelper = this.traceHelper, + }); - await this.loadMonitorHost.RegisterEventProcessorFactoryAsync( - new LoadMonitorEventProcessorFactory(this), - processorOptions); + await this.loadMonitorHost.StartProcessingAsync(this.shutdownToken).ConfigureAwait(false); this.traceHelper.LogInformation($"EventHubsTransport started LoadMonitorHost"); } + + async Task StopHostingPartitionsAsync() + { + this.traceHelper.LogInformation($"EventHubsTransport is stopping EventProcessorHost"); + + await this.eventProcessorHost.StopProcessingAsync(); + + this.traceHelper.LogInformation($"EventHubsTransport stopped EventProcessorHost"); + } + + async Task StopHostingLoadMonitorAsync() + { + this.traceHelper.LogInformation($"EventHubsTransport is stopping LoadMonitorHost"); + + await this.loadMonitorHost.StopProcessingAsync(); + + this.traceHelper.LogInformation($"EventHubsTransport stopped LoadMonitorHost"); + } - class PartitionEventProcessorFactory : IEventProcessorFactory + public class PartitionEventProcessorFactory : IEventProcessorFactory { readonly EventHubsPartitionManager manager; @@ -130,18 +137,19 @@ public PartitionEventProcessorFactory(EventHubsPartitionManager transport) this.manager = transport; } - public IEventProcessor CreateEventProcessor(PartitionContext context) + public IEventProcessor CreateEventProcessor(EventProcessorClient client, string partitionId) { - return new EventHubsProcessor( + return new PartitionProcessor( this.manager.host, this.manager.transport, this.manager.parameters, - context, + client, + partitionId, this.manager.settings, this.manager.transport, this.manager.traceHelper, this.manager.shutdownToken); - } + } } class LoadMonitorEventProcessorFactory : IEventProcessorFactory @@ -153,16 +161,18 @@ public LoadMonitorEventProcessorFactory(EventHubsPartitionManager transport) this.manager = transport; } - public IEventProcessor CreateEventProcessor(PartitionContext context) + public IEventProcessor CreateEventProcessor(EventProcessorClient client, string partitionId) { - return new LoadMonitorProcessor( - this.manager.host, - this.manager.transport, - this.manager.parameters, - context, - this.manager.settings, - this.manager.traceHelper, - this.manager.shutdownToken); + return new LoadMonitorProcessor( + this.manager.host, + this.manager.transport, + this.manager.parameters, + client, + partitionId, + this.manager.settings, + this.manager.transport, + this.manager.traceHelper, + this.manager.shutdownToken); } } } diff --git a/src/DurableTask.Netherite/TransportLayer/EventHubs/EventHubsSender.cs b/src/DurableTask.Netherite/TransportLayer/EventHubs/EventHubsSender.cs index a7fa06f5..535f409b 100644 --- a/src/DurableTask.Netherite/TransportLayer/EventHubs/EventHubsSender.cs +++ b/src/DurableTask.Netherite/TransportLayer/EventHubs/EventHubsSender.cs @@ -9,14 +9,14 @@ namespace DurableTask.Netherite.EventHubsTransport using System.IO; using System.Threading; using System.Threading.Tasks; - using Azure.Messaging.EventHubs.Consumer; + using Azure.Messaging.EventHubs; + using Azure.Messaging.EventHubs.Producer; using DurableTask.Core.Common; - using Microsoft.Azure.EventHubs; using Microsoft.Extensions.Logging; class EventHubsSender : BatchWorker where T : Event { - readonly PartitionSender sender; + readonly EventHubProducerClient client; readonly TransportAbstraction.IHost host; readonly byte[] guid; readonly EventHubsTraceHelper traceHelper; @@ -26,21 +26,39 @@ class EventHubsSender : BatchWorker where T : Event readonly TimeSpan backoff = TimeSpan.FromSeconds(5); readonly MemoryStream stream = new MemoryStream(); // reused for all packets readonly Stopwatch stopwatch = new Stopwatch(); - readonly BlobBatchSender blobBatchSender; + readonly BlobBatchSender blobBatchSender; + readonly CreateBatchOptions batchOptions; + readonly SendEventOptions sendOptions; - public EventHubsSender(TransportAbstraction.IHost host, byte[] guid, PartitionSender sender, CancellationToken shutdownToken, EventHubsTraceHelper traceHelper, NetheriteOrchestrationServiceSettings settings) - : base($"EventHubsSender {sender.EventHubClient.EventHubName}/{sender.PartitionId}", false, 2000, shutdownToken, traceHelper) + public EventHubsSender(TransportAbstraction.IHost host, byte[] guid, EventHubConnection connection, string partitionId, CancellationToken shutdownToken, EventHubsTraceHelper traceHelper, NetheriteOrchestrationServiceSettings settings) + : base($"EventHubsSender {connection.EventHubName}/{partitionId}", false, 2000, shutdownToken, traceHelper) { this.host = host; this.guid = guid; - this.sender = sender; + this.client = new EventHubProducerClient(connection); this.traceHelper = traceHelper; this.lowestTraceLevel = traceHelper.IsEnabled(LogLevel.Trace) ? traceHelper : null; - this.eventHubName = this.sender.EventHubClient.EventHubName; - this.eventHubPartition = this.sender.PartitionId; + this.eventHubName = connection.EventHubName; + this.eventHubPartition = partitionId; this.blobBatchSender = new BlobBatchSender($"EventHubsSender {this.eventHubName}/{this.eventHubPartition}", this.traceHelper, settings); + this.batchOptions = new CreateBatchOptions() + { + PartitionId = partitionId, + }; + this.sendOptions = new SendEventOptions() + { + PartitionId = partitionId, + }; + } + + public override async Task WaitForShutdownAsync() + { + await base.WaitForShutdownAsync(); + await this.client.CloseAsync(); } + public string EventHubsClientIdentifier => this.client.Identifier; + protected override async Task Process(IList toSend) { if (toSend.Count == 0) @@ -106,7 +124,7 @@ void CollectBatchContent(bool specificallyForBlob) { // we don't have a lot of bytes or messages to send // send them all in a single EH batch - using var batch = this.sender.CreateBatch(); + using var batch = await this.client.CreateBatchAsync(this.batchOptions, this.cancellationToken).ConfigureAwait(false); long maxPosition = this.stream.Position; this.stream.Seek(0, SeekOrigin.Begin); var buffer = this.stream.GetBuffer(); @@ -129,10 +147,10 @@ void CollectBatchContent(bool specificallyForBlob) } maybeSent = index - 1; this.stopwatch.Restart(); - await this.sender.SendAsync(batch).ConfigureAwait(false); + await this.client.SendAsync(batch, this.cancellationToken).ConfigureAwait(false); this.stopwatch.Stop(); sentSuccessfully = index - 1; - this.traceHelper.LogDebug("EventHubsSender {eventHubName}/{eventHubPartitionId} sent batch of {numPackets} packets ({size} bytes) in {latencyMs:F2}ms, throughput={throughput:F2}MB/s", this.eventHubName, this.eventHubPartition, batch.Count, batch.Size, this.stopwatch.Elapsed.TotalMilliseconds, batch.Size / (1024 * 1024 * this.stopwatch.Elapsed.TotalSeconds)); + this.traceHelper.LogDebug("EventHubsSender {eventHubName}/{eventHubPartitionId} sent batch of {numPackets} packets ({size} bytes) in {latencyMs:F2}ms, throughput={throughput:F2}MB/s", this.eventHubName, this.eventHubPartition, batch.Count, batch.SizeInBytes, this.stopwatch.Elapsed.TotalMilliseconds, batch.SizeInBytes / (1024 * 1024 * this.stopwatch.Elapsed.TotalSeconds)); break; // all messages were sent } else @@ -145,7 +163,7 @@ void CollectBatchContent(bool specificallyForBlob) this.stopwatch.Restart(); EventData blobMessage = await this.blobBatchSender.UploadEventsAsync(this.stream, packetOffsets, this.guid, this.cancellationToken); maybeSent = index - 1; - await this.sender.SendAsync(blobMessage); + await this.client.SendAsync(new EventData[] { blobMessage }, this.sendOptions, this.cancellationToken); this.stopwatch.Stop(); sentSuccessfully = index - 1; this.traceHelper.LogDebug("EventHubsSender {eventHubName}/{eventHubPartitionId} sent blob-batch of {numPackets} packets ({size} bytes) in {latencyMs:F2}ms, throughput={throughput:F2}MB/s", this.eventHubName, this.eventHubPartition, packetOffsets.Count + 1, this.stream.Position, this.stopwatch.Elapsed.TotalMilliseconds, this.stream.Position / (1024 * 1024 * this.stopwatch.Elapsed.TotalSeconds)); diff --git a/src/DurableTask.Netherite/TransportLayer/EventHubs/EventHubsTransport.cs b/src/DurableTask.Netherite/TransportLayer/EventHubs/EventHubsTransport.cs index 7a2a0b11..5366ca03 100644 --- a/src/DurableTask.Netherite/TransportLayer/EventHubs/EventHubsTransport.cs +++ b/src/DurableTask.Netherite/TransportLayer/EventHubs/EventHubsTransport.cs @@ -7,25 +7,22 @@ namespace DurableTask.Netherite.EventHubsTransport using System.Collections.Generic; using System.Threading; using System.Threading.Tasks; - using Microsoft.Azure.EventHubs; - using Microsoft.Azure.EventHubs.Processor; using Microsoft.Extensions.Logging; - using Microsoft.Azure.Storage; - using Microsoft.Azure.Storage.Blob; - using Newtonsoft.Json; using DurableTask.Netherite.Faster; using System.Linq; using System.Threading.Channels; using DurableTask.Netherite.Abstractions; using System.Diagnostics; using Azure.Storage.Blobs.Specialized; + using Azure.Messaging.EventHubs; + using Azure.Messaging.EventHubs.Primitives; + using Azure.Storage.Blobs; /// /// The EventHubs transport implementation. /// class EventHubsTransport : ITransportLayer, - IEventProcessorFactory, TransportAbstraction.ISender { readonly TransportAbstraction.IHost host; @@ -49,7 +46,7 @@ class EventHubsTransport : Task[] clientConnectionsEstablished; CancellationTokenSource shutdownSource; - CloudBlobContainer cloudBlobContainer; + BlobContainerClient cloudBlobContainer; IPartitionManager partitionManager; int shutdownTriggered; @@ -102,9 +99,8 @@ async Task ITransportLayer.StartAsync() (string containerName, string path) = this.storage.GetTaskhubPathPrefix(this.parameters); this.pathPrefix = path; - var cloudStorageAccount = await this.settings.BlobStorageConnection.GetAzureStorageV11AccountAsync(); - var cloudBlobClient = cloudStorageAccount.CreateCloudBlobClient(); - this.cloudBlobContainer = cloudBlobClient.GetContainerReference(containerName); + var blobServiceClient = this.settings.BlobStorageConnection.GetAzureStorageV12BlobServiceClient(new Azure.Storage.Blobs.BlobClientOptions()); + this.cloudBlobContainer = blobServiceClient.GetBlobContainerClient(containerName); // check that the storage format is supported, and load the relevant FASTER tuning parameters BlobManager.LoadAndCheckStorageFormat(this.parameters.StorageFormat, this.settings, this.host.TraceWarning); @@ -214,8 +210,11 @@ internal async Task ExitProcess(bool deletePartitionsFirst) await this.connections.DeletePartitions(); } - this.traceHelper.LogError("EventHubsTransport is killing process in 10 seconds"); - await Task.Delay(TimeSpan.FromSeconds(10)); + this.traceHelper.LogError("EventHubsTransport is killing process in 5 seconds"); + // release leases before exiting the process. This speeds up subsequent recovery. + Task _ = ((ITransportLayer)this).StopAsync(true); + + await Task.Delay(TimeSpan.FromSeconds(5)); System.Environment.Exit(222); } @@ -259,12 +258,6 @@ async Task StopClientsAndConnectionsAsync() this.traceHelper.LogDebug("EventHubsTransport stopped clients"); } - IEventProcessor IEventProcessorFactory.CreateEventProcessor(PartitionContext partitionContext) - { - var processor = new EventHubsProcessor(this.host, this, this.parameters, partitionContext, this.settings, this, this.traceHelper, this.shutdownSource.Token); - return processor; - } - void TransportAbstraction.ISender.Submit(Event evt) { if (!this.useRealEventHubsConnection) @@ -304,7 +297,7 @@ async Task ClientEstablishConnectionAsync(int index, PartitionReceiver receiver) this.traceHelper.LogDebug("Client.{clientId}.ch{index} establishing connection", this.shortClientId, index); // receive a dummy packet to establish connection // (the packet, if any, cannot be for this receiver because it is fresh) - await receiver.ReceiveAsync(1, TimeSpan.FromMilliseconds(1)); + await receiver.ReceiveBatchAsync(1, TimeSpan.FromMilliseconds(1), this.shutdownSource.Token); this.traceHelper.LogDebug("Client.{clientId}.ch{index} connection established", this.shortClientId, index); } catch (Exception exception) @@ -334,7 +327,7 @@ async Task ClientReceiveLoopAsync(int index, PartitionReceiver receiver, Channel { this.traceHelper.LogTrace("Client{clientId}.ch{index} waiting for new packets", this.shortClientId, index); - packets = await receiver.ReceiveAsync(1000, longPollingInterval); + packets = await receiver.ReceiveBatchAsync(1000, longPollingInterval, this.shutdownSource.Token); backoffDelay = TimeSpan.Zero; } diff --git a/src/DurableTask.Netherite/TransportLayer/EventHubs/EventProcessorHost.cs b/src/DurableTask.Netherite/TransportLayer/EventHubs/EventProcessorHost.cs new file mode 100644 index 00000000..ce75458e --- /dev/null +++ b/src/DurableTask.Netherite/TransportLayer/EventHubs/EventProcessorHost.cs @@ -0,0 +1,145 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT License. + +namespace DurableTask.Netherite.EventHubsTransport +{ + using System; + using System.Collections.Generic; + using System.Net.Sockets; + using System.Text; + using System.Threading; + using System.Threading.Tasks; + using Azure.Messaging.EventHubs; + using Azure.Messaging.EventHubs.Consumer; + using Azure.Messaging.EventHubs.Primitives; + using Azure.Messaging.EventHubs.Processor; + using Azure.Storage.Blobs; + using Microsoft.Extensions.Logging; + + class EventProcessorHost : EventProcessorClient + { + readonly IEventProcessor[] eventProcessors; + readonly IEventProcessorFactory factory; + readonly EventHubsTraceHelper traceHelper; + readonly CancellationToken shutdownToken; + const int maxPartitions = 32; + + public EventProcessorHost(ConstructorArguments args, string connectionString) + : base(args.BlobContainerClient, args.ConsumerGroup, connectionString, args.EventHubName, args.ClientOptions) + { + this.eventProcessors = new IEventProcessor[maxPartitions]; + this.factory = args.Factory; + this.traceHelper = args.TraceHelper; + this.PartitionInitializingAsync += this.OpenPartitionAsync; + this.ProcessEventAsync += this.ProcessPartitionEventAsync; + this.PartitionClosingAsync += this.ClosePartitionAsync; + this.ProcessErrorAsync += this.ErrorAsync; + } + + public EventProcessorHost(ConstructorArguments args, string fullyQualifiedNamespace, Azure.Core.TokenCredential tokenCredential) + : base(args.BlobContainerClient, args.ConsumerGroup, fullyQualifiedNamespace, args.EventHubName, tokenCredential, args.ClientOptions) + { + this.eventProcessors = new IEventProcessor[maxPartitions]; + this.factory = args.Factory; + this.traceHelper = args.TraceHelper; + this.PartitionInitializingAsync += this.OpenPartitionAsync; + this.PartitionClosingAsync += this.ClosePartitionAsync; + this.ProcessErrorAsync += this.ErrorAsync; + } + + public struct ConstructorArguments + { + public string EventHubName; + public EventHubsTraceHelper TraceHelper; + public string ConsumerGroup; + public EventProcessorClientOptions ClientOptions; + public BlobContainerClient BlobContainerClient; + public IEventProcessorFactory Factory; + }; + + async Task OpenPartitionAsync(PartitionInitializingEventArgs args) + { + if (this.shutdownToken.IsCancellationRequested) + { + return; + } + + int partitionId = int.Parse(args.PartitionId); + IEventProcessor processor = this.eventProcessors[partitionId]; + + if (processor == null) + { + this.eventProcessors[partitionId] = processor = this.factory.CreateEventProcessor(this, args.PartitionId); + EventPosition startPosition = await processor.OpenAsync(args.CancellationToken).ConfigureAwait(false); + args.DefaultStartingPosition = startPosition; + } + else + { + this.traceHelper.LogError("EventHubsProcessor {eventHubName}/{eventHubPartition} received open for already active partition", this.EventHubName, partitionId); + // no point in rethrowing - EH cannot deal with exceptions + } + } + + async Task ClosePartitionAsync(PartitionClosingEventArgs args) + { + int partitionId = int.Parse(args.PartitionId); + IEventProcessor processor = this.eventProcessors[partitionId]; + this.eventProcessors[partitionId] = null; + + if (processor != null) + { + await processor.CloseAsync(args.Reason, args.CancellationToken).ConfigureAwait(false); + } + else + { + this.traceHelper.LogError("EventHubsProcessor {eventHubName}/{eventHubPartition} received close for inactive partition", this.EventHubName, partitionId); + // no point in rethrowing - cannot deal with exceptions + } + } + + async Task ErrorAsync(ProcessErrorEventArgs args) + { + if (args.PartitionId != null) + { + int partitionId = int.Parse(args.PartitionId); + IEventProcessor processor = this.eventProcessors[partitionId]; + + if (processor != null) + { + await processor.ProcessErrorAsync(args.Exception, args.CancellationToken).ConfigureAwait(false); + } + else + { + this.traceHelper.LogWarning("EventHubsProcessor {eventHubName}/{eventHubPartition} received event hubs error indication: {exception}", this.EventHubName, args.PartitionId, args.Exception); + } + } + else + { + this.traceHelper.LogWarning("EventHubsProcessor {eventHubName} received event hubs error indication: {exception}", this.EventHubName, args.Exception); + } + } + + Task ProcessPartitionEventAsync(ProcessEventArgs args) + { + // this should never get called because we override OnProcessingEventBatchAsync, see below + this.traceHelper.LogError("EventHubsProcessor {eventHubName}/{eventHubPartition} received invalid call to overridden ProcessEventAsync", this.EventHubName, args.Partition.PartitionId); + return Task.CompletedTask; + } + + protected override async Task OnProcessingEventBatchAsync(IEnumerable events, EventProcessorPartition partition, CancellationToken cancellationToken) + { + int partitionId = int.Parse(partition.PartitionId); + IEventProcessor processor = this.eventProcessors[partitionId]; + + if (processor != null) + { + await processor.ProcessEventBatchAsync(events, cancellationToken).ConfigureAwait(false); + } + else + { + this.traceHelper.LogError("EventHubsProcessor {eventHubName}/{eventHubPartition} received events for inactive partition", this.EventHubName, partitionId); + // no point in rethrowing - EH cannot deal with exceptions + } + } + } +} diff --git a/src/DurableTask.Netherite/TransportLayer/EventHubs/IEventProcessor.cs b/src/DurableTask.Netherite/TransportLayer/EventHubs/IEventProcessor.cs new file mode 100644 index 00000000..07974dd4 --- /dev/null +++ b/src/DurableTask.Netherite/TransportLayer/EventHubs/IEventProcessor.cs @@ -0,0 +1,27 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT License. + +namespace DurableTask.Netherite.EventHubsTransport +{ + using System; + using System.Collections.Generic; + using System.Threading; + using System.Threading.Tasks; + using Azure.Messaging.EventHubs; + using Azure.Messaging.EventHubs.Consumer; + using Azure.Messaging.EventHubs.Processor; + + /// + /// Processes events for a specific partition. Gets opened and closed by the EventProcessorHost. + /// + interface IEventProcessor + { + Task OpenAsync(CancellationToken cancellationToken); + + Task ProcessEventBatchAsync(IEnumerable events, CancellationToken cancellationToken); + + Task CloseAsync(ProcessingStoppedReason reason, CancellationToken cancellationToken); + + Task ProcessErrorAsync(Exception error, CancellationToken cancellationToken); + } +} diff --git a/src/DurableTask.Netherite/TransportLayer/EventHubs/IEventProcessorFactory.cs b/src/DurableTask.Netherite/TransportLayer/EventHubs/IEventProcessorFactory.cs new file mode 100644 index 00000000..acb637f2 --- /dev/null +++ b/src/DurableTask.Netherite/TransportLayer/EventHubs/IEventProcessorFactory.cs @@ -0,0 +1,14 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT License. + +namespace DurableTask.Netherite.EventHubsTransport +{ + using System.Threading; + using System.Threading.Tasks; + using Azure.Messaging.EventHubs; + + interface IEventProcessorFactory + { + IEventProcessor CreateEventProcessor(EventProcessorClient eventProcessorClient, string partitionId); + } +} diff --git a/src/DurableTask.Netherite/TransportLayer/EventHubs/LoadMonitorProcessor.cs b/src/DurableTask.Netherite/TransportLayer/EventHubs/LoadMonitorProcessor.cs index d2aa0545..582095ec 100644 --- a/src/DurableTask.Netherite/TransportLayer/EventHubs/LoadMonitorProcessor.cs +++ b/src/DurableTask.Netherite/TransportLayer/EventHubs/LoadMonitorProcessor.cs @@ -11,11 +11,12 @@ namespace DurableTask.Netherite.EventHubsTransport using System.Linq; using System.Threading; using System.Threading.Tasks; + using Azure.Messaging.EventHubs; + using Azure.Messaging.EventHubs.Consumer; + using Azure.Messaging.EventHubs.Processor; using Azure.Storage.Blobs.Specialized; using DurableTask.Core.Common; using DurableTask.Netherite.Abstractions; - using Microsoft.Azure.EventHubs; - using Microsoft.Azure.EventHubs.Processor; using Microsoft.Extensions.Logging; class LoadMonitorProcessor : IEventProcessor @@ -24,14 +25,15 @@ class LoadMonitorProcessor : IEventProcessor readonly TransportAbstraction.ISender sender; readonly TaskhubParameters parameters; readonly EventHubsTraceHelper traceHelper; + readonly EventHubsTransport eventHubsTransport; readonly NetheriteOrchestrationServiceSettings settings; - readonly PartitionContext partitionContext; readonly string eventHubName; readonly string eventHubPartition; readonly byte[] taskHubGuid; readonly uint partitionId; readonly CancellationToken shutdownToken; readonly BlobBatchReceiver blobBatchReceiver; + readonly EventProcessorClient client; TransportAbstraction.ILoadMonitor loadMonitor; DateTime lastGarbageCheck = DateTime.MinValue; @@ -42,64 +44,73 @@ public LoadMonitorProcessor( TransportAbstraction.IHost host, TransportAbstraction.ISender sender, TaskhubParameters parameters, - PartitionContext partitionContext, + EventProcessorClient client, + string partitionId, NetheriteOrchestrationServiceSettings settings, + EventHubsTransport eventHubsTransport, EventHubsTraceHelper traceHelper, CancellationToken shutdownToken) { this.host = host; this.sender = sender; this.parameters = parameters; - this.partitionContext = partitionContext; this.settings = settings; - this.eventHubName = this.partitionContext.EventHubPath; - this.eventHubPartition = this.partitionContext.PartitionId; + this.client = client; + this.eventHubName = this.client.EventHubName; + this.eventHubPartition = partitionId; this.taskHubGuid = parameters.TaskhubGuid.ToByteArray(); this.partitionId = uint.Parse(this.eventHubPartition); this.traceHelper = new EventHubsTraceHelper(traceHelper, this.partitionId); + this.eventHubsTransport = eventHubsTransport; this.shutdownToken = shutdownToken; this.blobBatchReceiver = new BlobBatchReceiver("LoadMonitor", traceHelper, settings); } - Task IEventProcessor.OpenAsync(PartitionContext context) + Task IEventProcessor.OpenAsync(CancellationToken cancellationToken) { this.traceHelper.LogInformation("LoadMonitor is opening", this.eventHubName, this.eventHubPartition); this.loadMonitor = this.host.AddLoadMonitor(this.parameters.TaskhubGuid, this.sender); this.traceHelper.LogInformation("LoadMonitor opened", this.eventHubName, this.eventHubPartition); this.PeriodicGarbageCheck(); - return Task.CompletedTask; + return Task.FromResult(EventPosition.FromEnqueuedTime(DateTime.UtcNow - TimeSpan.FromSeconds(30))); } - async Task IEventProcessor.CloseAsync(PartitionContext context, CloseReason reason) + async Task IEventProcessor.CloseAsync(ProcessingStoppedReason reason, CancellationToken cancellationToken) { this.traceHelper.LogInformation("LoadMonitor is closing", this.eventHubName, this.eventHubPartition); await this.loadMonitor.StopAsync(); this.traceHelper.LogInformation("LoadMonitor closed", this.eventHubName, this.eventHubPartition); } - Task IEventProcessor.ProcessErrorAsync(PartitionContext context, Exception exception) + async Task IEventProcessor.ProcessErrorAsync(Exception exception, CancellationToken cancellationToken) { - LogLevel logLevel; - - switch (exception) + if (exception is OperationCanceledException && this.shutdownToken.IsCancellationRequested) { - case ReceiverDisconnectedException: - // occurs when partitions are being rebalanced by EventProcessorHost - logLevel = LogLevel.Information; - break; - - default: - logLevel = LogLevel.Warning; - break; + // normal to see some cancellations during shutdown } - this.traceHelper.Log(logLevel, "LoadMonitor received internal error indication from EventProcessorHost: {exception}", exception); - return Task.CompletedTask; + if (exception is Azure.Messaging.EventHubs.EventHubsException eventHubsException) + { + switch (eventHubsException.Reason) + { + case EventHubsException.FailureReason.ConsumerDisconnected: + this.traceHelper.LogInformation("LoadMonitor received ConsumerDisconnected notification"); + return; + + case EventHubsException.FailureReason.InvalidClientState: + // something is permantently broken inside EH client, let's try to recover via restart + this.traceHelper.LogError("LoadMonitor received InvalidClientState notification, initiating recovery via restart"); + await this.eventHubsTransport.ExitProcess(false); + return; + } + } + + this.traceHelper.LogWarning("LoadMonitor received internal error indication from EventProcessorHost: {exception}", exception); } - async Task IEventProcessor.ProcessEventsAsync(PartitionContext context, IEnumerable packets) + async Task IEventProcessor.ProcessEventBatchAsync(IEnumerable packets, CancellationToken cancellationToken) { - this.traceHelper.LogTrace("LoadMonitor receiving #{seqno}", packets.First().SystemProperties.SequenceNumber); + this.traceHelper.LogTrace("LoadMonitor receiving #{seqno}", packets.First().SequenceNumber); try { EventData last = null; @@ -128,7 +139,7 @@ async Task IEventProcessor.ProcessEventsAsync(PartitionContext context, IEnumera if (last != null) { - this.traceHelper.LogDebug("LoadMonitor received {totalEvents} events in {latencyMs:F2}ms, through #{seqno}", totalEvents, stopwatch.Elapsed.TotalMilliseconds, last.SystemProperties.SequenceNumber); + this.traceHelper.LogDebug("LoadMonitor received {totalEvents} events in {latencyMs:F2}ms, through #{seqno}", totalEvents, stopwatch.Elapsed.TotalMilliseconds, last.SequenceNumber); } else { diff --git a/src/DurableTask.Netherite/TransportLayer/EventHubs/LoadMonitorSender.cs b/src/DurableTask.Netherite/TransportLayer/EventHubs/LoadMonitorSender.cs index fe216cb4..786e02f3 100644 --- a/src/DurableTask.Netherite/TransportLayer/EventHubs/LoadMonitorSender.cs +++ b/src/DurableTask.Netherite/TransportLayer/EventHubs/LoadMonitorSender.cs @@ -3,8 +3,9 @@ namespace DurableTask.Netherite.EventHubsTransport { + using Azure.Messaging.EventHubs; + using Azure.Messaging.EventHubs.Producer; using DurableTask.Core.Common; - using Microsoft.Azure.EventHubs; using Microsoft.Extensions.Logging; using System; using System.Collections.Generic; @@ -15,26 +16,32 @@ namespace DurableTask.Netherite.EventHubsTransport class LoadMonitorSender : BatchWorker { - readonly PartitionSender sender; + readonly EventHubProducerClient client; readonly TransportAbstraction.IHost host; readonly byte[] taskHubGuid; readonly EventHubsTraceHelper traceHelper; readonly string eventHubName; readonly string eventHubPartition; readonly TimeSpan backoff = TimeSpan.FromSeconds(5); - const int maxFragmentSize = 500 * 1024; // account for very non-optimal serialization of event readonly MemoryStream stream = new MemoryStream(); // reused for all packets readonly Stopwatch stopwatch = new Stopwatch(); + readonly List events = new List(); - public LoadMonitorSender(TransportAbstraction.IHost host, byte[] taskHubGuid, PartitionSender sender, CancellationToken shutdownToken, EventHubsTraceHelper traceHelper) - : base($"EventHubsSender {sender.EventHubClient.EventHubName}/{sender.PartitionId}", false, 2000, shutdownToken, traceHelper) + public LoadMonitorSender(TransportAbstraction.IHost host, byte[] taskHubGuid, EventHubConnection connection, CancellationToken shutdownToken, EventHubsTraceHelper traceHelper) + : base($"EventHubsSender {connection.EventHubName}/0", false, 2000, shutdownToken, traceHelper) { this.host = host; this.taskHubGuid = taskHubGuid; - this.sender = sender; + this.client = new EventHubProducerClient(connection); this.traceHelper = traceHelper; - this.eventHubName = this.sender.EventHubClient.EventHubName; - this.eventHubPartition = this.sender.PartitionId; + this.eventHubName = connection.EventHubName; + this.eventHubPartition = "0"; + } + + public override async Task WaitForShutdownAsync() + { + await base.WaitForShutdownAsync(); + await this.client.CloseAsync(); } protected override async Task Process(IList toSend) @@ -54,8 +61,7 @@ protected override async Task Process(IList toSend) bool[] sentLoadInformationReceived = new bool[32]; bool[] sentPositionsReceived = new bool[32]; - this.stopwatch.Restart(); - int numEvents = 0; + this.stopwatch.Restart(); for (int i = toSend.Count - 1; i >= 0; i--) { @@ -85,20 +91,18 @@ protected override async Task Process(IList toSend) } } + int startPos = (int)this.stream.Position; Packet.Serialize(evt, this.stream, this.taskHubGuid); - int length = (int)(this.stream.Position); - var arraySegment = new ArraySegment(this.stream.GetBuffer(), 0, length); - var eventData = new EventData(arraySegment); - this.cancellationToken.ThrowIfCancellationRequested(); - await this.sender.SendAsync(eventData); - this.traceHelper.LogTrace("EventHubsSender {eventHubName}/{eventHubPartitionId} sent packet ({size} bytes) id={eventId}", this.eventHubName, this.eventHubPartition, length, evt.EventIdString); - this.stream.Seek(0, SeekOrigin.Begin); - numEvents++; + int length = (int)(this.stream.Position) - startPos; + var arraySegment = new ArraySegment(this.stream.GetBuffer(), startPos, length); + this.events.Add(new EventData(arraySegment)); + this.traceHelper.LogTrace("EventHubsSender {eventHubName}/{eventHubPartitionId} added packet ({size} bytes) id={eventId} to batch", this.eventHubName, this.eventHubPartition, length, evt.EventIdString); } + await this.client.SendAsync(this.events, this.cancellationToken); long elapsed = this.stopwatch.ElapsedMilliseconds; - this.traceHelper.LogDebug("EventHubsSender {eventHubName}/{eventHubPartitionId} sent info numEvents={numEvents} latencyMs={latencyMs}", this.eventHubName, this.eventHubPartition, numEvents, elapsed); - + this.traceHelper.LogDebug("EventHubsSender {eventHubName}/{eventHubPartitionId} sent info numEvents={numEvents} latencyMs={latencyMs}", this.eventHubName, this.eventHubPartition, this.events.Count, elapsed); + // rate limit this sender by making each iteration last at least 100ms duration long toSpare = 100 - elapsed; if (toSpare > 10) @@ -121,6 +125,7 @@ protected override async Task Process(IList toSend) { // we don't need the contents of the stream anymore. this.stream.SetLength(0); + this.events.Clear(); } } } diff --git a/src/DurableTask.Netherite/TransportLayer/EventHubs/EventHubsProcessor.cs b/src/DurableTask.Netherite/TransportLayer/EventHubs/PartitionProcessor.cs similarity index 55% rename from src/DurableTask.Netherite/TransportLayer/EventHubs/EventHubsProcessor.cs rename to src/DurableTask.Netherite/TransportLayer/EventHubs/PartitionProcessor.cs index 25060eb8..cbfdb30b 100644 --- a/src/DurableTask.Netherite/TransportLayer/EventHubs/EventHubsProcessor.cs +++ b/src/DurableTask.Netherite/TransportLayer/EventHubs/PartitionProcessor.cs @@ -11,14 +11,18 @@ namespace DurableTask.Netherite.EventHubsTransport using System.Linq; using System.Threading; using System.Threading.Tasks; + using Azure.Messaging.EventHubs; + using EventPosition = Azure.Messaging.EventHubs.Consumer.EventPosition; using Azure.Storage.Blobs.Specialized; using DurableTask.Core.Common; using DurableTask.Netherite.Abstractions; - using Microsoft.Azure.EventHubs; - using Microsoft.Azure.EventHubs.Processor; using Microsoft.Extensions.Logging; + using Azure.Messaging.EventHubs.Processor; - class EventHubsProcessor : IEventProcessor, TransportAbstraction.IDurabilityListener + /// + /// Delivers events targeting a specific Netherite partition. + /// + class PartitionProcessor : IEventProcessor, TransportAbstraction.IDurabilityListener { readonly TransportAbstraction.IHost host; readonly TransportAbstraction.ISender sender; @@ -26,24 +30,21 @@ class EventHubsProcessor : IEventProcessor, TransportAbstraction.IDurabilityList readonly EventHubsTraceHelper traceHelper; readonly NetheriteOrchestrationServiceSettings settings; readonly EventHubsTransport eventHubsTransport; - readonly PartitionContext partitionContext; readonly string eventHubName; readonly string eventHubPartition; readonly byte[] taskHubGuid; readonly uint partitionId; - readonly CancellationToken shutdownToken; readonly BlobBatchReceiver blobBatchReceiver; + readonly EventProcessorClient client; + readonly TaskCompletionSource firstPacketToReceive; + readonly CancellationTokenSource shutdownSource; + readonly CancellationTokenRegistration? shutdownRegistration; - //private uint partitionId; - CancellationTokenSource eventProcessorShutdown; // we set this task once shutdown has been initiated Task shutdownTask = null; - // we occasionally checkpoint received packets with eventhubs. It is not required for correctness - // as we filter duplicates anyway, but it will help startup time. - long persistedSequenceNumber; - long persistedOffset; - long? lastCheckpointedOffset; + // we continuously validate the sequence numbers that are being delivered + long nextToReceive; // since EventProcessorHost does not redeliver packets, we need to keep them around until we are sure // they are processed durably, so we can redeliver them when recycling/recovering a partition @@ -51,11 +52,12 @@ class EventHubsProcessor : IEventProcessor, TransportAbstraction.IDurabilityList readonly ConcurrentQueue pendingDelivery; AsyncLock deliveryLock; - record struct EventEntry(long SeqNo, long BatchPos, PartitionEvent Event, bool LastInBatch, long Offset, BlockBlobClient BatchBlob) + + record struct EventEntry(long SeqNo, long BatchPos, PartitionEvent Event, bool LastInBatch, BlockBlobClient BatchBlob) { - public EventEntry(long seqNo, long batchPos, PartitionEvent evt) : this(seqNo, batchPos, evt, false, 0, null) { } + public EventEntry(long seqNo, long batchPos, PartitionEvent evt) : this(seqNo, batchPos, evt, false, null) { } - public EventEntry(long seqNo, long batchPos, PartitionEvent evt, long offset, BlockBlobClient blob) : this(seqNo, batchPos, evt, true, 0, blob) { } + public EventEntry(long seqNo, long batchPos, PartitionEvent evt, BlockBlobClient blob) : this(seqNo, batchPos, evt, true, blob) { } } // this points to the latest incarnation of this partition; it gets @@ -78,11 +80,12 @@ class PartitionIncarnation readonly Dictionary reassembly = new Dictionary(); - public EventHubsProcessor( + public PartitionProcessor( TransportAbstraction.IHost host, TransportAbstraction.ISender sender, TaskhubParameters parameters, - PartitionContext partitionContext, + EventProcessorClient client, + string partitionId, NetheriteOrchestrationServiceSettings settings, EventHubsTransport eventHubsTransport, EventHubsTraceHelper traceHelper, @@ -92,54 +95,71 @@ public EventHubsProcessor( this.sender = sender; this.parameters = parameters; this.pendingDelivery = new(); - this.partitionContext = partitionContext; this.settings = settings; this.eventHubsTransport = eventHubsTransport; - this.eventHubName = this.partitionContext.EventHubPath; - this.eventHubPartition = this.partitionContext.PartitionId; + this.client = client; + this.eventHubName = client.EventHubName; + this.eventHubPartition = partitionId; this.taskHubGuid = parameters.TaskhubGuid.ToByteArray(); this.partitionId = uint.Parse(this.eventHubPartition); this.traceHelper = new EventHubsTraceHelper(traceHelper, this.partitionId); - this.shutdownToken = shutdownToken; + this.shutdownSource = new CancellationTokenSource(); string traceContext = $"EventHubsProcessor {this.eventHubName}/{this.eventHubPartition}"; this.blobBatchReceiver = new BlobBatchReceiver(traceContext, this.traceHelper, this.settings); + this.firstPacketToReceive = new TaskCompletionSource(); - var _ = shutdownToken.Register( - () => { var _ = Task.Run(() => this.IdempotentShutdown("shutdownToken", eventHubsTransport.FatalExceptionObserved)); }, - useSynchronizationContext: false); + this.shutdownRegistration = shutdownToken.Register(() => this.IdempotentShutdown("EventHubsProcessor.shutdownToken", eventHubsTransport.FatalExceptionObserved)); } - Task IEventProcessor.OpenAsync(PartitionContext context) + async Task IEventProcessor.OpenAsync(CancellationToken cancellationToken) { - this.traceHelper.LogInformation("EventHubsProcessor {eventHubName}/{eventHubPartition} is opening", this.eventHubName, this.eventHubPartition); - this.eventProcessorShutdown = new CancellationTokenSource(); - this.deliveryLock = new AsyncLock(); + this.traceHelper.LogInformation("EventHubsProcessor {eventHubName}/{eventHubPartition} OpenAsync called", this.eventHubName, this.eventHubPartition); + this.deliveryLock = new AsyncLock(this.shutdownSource.Token); + + try + { + // we kick off the start-and-retry mechanism for the partition incarnations + this.currentIncarnation = Task.Run(() => this.StartPartitionAsync()); + + // if the cancellation token fires, shut everything down + // this must be registered AFTER the current incarnation is assigned + // since it may fire immediately if already canceled at this point + using var _ = cancellationToken.Register(() => this.IdempotentShutdown("PartitionProcessor.OpenAsync cancellationToken", true)); + + // then we wait for a successful partition creation or recovery so we can tell where to resume packet processing + var firstPacketToReceive = await this.firstPacketToReceive.Task; - // make sure we shut down as soon as the partition is closing - var _ = context.CancellationToken.Register( - () => { var _ = Task.Run(() => this.IdempotentShutdown("context.CancellationToken", true)); }, - useSynchronizationContext: false); + this.nextToReceive = firstPacketToReceive; - // we kick off the start-and-retry mechanism for the partition, but don't wait for it to be fully started. - // instead, we save the task and wait for it when we need it - this.currentIncarnation = Task.Run(() => this.StartPartitionAsync()); + this.traceHelper.LogInformation("EventHubsProcessor {eventHubName}/{eventHubPartition} OpenAsync returned, first packet to receive is #{seqno}", this.eventHubName, this.eventHubPartition, firstPacketToReceive); - this.traceHelper.LogInformation("EventHubsProcessor {eventHubName}/{eventHubPartition} opened", this.eventHubName, this.eventHubPartition); - return Task.CompletedTask; + return EventPosition.FromSequenceNumber(firstPacketToReceive - 1, isInclusive: false); + } + catch (OperationCanceledException) when (this.shutdownSource.IsCancellationRequested) + { + this.traceHelper.LogInformation("EventHubsProcessor {eventHubName}/{eventHubPartition} OpenAsync canceled by shutdown", this.eventHubName, this.eventHubPartition); + + return EventPosition.Latest; // does not matter since we are already shut down + } } public void ConfirmDurable(Event evt) - { + { + if (this.shutdownSource.IsCancellationRequested) + { + return; + } + List obsoleteBatches = null; if (this.traceHelper.IsEnabled(LogLevel.Trace)) { - this.traceHelper.LogTrace("EventHubsProcessor {eventHubName}/{eventHubPartition} confirmed event nextInputQueuePosition={nextInputQueuePosition}", this.eventHubName, this.eventHubPartition, ((PartitionEvent)evt).NextInputQueuePositionTuple); + this.traceHelper.LogTrace("EventHubsProcessor {eventHubName}/{eventHubPartition} ConfirmDurable nextInputQueuePosition={nextInputQueuePosition}", this.eventHubName, this.eventHubPartition, ((PartitionEvent)evt).NextInputQueuePositionTuple); } // this is called after an event has committed (i.e. has been durably persisted in the recovery log). - // so we know we will never need to deliver it again. We remove it from the local buffer, update the fields that - // track the last persisted position, and delete the blob batch if this was the last event in the batch. + // so we know we will never need to deliver it again. We remove it from the local buffer + // and delete the blob batch if this was the last event in the batch. while (this.pendingDelivery.TryPeek(out var front) && (front.Event == evt || front.Event.NextInputQueuePositionTuple.CompareTo(((PartitionEvent) evt).NextInputQueuePositionTuple) < 0)) { @@ -147,15 +167,11 @@ public void ConfirmDurable(Event evt) { if (this.traceHelper.IsEnabled(LogLevel.Trace)) { - this.traceHelper.LogTrace("EventHubsProcessor {eventHubName}/{eventHubPartition} discarding buffered event nextInputQueuePosition={nextInputQueuePosition} lastInBatch={lastInBatch} seqno={seqNo} offset={offset} batchBlob={batchBlob}", this.eventHubName, this.eventHubPartition, front.Event.NextInputQueuePositionTuple, confirmed.LastInBatch, confirmed.SeqNo, confirmed.Offset, confirmed.BatchBlob); + this.traceHelper.LogTrace("EventHubsProcessor {eventHubName}/{eventHubPartition} discarding buffered event nextInputQueuePosition={nextInputQueuePosition} lastInBatch={lastInBatch} seqno={seqNo} batchBlob={batchBlob}", this.eventHubName, this.eventHubPartition, front.Event.NextInputQueuePositionTuple, confirmed.LastInBatch, confirmed.SeqNo, confirmed.BatchBlob?.Name); } if (confirmed.LastInBatch) { - // to create EH checkpoints we need to record both the EH sequence number and the EH offset (see SaveEventHubsReceiverCheckpoint) - this.persistedOffset = Math.Max(this.persistedOffset, confirmed.Offset); - this.persistedSequenceNumber = Math.Max(this.persistedSequenceNumber, confirmed.SeqNo); - if (confirmed.BatchBlob != null) { (obsoleteBatches ??= new()).Add(confirmed.BatchBlob); @@ -192,7 +208,7 @@ async Task StartPartitionAsync(PartitionIncarnation prior { } - if (!this.eventProcessorShutdown.IsCancellationRequested) + if (!this.shutdownSource.IsCancellationRequested) { // we are now becoming the current incarnation this.currentIncarnation = prior.Next; @@ -227,7 +243,7 @@ async Task StartPartitionAsync(PartitionIncarnation prior } // check that we are not already shutting down before even starting this - if (this.eventProcessorShutdown.IsCancellationRequested) + if (this.shutdownSource.IsCancellationRequested) { this.traceHelper.LogDebug("EventHubsProcessor {eventHubName}/{eventHubPartition} is cancelling startup of incarnation {incarnation}", this.eventHubName, this.eventHubPartition, c.Incarnation); return null; @@ -241,7 +257,7 @@ async Task StartPartitionAsync(PartitionIncarnation prior this.traceHelper.LogDebug("EventHubsProcessor {eventHubName}/{eventHubPartition} is starting partition (incarnation {incarnation})", this.eventHubName, this.eventHubPartition, c.Incarnation); // to handle shutdown before startup completes, register a force-termination - using var registration = this.eventProcessorShutdown.Token.Register( + using var registration = this.shutdownSource.Token.Register( () => c.ErrorHandler.HandleError( nameof(StartPartitionAsync), "EventHubsProcessor shut down before partition fully started", @@ -255,30 +271,36 @@ async Task StartPartitionAsync(PartitionIncarnation prior this.traceHelper.LogInformation("EventHubsProcessor {eventHubName}/{eventHubPartition} started partition (incarnation {incarnation}), next expected packet is #{nextSeqno}", this.eventHubName, this.eventHubPartition, c.Incarnation, c.NextPacketToReceive); - // receive packets already sitting in the buffer; use lock to prevent race with new packets being delivered - using (await this.deliveryLock.LockAsync()) + if (!this.firstPacketToReceive.TrySetResult(c.NextPacketToReceive.seqNo)) { - this.traceHelper.LogDebug("EventHubsProcessor {eventHubName}/{eventHubPartition} checking for packets requiring redelivery (incarnation {incarnation})", this.eventHubName, this.eventHubPartition, c.Incarnation); - var batch = this.pendingDelivery - .Select(x => x.Event) - .Where(evt => (evt.NextInputQueuePosition, evt.NextInputQueueBatchPosition).CompareTo(c.NextPacketToReceive) > 0) - .ToList(); - if (batch.Count > 0) - { - var lastInBatch = batch[batch.Count - 1]; - c.NextPacketToReceive = (lastInBatch.NextInputQueuePosition, lastInBatch.NextInputQueueBatchPosition); - c.Partition.SubmitEvents(batch); - this.traceHelper.LogDebug("EventHubsProcessor {eventHubName}/{eventHubPartition} redelivered {batchsize} packets, starting with #{seqno}, next expected packet is #{nextSeqno} (incarnation {incarnation})", this.eventHubName, this.eventHubPartition, batch.Count, batch[0].NextInputQueuePosition - 1, c.NextPacketToReceive, c.Incarnation); - } - else + this.shutdownSource.Token.ThrowIfCancellationRequested(); + + // this is not the first incarnation so there may be some packets that were received earlier and are sitting in the buffer; + // use lock to prevent race with new packets being delivered + using (await this.deliveryLock.LockAsync()) { - this.traceHelper.LogDebug("EventHubsProcessor {eventHubName}/{eventHubPartition} has no packets requiring redelivery ", this.eventHubName, this.eventHubPartition); + this.traceHelper.LogDebug("EventHubsProcessor {eventHubName}/{eventHubPartition} checking for packets requiring redelivery (incarnation {incarnation})", this.eventHubName, this.eventHubPartition, c.Incarnation); + var batch = this.pendingDelivery + .Select(x => x.Event) + .Where(evt => (evt.NextInputQueuePosition, evt.NextInputQueueBatchPosition).CompareTo(c.NextPacketToReceive) > 0) + .ToList(); + if (batch.Count > 0) + { + var lastInBatch = batch[batch.Count - 1]; + c.NextPacketToReceive = (lastInBatch.NextInputQueuePosition, lastInBatch.NextInputQueueBatchPosition); + c.Partition.SubmitEvents(batch); + this.traceHelper.LogDebug("EventHubsProcessor {eventHubName}/{eventHubPartition} redelivered {batchsize} packets, starting with #{seqno}, next expected packet is #{nextSeqno} (incarnation {incarnation})", this.eventHubName, this.eventHubPartition, batch.Count, batch[0].NextInputQueuePosition - 1, c.NextPacketToReceive, c.Incarnation); + } + else + { + this.traceHelper.LogDebug("EventHubsProcessor {eventHubName}/{eventHubPartition} has no packets requiring redelivery ", this.eventHubName, this.eventHubPartition); + } } - } + } this.traceHelper.LogDebug("EventHubsProcessor {eventHubName}/{eventHubPartition} completed partition startup (incarnation {incarnation})", this.eventHubName, this.eventHubPartition, c.Incarnation); } - catch (OperationCanceledException) when (c.ErrorHandler.IsTerminated) + catch (OperationCanceledException) when (c.ErrorHandler.IsTerminated || this.shutdownSource.Token.IsCancellationRequested) { // the partition startup was canceled this.traceHelper.LogDebug("EventHubsProcessor {eventHubName}/{eventHubPartition} canceled partition startup (incarnation {incarnation})", this.eventHubName, this.eventHubPartition, c.Incarnation); @@ -293,142 +315,138 @@ async Task StartPartitionAsync(PartitionIncarnation prior return c; } - async Task IdempotentShutdown(string reason, bool quickly) + void IdempotentShutdown(string reason, bool quickly) { + var waitForConfirmation = new TaskCompletionSource(); + var task = ShutdownAsync(); + var originalValue = Interlocked.CompareExchange(ref this.shutdownTask, task, null); + bool isFirst = (originalValue == null); + + this.traceHelper.LogInformation("EventHubsProcessor {eventHubName}/{eventHubPartition} triggered shutdown (reason: {reason}, quickly: {quickly}, isFirst:{isFirst})", this.eventHubName, this.eventHubPartition, reason, quickly, isFirst); + + waitForConfirmation.SetResult(isFirst); // only the first shutdown should proceed + async Task ShutdownAsync() { - this.traceHelper.LogInformation("EventHubsProcessor {eventHubName}/{eventHubPartition} is shutting down (reason: {reason}, quickly: {quickly})", this.eventHubName, this.eventHubPartition, reason, quickly); + if (await waitForConfirmation.Task) // only the first shutdown should proceed + { + try + { + this.traceHelper.LogInformation("EventHubsProcessor {eventHubName}/{eventHubPartition} is shutting down (reason: {reason}, quickly: {quickly})", this.eventHubName, this.eventHubPartition, reason, quickly); - this.eventProcessorShutdown.Cancel(); // stops reincarnations + this.shutdownSource.Cancel(); // stops all reincarnations, among other things - PartitionIncarnation current = await this.currentIncarnation; + this.firstPacketToReceive.TrySetCanceled(); // cancel partition opening if we are still waiting for it - while (current != null && current.ErrorHandler.IsTerminated) - { - current = await current.Next; - } + PartitionIncarnation current = await this.currentIncarnation; - if (current == null) - { - this.traceHelper.LogDebug("EventHubsProcessor {eventHubName}/{eventHubPartition} already canceled or terminated", this.eventHubName, this.eventHubPartition); - } - else - { - this.traceHelper.LogDebug("EventHubsProcessor {eventHubName}/{eventHubPartition} stopping partition (incarnation: {incarnation}, quickly: {quickly})", this.eventHubName, this.eventHubPartition, current.Incarnation, quickly); - await current.Partition.StopAsync(quickly); - this.traceHelper.LogDebug("EventHubsProcessor {eventHubName}/{eventHubPartition} stopped partition (incarnation {incarnation})", this.eventHubName, this.eventHubPartition, current.Incarnation); - } + while (current != null && current.ErrorHandler.IsTerminated) + { + current = await current.Next; + } - this.traceHelper.LogInformation("EventHubsProcessor {eventHubName}/{eventHubPartition} is shut down", this.eventHubName, this.eventHubPartition); - } + if (current == null) + { + this.traceHelper.LogDebug("EventHubsProcessor {eventHubName}/{eventHubPartition} already canceled or terminated", this.eventHubName, this.eventHubPartition); + } + else + { + this.traceHelper.LogDebug("EventHubsProcessor {eventHubName}/{eventHubPartition} stopping partition (incarnation: {incarnation}, quickly: {quickly})", this.eventHubName, this.eventHubPartition, current.Incarnation, quickly); + await current.Partition.StopAsync(quickly); + this.traceHelper.LogDebug("EventHubsProcessor {eventHubName}/{eventHubPartition} stopped partition (incarnation {incarnation})", this.eventHubName, this.eventHubPartition, current.Incarnation); + } - using (await this.deliveryLock.LockAsync()) - { - if (this.shutdownTask == null) - { - this.traceHelper.LogDebug("EventHubsProcessor {eventHubName}/{eventHubPartition} starting shutdown task", this.eventHubName, this.eventHubPartition); - this.shutdownTask = Task.Run(() => ShutdownAsync()); - } - else - { - this.traceHelper.LogDebug("EventHubsProcessor {eventHubName}/{eventHubPartition} shutdown task already started", this.eventHubName, this.eventHubPartition); + this.traceHelper.LogInformation("EventHubsProcessor {eventHubName}/{eventHubPartition} successfully shut down", this.eventHubName, this.eventHubPartition); + } + catch (Exception exception) + { + this.traceHelper.LogInformation("EventHubsProcessor {eventHubName}/{eventHubPartition} failed to shut down: {exception})", this.eventHubName, this.eventHubPartition, exception); + } + finally + { + this.deliveryLock.Dispose(); + } } } - - await this.shutdownTask; } - async Task IEventProcessor.CloseAsync(PartitionContext context, CloseReason reason) + async Task IEventProcessor.CloseAsync(ProcessingStoppedReason reason, CancellationToken cancellationToken) { - this.traceHelper.LogInformation("EventHubsProcessor {eventHubName}/{eventHubPartition} is closing (reason: {reason})", this.eventHubName, this.eventHubPartition, reason); - - if (reason != CloseReason.LeaseLost) - { - await this.SaveEventHubsReceiverCheckpoint(context, 0); - } + this.traceHelper.LogInformation("EventHubsProcessor {eventHubName}/{eventHubPartition} CloseAsync called (reason: {reason})", this.eventHubName, this.eventHubPartition, reason); - await this.IdempotentShutdown("CloseAsync", reason == CloseReason.LeaseLost); + this.IdempotentShutdown("CloseAsync", reason == ProcessingStoppedReason.OwnershipLost); - this.deliveryLock.Dispose(); + await this.shutdownTask; - this.traceHelper.LogInformation("EventHubsProcessor {eventHubName}/{eventHubPartition} closed", this.eventHubName, this.eventHubPartition); - } + this.traceHelper.LogInformation("EventHubsProcessor {eventHubName}/{eventHubPartition} CloseAsync returned", this.eventHubName, this.eventHubPartition); + } - async ValueTask SaveEventHubsReceiverCheckpoint(PartitionContext context, long byteThreshold) + async Task IEventProcessor.ProcessErrorAsync(Exception exception, CancellationToken cancellationToken) { - if (this.lastCheckpointedOffset.HasValue - && this.persistedOffset - this.lastCheckpointedOffset.Value > byteThreshold - && !context.CancellationToken.IsCancellationRequested) + if (exception is OperationCanceledException && this.shutdownSource.IsCancellationRequested) { - var checkpoint = new Checkpoint(this.partitionId.ToString(), this.persistedOffset.ToString(), this.persistedSequenceNumber); + // normal to see some cancellations during shutdown + } - this.traceHelper.LogInformation("EventHubsProcessor {eventHubName}/{eventHubPartition} is checkpointing receive position through #{seqno}", this.eventHubName, this.eventHubPartition, checkpoint.SequenceNumber); - try - { - await context.CheckpointAsync(checkpoint); - this.lastCheckpointedOffset = long.Parse(checkpoint.Offset); - } - catch (Exception e) + if (exception is Azure.Messaging.EventHubs.EventHubsException eventHubsException) + { + switch (eventHubsException.Reason) { - // updating EventHubs checkpoints has been known to fail occasionally due to leases shifting around; since it is optional anyway - // we don't want this exception to cause havoc - this.traceHelper.LogWarning("EventHubsProcessor {eventHubName}/{eventHubPartition} failed to checkpoint receive position: {e}", this.eventHubName, this.eventHubPartition, e); - - if (Utils.IsFatal(e)) - { - this.host.OnFatalExceptionObserved(e); - } + case EventHubsException.FailureReason.ResourceNotFound: + // occurs when partition hubs was deleted either accidentally, or intentionally after messages were lost due to the retention limit + this.traceHelper.LogError("EventHubsProcessor {eventHubName}/{eventHubPartition} EventHub was deleted, initiating recovery via restart", this.eventHubName, this.eventHubPartition); + await this.eventHubsTransport.ExitProcess(false); + return; + + case EventHubsException.FailureReason.ConsumerDisconnected: + // since this processor is no longer going to receive events, let's shut it down + this.traceHelper.LogError("EventHubsProcessor {eventHubName}/{eventHubPartition} received ConsumerDisconnected notification", this.eventHubName, this.eventHubPartition); + this.IdempotentShutdown("Receiver was disconnected", true); + return; + + case EventHubsException.FailureReason.InvalidClientState: + // something is permantently broken inside EH client, let's try to recover via restart + this.traceHelper.LogError("EventHubsProcessor {eventHubName}/{eventHubPartition} received InvalidClientState notification, initiating recovery via restart", this.eventHubName, this.eventHubPartition); + await this.eventHubsTransport.ExitProcess(false); + return; } } + + this.traceHelper.LogWarning("EventHubsProcessor {eventHubName}/{eventHubPartition} received internal error indication from EventProcessorHost: {exception}", this.eventHubName, this.eventHubPartition, exception); } - async Task IEventProcessor.ProcessErrorAsync(PartitionContext context, Exception exception) + async Task IEventProcessor.ProcessEventBatchAsync(IEnumerable packets, CancellationToken cancellationToken) { - - LogLevel logLevel; - - switch (exception) + if (this.shutdownSource.IsCancellationRequested) { - case ReceiverDisconnectedException: - - // occurs when partitions are being rebalanced by EventProcessorHost - logLevel = LogLevel.Information; - - // since this processor is no longer going to receive events, let's shut it down - // one would expect that this is redundant with EventProcessHost calling close - // but empirically we have observed that the latter does not always happen in this situation - var _ = Task.Run(() => this.IdempotentShutdown("Receiver was disconnected", true)); - break; - - case Microsoft.Azure.EventHubs.MessagingEntityNotFoundException: - - // occurs when partition hubs was deleted either accidentally, or intentionally after messages were lost due to the retention limit - logLevel = LogLevel.Warning; - this.traceHelper.LogError("EventHubsProcessor {eventHubName}/{eventHubPartition} EventHub was deleted, initiating recovery via restart", this.eventHubName, this.eventHubPartition); - await this.eventHubsTransport.ExitProcess(false); - break; - - default: - logLevel = LogLevel.Warning; - break; + this.traceHelper.LogDebug("EventHubsProcessor {eventHubName}/{eventHubPartition} ProcessEventBatchAsync canceled (already shut down)", this.eventHubName, this.eventHubPartition); + return; } - this.traceHelper.Log(logLevel, "EventHubsProcessor {eventHubName}/{eventHubPartition} received internal error indication from EventProcessorHost: {exception}", this.eventHubName, this.eventHubPartition, exception); - } + // if the cancellation token fires, shut everything down + using var _ = cancellationToken.Register(() => this.IdempotentShutdown("ProcessEventBatchAsync cancellationToken", true)); - async Task IEventProcessor.ProcessEventsAsync(PartitionContext context, IEnumerable packets) - { - var first = packets.FirstOrDefault(); - long firstSequenceNumber = first?.SystemProperties.SequenceNumber ?? 0; - - this.traceHelper.LogDebug("EventHubsProcessor {eventHubName}/{eventHubPartition} is receiving events starting with #{seqno}", this.eventHubName, this.eventHubPartition, firstSequenceNumber); + EventData first = packets.FirstOrDefault(); - if (this.shutdownToken.IsCancellationRequested) + if (first == null) { - this.traceHelper.LogDebug("EventHubsProcessor {eventHubName}/{eventHubPartition} already shut down", this.eventHubName, this.eventHubPartition); + this.traceHelper.LogDebug("EventHubsProcessor {eventHubName}/{eventHubPartition} received empty batch", this.eventHubName, this.eventHubPartition); return; } + this.traceHelper.LogDebug("EventHubsProcessor {eventHubName}/{eventHubPartition} is receiving events starting with #{seqno}", this.eventHubName, this.eventHubPartition, first.SequenceNumber); + + if (first.SequenceNumber > this.nextToReceive) + { + this.traceHelper.LogError("EventHubsProcessor {eventHubName}/{eventHubPartition} missing packets in sequence, #{seqno} instead of #{expected}. Initiating recovery via delete and restart.", this.eventHubName, this.eventHubPartition, first.SequenceNumber, this.nextToReceive); + await this.eventHubsTransport.ExitProcess(true); + return; + } + else if (first.SequenceNumber < this.nextToReceive) + { + this.traceHelper.LogError("EventHubsProcessor {eventHubName}/{eventHubPartition} duplicate packet in sequence, #{seqno} instead of #{expected}.", this.eventHubName, this.eventHubPartition, first.SequenceNumber, this.nextToReceive); + } + PartitionIncarnation current = await this.currentIncarnation; while (current != null && current.ErrorHandler.IsTerminated) @@ -439,32 +457,20 @@ async Task IEventProcessor.ProcessEventsAsync(PartitionContext context, IEnumera if (current == null) { this.traceHelper.LogWarning("EventHubsProcessor {eventHubName}/{eventHubPartition} received packets for closed processor, discarded", this.eventHubName, this.eventHubPartition); - return; } else { this.traceHelper.LogTrace("EventHubsProcessor {eventHubName}/{eventHubPartition} is delivering to incarnation {seqno}", this.eventHubName, this.eventHubPartition, current.Incarnation); } - if (!this.lastCheckpointedOffset.HasValue) - { - // the first packet we receive indicates what our last checkpoint was - this.lastCheckpointedOffset = first == null ? null : long.Parse(first.SystemProperties.Offset); - - // we may be missing packets if the service was down for longer than EH retention - if (firstSequenceNumber > current.NextPacketToReceive.seqNo) - { - this.traceHelper.LogError("EventHubsProcessor {eventHubName}/{eventHubPartition} missing packets in sequence, #{seqno} instead of #{expected}. Initiating recovery via delete and restart.", this.eventHubName, this.eventHubPartition, firstSequenceNumber, current.NextPacketToReceive); - await this.eventHubsTransport.ExitProcess(true); - } - } - try { var receivedTimestamp = current.Partition.CurrentTimeMs; int totalEvents = 0; Stopwatch stopwatch = Stopwatch.StartNew(); + this.shutdownSource.Token.ThrowIfCancellationRequested(); + using (await this.deliveryLock.LockAsync()) // must prevent rare race with a partition that is currently restarting. Contention is very unlikely. { this.traceHelper.LogDebug("EventHubsProcessor {eventHubName}/{eventHubPartition}({incarnation}) is processing packets", this.eventHubName, this.eventHubPartition, current.Incarnation); @@ -473,7 +479,7 @@ async Task IEventProcessor.ProcessEventsAsync(PartitionContext context, IEnumera // iterators do not support ref arguments, so we define a Position object with mutable fields to work around this limitation Position nextPacketToReceive = new Position() { SeqNo = current.NextPacketToReceive.seqNo, BatchPos = current.NextPacketToReceive.batchPos }; - await foreach ((EventData eventData, PartitionEvent[] events, long seqNo, BlockBlobClient blob) in this.blobBatchReceiver.ReceiveEventsAsync(this.taskHubGuid, packets, this.shutdownToken, current.ErrorHandler, nextPacketToReceive)) + await foreach ((EventData eventData, PartitionEvent[] events, long seqNo, BlockBlobClient blob) in this.blobBatchReceiver.ReceiveEventsAsync(this.taskHubGuid, packets, this.shutdownSource.Token, current.ErrorHandler, nextPacketToReceive)) { int numSkipped = 0; @@ -486,7 +492,7 @@ async Task IEventProcessor.ProcessEventsAsync(PartitionContext context, IEnumera numSkipped++; continue; // was skipped over by the batch receiver because it is already processed } - + if (i < events.Length - 1) // this is not the last event in the batch { // the next input queue position is the next position within the same batch @@ -499,7 +505,7 @@ async Task IEventProcessor.ProcessEventsAsync(PartitionContext context, IEnumera // the next input queue position is the first entry of of the next batch evt.NextInputQueuePosition = seqNo + 1; evt.NextInputQueueBatchPosition = 0; - this.pendingDelivery.Enqueue(new EventEntry(seqNo, i, evt, long.Parse(eventData.SystemProperties.Offset), blob)); + this.pendingDelivery.Enqueue(new EventEntry(seqNo, i, evt, blob)); } if (this.traceHelper.IsEnabled(LogLevel.Trace)) @@ -516,23 +522,24 @@ async Task IEventProcessor.ProcessEventsAsync(PartitionContext context, IEnumera } current.Partition.SubmitEvents(numSkipped == 0 ? events : events.Skip(numSkipped).ToList()); + + this.nextToReceive = eventData.SequenceNumber + 1; } current.NextPacketToReceive = (nextPacketToReceive.SeqNo, nextPacketToReceive.BatchPos); } - this.traceHelper.LogDebug("EventHubsProcessor {eventHubName}/{eventHubPartition}({incarnation}) received {totalEvents} events in {latencyMs:F2}ms, starting with #{seqno}, next expected packet is #{nextSeqno}", this.eventHubName, this.eventHubPartition, current.Incarnation, totalEvents, stopwatch.Elapsed.TotalMilliseconds, firstSequenceNumber, current.NextPacketToReceive); - - await this.SaveEventHubsReceiverCheckpoint(context, 600000); + this.traceHelper.LogDebug("EventHubsProcessor {eventHubName}/{eventHubPartition}({incarnation}) received {totalEvents} events in {latencyMs:F2}ms, starting with #{seqno}, next expected packet is #{nextSeqno}", this.eventHubName, this.eventHubPartition, current.Incarnation, totalEvents, stopwatch.Elapsed.TotalMilliseconds, first.SequenceNumber, current.NextPacketToReceive); } - catch (OperationCanceledException) when (this.shutdownToken.IsCancellationRequested) // we should only ignore these exceptions during VM shutdowns. See : https://github.com/microsoft/durabletask-netherite/pull/347 + catch (OperationCanceledException) when (this.shutdownSource.IsCancellationRequested) // we should only ignore these exceptions during eventProcessor shutdowns. See : https://github.com/microsoft/durabletask-netherite/pull/347 { this.traceHelper.LogInformation("EventHubsProcessor {eventHubName}/{eventHubPartition}({incarnation}) was terminated", this.eventHubName, this.eventHubPartition, current.Incarnation); } catch (Exception exception) { - this.traceHelper.LogError("EventHubsProcessor {eventHubName}/{eventHubPartition}({incarnation}) encountered an exception while processing packets : {exception}", this.eventHubName, this.eventHubPartition, current.Incarnation, exception); - current?.ErrorHandler.HandleError("IEventProcessor.ProcessEventsAsync", "Encountered exception while processing events", exception, true, false); + this.traceHelper.LogError("EventHubsProcessor {eventHubName}/{eventHubPartition}({incarnation}) ProcessEventsAsync encountered an exception while processing packets : {exception}", this.eventHubName, this.eventHubPartition, current.Incarnation, exception); + current?.ErrorHandler.HandleError("PartitionProcessor.ProcessEventsAsync", "Encountered exception while processing events", exception, true, false); + this.nextToReceive = packets.LastOrDefault().SequenceNumber; } } } diff --git a/src/DurableTask.Netherite/TransportLayer/EventHubs/RecoveryTester.cs b/src/DurableTask.Netherite/TransportLayer/EventHubs/RecoveryTester.cs index 8594420f..37e91c3c 100644 --- a/src/DurableTask.Netherite/TransportLayer/EventHubs/RecoveryTester.cs +++ b/src/DurableTask.Netherite/TransportLayer/EventHubs/RecoveryTester.cs @@ -3,16 +3,10 @@ namespace DurableTask.Netherite.EventHubsTransport { - using DurableTask.Core.Common; using DurableTask.Netherite.Abstractions; - using Microsoft.Azure.EventHubs; - using Microsoft.Azure.Storage.Blob; using Microsoft.Extensions.Logging; using System; using System.Collections.Generic; - using System.Diagnostics; - using System.Linq; - using System.Threading; using System.Threading.Tasks; /// diff --git a/src/DurableTask.Netherite/TransportLayer/SingleHost/PartitionQueue.cs b/src/DurableTask.Netherite/TransportLayer/SingleHost/PartitionQueue.cs index e8ed541e..6f7919ea 100644 --- a/src/DurableTask.Netherite/TransportLayer/SingleHost/PartitionQueue.cs +++ b/src/DurableTask.Netherite/TransportLayer/SingleHost/PartitionQueue.cs @@ -4,13 +4,9 @@ namespace DurableTask.Netherite.SingleHostTransport { using DurableTask.Netherite.Abstractions; - using Microsoft.Azure.EventHubs; - using Microsoft.Extensions.Azure; using Microsoft.Extensions.Logging; using System; - using System.Collections; using System.Collections.Generic; - using System.Diagnostics; using System.IO; using System.Linq; using System.Threading; diff --git a/src/DurableTask.Netherite/TransportLayer/SingleHost/SingleHostTransportProvider.cs b/src/DurableTask.Netherite/TransportLayer/SingleHost/SingleHostTransportProvider.cs index 2e8951ce..c1da78da 100644 --- a/src/DurableTask.Netherite/TransportLayer/SingleHost/SingleHostTransportProvider.cs +++ b/src/DurableTask.Netherite/TransportLayer/SingleHost/SingleHostTransportProvider.cs @@ -11,7 +11,6 @@ namespace DurableTask.Netherite.SingleHostTransport using System.Threading.Tasks; using DurableTask.Netherite.Abstractions; using DurableTask.Netherite.Faster; - using Microsoft.Azure.EventHubs; using Microsoft.Extensions.Logging; /// diff --git a/src/DurableTask.Netherite/Util/AsyncLock.cs b/src/DurableTask.Netherite/Util/AsyncLock.cs index b0ebe85b..d4ae9ff9 100644 --- a/src/DurableTask.Netherite/Util/AsyncLock.cs +++ b/src/DurableTask.Netherite/Util/AsyncLock.cs @@ -15,9 +15,11 @@ namespace DurableTask.Netherite class AsyncLock : SemaphoreSlim, IDisposable { readonly AcquisitionToken token; + readonly CancellationToken shutdownToken; - public AsyncLock() : base(1, 1) + public AsyncLock(CancellationToken shutdownToken) : base(1, 1) { + this.shutdownToken = shutdownToken; this.token = new AcquisitionToken() { AsyncLock = this @@ -26,17 +28,37 @@ public AsyncLock() : base(1, 1) public async ValueTask LockAsync() { - await base.WaitAsync(); + try + { + await base.WaitAsync(this.shutdownToken); + } + catch (ObjectDisposedException) when (this.shutdownToken.IsCancellationRequested) + { + this.shutdownToken.ThrowIfCancellationRequested(); + } + return this.token; } + void ReleaseSemaphore() + { + try + { + base.Release(); + } + catch (ObjectDisposedException) when (this.shutdownToken.IsCancellationRequested) + { + this.shutdownToken.ThrowIfCancellationRequested(); + } + } + internal struct AcquisitionToken : IDisposable { public AsyncLock AsyncLock; public void Dispose() { - this.AsyncLock.Release(); + this.AsyncLock.ReleaseSemaphore(); } } } diff --git a/src/DurableTask.Netherite/Util/BatchWorker.cs b/src/DurableTask.Netherite/Util/BatchWorker.cs index 3e8ab616..fe1ef9b2 100644 --- a/src/DurableTask.Netherite/Util/BatchWorker.cs +++ b/src/DurableTask.Netherite/Util/BatchWorker.cs @@ -95,7 +95,7 @@ public virtual Task WaitForCompletionAsync() return tcs.Task; } - public Task WaitForShutdownAsync() + public virtual Task WaitForShutdownAsync() { if (!this.cancellationToken.IsCancellationRequested) { diff --git a/test/DurableTask.Netherite.AzureFunctions.Tests/CoreScenarios.cs b/test/DurableTask.Netherite.AzureFunctions.Tests/CoreScenarios.cs index 88d7dcc3..19b31691 100644 --- a/test/DurableTask.Netherite.AzureFunctions.Tests/CoreScenarios.cs +++ b/test/DurableTask.Netherite.AzureFunctions.Tests/CoreScenarios.cs @@ -6,6 +6,7 @@ namespace DurableTask.Netherite.AzureFunctions.Tests using System; using System.Collections.Generic; using System.Linq; + using System.Threading; using System.Threading.Tasks; using DurableTask.Netherite.Tests; using Microsoft.Azure.WebJobs; @@ -25,80 +26,104 @@ public CoreScenarios(ITestOutputHelper output) this.AddFunctions(typeof(Functions)); } + static readonly TimeSpan defaultTimeout = TimeSpan.FromMinutes(1); + [Fact] - public void HostCanStartAndStop() + public Task HostCanStartAndStop() { - // Ensure (via logs) that the Durable extension is loaded - IEnumerable extensionLogs = this.GetExtensionLogs(); - Assert.NotEmpty(extensionLogs); - - // Ensure (via logs) that the Netherite provider correctly loaded. - IEnumerable providerLogs = this.GetProviderLogs(); - Assert.NotEmpty(providerLogs); + return Common.WithTimeoutAsync(defaultTimeout, () => + { + // Ensure (via logs) that the Durable extension is loaded + IEnumerable extensionLogs = this.GetExtensionLogs(); + Assert.NotEmpty(extensionLogs); + + // Ensure (via logs) that the Netherite provider correctly loaded. + IEnumerable providerLogs = this.GetProviderLogs(); + Assert.NotEmpty(providerLogs); + }); } [Fact] - public async Task CanRunActivitySequences() + public Task CanRunActivitySequences() { - DurableOrchestrationStatus status = await this.RunOrchestrationAsync(nameof(Functions.Sequence)); - Assert.Equal(OrchestrationRuntimeStatus.Completed, status.RuntimeStatus); - Assert.Equal(10, (int)status.Output); + return Common.WithTimeoutAsync(defaultTimeout, async () => + { + DurableOrchestrationStatus status = await this.RunOrchestrationAsync(nameof(Functions.Sequence)); + Assert.Equal(OrchestrationRuntimeStatus.Completed, status.RuntimeStatus); + Assert.Equal(10, (int)status.Output); + }); } [Fact] - public async Task CanPurgeAndListInstances() + public Task CanPurgeAndListInstances() { - await this.PurgeAllAsync(); - var results = await this.GetInstancesAsync(new OrchestrationStatusQueryCondition()); - Assert.Empty(results); + return Common.WithTimeoutAsync(defaultTimeout, async () => + { + await this.PurgeAllAsync(); + var results = await this.GetInstancesAsync(new OrchestrationStatusQueryCondition()); + Assert.Empty(results); + }); } + [Fact] - public async Task CanRunFanOutFanIn() + public Task CanRunFanOutFanIn() { - DurableOrchestrationStatus status = await this.RunOrchestrationAsync(nameof(Functions.FanOutFanIn)); - Assert.Equal(OrchestrationRuntimeStatus.Completed, status.RuntimeStatus); - Assert.Equal( - expected: @"[""9"",""8"",""7"",""6"",""5"",""4"",""3"",""2"",""1"",""0""]", - actual: status.Output?.ToString(Formatting.None)); + return Common.WithTimeoutAsync(defaultTimeout, async () => + { + DurableOrchestrationStatus status = await this.RunOrchestrationAsync(nameof(Functions.FanOutFanIn)); + Assert.Equal(OrchestrationRuntimeStatus.Completed, status.RuntimeStatus); + Assert.Equal( + expected: @"[""9"",""8"",""7"",""6"",""5"",""4"",""3"",""2"",""1"",""0""]", + actual: status.Output?.ToString(Formatting.None)); + }); } [Fact] - public async Task CanOrchestrateEntities() + public Task CanOrchestrateEntities() { - DurableOrchestrationStatus status = await this.RunOrchestrationAsync(nameof(Functions.OrchestrateCounterEntity)); - Assert.Equal(OrchestrationRuntimeStatus.Completed, status.RuntimeStatus); - Assert.Equal(7, (int)status.Output); + return Common.WithTimeoutAsync(defaultTimeout, async () => + { + DurableOrchestrationStatus status = await this.RunOrchestrationAsync(nameof(Functions.OrchestrateCounterEntity)); + Assert.Equal(OrchestrationRuntimeStatus.Completed, status.RuntimeStatus); + Assert.Equal(7, (int)status.Output); + }); } [Fact] - public async Task CanClientInteractWithEntities() + public Task CanClientInteractWithEntities() { - IDurableClient client = await this.GetDurableClientAsync(); + return Common.WithTimeoutAsync(defaultTimeout, async () => + { + IDurableClient client = await this.GetDurableClientAsync(); - var entityId = new EntityId(nameof(Functions.Counter), Guid.NewGuid().ToString("N")); - EntityStateResponse result = await client.ReadEntityStateAsync(entityId); - Assert.False(result.EntityExists); + var entityId = new EntityId(nameof(Functions.Counter), Guid.NewGuid().ToString("N")); + EntityStateResponse result = await client.ReadEntityStateAsync(entityId); + Assert.False(result.EntityExists); - await Task.WhenAll( - client.SignalEntityAsync(entityId, "incr"), - client.SignalEntityAsync(entityId, "incr"), - client.SignalEntityAsync(entityId, "incr"), - client.SignalEntityAsync(entityId, "add", 4)); + await Task.WhenAll( + client.SignalEntityAsync(entityId, "incr"), + client.SignalEntityAsync(entityId, "incr"), + client.SignalEntityAsync(entityId, "incr"), + client.SignalEntityAsync(entityId, "add", 4)); - await Task.Delay(TimeSpan.FromSeconds(5)); + await Task.Delay(TimeSpan.FromSeconds(5)); - result = await client.ReadEntityStateAsync(entityId); - Assert.True(result.EntityExists); - Assert.Equal(7, result.EntityState); + result = await client.ReadEntityStateAsync(entityId); + Assert.True(result.EntityExists); + Assert.Equal(7, result.EntityState); + }); } [Fact] - public async Task CanOrchestrationInteractWithEntities() + public Task CanOrchestrationInteractWithEntities() { - DurableOrchestrationStatus status = await this.RunOrchestrationAsync(nameof(Functions.IncrementThenGet)); - Assert.Equal(OrchestrationRuntimeStatus.Completed, status.RuntimeStatus); - Assert.Equal(1, (int)status.Output); + return Common.WithTimeoutAsync(defaultTimeout, async () => + { + DurableOrchestrationStatus status = await this.RunOrchestrationAsync(nameof(Functions.IncrementThenGet)); + Assert.Equal(OrchestrationRuntimeStatus.Completed, status.RuntimeStatus); + Assert.Equal(1, (int)status.Output); + }); } static class Functions diff --git a/test/DurableTask.Netherite.AzureFunctions.Tests/IntegrationTestBase.cs b/test/DurableTask.Netherite.AzureFunctions.Tests/IntegrationTestBase.cs index 74d31d0d..481bb4d8 100644 --- a/test/DurableTask.Netherite.AzureFunctions.Tests/IntegrationTestBase.cs +++ b/test/DurableTask.Netherite.AzureFunctions.Tests/IntegrationTestBase.cs @@ -10,6 +10,7 @@ namespace DurableTask.Netherite.AzureFunctions.Tests using System.Threading; using System.Threading.Tasks; using DurableTask.Netherite.AzureFunctions.Tests.Logging; + using DurableTask.Netherite.Tests; using Microsoft.Azure.WebJobs; using Microsoft.Azure.WebJobs.Extensions.DurableTask; using Microsoft.Extensions.DependencyInjection; @@ -74,21 +75,23 @@ public IntegrationTestBase(ITestOutputHelper output) this.AddFunctions(typeof(ClientFunctions)); } - async Task IAsyncLifetime.InitializeAsync() + Task IAsyncLifetime.InitializeAsync() { - await this.functionsHost.StartAsync(); + return Common.WithTimeoutAsync(TimeSpan.FromMinutes(1), () => this.functionsHost.StartAsync()); } - async Task IAsyncLifetime.DisposeAsync() + Task IAsyncLifetime.DisposeAsync() { - try + return Common.WithTimeoutAsync(TimeSpan.FromMinutes(1), async () => { - await this.functionsHost.StopAsync(); - } - catch(OperationCanceledException) - { - - } + try + { + await this.functionsHost.StopAsync(); + } + catch (OperationCanceledException) + { + } + }); } protected void AddFunctions(Type functionType) => this.typeLocator.AddFunctionType(functionType); diff --git a/test/DurableTask.Netherite.Tests/Common.cs b/test/DurableTask.Netherite.Tests/Common.cs index d7e5be05..a65737ba 100644 --- a/test/DurableTask.Netherite.Tests/Common.cs +++ b/test/DurableTask.Netherite.Tests/Common.cs @@ -9,7 +9,7 @@ namespace DurableTask.Netherite.Tests using System.Threading; using System.Threading.Tasks; - static class Common + public static class Common { public static async Task> ParallelForEachAsync(this IEnumerable items, int maxConcurrency, bool useThreadpool, Func> action) { @@ -52,5 +52,44 @@ static async Task InvokeThrottledAction(T item, Func> action, semaphore.Release(); } } + + public static async Task WithTimeoutAsync(TimeSpan timeout, Func taskFactory) + { + using CancellationTokenSource cts = new CancellationTokenSource(); + Task timeoutTask = RunTimeoutCheckerAsync(); + + async Task RunTimeoutCheckerAsync() + { + try + { + await Task.Delay(timeout, cts.Token); + throw new TimeoutException($"task did not complete within {timeout}"); + } + catch (OperationCanceledException) + { + // we cancel this if the task completes before the timeout + } + } + + try + { + await Task.WhenAny(taskFactory(), timeoutTask); + } + finally + { + cts.Cancel(); + await timeoutTask; + } + } + + public static Task WithTimeoutAsync(TimeSpan timeout, Action action) + { + return WithTimeoutAsync(timeout, () => Task.Run(action)); + } + + public static void WithTimeout(TimeSpan timeout, Action action) + { + WithTimeoutAsync(timeout, action).Wait(); + } } } \ No newline at end of file diff --git a/test/DurableTask.Netherite.Tests/ConcurrentTestsFaster.cs b/test/DurableTask.Netherite.Tests/ConcurrentTestsFaster.cs index 5b4149ca..7ed1c752 100644 --- a/test/DurableTask.Netherite.Tests/ConcurrentTestsFaster.cs +++ b/test/DurableTask.Netherite.Tests/ConcurrentTestsFaster.cs @@ -158,7 +158,7 @@ async Task RunAsync() useReplayChecker, restrictMemory ? (int?)0 : null, startupTimeout, - (msg) => this.outputHelper?.WriteLine(msg))) + (msg) => this.outputHelper?.WriteLine(msg))) { var scenarios = new ScenarioTests(fixture, this.outputHelper); @@ -166,10 +166,10 @@ async Task RunAsync() for (int i = 0; i < multiplicity; i++) { - foreach((string name, Task task) in scenarios.StartAllScenarios(false, false)) + foreach ((string name, Task task) in scenarios.StartAllScenarios(false, false)) { Trace.WriteLine($"TestProgress: Adding {name}"); - tests.Add((name,task)); + tests.Add((name, task)); } } @@ -197,34 +197,37 @@ async Task RunAsync() //[InlineData(6)] //[InlineData(7)] //[InlineData(8)] - public async Task ReproHangingReads(int sequenceNumber) + public Task ReproHangingReads(int sequenceNumber) { - // running a single test is usually not enough to repro, so we run the same test multiple times - this.outputHelper.WriteLine($"starting test {sequenceNumber}"); + return Common.WithTimeoutAsync(TimeSpan.FromMinutes(5), async () => + { + // running a single test is usually not enough to repro, so we run the same test multiple times + this.outputHelper.WriteLine($"starting test {sequenceNumber}"); - // disable checkpoints since they are not needed to trigger the bug - this.settings.MaxNumberBytesBetweenCheckpoints = 1024L * 1024 * 1024 * 1024; - this.settings.MaxNumberEventsBetweenCheckpoints = 10000000000L; - this.settings.IdleCheckpointFrequencyMs = (long)TimeSpan.FromDays(1).TotalMilliseconds; + // disable checkpoints since they are not needed to trigger the bug + this.settings.MaxNumberBytesBetweenCheckpoints = 1024L * 1024 * 1024 * 1024; + this.settings.MaxNumberEventsBetweenCheckpoints = 10000000000L; + this.settings.IdleCheckpointFrequencyMs = (long)TimeSpan.FromDays(1).TotalMilliseconds; - this.settings.PartitionCount = 4; + this.settings.PartitionCount = 4; - using var _ = TestOrchestrationClient.WithExtraTime(TimeSpan.FromMinutes(3)); - using var fixture = await HostFixture.StartNew(this.settings, true, false, 0, TimeSpan.FromMinutes(5), (msg) => this.outputHelper?.WriteLine(msg)); + using var _ = TestOrchestrationClient.WithExtraTime(TimeSpan.FromMinutes(3)); + using var fixture = await HostFixture.StartNew(this.settings, true, false, 0, TimeSpan.FromMinutes(5), (msg) => this.outputHelper?.WriteLine(msg)); - this.settings.TestHooks.CacheDebugger.EnableSizeChecking = false; + this.settings.TestHooks.CacheDebugger.EnableSizeChecking = false; - var scenarios = new ScenarioTests(fixture, this.outputHelper); + var scenarios = new ScenarioTests(fixture, this.outputHelper); - var tests = new List<(string, Task)>(); + var tests = new List<(string, Task)>(); - for (int i = 0; i < 20; i++) - { - tests.AddRange(scenarios.StartAllScenarios(false, false)); - } + for (int i = 0; i < 20; i++) + { + tests.AddRange(scenarios.StartAllScenarios(false, false)); + } - await this.WaitForCompletion(tests, TimeSpan.FromMinutes(10)); + await this.WaitForCompletion(tests, TimeSpan.FromMinutes(10)); + }); } } } diff --git a/test/DurableTask.Netherite.Tests/FasterPartitionTests.cs b/test/DurableTask.Netherite.Tests/FasterPartitionTests.cs index 2e7ac8dc..90b67df4 100644 --- a/test/DurableTask.Netherite.Tests/FasterPartitionTests.cs +++ b/test/DurableTask.Netherite.Tests/FasterPartitionTests.cs @@ -34,6 +34,8 @@ public class FasterPartitionTests : IDisposable ITestOutputHelper outputHelper; string errorInTestHooks; + static readonly TimeSpan DefaultTimeout = TimeSpan.FromMinutes(5); + const int extraLogEntrySize = 96; // since v2 Faster puts extra stuff in the log. Empirically determined. public FasterPartitionTests(ITestOutputHelper outputHelper) @@ -61,6 +63,12 @@ public FasterPartitionTests(ITestOutputHelper outputHelper) }; } + // to more easily debug hangs in CI, we add progress messages to our unit tests so we can tell where it hangs + void Progress(string message) + { + this.output?.Invoke($"%%% {message}"); + } + public void Dispose() { this.outputHelper = null; @@ -125,318 +133,334 @@ void SetCheckpointFrequency(CheckpointFrequency frequency) /// Create a partition and then restore it. /// [Fact] - public async Task CreateThenRestore() + public Task CreateThenRestore() { - this.settings.PartitionCount = 1; - var orchestrationType = typeof(ScenarioTests.Orchestrations.SayHelloInline); + return Common.WithTimeoutAsync(DefaultTimeout, async () => { - // start the service - var (service, client) = await this.StartService(recover: false, orchestrationType); + this.settings.PartitionCount = 1; + var orchestrationType = typeof(ScenarioTests.Orchestrations.SayHelloInline); + { + // start the service + var (service, client) = await this.StartService(recover: false, orchestrationType); - // do orchestration - var instance = await client.CreateOrchestrationInstanceAsync(orchestrationType, "0", "0"); - await client.WaitForOrchestrationAsync(instance, TimeSpan.FromSeconds(20)); + // do orchestration + var instance = await client.CreateOrchestrationInstanceAsync(orchestrationType, "0", "0"); + await client.WaitForOrchestrationAsync(instance, TimeSpan.FromSeconds(20)); - // stop the service - await service.StopAsync(); - } - { - // start the service - var (service, client) = await this.StartService(recover: true, orchestrationType); - var orchestrationState = await client.GetOrchestrationStateAsync("0"); - Assert.Equal(OrchestrationStatus.Completed, orchestrationState?.OrchestrationStatus); + // stop the service + await service.StopAsync(); + } + { + // start the service + var (service, client) = await this.StartService(recover: true, orchestrationType); + var orchestrationState = await client.GetOrchestrationStateAsync("0"); + Assert.Equal(OrchestrationStatus.Completed, orchestrationState?.OrchestrationStatus); - // stop the service - await service.StopAsync(); - } + // stop the service + await service.StopAsync(); + } + }); } /// /// Run a number of orchestrations that requires more memory than available for FASTER /// [Fact()] - public async Task LimitedMemory() + public Task LimitedMemory() { - this.settings.PartitionCount = 1; - this.SetCheckpointFrequency(CheckpointFrequency.None); - - // set the memory size very small so we can force evictions - this.settings.FasterTuningParameters = new Faster.BlobManager.FasterTuningParameters() + return Common.WithTimeoutAsync(DefaultTimeout, async () => { - StoreLogPageSizeBits = 10, // 1 KB - StoreLogMemorySizeBits = 12, // 4 KB, which means only about 166 entries fit into memory - }; + this.settings.PartitionCount = 1; + this.SetCheckpointFrequency(CheckpointFrequency.None); - // we use the standard hello orchestration from the samples, which calls 5 activities in sequence - var orchestrationType = typeof(ScenarioTests.Orchestrations.Hello5); - var activityType = typeof(ScenarioTests.Activities.Hello); - string InstanceId(int i) => $"Orch{i:D5}"; - int OrchestrationCount = 100; // requires 200 FASTER key-value pairs so it does not fit into memory + // set the memory size very small so we can force evictions + this.settings.FasterTuningParameters = new Faster.BlobManager.FasterTuningParameters() + { + StoreLogPageSizeBits = 10, // 1 KB + StoreLogMemorySizeBits = 12, // 4 KB, which means only about 166 entries fit into memory + }; - // start the service - var (service, client) = await this.StartService(recover: false, orchestrationType, activityType); + // we use the standard hello orchestration from the samples, which calls 5 activities in sequence + var orchestrationType = typeof(ScenarioTests.Orchestrations.Hello5); + var activityType = typeof(ScenarioTests.Activities.Hello); + string InstanceId(int i) => $"Orch{i:D5}"; + int OrchestrationCount = 100; // requires 200 FASTER key-value pairs so it does not fit into memory - // start all orchestrations - { - var tasks = new Task[OrchestrationCount]; - for (int i = 0; i < OrchestrationCount; i++) - tasks[i] = client.CreateOrchestrationInstanceAsync(orchestrationType, InstanceId(i), null); - - var timeout = TimeSpan.FromMinutes(3); - var terminationTask = Task.Delay(timeout, this.cts.Token); - var completionTask = Task.WhenAll(tasks); - var firstTask = await Task.WhenAny(terminationTask, completionTask); - Assert.True(this.errorInTestHooks == null, $"while starting orchestrations: {this.errorInTestHooks}"); - Assert.True(firstTask != terminationTask, $"timed out after {timeout} while starting orchestrations"); - } + // start the service + var (service, client) = await this.StartService(recover: false, orchestrationType, activityType); - // wait for all orchestrations to finish executing - this.output?.Invoke("waiting for orchestrations to finish executing..."); - try - { - async Task WaitFor(int i) + // start all orchestrations { - try - { - await client.WaitForOrchestrationAsync(new OrchestrationInstance { InstanceId = InstanceId(i) }, TimeSpan.FromMinutes(10)); - } - catch (Exception e) - { - this.output?.Invoke($"Orchestration {InstanceId(i)} failed with {e.GetType()}: {e.Message}"); - } - } + var tasks = new Task[OrchestrationCount]; + for (int i = 0; i < OrchestrationCount; i++) + tasks[i] = client.CreateOrchestrationInstanceAsync(orchestrationType, InstanceId(i), null); - var tasks = new Task[OrchestrationCount]; - var timeout = Debugger.IsAttached ? TimeSpan.FromMinutes(15) : TimeSpan.FromMinutes(3); - for (int i = 0; i < OrchestrationCount; i++) - tasks[i] = WaitFor(i); + var timeout = TimeSpan.FromMinutes(3); + var terminationTask = Task.Delay(timeout, this.cts.Token); + var completionTask = Task.WhenAll(tasks); + var firstTask = await Task.WhenAny(terminationTask, completionTask); + Assert.True(this.errorInTestHooks == null, $"while starting orchestrations: {this.errorInTestHooks}"); + Assert.True(firstTask != terminationTask, $"timed out after {timeout} while starting orchestrations"); + } - void PrintUnfinished() + // wait for all orchestrations to finish executing + this.output?.Invoke("waiting for orchestrations to finish executing..."); + try { - var sb = new StringBuilder(); - sb.Append("Waiting for orchestrations:"); - for (int i = 0; i < OrchestrationCount; i++) + async Task WaitFor(int i) { - if (!tasks[i].IsCompleted) + try + { + await client.WaitForOrchestrationAsync(new OrchestrationInstance { InstanceId = InstanceId(i) }, TimeSpan.FromMinutes(10)); + } + catch (Exception e) { - sb.Append(' '); - sb.Append(InstanceId(i)); + this.output?.Invoke($"Orchestration {InstanceId(i)} failed with {e.GetType()}: {e.Message}"); } } - this.output?.Invoke(sb.ToString()); - } - void ProgressReportThread() - { - Stopwatch elapsed = new Stopwatch(); - elapsed.Start(); + var tasks = new Task[OrchestrationCount]; + var timeout = Debugger.IsAttached ? TimeSpan.FromMinutes(15) : TimeSpan.FromMinutes(3); + for (int i = 0; i < OrchestrationCount; i++) + tasks[i] = WaitFor(i); - while (elapsed.Elapsed < timeout) + void PrintUnfinished() { - Thread.Sleep(10000); - PrintUnfinished(); + var sb = new StringBuilder(); + sb.Append("Waiting for orchestrations:"); + for (int i = 0; i < OrchestrationCount; i++) + { + if (!tasks[i].IsCompleted) + { + sb.Append(' '); + sb.Append(InstanceId(i)); + } + } + this.output?.Invoke(sb.ToString()); } - this.cts.Cancel(); - } - var thread = TrackedThreads.MakeTrackedThread(ProgressReportThread, "ProgressReportThread"); - thread.Start(); + void ProgressReportThread() + { + Stopwatch elapsed = new Stopwatch(); + elapsed.Start(); + + while (elapsed.Elapsed < timeout) + { + Thread.Sleep(10000); + PrintUnfinished(); + } - var terminationTask = Task.Delay(timeout, this.cts.Token); - var completionTask = Task.WhenAll(tasks); - var firstTask = await Task.WhenAny(terminationTask, completionTask); - Assert.True(this.errorInTestHooks == null, $"while executing orchestrations: {this.errorInTestHooks}"); + this.cts.Cancel(); + } + var thread = TrackedThreads.MakeTrackedThread(ProgressReportThread, "ProgressReportThread"); + thread.Start(); - PrintUnfinished(); + var terminationTask = Task.Delay(timeout, this.cts.Token); + var completionTask = Task.WhenAll(tasks); + var firstTask = await Task.WhenAny(terminationTask, completionTask); + Assert.True(this.errorInTestHooks == null, $"while executing orchestrations: {this.errorInTestHooks}"); - Assert.True(firstTask != terminationTask, $"timed out after {timeout} while executing orchestrations"); + PrintUnfinished(); - foreach (var line in this.cacheDebugger.Dump()) - { - this.output?.Invoke(line); + Assert.True(firstTask != terminationTask, $"timed out after {timeout} while executing orchestrations"); + + foreach (var line in this.cacheDebugger.Dump()) + { + this.output?.Invoke(line); + } } - } - catch (Exception e) - { - this.output?.Invoke($"exception thrown while executing orchestrations: {e}"); - foreach (var line in this.cacheDebugger.Dump()) + catch (Exception e) { - this.output?.Invoke(line); + this.output?.Invoke($"exception thrown while executing orchestrations: {e}"); + foreach (var line in this.cacheDebugger.Dump()) + { + this.output?.Invoke(line); + } + throw; } - throw; - } - // shut down the service - await service.StopAsync(); + // shut down the service + await service.StopAsync(); + }); } /// /// Create a partition and then restore it, and use the size tracker again. /// [Fact] - public async Task CheckSizeTrackerOnRecovery() + public Task CheckSizeTrackerOnRecovery() { - this.settings.PartitionCount = 1; - this.SetCheckpointFrequency(CheckpointFrequency.None); - - // set the memory size very small so we can force evictions - this.settings.FasterTuningParameters = new Faster.BlobManager.FasterTuningParameters() + return Common.WithTimeoutAsync(DefaultTimeout, async () => { - StoreLogPageSizeBits = 10, // 1 KB - StoreLogMemorySizeBits = 12, // 4 KB, which means only about 166 entries fit into memory - }; + this.settings.PartitionCount = 1; + this.SetCheckpointFrequency(CheckpointFrequency.None); - // we use the standard hello orchestration from the samples, which calls 5 activities in sequence - var orchestrationType = typeof(ScenarioTests.Orchestrations.Hello5); - var activityType = typeof(ScenarioTests.Activities.Hello); - string InstanceId(int i) => $"Orch{i:D5}"; - int OrchestrationCount = 100; // requires 200 FASTER key-value pairs so it does not fit into memory + // set the memory size very small so we can force evictions + this.settings.FasterTuningParameters = new Faster.BlobManager.FasterTuningParameters() + { + StoreLogPageSizeBits = 10, // 1 KB + StoreLogMemorySizeBits = 12, // 4 KB, which means only about 166 entries fit into memory + }; - { - // start the service - var (service, client) = await this.StartService(recover: false, orchestrationType, activityType); + // we use the standard hello orchestration from the samples, which calls 5 activities in sequence + var orchestrationType = typeof(ScenarioTests.Orchestrations.Hello5); + var activityType = typeof(ScenarioTests.Activities.Hello); + string InstanceId(int i) => $"Orch{i:D5}"; + int OrchestrationCount = 100; // requires 200 FASTER key-value pairs so it does not fit into memory - // start all orchestrations { - var tasks = new Task[OrchestrationCount]; - for (int i = 0; i < OrchestrationCount; i++) - tasks[i] = client.CreateOrchestrationInstanceAsync(orchestrationType, InstanceId(i), null); + // start the service + var (service, client) = await this.StartService(recover: false, orchestrationType, activityType); - await Task.WhenAll(tasks); - Assert.True(this.errorInTestHooks == null, $"TestHooks detected problem while starting orchestrations: {this.errorInTestHooks}"); - } + // start all orchestrations + { + var tasks = new Task[OrchestrationCount]; + for (int i = 0; i < OrchestrationCount; i++) + tasks[i] = client.CreateOrchestrationInstanceAsync(orchestrationType, InstanceId(i), null); - // wait for all orchestrations to finish executing - { - async Task WaitFor(int i) + await Task.WhenAll(tasks); + Assert.True(this.errorInTestHooks == null, $"TestHooks detected problem while starting orchestrations: {this.errorInTestHooks}"); + } + + // wait for all orchestrations to finish executing { - try - { - await client.WaitForOrchestrationAsync(new OrchestrationInstance { InstanceId = InstanceId(i) }, TimeSpan.FromMinutes(10)); - } - catch (Exception e) + async Task WaitFor(int i) { - this.output?.Invoke($"Orchestration {InstanceId(i)} failed with {e.GetType()}: {e.Message}"); + try + { + await client.WaitForOrchestrationAsync(new OrchestrationInstance { InstanceId = InstanceId(i) }, TimeSpan.FromMinutes(10)); + } + catch (Exception e) + { + this.output?.Invoke($"Orchestration {InstanceId(i)} failed with {e.GetType()}: {e.Message}"); + } } + + var tasks = new Task[OrchestrationCount]; + for (int i = 0; i < OrchestrationCount; i++) + tasks[i] = WaitFor(i); + await Task.WhenAll(tasks); + Assert.True(this.errorInTestHooks == null, $"TestHooks detected problem while executing orchestrations: {this.errorInTestHooks}"); } - var tasks = new Task[OrchestrationCount]; - for (int i = 0; i < OrchestrationCount; i++) - tasks[i] = WaitFor(i); - await Task.WhenAll(tasks); - Assert.True(this.errorInTestHooks == null, $"TestHooks detected problem while executing orchestrations: {this.errorInTestHooks}"); - } + this.Progress("shutting down"); + foreach (var line in this.cacheDebugger.Dump()) + { + this.output?.Invoke(line); + } - this.output?.Invoke("--- test progress: BEFORE SHUTDOWN ------------------------------------"); - foreach (var line in this.cacheDebugger.Dump()) - { - this.output?.Invoke(line); - } + // shut down the service + await service.StopAsync(); - // shut down the service - await service.StopAsync(); - } + this.Progress("shut down"); + } - { - this.output?.Invoke("--- test progress: BEFORE RECOVERY ------------------------------------"); + { + this.Progress("recovering"); - // recover the service - var (service, client) = await this.StartService(recover: true, orchestrationType, activityType); + // recover the service + var (service, client) = await this.StartService(recover: true, orchestrationType, activityType); - this.output?.Invoke("--- test progress: AFTER RECOVERY ------------------------------------"); + this.Progress("recovered"); + this.Progress("querying the status of all orchestrations"); + { + var tasks = new Task[OrchestrationCount]; + for (int i = 0; i < OrchestrationCount; i++) + tasks[i] = client.WaitForOrchestrationAsync(new OrchestrationInstance { InstanceId = InstanceId(i) }, TimeSpan.FromMinutes(10)); + await Task.WhenAll(tasks); + Assert.True(this.errorInTestHooks == null, $"TestHooks detected problem while querying orchestration states: {this.errorInTestHooks}"); + } + this.Progress("queries completed"); - // query the status of all orchestrations - { - var tasks = new Task[OrchestrationCount]; - for (int i = 0; i < OrchestrationCount; i++) - tasks[i] = client.WaitForOrchestrationAsync(new OrchestrationInstance { InstanceId = InstanceId(i) }, TimeSpan.FromMinutes(10)); - await Task.WhenAll(tasks); - Assert.True(this.errorInTestHooks == null, $"TestHooks detected problem while querying orchestration states: {this.errorInTestHooks}"); - } + foreach (var line in this.cacheDebugger.Dump()) + { + this.output?.Invoke(line); + } - this.output?.Invoke("--- test progress: AFTER QUERIES ------------------------------------"); - foreach (var line in this.cacheDebugger.Dump()) - { - this.output?.Invoke(line); + // shut down the service + await service.StopAsync(); } - - // shut down the service - await service.StopAsync(); - } + }); } /// /// Fill memory, then compute size, then reduce page count, and measure size again /// [Fact] - public async Task PipelinedStart() + public Task PipelinedStart() { - this.settings.PartitionCount = 1; - this.settings.InstanceCacheSizeMB = 2; - this.SetCheckpointFrequency(CheckpointFrequency.Frequent); + return Common.WithTimeoutAsync(DefaultTimeout, async () => + { + this.settings.PartitionCount = 1; + this.settings.InstanceCacheSizeMB = 2; + this.SetCheckpointFrequency(CheckpointFrequency.Frequent); - var orchestrationType = typeof(ScenarioTests.Orchestrations.Hello5); - var activityType = typeof(ScenarioTests.Activities.Hello); - string InstanceId(int i) => $"Orch{i:D5}"; - int numOrchestrations = 500; + var orchestrationType = typeof(ScenarioTests.Orchestrations.Hello5); + var activityType = typeof(ScenarioTests.Activities.Hello); + string InstanceId(int i) => $"Orch{i:D5}"; + int numOrchestrations = 500; - // start the service - var (service, client) = await this.StartService(recover: false, orchestrationType, activityType); + // start the service + var (service, client) = await this.StartService(recover: false, orchestrationType, activityType); - // start all orchestrations and then get the status of each one - { - var orchestrations = await Enumerable.Range(0, numOrchestrations).ParallelForEachAsync(200, true, (iteration) => + // start all orchestrations and then get the status of each one { - var orchestrationInstanceId = InstanceId(iteration); - return client.CreateOrchestrationInstanceAsync(orchestrationType, orchestrationInstanceId, null); - }); - Assert.True(this.errorInTestHooks == null, $"TestHooks detected problem while starting orchestrations: {this.errorInTestHooks}"); - await Enumerable.Range(0, numOrchestrations).ParallelForEachAsync(200, true, (iteration) => - { - return client.GetOrchestrationStateAsync(orchestrations[iteration]); - }); - Assert.True(this.errorInTestHooks == null, $"TestHooks detected problem while checking progress of orchestrations: {this.errorInTestHooks}"); - } + var orchestrations = await Enumerable.Range(0, numOrchestrations).ParallelForEachAsync(200, true, (iteration) => + { + var orchestrationInstanceId = InstanceId(iteration); + return client.CreateOrchestrationInstanceAsync(orchestrationType, orchestrationInstanceId, null); + }); + Assert.True(this.errorInTestHooks == null, $"TestHooks detected problem while starting orchestrations: {this.errorInTestHooks}"); + await Enumerable.Range(0, numOrchestrations).ParallelForEachAsync(200, true, (iteration) => + { + return client.GetOrchestrationStateAsync(orchestrations[iteration]); + }); + Assert.True(this.errorInTestHooks == null, $"TestHooks detected problem while checking progress of orchestrations: {this.errorInTestHooks}"); + } - await service.StopAsync(); + await service.StopAsync(); + }); } /// /// Repro fail on basic 1000 * hello /// [Fact] - public async Task CheckMemorySize() + public Task CheckMemorySize() { - this.settings.PartitionCount = 1; - this.SetCheckpointFrequency(CheckpointFrequency.None); + return Common.WithTimeoutAsync(DefaultTimeout, async () => + { + this.settings.PartitionCount = 1; + this.SetCheckpointFrequency(CheckpointFrequency.None); - var orchestrationType = typeof(ScenarioTests.Orchestrations.SemiLargePayloadFanOutFanIn); - var activityType = typeof(ScenarioTests.Activities.Echo); - string InstanceId(int i) => $"Orch{i:D5}"; - int OrchestrationCount = 30; - int FanOut = 7; - long historyAndStatusSize = OrchestrationCount * (FanOut * 50000 /* in history */ + 2 * 16000 /* in status */); + var orchestrationType = typeof(ScenarioTests.Orchestrations.SemiLargePayloadFanOutFanIn); + var activityType = typeof(ScenarioTests.Activities.Echo); + string InstanceId(int i) => $"Orch{i:D5}"; + int OrchestrationCount = 30; + int FanOut = 7; + long historyAndStatusSize = OrchestrationCount * (FanOut * 50000 /* in history */ + 2 * 16000 /* in status */); - // start the service - var (service, client) = await this.StartService(recover: false, orchestrationType, activityType); + // start the service + var (service, client) = await this.StartService(recover: false, orchestrationType, activityType); - // run all orchestrations - { - var tasks = new Task[OrchestrationCount]; - for (int i = 0; i < OrchestrationCount; i++) - tasks[i] = client.CreateOrchestrationInstanceAsync(orchestrationType, InstanceId(i), FanOut); - await Task.WhenAll(tasks); - for (int i = 0; i < OrchestrationCount; i++) - tasks[i] = client.WaitForOrchestrationAsync(new OrchestrationInstance { InstanceId = InstanceId(i) }, TimeSpan.FromMinutes(3)); - await Task.WhenAll(tasks); - Assert.True(this.errorInTestHooks == null, $"TestHooks detected problem while starting orchestrations: {this.errorInTestHooks}"); - } + // run all orchestrations + { + var tasks = new Task[OrchestrationCount]; + for (int i = 0; i < OrchestrationCount; i++) + tasks[i] = client.CreateOrchestrationInstanceAsync(orchestrationType, InstanceId(i), FanOut); + await Task.WhenAll(tasks); + for (int i = 0; i < OrchestrationCount; i++) + tasks[i] = client.WaitForOrchestrationAsync(new OrchestrationInstance { InstanceId = InstanceId(i) }, TimeSpan.FromMinutes(3)); + await Task.WhenAll(tasks); + Assert.True(this.errorInTestHooks == null, $"TestHooks detected problem while starting orchestrations: {this.errorInTestHooks}"); + } - (int numPages, long memorySize) = this.cacheDebugger.MemoryTracker.GetMemorySize(); + (int numPages, long memorySize) = this.cacheDebugger.MemoryTracker.GetMemorySize(); - Assert.InRange(memorySize, historyAndStatusSize, 1.05 * historyAndStatusSize); - await service.StopAsync(); + Assert.InRange(memorySize, historyAndStatusSize, 1.05 * historyAndStatusSize); + await service.StopAsync(); + }); } @@ -444,64 +468,67 @@ public async Task CheckMemorySize() /// Fill up memory, then compute size, then reduce page count, and measure size again /// [Fact] - public async Task CheckMemoryReduction() + public Task CheckMemoryReduction() { - this.settings.PartitionCount = 1; - this.SetCheckpointFrequency(CheckpointFrequency.None); + return Common.WithTimeoutAsync(DefaultTimeout, async () => + { + this.settings.PartitionCount = 1; + this.SetCheckpointFrequency(CheckpointFrequency.None); - int pageCountBits = 3; - int pageCount = 1 << pageCountBits; + int pageCountBits = 3; + int pageCount = 1 << pageCountBits; - // set the memory size very small so we can force evictions - this.settings.FasterTuningParameters = new Faster.BlobManager.FasterTuningParameters() - { - StoreLogPageSizeBits = 9, // 512 B - StoreLogMemorySizeBits = 9 + pageCountBits, - }; + // set the memory size very small so we can force evictions + this.settings.FasterTuningParameters = new Faster.BlobManager.FasterTuningParameters() + { + StoreLogPageSizeBits = 9, // 512 B + StoreLogMemorySizeBits = 9 + pageCountBits, + }; - var orchestrationType = typeof(ScenarioTests.Orchestrations.SemiLargePayloadFanOutFanIn); - var activityType = typeof(ScenarioTests.Activities.Echo); - string InstanceId(int i) => $"Orch{i:D5}"; - int OrchestrationCount = 50; - int FanOut = 3; - long historyAndStatusSize = OrchestrationCount * (FanOut * 50000 /* in history */ + 2*16000 /* in status */); + var orchestrationType = typeof(ScenarioTests.Orchestrations.SemiLargePayloadFanOutFanIn); + var activityType = typeof(ScenarioTests.Activities.Echo); + string InstanceId(int i) => $"Orch{i:D5}"; + int OrchestrationCount = 50; + int FanOut = 3; + long historyAndStatusSize = OrchestrationCount * (FanOut * 50000 /* in history */ + 2 * 16000 /* in status */); - // start the service - var (service, client) = await this.StartService(recover: false, orchestrationType, activityType); + // start the service + var (service, client) = await this.StartService(recover: false, orchestrationType, activityType); - // run all orchestrations - { - var tasks = new Task[OrchestrationCount]; - for (int i = 0; i < OrchestrationCount; i++) - tasks[i] = client.CreateOrchestrationInstanceAsync(orchestrationType, InstanceId(i), FanOut); - await Task.WhenAll(tasks); - for (int i = 0; i < OrchestrationCount; i++) - tasks[i] = client.WaitForOrchestrationAsync(new OrchestrationInstance { InstanceId = InstanceId(i) }, TimeSpan.FromMinutes(3)); - await Task.WhenAll(tasks); - Assert.True(this.errorInTestHooks == null, $"TestHooks detected problem while starting orchestrations: {this.errorInTestHooks}"); - } + // run all orchestrations + { + var tasks = new Task[OrchestrationCount]; + for (int i = 0; i < OrchestrationCount; i++) + tasks[i] = client.CreateOrchestrationInstanceAsync(orchestrationType, InstanceId(i), FanOut); + await Task.WhenAll(tasks); + for (int i = 0; i < OrchestrationCount; i++) + tasks[i] = client.WaitForOrchestrationAsync(new OrchestrationInstance { InstanceId = InstanceId(i) }, TimeSpan.FromMinutes(3)); + await Task.WhenAll(tasks); + Assert.True(this.errorInTestHooks == null, $"TestHooks detected problem while starting orchestrations: {this.errorInTestHooks}"); + } - { - (int numPages, long memorySize) = this.cacheDebugger.MemoryTracker.GetMemorySize(); - Assert.InRange(numPages, 1, pageCount); - Assert.InRange(memorySize, 0, historyAndStatusSize * 1.1); - } + { + (int numPages, long memorySize) = this.cacheDebugger.MemoryTracker.GetMemorySize(); + Assert.InRange(numPages, 1, pageCount); + Assert.InRange(memorySize, 0, historyAndStatusSize * 1.1); + } - int emptyPageCount = 0; - int tolerance = 1; + int emptyPageCount = 0; + int tolerance = 1; - for (int i = 0; i < 4; i++) - { - emptyPageCount++; - this.cacheDebugger.MemoryTracker.SetEmptyPageCount(emptyPageCount); - await Task.Delay(TimeSpan.FromSeconds(20)); - (int numPages, long memorySize) = this.cacheDebugger.MemoryTracker.GetMemorySize(); - Assert.InRange(numPages, 1, pageCount - emptyPageCount + tolerance); - Assert.InRange(memorySize, 0, historyAndStatusSize * 1.1); - } + for (int i = 0; i < 4; i++) + { + emptyPageCount++; + this.cacheDebugger.MemoryTracker.SetEmptyPageCount(emptyPageCount); + await Task.Delay(TimeSpan.FromSeconds(20)); + (int numPages, long memorySize) = this.cacheDebugger.MemoryTracker.GetMemorySize(); + Assert.InRange(numPages, 1, pageCount - emptyPageCount + tolerance); + Assert.InRange(memorySize, 0, historyAndStatusSize * 1.1); + } - await service.StopAsync(); + await service.StopAsync(); + }); } /// @@ -510,182 +537,198 @@ public async Task CheckMemoryReduction() [Theory] [InlineData(false)] [InlineData(true)] - public async Task CheckMemoryControl(bool useSpaceConsumingOrchestrations) + public Task CheckMemoryControl(bool useSpaceConsumingOrchestrations) { - this.settings.PartitionCount = 1; - this.settings.FasterTuningParameters = new BlobManager.FasterTuningParameters() + return Common.WithTimeoutAsync(DefaultTimeout, async () => { - StoreLogPageSizeBits = 10 - }; - this.SetCheckpointFrequency(CheckpointFrequency.None); - this.settings.TestHooks.CacheDebugger.EnableSizeChecking = false; // our size checker is not entirely accurate in low-memory situationss - - Type orchestrationType, activityType; - long SizePerInstance; - object input; - int portionSize; - double uppertolerance; - double lowertolerance; + this.settings.PartitionCount = 1; + this.settings.FasterTuningParameters = new BlobManager.FasterTuningParameters() + { + StoreLogPageSizeBits = 10 + }; + this.SetCheckpointFrequency(CheckpointFrequency.None); + this.settings.TestHooks.CacheDebugger.EnableSizeChecking = false; // our size checker is not entirely accurate in low-memory situationss + + Type orchestrationType, activityType; + long SizePerInstance; + object input; + int portionSize; + double uppertolerance; + double lowertolerance; + + if (useSpaceConsumingOrchestrations) + { + this.settings.InstanceCacheSizeMB = 4; + orchestrationType = typeof(ScenarioTests.Orchestrations.SemiLargePayloadFanOutFanIn); + activityType = typeof(ScenarioTests.Activities.Echo); + int FanOut = 1; + input = FanOut; + SizePerInstance = FanOut * 50000 /* in history */ + 16000 /* in status */; + portionSize = 50; + uppertolerance = 1.1; + lowertolerance = 0; - if (useSpaceConsumingOrchestrations) - { - this.settings.InstanceCacheSizeMB = 4; - orchestrationType = typeof(ScenarioTests.Orchestrations.SemiLargePayloadFanOutFanIn); - activityType = typeof(ScenarioTests.Activities.Echo); - int FanOut = 1; - input = FanOut; - SizePerInstance = FanOut * 50000 /* in history */ + 16000 /* in status */; - portionSize = 50; - uppertolerance = 1.1; - lowertolerance = 0; + } + else + { + this.settings.InstanceCacheSizeMB = 2; + orchestrationType = typeof(ScenarioTests.Orchestrations.Hello5); + activityType = typeof(ScenarioTests.Activities.Hello); + SizePerInstance = 3610 /* empiric */; + input = null; + portionSize = 300; + uppertolerance = 1.1; + lowertolerance = 0.5; + } - } - else - { - this.settings.InstanceCacheSizeMB = 2; - orchestrationType = typeof(ScenarioTests.Orchestrations.Hello5); - activityType = typeof(ScenarioTests.Activities.Hello); - SizePerInstance = 3610 /* empiric */; - input = null; - portionSize = 300; - uppertolerance = 1.1; - lowertolerance = 0.5; - } + // start the service + var (service, client) = await this.StartService(recover: false, orchestrationType, activityType); - // start the service - var (service, client) = await this.StartService(recover: false, orchestrationType, activityType); - - int logBytesPerInstance = 2 * 40; - long memoryPerPage = ((1 << this.settings.FasterTuningParameters.StoreLogPageSizeBits.Value) / logBytesPerInstance) * SizePerInstance; - double memoryRangeTo = (this.settings.InstanceCacheSizeMB.Value - 1) * 1024 * 1024; - double memoryRangeFrom = (memoryRangeTo - memoryPerPage); - memoryRangeTo = Math.Max(memoryRangeTo, MemoryTracker.MinimumMemoryPages * memoryPerPage); - memoryRangeTo = uppertolerance * memoryRangeTo; - memoryRangeFrom = lowertolerance * memoryRangeFrom; - double pageRangeFrom = Math.Max(MemoryTracker.MinimumMemoryPages, Math.Floor(memoryRangeFrom / memoryPerPage)); - double pageRangeTo = Math.Ceiling(memoryRangeTo / memoryPerPage); - - async Task AddOrchestrationsAsync(int numOrchestrations) - { - var tasks = new Task[numOrchestrations]; - for (int i = 0; i < numOrchestrations; i++) - tasks[i] = client.CreateOrchestrationInstanceAsync(orchestrationType, Guid.NewGuid().ToString(), input); - await Task.WhenAll(tasks); - Assert.True(this.errorInTestHooks == null, $"TestHooks detected problem while starting orchestrations: {this.errorInTestHooks}"); - var tasks2 = new Task[numOrchestrations]; - for (int i = 0; i < numOrchestrations; i++) - tasks2[i] = client.WaitForOrchestrationAsync(tasks[i].Result, TimeSpan.FromMinutes(3)); - await Task.WhenAll(tasks2); - Assert.True(this.errorInTestHooks == null, $"TestHooks detected problem while waiting for orchestrations: {this.errorInTestHooks}"); - } + int logBytesPerInstance = 2 * 40; + long memoryPerPage = ((1 << this.settings.FasterTuningParameters.StoreLogPageSizeBits.Value) / logBytesPerInstance) * SizePerInstance; + double memoryRangeTo = (this.settings.InstanceCacheSizeMB.Value - 1) * 1024 * 1024; + double memoryRangeFrom = (memoryRangeTo - memoryPerPage); + memoryRangeTo = Math.Max(memoryRangeTo, MemoryTracker.MinimumMemoryPages * memoryPerPage); + memoryRangeTo = uppertolerance * memoryRangeTo; + memoryRangeFrom = lowertolerance * memoryRangeFrom; + double pageRangeFrom = Math.Max(MemoryTracker.MinimumMemoryPages, Math.Floor(memoryRangeFrom / memoryPerPage)); + double pageRangeTo = Math.Ceiling(memoryRangeTo / memoryPerPage); + + async Task AddOrchestrationsAsync(int numOrchestrations) + { + var tasks = new Task[numOrchestrations]; + for (int i = 0; i < numOrchestrations; i++) + tasks[i] = client.CreateOrchestrationInstanceAsync(orchestrationType, Guid.NewGuid().ToString(), input); + await Task.WhenAll(tasks); + Assert.True(this.errorInTestHooks == null, $"TestHooks detected problem while starting orchestrations: {this.errorInTestHooks}"); + var tasks2 = new Task[numOrchestrations]; + for (int i = 0; i < numOrchestrations; i++) + tasks2[i] = client.WaitForOrchestrationAsync(tasks[i].Result, TimeSpan.FromMinutes(3)); + await Task.WhenAll(tasks2); + Assert.True(this.errorInTestHooks == null, $"TestHooks detected problem while waiting for orchestrations: {this.errorInTestHooks}"); + } - for (int i = 0; i < 4; i++) - { - this.output("memory control ------------------- Add orchestrations"); + for (int i = 0; i < 4; i++) { - await AddOrchestrationsAsync(portionSize); + this.output("memory control ------------------- Add orchestrations"); + { + await AddOrchestrationsAsync(portionSize); - this.output("memory control -------- wait for effect"); - await Task.Delay(TimeSpan.FromSeconds(10)); + this.output("memory control -------- wait for effect"); + await Task.Delay(TimeSpan.FromSeconds(10)); - this.output("memory control -------- check memory size"); - (int numPages, long memorySize) = this.cacheDebugger.MemoryTracker.GetMemorySize(); + this.output("memory control -------- check memory size"); + (int numPages, long memorySize) = this.cacheDebugger.MemoryTracker.GetMemorySize(); - Assert.InRange(numPages, pageRangeFrom, pageRangeTo); - Assert.InRange(memorySize, memoryRangeFrom, memoryRangeTo); + Assert.InRange(numPages, pageRangeFrom, pageRangeTo); + Assert.InRange(memorySize, memoryRangeFrom, memoryRangeTo); + } } - } - await service.StopAsync(); + await service.StopAsync(); + }); } /// /// Test behavior of queries and point queries /// [Fact] - public async Task QueriesCopyToTail() + public Task QueriesCopyToTail() { - this.settings.PartitionCount = 1; - this.SetCheckpointFrequency(CheckpointFrequency.None); + return Common.WithTimeoutAsync(DefaultTimeout, async () => + { + this.settings.PartitionCount = 1; + this.SetCheckpointFrequency(CheckpointFrequency.None); - var checkpointInjector = this.settings.TestHooks.CheckpointInjector = new Faster.CheckpointInjector(this.settings.TestHooks); + var checkpointInjector = this.settings.TestHooks.CheckpointInjector = new Faster.CheckpointInjector(this.settings.TestHooks); - var orchestrationType = typeof(ScenarioTests.Orchestrations.SayHelloFanOutFanIn); - var orchestrationType2 = typeof(ScenarioTests.Orchestrations.SayHelloInline); + var orchestrationType = typeof(ScenarioTests.Orchestrations.SayHelloFanOutFanIn); + var orchestrationType2 = typeof(ScenarioTests.Orchestrations.SayHelloInline); - { - // start the service - var service = new NetheriteOrchestrationService(this.settings, this.loggerFactory); - var orchestrationService = (IOrchestrationService)service; - var orchestrationServiceClient = (IOrchestrationServiceClient)service; - var orchestrationServiceQueryClient = (IOrchestrationServiceQueryClient)service; - await orchestrationService.CreateAsync(); - await orchestrationService.StartAsync(); - var host = (TransportAbstraction.IHost)service; - Assert.Equal(1u, service.NumberPartitions); - var worker = new TaskHubWorker(service); - var client = new TaskHubClient(service); - worker.AddTaskOrchestrations(orchestrationType); - worker.AddTaskOrchestrations(orchestrationType2); - await worker.StartAsync(); - - int numExtraEntries = 0; - - // check that log contains no records - var log = await checkpointInjector.InjectAsync(log => (Faster.StoreWorker.CheckpointTrigger.None, null)); - Assert.Equal(0 * log.FixedRecordSize + numExtraEntries * extraLogEntrySize, log.TailAddress - log.BeginAddress); - - // create 100 instances - var instance = await client.CreateOrchestrationInstanceAsync(orchestrationType, "parent", 99); - await client.WaitForOrchestrationAsync(instance, TimeSpan.FromSeconds(40)); - var instances = await orchestrationServiceQueryClient.GetAllOrchestrationStatesAsync(CancellationToken.None); - numExtraEntries += 2; - Assert.Equal(100, instances.Count); - - // check that log contains 200 records - log = await checkpointInjector.InjectAsync(log => (Faster.StoreWorker.CheckpointTrigger.None, null)); - Assert.Equal(200 * log.FixedRecordSize + numExtraEntries * extraLogEntrySize, log.TailAddress - log.BeginAddress); - Assert.Equal(log.ReadOnlyAddress, log.BeginAddress); - - // take a foldover checkpoint - log = await checkpointInjector.InjectAsync(log => (Faster.StoreWorker.CheckpointTrigger.Idle, null)); - Assert.Equal(200 * log.FixedRecordSize + numExtraEntries * extraLogEntrySize, log.TailAddress - log.BeginAddress); - Assert.Equal(log.ReadOnlyAddress, log.TailAddress); - - // read all instances using a query and check that the log did not grow - // (because queries do not copy to tail) - instances = await orchestrationServiceQueryClient.GetAllOrchestrationStatesAsync(CancellationToken.None); - Assert.Equal(100, instances.Count); - log = await checkpointInjector.InjectAsync(log => (Faster.StoreWorker.CheckpointTrigger.None, null)); - Assert.Equal(200 * log.FixedRecordSize + numExtraEntries * extraLogEntrySize, log.TailAddress - log.BeginAddress); - Assert.Equal(log.ReadOnlyAddress, log.TailAddress); - - // read all instances using point queries and check that the log grew by one record per instance - // (because point queries read the InstanceState on the main session, which copies it to the tail) - var tasks = instances.Select(instance => orchestrationServiceClient.GetOrchestrationStateAsync(instance.OrchestrationInstance.InstanceId, false)); - await Task.WhenAll(tasks); - log = await checkpointInjector.InjectAsync(log => (Faster.StoreWorker.CheckpointTrigger.None, null)); - numExtraEntries += 1; - Assert.Equal(300 * log.FixedRecordSize + numExtraEntries * extraLogEntrySize, log.TailAddress - log.BeginAddress); - Assert.Equal(log.ReadOnlyAddress, log.TailAddress - 100 * log.FixedRecordSize - 1 * extraLogEntrySize); - - // doing the same again has no effect - // (because all instances are already in the mutable section) - tasks = instances.Select(instance => orchestrationServiceClient.GetOrchestrationStateAsync(instance.OrchestrationInstance.InstanceId, false)); - await Task.WhenAll(tasks); - log = await checkpointInjector.InjectAsync(log => (Faster.StoreWorker.CheckpointTrigger.None, null)); - Assert.Equal(300 * log.FixedRecordSize + numExtraEntries * extraLogEntrySize, log.TailAddress - log.BeginAddress); - Assert.Equal(log.ReadOnlyAddress, log.TailAddress - 100 * log.FixedRecordSize - 1 * extraLogEntrySize); - - // take a foldover checkpoint - // this moves the readonly section back to the end - log = await checkpointInjector.InjectAsync(log => (Faster.StoreWorker.CheckpointTrigger.Idle, null)); - Assert.Equal(300 * log.FixedRecordSize + numExtraEntries * extraLogEntrySize, log.TailAddress - log.BeginAddress); - Assert.Equal(log.ReadOnlyAddress, log.TailAddress); - - // stop the service - await orchestrationService.StopAsync(); - } + { + this.Progress("start the service"); + var service = new NetheriteOrchestrationService(this.settings, this.loggerFactory); + var orchestrationService = (IOrchestrationService)service; + var orchestrationServiceClient = (IOrchestrationServiceClient)service; + var orchestrationServiceQueryClient = (IOrchestrationServiceQueryClient)service; + await orchestrationService.CreateAsync(); + await orchestrationService.StartAsync(); + var host = (TransportAbstraction.IHost)service; + Assert.Equal(1u, service.NumberPartitions); + var worker = new TaskHubWorker(service); + var client = new TaskHubClient(service); + worker.AddTaskOrchestrations(orchestrationType); + worker.AddTaskOrchestrations(orchestrationType2); + await worker.StartAsync(); + + int numExtraEntries = 0; + + this.Progress("check that log contains no records"); + var log = await checkpointInjector.InjectAsync(log => (Faster.StoreWorker.CheckpointTrigger.None, null)); + this.Progress("Injection complete"); + Assert.Equal(0 * log.FixedRecordSize + numExtraEntries * extraLogEntrySize, log.TailAddress - log.BeginAddress); + + this.Progress("create 100 instances"); + var instance = await client.CreateOrchestrationInstanceAsync(orchestrationType, "parent", 99); + await client.WaitForOrchestrationAsync(instance, TimeSpan.FromSeconds(40)); + var instances = await orchestrationServiceQueryClient.GetAllOrchestrationStatesAsync(CancellationToken.None); + this.Progress("Injection complete"); + numExtraEntries += 2; + Assert.Equal(100, instances.Count); + + this.Progress("check that log contains 100 records"); + log = await checkpointInjector.InjectAsync(log => (Faster.StoreWorker.CheckpointTrigger.None, null)); + this.Progress("Injection complete"); + Assert.Equal(200 * log.FixedRecordSize + numExtraEntries * extraLogEntrySize, log.TailAddress - log.BeginAddress); + Assert.Equal(log.ReadOnlyAddress, log.BeginAddress); + + this.Progress("take a foldover checkpoint"); + log = await checkpointInjector.InjectAsync(log => (Faster.StoreWorker.CheckpointTrigger.Idle, null)); + this.Progress("Injection complete"); + Assert.Equal(200 * log.FixedRecordSize + numExtraEntries * extraLogEntrySize, log.TailAddress - log.BeginAddress); + Assert.Equal(log.ReadOnlyAddress, log.TailAddress); + + this.Progress("read all instances using a query and check that the log did not grow"); + this.Progress("(because queries do not copy to tail)"); + instances = await orchestrationServiceQueryClient.GetAllOrchestrationStatesAsync(CancellationToken.None); + this.Progress("query complete"); + Assert.Equal(100, instances.Count); + log = await checkpointInjector.InjectAsync(log => (Faster.StoreWorker.CheckpointTrigger.None, null)); + Assert.Equal(200 * log.FixedRecordSize + numExtraEntries * extraLogEntrySize, log.TailAddress - log.BeginAddress); + Assert.Equal(log.ReadOnlyAddress, log.TailAddress); + + this.Progress("read all instances using point queries and check that the log grew by one record per instance"); + this.Progress("(because point queries read the InstanceState on the main session, which copies it to the tail)"); + var tasks = instances.Select(instance => orchestrationServiceClient.GetOrchestrationStateAsync(instance.OrchestrationInstance.InstanceId, false)); + await Task.WhenAll(tasks); + this.Progress("queries complete"); + log = await checkpointInjector.InjectAsync(log => (Faster.StoreWorker.CheckpointTrigger.None, null)); + this.Progress("Injection complete"); + numExtraEntries += 1; + Assert.Equal(300 * log.FixedRecordSize + numExtraEntries * extraLogEntrySize, log.TailAddress - log.BeginAddress); + Assert.Equal(log.ReadOnlyAddress, log.TailAddress - 100 * log.FixedRecordSize - 1 * extraLogEntrySize); + + this.Progress("doing the same again has no effect"); + this.Progress("because all instances are already in the mutable section)"); + tasks = instances.Select(instance => orchestrationServiceClient.GetOrchestrationStateAsync(instance.OrchestrationInstance.InstanceId, false)); + await Task.WhenAll(tasks); + log = await checkpointInjector.InjectAsync(log => (Faster.StoreWorker.CheckpointTrigger.None, null)); + this.Progress("Injection complete"); + Assert.Equal(300 * log.FixedRecordSize + numExtraEntries * extraLogEntrySize, log.TailAddress - log.BeginAddress); + Assert.Equal(log.ReadOnlyAddress, log.TailAddress - 100 * log.FixedRecordSize - 1 * extraLogEntrySize); + + this.Progress("take a foldover checkpoint"); + this.Progress("this moves the readonly section back to the end"); + log = await checkpointInjector.InjectAsync(log => (Faster.StoreWorker.CheckpointTrigger.Idle, null)); + this.Progress("Injection complete"); + Assert.Equal(300 * log.FixedRecordSize + numExtraEntries * extraLogEntrySize, log.TailAddress - log.BeginAddress); + Assert.Equal(log.ReadOnlyAddress, log.TailAddress); + + this.Progress("stop the service"); + await orchestrationService.StopAsync(); + this.Progress("service stopped"); + } + }); } @@ -693,187 +736,193 @@ public async Task QueriesCopyToTail() /// Test log compaction /// [Fact] - public async Task Compaction() + public Task Compaction() { - this.settings.PartitionCount = 1; - this.SetCheckpointFrequency(CheckpointFrequency.None); - - var checkpointInjector = this.settings.TestHooks.CheckpointInjector = new Faster.CheckpointInjector(this.settings.TestHooks); + return Common.WithTimeoutAsync(DefaultTimeout, async () => + { + this.settings.PartitionCount = 1; + this.SetCheckpointFrequency(CheckpointFrequency.None); - var orchestrationType = typeof(ScenarioTests.Orchestrations.SayHelloFanOutFanIn); - var orchestrationType2 = typeof(ScenarioTests.Orchestrations.SayHelloInline); + var checkpointInjector = this.settings.TestHooks.CheckpointInjector = new Faster.CheckpointInjector(this.settings.TestHooks); - long compactUntil = 0; - { - // start the service - var service = new NetheriteOrchestrationService(this.settings, this.loggerFactory); - var orchestrationService = (IOrchestrationService)service; - var orchestrationServiceClient = (IOrchestrationServiceClient)service; - var orchestrationServiceQueryClient = (IOrchestrationServiceQueryClient)service; - await orchestrationService.CreateAsync(); - await orchestrationService.StartAsync(); - var host = (TransportAbstraction.IHost)service; - Assert.Equal(1u, service.NumberPartitions); - var worker = new TaskHubWorker(service); - var client = new TaskHubClient(service); - worker.AddTaskOrchestrations(orchestrationType); - worker.AddTaskOrchestrations(orchestrationType2); - await worker.StartAsync(); - - // create 100 instances - var instance = await client.CreateOrchestrationInstanceAsync(orchestrationType, "parent", 99); - await client.WaitForOrchestrationAsync(instance, TimeSpan.FromSeconds(40)); - var instances = await orchestrationServiceQueryClient.GetAllOrchestrationStatesAsync(CancellationToken.None); - Assert.Equal(100, instances.Count); - - int numExtraEntries = 1; - - // repeat foldover and copy to tail to inflate the log - for (int i = 0; i < 4; i++) + var orchestrationType = typeof(ScenarioTests.Orchestrations.SayHelloFanOutFanIn); + var orchestrationType2 = typeof(ScenarioTests.Orchestrations.SayHelloInline); + + long compactUntil = 0; { - // take a foldover checkpoint - var log2 = await checkpointInjector.InjectAsync(log => (Faster.StoreWorker.CheckpointTrigger.Idle, null)); - numExtraEntries += 1; - Assert.Equal((200 + (100 * i)) * log2.FixedRecordSize + numExtraEntries * extraLogEntrySize, log2.TailAddress - log2.BeginAddress); - Assert.Equal(log2.ReadOnlyAddress, log2.TailAddress); + // start the service + var service = new NetheriteOrchestrationService(this.settings, this.loggerFactory); + var orchestrationService = (IOrchestrationService)service; + var orchestrationServiceClient = (IOrchestrationServiceClient)service; + var orchestrationServiceQueryClient = (IOrchestrationServiceQueryClient)service; + await orchestrationService.CreateAsync(); + await orchestrationService.StartAsync(); + var host = (TransportAbstraction.IHost)service; + Assert.Equal(1u, service.NumberPartitions); + var worker = new TaskHubWorker(service); + var client = new TaskHubClient(service); + worker.AddTaskOrchestrations(orchestrationType); + worker.AddTaskOrchestrations(orchestrationType2); + await worker.StartAsync(); + + // create 100 instances + var instance = await client.CreateOrchestrationInstanceAsync(orchestrationType, "parent", 99); + await client.WaitForOrchestrationAsync(instance, TimeSpan.FromSeconds(40)); + var instances = await orchestrationServiceQueryClient.GetAllOrchestrationStatesAsync(CancellationToken.None); + Assert.Equal(100, instances.Count); + + int numExtraEntries = 1; + + // repeat foldover and copy to tail to inflate the log + for (int i = 0; i < 4; i++) + { + // take a foldover checkpoint + var log2 = await checkpointInjector.InjectAsync(log => (Faster.StoreWorker.CheckpointTrigger.Idle, null)); + numExtraEntries += 1; + Assert.Equal((200 + (100 * i)) * log2.FixedRecordSize + numExtraEntries * extraLogEntrySize, log2.TailAddress - log2.BeginAddress); + Assert.Equal(log2.ReadOnlyAddress, log2.TailAddress); + + // read all instances using point queries to force copy to tail + var tasks = instances.Select(instance => orchestrationServiceClient.GetOrchestrationStateAsync(instance.OrchestrationInstance.InstanceId, false)); + await Task.WhenAll(tasks); + } - // read all instances using point queries to force copy to tail - var tasks = instances.Select(instance => orchestrationServiceClient.GetOrchestrationStateAsync(instance.OrchestrationInstance.InstanceId, false)); - await Task.WhenAll(tasks); + // do log compaction + var log = await checkpointInjector.InjectAsync(log => + { + compactUntil = 500 * log.FixedRecordSize + log.BeginAddress + numExtraEntries * extraLogEntrySize; + Assert.Equal(compactUntil, log.SafeReadOnlyAddress); + return (Faster.StoreWorker.CheckpointTrigger.Compaction, compactUntil); + }); + + // check that the compaction had the desired effect + numExtraEntries = 2; + Assert.Equal(200 * log.FixedRecordSize + numExtraEntries * extraLogEntrySize, log.TailAddress - log.BeginAddress); + Assert.Equal(compactUntil, log.BeginAddress); + Assert.Equal(log.ReadOnlyAddress, log.TailAddress); + + // stop the service + await orchestrationService.StopAsync(); } - - // do log compaction - var log = await checkpointInjector.InjectAsync(log => { - compactUntil = 500 * log.FixedRecordSize + log.BeginAddress + numExtraEntries * extraLogEntrySize; - Assert.Equal(compactUntil, log.SafeReadOnlyAddress); - return (Faster.StoreWorker.CheckpointTrigger.Compaction, compactUntil); - }); - - // check that the compaction had the desired effect - numExtraEntries = 2; - Assert.Equal(200 * log.FixedRecordSize + numExtraEntries * extraLogEntrySize, log.TailAddress - log.BeginAddress); - Assert.Equal(compactUntil, log.BeginAddress); - Assert.Equal(log.ReadOnlyAddress, log.TailAddress); - - // stop the service - await orchestrationService.StopAsync(); - } - { - // recover the service - var service = new NetheriteOrchestrationService(this.settings, this.loggerFactory); - var orchestrationService = (IOrchestrationService)service; - var orchestrationServiceQueryClient = (IOrchestrationServiceQueryClient)service; - await orchestrationService.StartAsync(); - var host = (TransportAbstraction.IHost)service; - Assert.Equal(1u, service.NumberPartitions); - - int numExtraEntries = 2; - - // check the log positions - var log = await checkpointInjector.InjectAsync(log => (Faster.StoreWorker.CheckpointTrigger.None, null)); - Assert.Equal(200 * log.FixedRecordSize + numExtraEntries * extraLogEntrySize, log.TailAddress - log.BeginAddress); - Assert.Equal(compactUntil, log.BeginAddress); - Assert.Equal(log.ReadOnlyAddress, log.TailAddress); - - // check the instance count - var instances = await orchestrationServiceQueryClient.GetAllOrchestrationStatesAsync(CancellationToken.None); - Assert.Equal(100, instances.Count); - - // stop the service - await orchestrationService.StopAsync(); - } + // recover the service + var service = new NetheriteOrchestrationService(this.settings, this.loggerFactory); + var orchestrationService = (IOrchestrationService)service; + var orchestrationServiceQueryClient = (IOrchestrationServiceQueryClient)service; + await orchestrationService.StartAsync(); + var host = (TransportAbstraction.IHost)service; + Assert.Equal(1u, service.NumberPartitions); + + int numExtraEntries = 2; + + // check the log positions + var log = await checkpointInjector.InjectAsync(log => (Faster.StoreWorker.CheckpointTrigger.None, null)); + Assert.Equal(200 * log.FixedRecordSize + numExtraEntries * extraLogEntrySize, log.TailAddress - log.BeginAddress); + Assert.Equal(compactUntil, log.BeginAddress); + Assert.Equal(log.ReadOnlyAddress, log.TailAddress); + + // check the instance count + var instances = await orchestrationServiceQueryClient.GetAllOrchestrationStatesAsync(CancellationToken.None); + Assert.Equal(100, instances.Count); + + // stop the service + await orchestrationService.StopAsync(); + } + }); } /// /// Test log compaction that fails right after compaction /// [Fact] - public async Task CompactThenFail() + public Task CompactThenFail() { - this.settings.PartitionCount = 1; - this.SetCheckpointFrequency(CheckpointFrequency.None); - - var checkpointInjector = this.settings.TestHooks.CheckpointInjector = new Faster.CheckpointInjector(this.settings.TestHooks); - - var orchestrationType = typeof(ScenarioTests.Orchestrations.SayHelloFanOutFanIn); - var orchestrationType2 = typeof(ScenarioTests.Orchestrations.SayHelloInline); - - long currentTail = 0; - long currentBegin = 0; - - long compactUntil = 0; + return Common.WithTimeoutAsync(DefaultTimeout, async () => { - // start the service - var service = new NetheriteOrchestrationService(this.settings, this.loggerFactory); - var orchestrationService = (IOrchestrationService)service; - var orchestrationServiceClient = (IOrchestrationServiceClient)service; - var orchestrationServiceQueryClient = (IOrchestrationServiceQueryClient)service; - await orchestrationService.CreateAsync(); - await orchestrationService.StartAsync(); - var host = (TransportAbstraction.IHost)service; - Assert.Equal(1u, service.NumberPartitions); - var worker = new TaskHubWorker(service); - var client = new TaskHubClient(service); - worker.AddTaskOrchestrations(orchestrationType); - worker.AddTaskOrchestrations(orchestrationType2); - await worker.StartAsync(); + this.settings.PartitionCount = 1; + this.SetCheckpointFrequency(CheckpointFrequency.None); - // create 100 instances - var instance = await client.CreateOrchestrationInstanceAsync(orchestrationType, "parent", 99); - await client.WaitForOrchestrationAsync(instance, TimeSpan.FromSeconds(40)); - var instances = await orchestrationServiceQueryClient.GetAllOrchestrationStatesAsync(CancellationToken.None); - Assert.Equal(100, instances.Count); + var checkpointInjector = this.settings.TestHooks.CheckpointInjector = new Faster.CheckpointInjector(this.settings.TestHooks); - int numExtraEntries = 1; + var orchestrationType = typeof(ScenarioTests.Orchestrations.SayHelloFanOutFanIn); + var orchestrationType2 = typeof(ScenarioTests.Orchestrations.SayHelloInline); + long currentTail = 0; + long currentBegin = 0; - // repeat foldover and copy to tail to inflate the log - for (int i = 0; i < 4; i++) + long compactUntil = 0; { - // take a foldover checkpoint - var log2 = await checkpointInjector.InjectAsync(log => (Faster.StoreWorker.CheckpointTrigger.Idle, null)); - numExtraEntries += 1; - Assert.Equal((200 + (100 * i)) * log2.FixedRecordSize + numExtraEntries * extraLogEntrySize, log2.TailAddress - log2.BeginAddress); - Assert.Equal(log2.ReadOnlyAddress, log2.TailAddress); + // start the service + var service = new NetheriteOrchestrationService(this.settings, this.loggerFactory); + var orchestrationService = (IOrchestrationService)service; + var orchestrationServiceClient = (IOrchestrationServiceClient)service; + var orchestrationServiceQueryClient = (IOrchestrationServiceQueryClient)service; + await orchestrationService.CreateAsync(); + await orchestrationService.StartAsync(); + var host = (TransportAbstraction.IHost)service; + Assert.Equal(1u, service.NumberPartitions); + var worker = new TaskHubWorker(service); + var client = new TaskHubClient(service); + worker.AddTaskOrchestrations(orchestrationType); + worker.AddTaskOrchestrations(orchestrationType2); + await worker.StartAsync(); + + // create 100 instances + var instance = await client.CreateOrchestrationInstanceAsync(orchestrationType, "parent", 99); + await client.WaitForOrchestrationAsync(instance, TimeSpan.FromSeconds(40)); + var instances = await orchestrationServiceQueryClient.GetAllOrchestrationStatesAsync(CancellationToken.None); + Assert.Equal(100, instances.Count); + + int numExtraEntries = 1; + + + // repeat foldover and copy to tail to inflate the log + for (int i = 0; i < 4; i++) + { + // take a foldover checkpoint + var log2 = await checkpointInjector.InjectAsync(log => (Faster.StoreWorker.CheckpointTrigger.Idle, null)); + numExtraEntries += 1; + Assert.Equal((200 + (100 * i)) * log2.FixedRecordSize + numExtraEntries * extraLogEntrySize, log2.TailAddress - log2.BeginAddress); + Assert.Equal(log2.ReadOnlyAddress, log2.TailAddress); + + currentTail = log2.TailAddress; + currentBegin = log2.BeginAddress; + + // read all instances using point queries to force copy to tail + var tasks = instances.Select(instance => orchestrationServiceClient.GetOrchestrationStateAsync(instance.OrchestrationInstance.InstanceId, false)); + await Task.WhenAll(tasks); + } - currentTail = log2.TailAddress; - currentBegin = log2.BeginAddress; + // do log compaction + var log = await checkpointInjector.InjectAsync(log => + { + compactUntil = 500 * log.FixedRecordSize + log.BeginAddress + numExtraEntries * extraLogEntrySize; + Assert.Equal(compactUntil, log.SafeReadOnlyAddress); + return (Faster.StoreWorker.CheckpointTrigger.Compaction, compactUntil); + }, + injectFailureAfterCompaction:true); - // read all instances using point queries to force copy to tail - var tasks = instances.Select(instance => orchestrationServiceClient.GetOrchestrationStateAsync(instance.OrchestrationInstance.InstanceId, false)); - await Task.WhenAll(tasks); + await orchestrationService.StopAsync(); } - // do log compaction - var log = await checkpointInjector.InjectAsync(log => { - compactUntil = 500 * log.FixedRecordSize + log.BeginAddress + numExtraEntries * extraLogEntrySize; - Assert.Equal(compactUntil, log.SafeReadOnlyAddress); - return (Faster.StoreWorker.CheckpointTrigger.Compaction, compactUntil); - }, - injectFailureAfterCompaction:true); - - await orchestrationService.StopAsync(); - } - - { - // recover the service - var service = new NetheriteOrchestrationService(this.settings, this.loggerFactory); - var orchestrationService = (IOrchestrationService)service; - var orchestrationServiceClient = (IOrchestrationServiceClient)service; - await orchestrationService.StartAsync(); - Assert.Equal(this.settings.PartitionCount, (int)service.NumberPartitions); - var worker = new TaskHubWorker(service); - var client = new TaskHubClient(service); - await worker.StartAsync(); - - // check that begin and tail are the same - var log = await checkpointInjector.InjectAsync(log => (Faster.StoreWorker.CheckpointTrigger.None, null)); - - Debug.Assert(log.BeginAddress == currentBegin); - Debug.Assert(log.TailAddress == currentTail); - } + // recover the service + var service = new NetheriteOrchestrationService(this.settings, this.loggerFactory); + var orchestrationService = (IOrchestrationService)service; + var orchestrationServiceClient = (IOrchestrationServiceClient)service; + await orchestrationService.StartAsync(); + Assert.Equal(this.settings.PartitionCount, (int)service.NumberPartitions); + var worker = new TaskHubWorker(service); + var client = new TaskHubClient(service); + await worker.StartAsync(); + + // check that begin and tail are the same + var log = await checkpointInjector.InjectAsync(log => (Faster.StoreWorker.CheckpointTrigger.None, null)); + + Debug.Assert(log.BeginAddress == currentBegin); + Debug.Assert(log.TailAddress == currentTail); + } + }); } } } \ No newline at end of file diff --git a/test/DurableTask.Netherite.Tests/HostFixture.cs b/test/DurableTask.Netherite.Tests/HostFixture.cs index aca6f29b..6e1ca4bf 100644 --- a/test/DurableTask.Netherite.Tests/HostFixture.cs +++ b/test/DurableTask.Netherite.Tests/HostFixture.cs @@ -17,9 +17,9 @@ namespace DurableTask.Netherite.Tests /// public class HostFixture : IDisposable { - readonly TestTraceListener traceListener; - readonly XunitLoggerProvider loggerProvider; - readonly CacheDebugger cacheDebugger; + TestTraceListener traceListener; + XunitLoggerProvider loggerProvider; + CacheDebugger cacheDebugger; internal TestOrchestrationHost Host { get; private set; } internal ILoggerFactory LoggerFactory { get; private set; } @@ -35,30 +35,33 @@ public HostFixture() internal HostFixture(NetheriteOrchestrationServiceSettings settings, bool useCacheDebugger, bool useReplayChecker, int? restrictMemory, Action output) { - this.LoggerFactory = new LoggerFactory(); - this.loggerProvider = new XunitLoggerProvider(); - this.LoggerFactory.AddProvider(this.loggerProvider); - this.traceListener = new TestTraceListener() { Output = output }; - Trace.Listeners.Add(this.traceListener); - string timestamp = DateTime.UtcNow.ToString("yyyyMMdd-HHmmss-fffffff"); - settings.HubName = $"HostFixture-{timestamp}"; - settings.PartitionManagement = PartitionManagementOptions.EventProcessorHost; - settings.InstanceCacheSizeMB = restrictMemory; - if (useCacheDebugger) + Common.WithTimeout(TimeSpan.FromMinutes(1), () => { - this.cacheDebugger = settings.TestHooks.CacheDebugger = new Faster.CacheDebugger(settings.TestHooks); - } - if (useReplayChecker) - { - settings.TestHooks.ReplayChecker = new Faster.ReplayChecker(settings.TestHooks); - } - settings.TestHooks.OnError += (message) => - { - Trace.WriteLine($"TESTHOOKS: {message}"); - this.TestHooksError ??= message; - }; - // start the host - this.Host = new TestOrchestrationHost(settings, this.LoggerFactory); + this.LoggerFactory = new LoggerFactory(); + this.loggerProvider = new XunitLoggerProvider(); + this.LoggerFactory.AddProvider(this.loggerProvider); + this.traceListener = new TestTraceListener() { Output = output }; + Trace.Listeners.Add(this.traceListener); + string timestamp = DateTime.UtcNow.ToString("yyyyMMdd-HHmmss-fffffff"); + settings.HubName = $"HostFixture-{timestamp}"; + settings.PartitionManagement = PartitionManagementOptions.EventProcessorHost; + settings.InstanceCacheSizeMB = restrictMemory; + if (useCacheDebugger) + { + this.cacheDebugger = settings.TestHooks.CacheDebugger = new Faster.CacheDebugger(settings.TestHooks); + } + if (useReplayChecker) + { + settings.TestHooks.ReplayChecker = new Faster.ReplayChecker(settings.TestHooks); + } + settings.TestHooks.OnError += (message) => + { + Trace.WriteLine($"TESTHOOKS: {message}"); + this.TestHooksError ??= message; + }; + // start the host + this.Host = new TestOrchestrationHost(settings, this.LoggerFactory); + }); } public static async Task StartNew(NetheriteOrchestrationServiceSettings settings, bool useCacheDebugger, bool useReplayChecker, int? restrictMemory, TimeSpan timeout, Action output) @@ -86,9 +89,12 @@ public void DumpCacheDebugger() public void Dispose() { - this.Host.StopAsync(false).Wait(); - this.Host.Dispose(); - Trace.Listeners.Remove(this.traceListener); + Common.WithTimeout(TimeSpan.FromMinutes(1), () => + { + this.Host.StopAsync(false).Wait(); + this.Host.Dispose(); + Trace.Listeners.Remove(this.traceListener); + }); } public bool HasError(out string error) diff --git a/test/DurableTask.Netherite.Tests/OrchestrationServiceTests.cs b/test/DurableTask.Netherite.Tests/OrchestrationServiceTests.cs index 862c9c7c..835ba934 100644 --- a/test/DurableTask.Netherite.Tests/OrchestrationServiceTests.cs +++ b/test/DurableTask.Netherite.Tests/OrchestrationServiceTests.cs @@ -28,21 +28,27 @@ public OrchestrationServiceTests(ITestOutputHelper outputHelper) } [Fact] - public async Task StopAsync_IsIdempotent() + public Task StopAsync_IsIdempotent() { - int numStops = 3; - IOrchestrationService service = TestConstants.GetTestOrchestrationService(this.loggerFactory); - for (int i =0; i < numStops; i++) + return Common.WithTimeoutAsync(TimeSpan.FromMinutes(1), async () => { - await service.StopAsync(); - } + int numStops = 3; + IOrchestrationService service = TestConstants.GetTestOrchestrationService(this.loggerFactory); + for (int i = 0; i < numStops; i++) + { + await service.StopAsync(); + } + }); } [Fact] - public async Task UnstartedService_CanBeSafelyStopped() + public Task UnstartedService_CanBeSafelyStopped() { - IOrchestrationService service = TestConstants.GetTestOrchestrationService(this.loggerFactory); - await service.StopAsync(); + return Common.WithTimeoutAsync(TimeSpan.FromMinutes(1), async () => + { + IOrchestrationService service = TestConstants.GetTestOrchestrationService(this.loggerFactory); + await service.StopAsync(); + }); } } } diff --git a/test/DurableTask.Netherite.Tests/QueryTests.cs b/test/DurableTask.Netherite.Tests/QueryTests.cs index 4dcde788..1bd167ce 100644 --- a/test/DurableTask.Netherite.Tests/QueryTests.cs +++ b/test/DurableTask.Netherite.Tests/QueryTests.cs @@ -28,6 +28,8 @@ public partial class QueryTests : IClassFixture, IDisposable readonly Action output; ITestOutputHelper outputHelper; + static readonly TimeSpan DefaultTimeout = TimeSpan.FromMinutes(1); + public QueryTests(HostFixture fixture, ITestOutputHelper outputHelper) { this.fixture = fixture; @@ -54,7 +56,6 @@ public void Dispose() Assert.False(this.fixture.HasError(out var error), $"detected test failure: {error}"); - // purge all instances after each test // this helps to catch "bad states" (e.g. hung workers) caused by the tests if (!this.host.PurgeAllAsync().Wait(TimeSpan.FromMinutes(3))) @@ -74,159 +75,170 @@ public void Dispose() /// Ported from AzureStorageScenarioTests /// [Fact] - public async Task QueryOrchestrationInstancesByDateRange() + public Task QueryOrchestrationInstancesByDateRange() { - const int numInstances = 3; - string getPrefix(int ii) => $"@inst{ii}@__"; - - // Start multiple orchestrations. Use 1-based to avoid confusion where we use explicit values in asserts. - for (var ii = 1; ii <= numInstances; ++ii) - { - var client = await this.host.StartOrchestrationAsync(typeof(Orchestrations.SayHelloInline), $"world {ii}", $"{getPrefix(ii)}__{Guid.NewGuid()}"); - await Task.Delay(100); // To ensure time separation for the date time range queries - await client.WaitForCompletionAsync(TimeSpan.FromSeconds(30)); - } - - var results = await this.host.GetAllOrchestrationInstancesAsync(); - Assert.Equal(numInstances, results.Count); - for (var ii = 1; ii <= numInstances; ++ii) - { - Assert.NotNull(results.SingleOrDefault(r => r.Output == $"\"Hello, world {ii}!\"")); - } - - // Select the middle instance for the time range query. - var middleInstance = results.SingleOrDefault(r => r.Output == $"\"Hello, world 2!\""); - void assertIsMiddleInstance(IList testStates) + return Common.WithTimeoutAsync(DefaultTimeout, async () => { - Assert.Equal(1, testStates.Count); - Assert.Equal(testStates[0].OrchestrationInstance.InstanceId, middleInstance.OrchestrationInstance.InstanceId); - } - - assertIsMiddleInstance(await this.host.GetOrchestrationStateAsync(CreatedTimeFrom: middleInstance.CreatedTime, - CreatedTimeTo: middleInstance.CreatedTime.AddMilliseconds(50))); - assertIsMiddleInstance(await this.host.GetOrchestrationStateAsync(InstanceIdPrefix: getPrefix(2))); + const int numInstances = 3; + string getPrefix(int ii) => $"@inst{ii}@__"; + + // Start multiple orchestrations. Use 1-based to avoid confusion where we use explicit values in asserts. + for (var ii = 1; ii <= numInstances; ++ii) + { + var client = await this.host.StartOrchestrationAsync(typeof(Orchestrations.SayHelloInline), $"world {ii}", $"{getPrefix(ii)}__{Guid.NewGuid()}"); + await Task.Delay(100); // To ensure time separation for the date time range queries + await client.WaitForCompletionAsync(TimeSpan.FromSeconds(30)); + } + + var results = await this.host.GetAllOrchestrationInstancesAsync(); + Assert.Equal(numInstances, results.Count); + for (var ii = 1; ii <= numInstances; ++ii) + { + Assert.NotNull(results.SingleOrDefault(r => r.Output == $"\"Hello, world {ii}!\"")); + } + + // Select the middle instance for the time range query. + var middleInstance = results.SingleOrDefault(r => r.Output == $"\"Hello, world 2!\""); + void assertIsMiddleInstance(IList testStates) + { + Assert.Equal(1, testStates.Count); + Assert.Equal(testStates[0].OrchestrationInstance.InstanceId, middleInstance.OrchestrationInstance.InstanceId); + } + + assertIsMiddleInstance(await this.host.GetOrchestrationStateAsync(CreatedTimeFrom: middleInstance.CreatedTime, + CreatedTimeTo: middleInstance.CreatedTime.AddMilliseconds(50))); + assertIsMiddleInstance(await this.host.GetOrchestrationStateAsync(InstanceIdPrefix: getPrefix(2))); + }); } /// /// Validate query functions. /// [Fact] - public async Task QueryOrchestrationInstanceByRuntimeStatus() + public Task QueryOrchestrationInstanceByRuntimeStatus() { - // Reuse counter as it provides a wait for the actor to complete itself. - var client = await this.host.StartOrchestrationAsync(typeof(Orchestrations.Counter), 0); - - // Need to wait for the instance to start before sending events to it. - // TODO: This requirement may not be ideal and should be revisited. - await client.WaitForStartupAsync(TimeSpan.FromSeconds(10)); - Trace.TraceInformation($"TestProgress: Counter is running."); - - // We should have one orchestration state - var instanceStates = await this.host.GetAllOrchestrationInstancesAsync(); - Assert.Equal(1, instanceStates.Count); - - var inProgressStatus = new[] { OrchestrationStatus.Running, OrchestrationStatus.ContinuedAsNew }; - var completedStatus = new[] { OrchestrationStatus.Completed }; - - async Task assertCounts(int running, int completed) + return Common.WithTimeoutAsync(DefaultTimeout, async () => { - var runningStates = await this.host.GetOrchestrationStateAsync(RuntimeStatus: inProgressStatus); - Assert.Equal(running, runningStates.Count); - var completedStates = await this.host.GetOrchestrationStateAsync(RuntimeStatus: completedStatus); - Assert.Equal(completed, completedStates.Count); - } - - // Make sure the client and instance are still running and didn't complete early (or fail). - var status = await client.GetStateAsync(); - Assert.NotNull(status); - Assert.Contains(status.OrchestrationStatus, inProgressStatus); - await assertCounts(1, 0); - - // The end message will cause the actor to complete itself. - Trace.TraceInformation($"TestProgress: Sending event to Counter."); - await client.RaiseEventAsync(Orchestrations.Counter.OpEventName, Orchestrations.Counter.OpEnd); - status = await client.WaitForCompletionAsync(TimeSpan.FromSeconds(30)); - Trace.TraceInformation($"TestProgress: Counter completed."); - - // The client and instance should be Completed - Assert.NotNull(status); - Assert.Contains(status.OrchestrationStatus, completedStatus); - await assertCounts(0, 1); + // Reuse counter as it provides a wait for the actor to complete itself. + var client = await this.host.StartOrchestrationAsync(typeof(Orchestrations.Counter), 0); + + // Need to wait for the instance to start before sending events to it. + // TODO: This requirement may not be ideal and should be revisited. + await client.WaitForStartupAsync(TimeSpan.FromSeconds(10)); + Trace.TraceInformation($"TestProgress: Counter is running."); + + // We should have one orchestration state + var instanceStates = await this.host.GetAllOrchestrationInstancesAsync(); + Assert.Equal(1, instanceStates.Count); + + var inProgressStatus = new[] { OrchestrationStatus.Running, OrchestrationStatus.ContinuedAsNew }; + var completedStatus = new[] { OrchestrationStatus.Completed }; + + async Task assertCounts(int running, int completed) + { + var runningStates = await this.host.GetOrchestrationStateAsync(RuntimeStatus: inProgressStatus); + Assert.Equal(running, runningStates.Count); + var completedStates = await this.host.GetOrchestrationStateAsync(RuntimeStatus: completedStatus); + Assert.Equal(completed, completedStates.Count); + } + + // Make sure the client and instance are still running and didn't complete early (or fail). + var status = await client.GetStateAsync(); + Assert.NotNull(status); + Assert.Contains(status.OrchestrationStatus, inProgressStatus); + await assertCounts(1, 0); + + // The end message will cause the actor to complete itself. + Trace.TraceInformation($"TestProgress: Sending event to Counter."); + await client.RaiseEventAsync(Orchestrations.Counter.OpEventName, Orchestrations.Counter.OpEnd); + status = await client.WaitForCompletionAsync(TimeSpan.FromSeconds(30)); + Trace.TraceInformation($"TestProgress: Counter completed."); + + // The client and instance should be Completed + Assert.NotNull(status); + Assert.Contains(status.OrchestrationStatus, completedStatus); + await assertCounts(0, 1); + }); } [Fact] - public async void NoInstancesCreated() + public Task NoInstancesCreated() { - var instanceStates = await this.host.GetAllOrchestrationInstancesAsync(); - Assert.Equal(0, instanceStates.Count); + return Common.WithTimeoutAsync(DefaultTimeout, async () => + { + var instanceStates = await this.host.GetAllOrchestrationInstancesAsync(); + Assert.Equal(0, instanceStates.Count); + }); } - [Fact] - public async Task PurgeInstanceHistoryForTimePeriodDeletePartially() + public Task PurgeInstanceHistoryForTimePeriodDeletePartially() { - DateTime startDateTime = DateTime.Now; - string firstInstanceId = Guid.NewGuid().ToString(); - TestOrchestrationClient client = await this.host.StartOrchestrationAsync(typeof(Orchestrations.FanOutFanIn), 50, firstInstanceId); - await client.WaitForCompletionAsync(TimeSpan.FromSeconds(30)); - DateTime endDateTime = DateTime.Now; - await Task.Delay(5000); - string secondInstanceId = Guid.NewGuid().ToString(); - client = await this.host.StartOrchestrationAsync(typeof(Orchestrations.FanOutFanIn), 50, secondInstanceId); - await client.WaitForCompletionAsync(TimeSpan.FromSeconds(30)); - string thirdInstanceId = Guid.NewGuid().ToString(); - client = await this.host.StartOrchestrationAsync(typeof(Orchestrations.FanOutFanIn), 50, thirdInstanceId); - await client.WaitForCompletionAsync(TimeSpan.FromSeconds(30)); - - IList results = await this.host.GetAllOrchestrationInstancesAsync(); - Assert.Equal(3, results.Count); - Assert.Equal("\"Done\"", results[0].Output); - Assert.Equal("\"Done\"", results[1].Output); - Assert.Equal("\"Done\"", results[2].Output); - - List firstHistoryEvents = await client.GetOrchestrationHistoryAsync(firstInstanceId); - Assert.True(firstHistoryEvents.Count > 0); - - List secondHistoryEvents = await client.GetOrchestrationHistoryAsync(secondInstanceId); - Assert.True(secondHistoryEvents.Count > 0); - - List thirdHistoryEvents = await client.GetOrchestrationHistoryAsync(thirdInstanceId); - Assert.True(secondHistoryEvents.Count > 0); - - IList firstOrchestrationStateList = await client.GetStateAsync(firstInstanceId); - Assert.Equal(1, firstOrchestrationStateList.Count); - Assert.Equal(firstInstanceId, firstOrchestrationStateList.First().OrchestrationInstance.InstanceId); - - IList secondOrchestrationStateList = await client.GetStateAsync(secondInstanceId); - Assert.Equal(1, secondOrchestrationStateList.Count); - Assert.Equal(secondInstanceId, secondOrchestrationStateList.First().OrchestrationInstance.InstanceId); - - IList thirdOrchestrationStateList = await client.GetStateAsync(thirdInstanceId); - Assert.Equal(1, thirdOrchestrationStateList.Count); - Assert.Equal(thirdInstanceId, thirdOrchestrationStateList.First().OrchestrationInstance.InstanceId); - - await client.PurgeInstanceHistoryByTimePeriod(startDateTime, endDateTime, new List { OrchestrationStatus.Completed, OrchestrationStatus.Terminated, OrchestrationStatus.Failed, OrchestrationStatus.Running }); - - List firstHistoryEventsAfterPurging = await client.GetOrchestrationHistoryAsync(firstInstanceId); - Assert.Empty(firstHistoryEventsAfterPurging); - - List secondHistoryEventsAfterPurging = await client.GetOrchestrationHistoryAsync(secondInstanceId); - Assert.True(secondHistoryEventsAfterPurging.Count > 0); - - List thirdHistoryEventsAfterPurging = await client.GetOrchestrationHistoryAsync(thirdInstanceId); - Assert.True(thirdHistoryEventsAfterPurging.Count > 0); - - firstOrchestrationStateList = await client.GetStateAsync(firstInstanceId); - Assert.Equal(0, firstOrchestrationStateList.Count); - Assert.Null(firstOrchestrationStateList.FirstOrDefault()); - - secondOrchestrationStateList = await client.GetStateAsync(secondInstanceId); - Assert.Equal(1, secondOrchestrationStateList.Count); - Assert.Equal(secondInstanceId, secondOrchestrationStateList.First().OrchestrationInstance.InstanceId); - - thirdOrchestrationStateList = await client.GetStateAsync(thirdInstanceId); - Assert.Equal(1, thirdOrchestrationStateList.Count); - Assert.Equal(thirdInstanceId, thirdOrchestrationStateList.First().OrchestrationInstance.InstanceId); + return Common.WithTimeoutAsync(DefaultTimeout, async () => + { + DateTime startDateTime = DateTime.Now; + string firstInstanceId = Guid.NewGuid().ToString(); + TestOrchestrationClient client = await this.host.StartOrchestrationAsync(typeof(Orchestrations.FanOutFanIn), 50, firstInstanceId); + await client.WaitForCompletionAsync(TimeSpan.FromSeconds(30)); + DateTime endDateTime = DateTime.Now; + await Task.Delay(5000); + string secondInstanceId = Guid.NewGuid().ToString(); + client = await this.host.StartOrchestrationAsync(typeof(Orchestrations.FanOutFanIn), 50, secondInstanceId); + await client.WaitForCompletionAsync(TimeSpan.FromSeconds(30)); + string thirdInstanceId = Guid.NewGuid().ToString(); + client = await this.host.StartOrchestrationAsync(typeof(Orchestrations.FanOutFanIn), 50, thirdInstanceId); + await client.WaitForCompletionAsync(TimeSpan.FromSeconds(30)); + + IList results = await this.host.GetAllOrchestrationInstancesAsync(); + Assert.Equal(3, results.Count); + Assert.Equal("\"Done\"", results[0].Output); + Assert.Equal("\"Done\"", results[1].Output); + Assert.Equal("\"Done\"", results[2].Output); + + List firstHistoryEvents = await client.GetOrchestrationHistoryAsync(firstInstanceId); + Assert.True(firstHistoryEvents.Count > 0); + + List secondHistoryEvents = await client.GetOrchestrationHistoryAsync(secondInstanceId); + Assert.True(secondHistoryEvents.Count > 0); + + List thirdHistoryEvents = await client.GetOrchestrationHistoryAsync(thirdInstanceId); + Assert.True(secondHistoryEvents.Count > 0); + + IList firstOrchestrationStateList = await client.GetStateAsync(firstInstanceId); + Assert.Equal(1, firstOrchestrationStateList.Count); + Assert.Equal(firstInstanceId, firstOrchestrationStateList.First().OrchestrationInstance.InstanceId); + + IList secondOrchestrationStateList = await client.GetStateAsync(secondInstanceId); + Assert.Equal(1, secondOrchestrationStateList.Count); + Assert.Equal(secondInstanceId, secondOrchestrationStateList.First().OrchestrationInstance.InstanceId); + + IList thirdOrchestrationStateList = await client.GetStateAsync(thirdInstanceId); + Assert.Equal(1, thirdOrchestrationStateList.Count); + Assert.Equal(thirdInstanceId, thirdOrchestrationStateList.First().OrchestrationInstance.InstanceId); + + await client.PurgeInstanceHistoryByTimePeriod(startDateTime, endDateTime, new List { OrchestrationStatus.Completed, OrchestrationStatus.Terminated, OrchestrationStatus.Failed, OrchestrationStatus.Running }); + + List firstHistoryEventsAfterPurging = await client.GetOrchestrationHistoryAsync(firstInstanceId); + Assert.Empty(firstHistoryEventsAfterPurging); + + List secondHistoryEventsAfterPurging = await client.GetOrchestrationHistoryAsync(secondInstanceId); + Assert.True(secondHistoryEventsAfterPurging.Count > 0); + + List thirdHistoryEventsAfterPurging = await client.GetOrchestrationHistoryAsync(thirdInstanceId); + Assert.True(thirdHistoryEventsAfterPurging.Count > 0); + + firstOrchestrationStateList = await client.GetStateAsync(firstInstanceId); + Assert.Equal(0, firstOrchestrationStateList.Count); + Assert.Null(firstOrchestrationStateList.FirstOrDefault()); + + secondOrchestrationStateList = await client.GetStateAsync(secondInstanceId); + Assert.Equal(1, secondOrchestrationStateList.Count); + Assert.Equal(secondInstanceId, secondOrchestrationStateList.First().OrchestrationInstance.InstanceId); + + thirdOrchestrationStateList = await client.GetStateAsync(thirdInstanceId); + Assert.Equal(1, thirdOrchestrationStateList.Count); + Assert.Equal(thirdInstanceId, thirdOrchestrationStateList.First().OrchestrationInstance.InstanceId); + }); } } @@ -263,21 +275,24 @@ public void Dispose() /// This exercises what LinqPAD queries do. /// [Fact] - public async void SingleServiceQuery() + public Task SingleServiceQuery() { - Trace.WriteLine("Starting the orchestration service..."); - var settings = TestConstants.GetNetheriteOrchestrationServiceSettings(emulationSpec: "SingleHost"); - var service = new NetheriteOrchestrationService(settings, this.loggerFactory); - var orchestrationService = (IOrchestrationService)service; - await orchestrationService.CreateAsync(true); - await orchestrationService.StartAsync(); - Trace.WriteLine("Orchestration service is started."); - - var _ = await ((IOrchestrationServiceQueryClient)service).GetOrchestrationStateAsync(); - - Trace.WriteLine("shutting down the orchestration service..."); - await orchestrationService.StopAsync(); - Trace.WriteLine("Orchestration service is shut down."); + return Common.WithTimeoutAsync(TimeSpan.FromMinutes(1), async () => + { + Trace.WriteLine("Starting the orchestration service..."); + var settings = TestConstants.GetNetheriteOrchestrationServiceSettings(emulationSpec: "SingleHost"); + var service = new NetheriteOrchestrationService(settings, this.loggerFactory); + var orchestrationService = (IOrchestrationService)service; + await orchestrationService.CreateAsync(true); + await orchestrationService.StartAsync(); + Trace.WriteLine("Orchestration service is started."); + + var _ = await ((IOrchestrationServiceQueryClient)service).GetOrchestrationStateAsync(); + + Trace.WriteLine("shutting down the orchestration service..."); + await orchestrationService.StopAsync(); + Trace.WriteLine("Orchestration service is shut down."); + }); } } } diff --git a/test/DurableTask.Netherite.Tests/TestConstants.cs b/test/DurableTask.Netherite.Tests/TestConstants.cs index 8b209938..abfb7f1c 100644 --- a/test/DurableTask.Netherite.Tests/TestConstants.cs +++ b/test/DurableTask.Netherite.Tests/TestConstants.cs @@ -64,8 +64,6 @@ public static NetheriteOrchestrationServiceSettings GetNetheriteOrchestrationSer public static NetheriteOrchestrationService GetTestOrchestrationService(ILoggerFactory loggerFactory) => new NetheriteOrchestrationService(GetNetheriteOrchestrationServiceSettings(), loggerFactory); - internal static TestOrchestrationHost GetTestOrchestrationHost(ILoggerFactory loggerFactory) - => new TestOrchestrationHost(GetNetheriteOrchestrationServiceSettings(), loggerFactory); internal static bool UsesEmulation(this NetheriteOrchestrationServiceSettings settings) { diff --git a/test/PerformanceTests/Benchmarks/EventHubs/HttpTriggers/Producers.cs b/test/PerformanceTests/Benchmarks/EventHubs/HttpTriggers/Producers.cs index b8f9c510..541276b4 100644 --- a/test/PerformanceTests/Benchmarks/EventHubs/HttpTriggers/Producers.cs +++ b/test/PerformanceTests/Benchmarks/EventHubs/HttpTriggers/Producers.cs @@ -4,12 +4,9 @@ namespace PerformanceTests.EventProducer { using System; - using System.Collections.Generic; using System.Threading.Tasks; - using Microsoft.Azure.EventHubs; using EventHubs; using Microsoft.AspNetCore.Mvc; - using Microsoft.AspNetCore.Routing; using Microsoft.Azure.WebJobs; using Microsoft.AspNetCore.Http; using Microsoft.Extensions.Logging; @@ -17,7 +14,6 @@ namespace PerformanceTests.EventProducer using Microsoft.Azure.WebJobs.Extensions.Http; using Newtonsoft.Json; using System.Linq; - using System.Diagnostics; using System.IO; using System.Net; diff --git a/test/PerformanceTests/PerformanceTests.csproj b/test/PerformanceTests/PerformanceTests.csproj index a20d999f..36f935dd 100644 --- a/test/PerformanceTests/PerformanceTests.csproj +++ b/test/PerformanceTests/PerformanceTests.csproj @@ -4,7 +4,7 @@ v4 - +