From 1b639d8fa77aef225d97835f37d2b268aed0fed3 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Thu, 24 Jul 2025 16:19:40 +1000 Subject: [PATCH 001/137] Adds Steve's redis queue --- source/Halibut/DataStream.cs | 16 +- source/Halibut/Halibut.csproj | 1 + .../IStoreDataStreamsForDistributedQueues.cs | 58 ++++ .../Halibut/Queue/QueueMessageSerializer.cs | 78 +++++ .../Redis/HalibutHalibutRedisTransport.cs | 171 +++++++++++ source/Halibut/Queue/Redis/RedisConnection.cs | 110 +++++++ .../Queue/Redis/RedisPendingRequestQueue.cs | 278 ++++++++++++++++++ .../ServiceModel/PendingRequestQueueAsync.cs | 2 + .../Transport/Protocol/MessageSerializer.cs | 7 - 9 files changed, 713 insertions(+), 8 deletions(-) create mode 100644 source/Halibut/Queue/IStoreDataStreamsForDistributedQueues.cs create mode 100644 source/Halibut/Queue/QueueMessageSerializer.cs create mode 100644 source/Halibut/Queue/Redis/HalibutHalibutRedisTransport.cs create mode 100644 source/Halibut/Queue/Redis/RedisConnection.cs create mode 100644 source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs diff --git a/source/Halibut/DataStream.cs b/source/Halibut/DataStream.cs index 414457fc0..9e413ef40 100644 --- a/source/Halibut/DataStream.cs +++ b/source/Halibut/DataStream.cs @@ -10,7 +10,7 @@ namespace Halibut { public class DataStream : IEquatable, IDataStreamInternal { - readonly Func writerAsync; + Func writerAsync; IDataStreamReceiver? receiver; [JsonConstructor] @@ -179,9 +179,23 @@ async Task IDataStreamInternal.TransmitAsync(Stream stream, CancellationToken ca await writerAsync(stream, cancellationToken); } + public async Task WriteData(Stream stream, CancellationToken cancellationToken) + { + await writerAsync(stream, cancellationToken); + } + void IDataStreamInternal.Received(IDataStreamReceiver attachedReceiver) { receiver = attachedReceiver; } + + /// + /// Be carefull + /// + /// + public void SetWriterAsync(Func writerAsync) + { + this.writerAsync = writerAsync; + } } } \ No newline at end of file diff --git a/source/Halibut/Halibut.csproj b/source/Halibut/Halibut.csproj index 3f77b994c..1163b6c28 100644 --- a/source/Halibut/Halibut.csproj +++ b/source/Halibut/Halibut.csproj @@ -30,6 +30,7 @@ + diff --git a/source/Halibut/Queue/IStoreDataStreamsForDistributedQueues.cs b/source/Halibut/Queue/IStoreDataStreamsForDistributedQueues.cs new file mode 100644 index 000000000..c5e19cacd --- /dev/null +++ b/source/Halibut/Queue/IStoreDataStreamsForDistributedQueues.cs @@ -0,0 +1,58 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; +using System.Collections.Generic; +using System.IO; +using System.Threading; +using System.Threading.Tasks; + +namespace Halibut.Queue +{ + public interface IStoreDataStreamsForDistributedQueues + { + public Task StoreDataStreams(IReadOnlyList dataStreams, CancellationToken cancellationToken); + + public Task ReHydrateDataStreams(IReadOnlyList dataStreams, CancellationToken cancellationToken); + } + + public class InMemoryStoreDataStreamsForDistributedQueues : IStoreDataStreamsForDistributedQueues + { + public IDictionary dataStreamsStored = new Dictionary(); + public async Task StoreDataStreams(IReadOnlyList dataStreams, CancellationToken cancellationToken) + { + await Task.CompletedTask; + foreach (var dataStream in dataStreams) + { + using var memoryStream = new MemoryStream(); + await dataStream.WriteData(memoryStream, cancellationToken); + dataStreamsStored[dataStream.Id] = memoryStream.ToArray(); + } + } + + public async Task ReHydrateDataStreams(IReadOnlyList dataStreams, CancellationToken cancellationToken) + { + await Task.CompletedTask; + foreach (var dataStream in dataStreams) + { + var bytes = dataStreamsStored[dataStream.Id]; + dataStreamsStored.Remove(dataStream.Id); + dataStream.SetWriterAsync(async (stream, ct) => + { + await stream.WriteAsync(bytes, 0, bytes.Length, ct); + }); + } + } + } +} \ No newline at end of file diff --git a/source/Halibut/Queue/QueueMessageSerializer.cs b/source/Halibut/Queue/QueueMessageSerializer.cs new file mode 100644 index 000000000..37b9d2164 --- /dev/null +++ b/source/Halibut/Queue/QueueMessageSerializer.cs @@ -0,0 +1,78 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; +using System.Collections.Generic; +using System.Globalization; +using System.IO; +using System.Text; +using Halibut.Transport.Protocol; +using Newtonsoft.Json; + +namespace Halibut.Queue +{ + public class QueueMessageSerializer + { + readonly Func createStreamCapturingSerializer; + + public QueueMessageSerializer(Func createStreamCapturingSerializer) + { + this.createStreamCapturingSerializer = createStreamCapturingSerializer; + } + + public (string, IReadOnlyList) WriteMessage(T message) + { + IReadOnlyList datatStreams; + + var sb = new StringBuilder(); + using var sw = new StringWriter(sb, CultureInfo.InvariantCulture); + using (var jsonTextWriter = new JsonTextWriter(sw) { CloseOutput = false }) + { + var streamCapturingSerializer = createStreamCapturingSerializer(); + streamCapturingSerializer.Serializer.Serialize(jsonTextWriter, new MessageEnvelope { Message = message! }); + datatStreams = streamCapturingSerializer.DataStreams; + } + + return (sb.ToString(), datatStreams); + } + + public (T Message, IReadOnlyList DataStreams) ReadMessage(string json) + { + using var reader = new JsonTextReader(new StringReader(json)); + { + + var streamCapturingSerializer = createStreamCapturingSerializer(); + var result = streamCapturingSerializer.Serializer.Deserialize>(reader); + + if (result == null) + { + throw new Exception("messageEnvelope is null"); + } + + return (result.Message, streamCapturingSerializer.DataStreams); + } + } + + + + // By making this a generic type, each message specifies the exact type it sends/expects + // And it is impossible to deserialize the wrong type - any mismatched type will refuse to deserialize + class MessageEnvelope + { +#pragma warning disable CS8618 // Non-nullable field must contain a non-null value when exiting constructor. Consider declaring as nullable. + public T Message { get; set; } +#pragma warning restore CS8618 // Non-nullable field must contain a non-null value when exiting constructor. Consider declaring as nullable. + } + } +} \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/HalibutHalibutRedisTransport.cs b/source/Halibut/Queue/Redis/HalibutHalibutRedisTransport.cs new file mode 100644 index 000000000..8fe55af6e --- /dev/null +++ b/source/Halibut/Queue/Redis/HalibutHalibutRedisTransport.cs @@ -0,0 +1,171 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; +using System.Diagnostics; +using System.Threading.Tasks; +using Newtonsoft.Json; + +namespace Halibut.Queue.Redis +{ + class HalibutHalibutRedisTransport : IHalibutRedisTransport + { + const string Namespace = "octopus:server:halibut"; + static readonly string RequestChannel = $"{Namespace}:request"; + static readonly string ResponseChannel = $"{Namespace}:response"; + static readonly string RequestPoppedChannel = $"{Namespace}:requestpopped"; + + readonly IRedisConnection connection; + + //TODO: Rethink .net event + public event EventHandler? NewRequestEvent; + public event EventHandler? NewResponseEvent; + public event EventHandler? RequestPoppedEvent; + + public HalibutHalibutRedisTransport(IRedisConnection connection) + { + this.connection = connection; + + SubscribeToChannels(); + } + + void SubscribeToChannels() + { + + connection.SubscribeToChannel( + RequestChannel, + message => + { + + var queueItem = JsonConvert.DeserializeObject(message.Message!); + NewRequestEvent?.Invoke(this, queueItem!); + }); + + connection.SubscribeToChannel( + ResponseChannel, + message => + { + var queueItem = JsonConvert.DeserializeObject(message.Message!); + NewResponseEvent?.Invoke(this, queueItem!); + }); + + connection.SubscribeToChannel( + RequestPoppedChannel, + message => + { + var queueItem = JsonConvert.DeserializeObject(message.Message!); + RequestPoppedEvent?.Invoke(this, queueItem!); + }); + } + + static string ResponseKey(string requestId) + { + var validRequestId = requestId.Replace(":", "_"); + return $"{Namespace}:response:{validRequestId}"; + } + + static string RequestKey(Uri endpoint) + { + return $"{Namespace}:request:{endpoint}"; + } + + public async Task SetResponse(Uri endpoint, string requestId, string payload) + { + var responseKey = ResponseKey(requestId); + var responseExpiry = TimeSpan.FromMinutes(10); + + await connection.StringSet(responseKey, payload, responseExpiry); + await PublishQueueItem(ResponseChannel, endpoint, requestId); + } + + public async Task GetDeleteResponse(string requestId) + { + var responseKey = ResponseKey(requestId); + var response = await connection.StringGetDelete(responseKey); + return response; + } + + public async Task PushRequest(Uri endpoint, string requestId, string payload) + { + var redisQueueItem = new RedisHalibuteQueueItem(requestId,payload); + + var serialisedQueueItem = JsonConvert.SerializeObject(redisQueueItem); + + var requestKey = RequestKey(endpoint); + + await connection.ListRightPush(requestKey, serialisedQueueItem); + + await PublishQueueItem(RequestChannel, endpoint, requestId); + } + + public async Task PopRequest(Uri endpoint) + { + var requestKey = RequestKey(endpoint); + + var requestMessage = await connection.ListLeftPopAsync(requestKey); + if (requestMessage is null) return null; + + var redisQueueItem = JsonConvert.DeserializeObject(requestMessage); + if (redisQueueItem is null) return null; + + await PublishQueueItem(RequestPoppedChannel, endpoint, redisQueueItem.RequestId); + return redisQueueItem; + } + + async Task PublishQueueItem(string channel, Uri endpoint, string requestId) + { + var queueItem = new RedisHalibutQueueItem( + endpoint, + requestId); + + await connection.PublishToChannel(channel, queueItem); + } + } + + interface IHalibutRedisTransport + { + event EventHandler NewRequestEvent; + event EventHandler NewResponseEvent; + event EventHandler RequestPoppedEvent; + + Task SetResponse(Uri endpoint, string requestId, string payload); + Task PushRequest(Uri endpoint, string requestId, string payload); + Task GetDeleteResponse(string requestId); + Task PopRequest(Uri endpoint); + } + + public class RedisHalibuteQueueItem + { + public RedisHalibuteQueueItem(string requestId, string payloadJson) + { + RequestId = requestId; + PayloadJson = payloadJson; + } + + public string RequestId { get; protected set; } + public string PayloadJson { get; protected set; } + } + + public class RedisHalibutQueueItem + { + public Uri Endpoint { get; protected set; } + public string RequestId { get; protected set; } + + public RedisHalibutQueueItem(Uri endpoint, string requestId) + { + Endpoint = endpoint; + RequestId = requestId; + } + } +} \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/RedisConnection.cs b/source/Halibut/Queue/Redis/RedisConnection.cs new file mode 100644 index 000000000..0e688dd3a --- /dev/null +++ b/source/Halibut/Queue/Redis/RedisConnection.cs @@ -0,0 +1,110 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; +using System.Threading.Tasks; +using Newtonsoft.Json; +using StackExchange.Redis; + +namespace Halibut.Queue.Redis +{ + class RedisConnection : IRedisConnection, IDisposable + { + readonly Lazy connection; + + ConnectionMultiplexer Connection => connection.Value; + + public RedisConnection(string redisHost) + { + + connection = new Lazy(() => + { + return ConnectionMultiplexer.Connect(redisHost); + }); + } + + public void Dispose() + { + if (connection.IsValueCreated) + { + connection.Value.Dispose(); + } + } + + public void SubscribeToChannel(string channel, Action onMessage) + { + Connection.GetSubscriber() + .Subscribe(channel) + .OnMessage(onMessage); + } + + public async Task PublishToChannel(string channel, T payload) + { + var serialized = JsonConvert.SerializeObject(payload); + await PublishToChannel(channel, serialized); + + } + + public async Task PublishToChannel(string channel, string payload) + { + var subscriber = Connection.GetSubscriber(); + await subscriber.PublishAsync(channel, payload); + } + + public async Task StringSet(string key, string payload, TimeSpan expiry) + { + var database = Connection.GetDatabase(); + await database.StringSetAsync(key, payload, expiry:expiry); + } + + public async Task ListRightPush(string key, string payload) + { + var database = Connection.GetDatabase(); + await database.ListRightPushAsync(key, payload); + } + + public async Task StringGetDelete(string key) + { + var database = Connection.GetDatabase(); + var payload = await database.StringGetDeleteAsync(key); + return payload; + } + + public async Task ListLeftPopAsync(string key) + { + var database = Connection.GetDatabase(); + var value = await database.ListLeftPopAsync(key); + if (value.IsNull) + { + return null; + } + + return value; + } + } + + public interface IRedisConnection + { + void SubscribeToChannel(string channel, Action onMessage); + + Task PublishToChannel(string channel, string payload); + Task PublishToChannel(string channel, T payload); + + Task StringSet(string key, string payload, TimeSpan expiry); + Task ListRightPush(string key, string payload); + Task StringGetDelete(string key); + + Task ListLeftPopAsync(string key); + } +} \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs new file mode 100644 index 000000000..afa1ca2b6 --- /dev/null +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -0,0 +1,278 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; +using System.Collections.Generic; +using System.Threading; +using System.Threading.Tasks; +using Halibut.Diagnostics; +using Halibut.ServiceModel; +using Halibut.Transport.Protocol; +using Nito.AsyncEx; + +namespace Halibut.Queue.Redis +{ + class RedisPendingRequestQueue : IPendingRequestQueue, IDisposable + { + readonly public static TimeSpan PollingRequestMaximumMessageProcessingTimeout = TimeSpan.FromHours(33); + + readonly Dictionary inProgress = new(); + //TODO: Do we really need the lock (which came from original implementation) + readonly object sync = new(); + readonly AsyncManualResetEvent hasItemsForEndpoint = new(); + readonly Uri endpoint; + readonly ILog log; + readonly IHalibutRedisTransport halibutRedisTransport; + + readonly QueueMessageSerializer queueMessageSerializer; + readonly IStoreDataStreamsForDistributedQueues storeDataStreamsForDistributedQueues; + + public delegate RedisPendingRequestQueue Factory(Uri endpoint, ILog log); + + public RedisPendingRequestQueue( + Uri endpoint, + ILog log, + IHalibutRedisTransport halibutRedisTransport, + QueueMessageSerializer queueMessageSerializer, + IStoreDataStreamsForDistributedQueues storeDataStreamsForDistributedQueues) + { + this.endpoint = endpoint; + this.log = log; + this.halibutRedisTransport = halibutRedisTransport; + this.queueMessageSerializer = queueMessageSerializer; + this.storeDataStreamsForDistributedQueues = storeDataStreamsForDistributedQueues; + + halibutRedisTransport.NewRequestEvent += HandleNewRequest; + halibutRedisTransport.NewResponseEvent += HandleNewResponse; + halibutRedisTransport.RequestPoppedEvent += HandleRequestPopped; + } + + public void Dispose() + { + halibutRedisTransport.NewRequestEvent += HandleNewRequest; + halibutRedisTransport.NewResponseEvent += HandleNewResponse; + } + + void HandleNewRequest(object? sender, RedisHalibutQueueItem e) + { + lock (sync) + { + if (e.Endpoint == endpoint) + { + hasItemsForEndpoint.Set(); + } + } + } + + void HandleNewResponse(object? sender, RedisHalibutQueueItem e) + { + lock (sync) + { + if (inProgress.TryGetValue(e.RequestId, out var pendingRequest)) + { + var responseMessage = halibutRedisTransport.GetDeleteResponse(e.RequestId).GetAwaiter().GetResult(); + + var (response, dataStreams) = this.queueMessageSerializer.ReadMessage(responseMessage!); + storeDataStreamsForDistributedQueues.ReHydrateDataStreams(dataStreams, CancellationToken.None).GetAwaiter().GetResult(); + + pendingRequest.SetResponse(response!); + } + } + } + + void HandleRequestPopped(object? sender, RedisHalibutQueueItem e) + { + lock (sync) + { + if (inProgress.TryGetValue(e.RequestId, out var pendingRequest)) + { + pendingRequest.BeginTransfer(); + } + } + } + + public Task ApplyResponse(ResponseMessage response, ServiceEndPoint destination) + { + throw new NotImplementedException(); + } + + public async Task QueueAndWaitAsync(RequestMessage request, CancellationToken cancellationToken) + { + var pending = new PendingRequest(request, log); + + lock (sync) + { + inProgress.Add(request.Id, pending); + } + + var (payload, dataStreams) = queueMessageSerializer.WriteMessage(request); + await storeDataStreamsForDistributedQueues.StoreDataStreams(dataStreams, cancellationToken); + + await halibutRedisTransport.PushRequest(endpoint, request.Id, payload); + + pending.WaitUntilComplete(cancellationToken); + + lock (sync) + { + inProgress.Remove(request.Id); + } + + return pending.Response!; + } + + public bool IsEmpty => throw new NotImplementedException(); + public int Count => throw new NotImplementedException(); + + public async Task DequeueAsync(CancellationToken cancellationToken) + { + var pending = await DequeueNextAsync(); + if (pending == null) return null; + return new RequestMessageWithCancellationToken(pending, CancellationToken.None); + } + + async Task DequeueNextAsync() + { + var first = await TakeFirst(); + if (first != null) return first; + + await Task.WhenAny(hasItemsForEndpoint.WaitAsync(), Task.Delay(new HalibutTimeoutsAndLimits().PollingQueueWaitTimeout)); + hasItemsForEndpoint.Reset(); + return await TakeFirst(); + } + + async Task TakeFirst() + { + var redisQueueItem = await halibutRedisTransport.PopRequest(endpoint); + + if (redisQueueItem is null) return null; + + var (request, dataStreams) = queueMessageSerializer.ReadMessage(redisQueueItem.PayloadJson); + storeDataStreamsForDistributedQueues.ReHydrateDataStreams(dataStreams, CancellationToken.None).GetAwaiter().GetResult(); + + return request; + } + + public void ApplyResponse(ResponseMessage response) + { + if (response == null) return; + + lock (sync) + { + var (payload, dataStreams) = queueMessageSerializer.WriteMessage(response); + storeDataStreamsForDistributedQueues.StoreDataStreams(dataStreams, CancellationToken.None).GetAwaiter().GetResult(); + + halibutRedisTransport.SetResponse(endpoint, response.Id, payload).GetAwaiter().GetResult(); + } + } + + + class PendingRequest + { + readonly RequestMessage request; + readonly ILog log; + readonly ManualResetEventSlim waiter; + readonly object sync = new object(); + bool transferBegun; + bool completed; + + public PendingRequest(RequestMessage request, ILog log) + { + this.request = request; + this.log = log; + waiter = new ManualResetEventSlim(false); + } + + + public void WaitUntilComplete(CancellationToken cancellationToken) + { + log.Write(EventType.MessageExchange, "Request {0} was queued", request); + + var success = waiter.Wait(request.Destination.PollingRequestQueueTimeout, cancellationToken); + if (success) + { + log.Write(EventType.MessageExchange, "Request {0} was collected by the polling endpoint", request); + return; + } + + var waitForTransferToComplete = false; + lock (sync) + { + if (transferBegun) + { + waitForTransferToComplete = true; + } + else + { + completed = true; + } + } + + if (waitForTransferToComplete) + { + success = waiter.Wait(PollingRequestMaximumMessageProcessingTimeout); + if (success) + { + log.Write(EventType.MessageExchange, "Request {0} was eventually collected by the polling endpoint", request); + } + else + { + SetResponse(ResponseMessage.FromException(request, new TimeoutException(string.Format("A request was sent to a polling endpoint, the polling endpoint collected it but did not respond in the allowed time ({0}), so the request timed out.", PollingRequestMaximumMessageProcessingTimeout)))); + } + } + else + { + log.Write(EventType.MessageExchange, "Request {0} timed out before it could be collected by the polling endpoint", request); + SetResponse(ResponseMessage.FromException(request, new TimeoutException(string.Format("A request was sent to a polling endpoint, but the polling endpoint did not collect the request within the allowed time ({0}), so the request timed out.", request.Destination.PollingRequestQueueTimeout)))); + } + } + + public bool BeginTransfer() + { + lock (sync) + { + if (completed) + return false; + + transferBegun = true; + return true; + } + } + + public ResponseMessage? Response { get; private set; } + + public void SetResponse(ResponseMessage response) + { + Response = response; + waiter.Set(); + } + } + + public ValueTask DisposeAsync() + { + throw new NotImplementedException(); + } + } + + interface IHalibutMessageQueueItemSerializer + { + T FromJson(string responseMessage); + string ToJson(object message); + } + + public enum MessageType + { + RequestMessage, + ResponseMessage + } +} \ No newline at end of file diff --git a/source/Halibut/ServiceModel/PendingRequestQueueAsync.cs b/source/Halibut/ServiceModel/PendingRequestQueueAsync.cs index b7a5c371f..b95bf3279 100644 --- a/source/Halibut/ServiceModel/PendingRequestQueueAsync.cs +++ b/source/Halibut/ServiceModel/PendingRequestQueueAsync.cs @@ -36,6 +36,8 @@ public PendingRequestQueueAsync(ILog log, TimeSpan pollingQueueWaitTimeout) public async Task QueueAndWaitAsync(RequestMessage request, CancellationToken cancellationToken) { + //cancellationToken = CancellationToken.None; + using var cancellationTokenSource = CancellationTokenSource.CreateLinkedTokenSource(cancellationToken, this.entireQueueCancellationTokenSource.Token); cancellationToken = cancellationTokenSource.Token; diff --git a/source/Halibut/Transport/Protocol/MessageSerializer.cs b/source/Halibut/Transport/Protocol/MessageSerializer.cs index a677dedd3..aedf9dbba 100644 --- a/source/Halibut/Transport/Protocol/MessageSerializer.cs +++ b/source/Halibut/Transport/Protocol/MessageSerializer.cs @@ -14,7 +14,6 @@ namespace Halibut.Transport.Protocol { public class MessageSerializer : IMessageSerializer { - readonly ITypeRegistry typeRegistry; readonly Func createStreamCapturingSerializer; readonly IMessageSerializerObserver observer; readonly long readIntoMemoryLimitBytes; @@ -29,7 +28,6 @@ internal MessageSerializer( long writeIntoMemoryLimitBytes, ILogFactory logFactory) { - this.typeRegistry = typeRegistry; this.createStreamCapturingSerializer = createStreamCapturingSerializer; this.observer = observer; this.readIntoMemoryLimitBytes = readIntoMemoryLimitBytes; @@ -37,11 +35,6 @@ internal MessageSerializer( deflateReflector = new DeflateStreamInputBufferReflector(logFactory.ForPrefix(nameof(MessageSerializer))); } - public void AddToMessageContract(params Type[] types) // kept for backwards compatibility - { - typeRegistry.AddToMessageContract(types); - } - public async Task> WriteMessageAsync(Stream stream, T message, CancellationToken cancellationToken) { IReadOnlyList serializedStreams; From 31704d33da22586c2448b4e9ff6473b21dbd5833 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Mon, 28 Jul 2025 12:25:50 +1000 Subject: [PATCH 002/137] A queue in halibut --- .../IComplexObjectService.cs | 10 + .../Redis/RedisPendingRequestQueueFixture.cs | 170 +++++++++++ .../PendingRequestQueueFixture.cs | 22 +- .../LatestClientAndLatestServiceBuilder.cs | 6 + .../Support/LatestClientBuilder.cs | 12 +- ...questDequeuedPendingRequestQueueFactory.cs | 4 +- ...RequestQueuedPendingRequestQueueFactory.cs | 2 +- source/Halibut/Halibut.csproj | 2 +- source/Halibut/HalibutRuntimeBuilder.cs | 18 +- .../Halibut/Queue/QueueMessageSerializer.cs | 7 +- .../Redis/HalibutHalibutRedisTransport.cs | 191 ++++++------ .../Queue/Redis/MessageReaderWriter.cs | 60 ++++ .../Queue/Redis/PendingRequestQueue.cs | 192 ++++++++++++ .../{RedisConnection.cs => RedisFacade.cs} | 75 +++-- .../Queue/Redis/RedisPendingRequest.cs | 114 +++++++ .../Queue/Redis/RedisPendingRequestQueue.cs | 278 ------------------ .../Redis/RedisPendingRequestQueueFactory.cs | 52 ++++ .../ServiceModel/IPendingRequestQueue.cs | 2 +- .../ServiceModel/PendingRequestQueueAsync.cs | 2 +- .../Protocol/MessageExchangeProtocol.cs | 4 +- .../Transport/Protocol/MessageSerializer.cs | 8 +- source/Halibut/Util/FuncAsyncDisposable.cs | 34 +++ source/Halibut/Util/StringExtensionMethods.cs | 29 ++ 23 files changed, 838 insertions(+), 456 deletions(-) create mode 100644 source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs create mode 100644 source/Halibut/Queue/Redis/MessageReaderWriter.cs create mode 100644 source/Halibut/Queue/Redis/PendingRequestQueue.cs rename source/Halibut/Queue/Redis/{RedisConnection.cs => RedisFacade.cs} (53%) create mode 100644 source/Halibut/Queue/Redis/RedisPendingRequest.cs delete mode 100644 source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs create mode 100644 source/Halibut/Queue/Redis/RedisPendingRequestQueueFactory.cs create mode 100644 source/Halibut/Util/FuncAsyncDisposable.cs create mode 100644 source/Halibut/Util/StringExtensionMethods.cs diff --git a/source/Halibut.TestUtils.Contracts/IComplexObjectService.cs b/source/Halibut.TestUtils.Contracts/IComplexObjectService.cs index 9a788d3cd..e6706b4ea 100644 --- a/source/Halibut.TestUtils.Contracts/IComplexObjectService.cs +++ b/source/Halibut.TestUtils.Contracts/IComplexObjectService.cs @@ -13,6 +13,16 @@ public interface IComplexObjectService public class ComplexObjectMultipleDataStreams { + public ComplexObjectMultipleDataStreams() + { + } + + public ComplexObjectMultipleDataStreams(DataStream? payload1, DataStream? payload2) + { + Payload1 = payload1; + Payload2 = payload2; + } + public DataStream? Payload1; public DataStream? Payload2; } diff --git a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs new file mode 100644 index 000000000..3f4daf4ae --- /dev/null +++ b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs @@ -0,0 +1,170 @@ +using System; +using System.Threading; +using System.Threading.Tasks; +using FluentAssertions; +using Halibut.Diagnostics; +using Halibut.Queue; +using Halibut.Queue.Redis; +using Halibut.Tests.Builders; +using Halibut.Tests.Support; +using Halibut.Tests.Support.TestAttributes; +using Halibut.Tests.Support.TestCases; +using Halibut.Tests.TestServices.Async; +using Halibut.Tests.Util; +using Halibut.TestUtils.Contracts; +using Halibut.Transport.Protocol; +using NSubstitute; +using NUnit.Framework; + +namespace Halibut.Tests.Queue.Redis +{ + public class RedisPendingRequestQueueFixture : BaseTest + { + + [Test] + public async Task DequeueAsync_ShouldReturnRequestFromRedis() + { + // Arrange + var endpoint = new Uri("poll://" + Guid.NewGuid()); + var log = Substitute.For(); + var redisTransport = new HalibutHalibutRedisTransport(new RedisFacade("localhost")); + var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); + var messageSerializer = new QueueMessageSerializerBuilder().Build(); + var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); + + var request = new RequestMessageBuilder("poll://test-endpoint").Build(); + + var sut = new RedisPendingRequestQueue(endpoint, log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + + var task = sut.QueueAndWaitAsync(request, CancellationToken.None); + + // Act + var result = await sut.DequeueAsync(CancellationToken); + + // Assert + result.Should().NotBeNull(); + result!.RequestMessage.Id.Should().Be(request.Id); + result.RequestMessage.MethodName.Should().Be(request.MethodName); + result.RequestMessage.ServiceName.Should().Be(request.ServiceName); + } + + [Test] + public async Task FullSendAndReceiveShouldWork() + { + // Arrange + var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); + var log = Substitute.For(); + var redisTransport = new HalibutHalibutRedisTransport(new RedisFacade("localhost")); + var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); + var messageSerializer = new QueueMessageSerializerBuilder().Build(); + var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); + + var request = new RequestMessageBuilder("poll://test-endpoint").Build(); + + var node1Sender = new RedisPendingRequestQueue(endpoint, log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + var node2Reciever = new RedisPendingRequestQueue(endpoint, log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + + var queueAndWaitAsync = node1Sender.QueueAndWaitAsync(request, CancellationToken.None); + + var requestMessageWithCancellationToken = await node2Reciever.DequeueAsync(CancellationToken); + + requestMessageWithCancellationToken.Should().NotBeNull(); + requestMessageWithCancellationToken!.RequestMessage.Id.Should().Be(request.Id); + requestMessageWithCancellationToken.RequestMessage.MethodName.Should().Be(request.MethodName); + requestMessageWithCancellationToken.RequestMessage.ServiceName.Should().Be(request.ServiceName); + + var response = ResponseMessage.FromResult(requestMessageWithCancellationToken.RequestMessage, "Yay"); + await node2Reciever.ApplyResponse(response, requestMessageWithCancellationToken.RequestMessage.ActivityId); + + var responseMessage = await queueAndWaitAsync; + + responseMessage.Result.Should().Be("Yay"); + } + + [Test] + public async Task FullSendAndReceiveWithDataStreamShouldWork() + { + // Arrange + var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); + var log = Substitute.For(); + var redisTransport = new HalibutHalibutRedisTransport(new RedisFacade("localhost")); + var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); + var messageSerializer = new QueueMessageSerializerBuilder().Build(); + var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); + + var request = new RequestMessageBuilder("poll://test-endpoint").Build(); + request.Params = new[] { new ComplexObjectMultipleDataStreams(DataStream.FromString("hello"), DataStream.FromString("world")) }; + + + var node1Sender = new RedisPendingRequestQueue(endpoint, log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + var node2Reciever = new RedisPendingRequestQueue(endpoint, log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + + var queueAndWaitAsync = node1Sender.QueueAndWaitAsync(request, CancellationToken.None); + + var requestMessageWithCancellationToken = await node2Reciever.DequeueAsync(CancellationToken); + + var objWithDataStreams = (ComplexObjectMultipleDataStreams) requestMessageWithCancellationToken!.RequestMessage.Params[0]; + (await objWithDataStreams.Payload1!.ReadAsString(CancellationToken)).Should().Be("hello"); + (await objWithDataStreams.Payload2!.ReadAsString(CancellationToken)).Should().Be("world"); + + var response = ResponseMessage.FromResult(requestMessageWithCancellationToken.RequestMessage, + new ComplexObjectMultipleDataStreams(DataStream.FromString("good"), DataStream.FromString("bye"))); + + await node2Reciever.ApplyResponse(response, requestMessageWithCancellationToken.RequestMessage.ActivityId); + + var responseMessage = await queueAndWaitAsync; + + var returnObject = (ComplexObjectMultipleDataStreams) responseMessage.Result!; + (await returnObject.Payload1!.ReadAsString(CancellationToken)).Should().Be("good"); + (await returnObject.Payload2!.ReadAsString(CancellationToken)).Should().Be("bye"); + } + + [Test] + [LatestClientAndLatestServiceTestCases(testNetworkConditions: false, testListening: false, testWebSocket: false)] + public async Task OctopusCanSendMessagesToTentacle_WithEchoService(ClientAndServiceTestCase clientAndServiceTestCase) + { + var redisTransport = new HalibutHalibutRedisTransport(new RedisFacade("localhost")); + var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); + + await using (var clientAndService = await clientAndServiceTestCase.CreateTestCaseBuilder() + .WithStandardServices() + .AsLatestClientAndLatestServiceBuilder() + .WithPendingRequestQueueFactory((queueMessageSerializer, logFactory) => + new RedisPendingRequestQueueFactory( + queueMessageSerializer, + dataStreamStore, + redisTransport, + new HalibutTimeoutsAndLimits(), + logFactory)) + .Build(CancellationToken)) + { + var echo = clientAndService.CreateAsyncClient(); + (await echo.SayHelloAsync("Deploy package A")).Should().Be("Deploy package A..."); + + for (var i = 0; i < clientAndServiceTestCase.RecommendedIterations; i++) + { + (await echo.SayHelloAsync($"Deploy package A {i}")).Should().Be($"Deploy package A {i}..."); + } + } + } + + public class QueueMessageSerializerBuilder + { + public QueueMessageSerializer Build() + { + var typeRegistry = new TypeRegistry(); + typeRegistry.Register(typeof(IComplexObjectService)); + + StreamCapturingJsonSerializer StreamCapturingSerializer() + { + var settings = MessageSerializerBuilder.CreateSerializer(); + var binder = new RegisteredSerializationBinder(typeRegistry); + settings.SerializationBinder = binder; + return new StreamCapturingJsonSerializer(settings); + } + + return new QueueMessageSerializer(StreamCapturingSerializer); + } + } + } +} \ No newline at end of file diff --git a/source/Halibut.Tests/ServiceModel/PendingRequestQueueFixture.cs b/source/Halibut.Tests/ServiceModel/PendingRequestQueueFixture.cs index 71063309d..0a886f012 100644 --- a/source/Halibut.Tests/ServiceModel/PendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/ServiceModel/PendingRequestQueueFixture.cs @@ -33,7 +33,7 @@ public async Task QueueAndWait_WillContinueWaitingUntilResponseIsApplied() await Task.Delay(1000, CancellationToken); queueAndWaitTask.IsCompleted.Should().BeFalse(); - await sut.ApplyResponse(expectedResponse, request.Destination); + await sut.ApplyResponse(expectedResponse, request.ActivityId); // Assert var response = await queueAndWaitTask; @@ -60,13 +60,13 @@ public async Task QueueAndWait_WillIgnoreUnrelatedApplyResponses_AndShouldContin queueAndWaitTask.IsCompleted.Should().BeFalse(); // Apply unrelated responses - await sut.ApplyResponse(null!, request.Destination); - await sut.ApplyResponse(unexpectedResponse, request.Destination); + await sut.ApplyResponse(null!, request.ActivityId); + await sut.ApplyResponse(unexpectedResponse, request.ActivityId); await Task.Delay(1000, CancellationToken); queueAndWaitTask.IsCompleted.Should().BeFalse(); - await sut.ApplyResponse(expectedResponse, request.Destination); + await sut.ApplyResponse(expectedResponse, request.ActivityId); // Assert @@ -153,7 +153,7 @@ public async Task QueueAndWait_WhenRequestIsDequeued_WithRelyOnConnectionTimeout finishedTask.Should().Be(delayTask); - await sut.ApplyResponse(expectedResponse, request.Destination); + await sut.ApplyResponse(expectedResponse, request.ActivityId); var response = await queueAndWaitTask; @@ -188,7 +188,7 @@ public async Task QueueAndWait_WhenRequestIsDequeued_ButPollingRequestQueueTimeo await Task.Delay(2000, CancellationToken); dequeued.CancellationToken.IsCancellationRequested.Should().BeFalse("Should not have cancelled the request after PollingRequestQueueTimeout is reached"); - await sut.ApplyResponse(expectedResponse, request.Destination); + await sut.ApplyResponse(expectedResponse, request.ActivityId); var response = await queueAndWaitTask; @@ -472,7 +472,7 @@ public async Task DequeueAsync_WithNothingQueued_WillWaitPollingQueueWaitTimeout var queueAndWaitTask = await StartQueueAndWaitAndWaitForRequestToBeQueued(sut, previousRequest, CancellationToken); await sut.DequeueAsync(CancellationToken); - await sut.ApplyResponse(expectedPreviousResponse, previousRequest.Destination); + await sut.ApplyResponse(expectedPreviousResponse, previousRequest.ActivityId); await queueAndWaitTask; // Act @@ -512,7 +512,7 @@ public async Task DequeueAsync_WillContinueWaitingUntilItemIsQueued() dequeuedRequest!.RequestMessage.Should().Be(request); // Apply a response so we can prove this counts as taking a message. - await sut.ApplyResponse(expectedResponse, request.Destination); + await sut.ApplyResponse(expectedResponse, request.ActivityId); var response = await queueAndWaitTask; response.Should().Be(expectedResponse); } @@ -539,7 +539,7 @@ public async Task DequeueAsync_WithMultipleDequeueCallsWaiting_WhenSingleRequest // Assert await Task.WhenAll(dequeueTasks); - await sut.ApplyResponse(expectedResponse, request.Destination); + await sut.ApplyResponse(expectedResponse, request.ActivityId); await queueAndWaitTask; var singleDequeuedRequest = await dequeueTasks.Should().ContainSingle(t => t.Result != null).Subject; @@ -577,7 +577,7 @@ public async Task ApplyResponse_AfterRequestHasBeenCollected_AndWaitingHasBeenCa // Act - await sut.ApplyResponse(expectedResponse, request.Destination); + await sut.ApplyResponse(expectedResponse, request.ActivityId); // Assert @@ -656,7 +656,7 @@ static async Task ApplyResponsesConcurrentlyAndEnsureAllQueueResponsesMatch( //Concurrently apply responses to prove this does not cause issues. var applyResponseTasks = requestsInOrder - .Select((r,i) => Task.Run(async () => await sut.ApplyResponse(expectedResponsesInOrder[i], r.Destination))) + .Select((r,i) => Task.Run(async () => await sut.ApplyResponse(expectedResponsesInOrder[i], r.ActivityId))) .ToList(); await Task.WhenAll(applyResponseTasks); diff --git a/source/Halibut.Tests/Support/LatestClientAndLatestServiceBuilder.cs b/source/Halibut.Tests/Support/LatestClientAndLatestServiceBuilder.cs index 2d702079a..6fb4af75e 100644 --- a/source/Halibut.Tests/Support/LatestClientAndLatestServiceBuilder.cs +++ b/source/Halibut.Tests/Support/LatestClientAndLatestServiceBuilder.cs @@ -4,6 +4,7 @@ using System.Threading.Tasks; using Halibut.Diagnostics; using Halibut.Logging; +using Halibut.Queue; using Halibut.ServiceModel; using Halibut.TestProxy; using Halibut.Tests.TestServices; @@ -221,6 +222,11 @@ public LatestClientAndLatestServiceBuilder WithProxy(out Reference pendingRequestQueueFactory) + { + return WithPendingRequestQueueFactory((_, logFactory) => pendingRequestQueueFactory(logFactory)); + } + + public LatestClientAndLatestServiceBuilder WithPendingRequestQueueFactory(Func pendingRequestQueueFactory) { clientBuilder.WithPendingRequestQueueFactory(pendingRequestQueueFactory); return this; diff --git a/source/Halibut.Tests/Support/LatestClientBuilder.cs b/source/Halibut.Tests/Support/LatestClientBuilder.cs index 3352eaeab..38af89104 100644 --- a/source/Halibut.Tests/Support/LatestClientBuilder.cs +++ b/source/Halibut.Tests/Support/LatestClientBuilder.cs @@ -5,6 +5,7 @@ using Halibut.Diagnostics; using Halibut.Diagnostics.LogCreators; using Halibut.Logging; +using Halibut.Queue; using Halibut.ServiceModel; using Halibut.Tests.Support.Logging; using Halibut.Transport.Observability; @@ -24,7 +25,7 @@ public class LatestClientBuilder : IClientBuilder IRpcObserver? clientRpcObserver; Func? portForwarderFactory; Reference? portForwarderReference; - Func? pendingRequestQueueFactory; + Func? pendingRequestQueueFactory; Action? pendingRequestQueueFactoryBuilder; ProxyDetails? proxyDetails; LogLevel halibutLogLevel = LogLevel.Trace; @@ -115,7 +116,7 @@ public LatestClientBuilder WithPortForwarding(out Reference portF return this; } - public LatestClientBuilder WithPendingRequestQueueFactory(Func pendingRequestQueueFactory) + public LatestClientBuilder WithPendingRequestQueueFactory(Func pendingRequestQueueFactory) { this.pendingRequestQueueFactory = pendingRequestQueueFactory; return this; @@ -184,12 +185,11 @@ public async Task Build(CancellationToken cancellationToken) { var octopusLogFactory = BuildClientLogger(); - var factory = CreatePendingRequestQueueFactory(octopusLogFactory); var clientBuilder = new HalibutRuntimeBuilder() .WithServerCertificate(clientCertAndThumbprint.Certificate2) .WithLogFactory(octopusLogFactory) - .WithPendingRequestQueueFactory(factory) + .WithPendingRequestQueueFactory(serializer => CreatePendingRequestQueueFactory(serializer, octopusLogFactory)) .WithTrustProvider(clientTrustProvider!) .WithStreamFactoryIfNotNull(clientStreamFactory) .WithControlMessageObserverIfNotNull(controlMessageObserver) @@ -248,11 +248,11 @@ public async Task Build(CancellationToken cancellationToken) return new LatestClient(client, clientListeningUri, clientTrustsThumbprint, portForwarder, proxyDetails, serviceConnectionType, disposableCollection); } - IPendingRequestQueueFactory CreatePendingRequestQueueFactory(ILogFactory octopusLogFactory) + IPendingRequestQueueFactory CreatePendingRequestQueueFactory(QueueMessageSerializer queueMessageSerializer, ILogFactory octopusLogFactory) { if (pendingRequestQueueFactory != null) { - return pendingRequestQueueFactory(octopusLogFactory); + return pendingRequestQueueFactory(queueMessageSerializer, octopusLogFactory); } var pendingRequestQueueFactoryBuilder = new PendingRequestQueueFactoryBuilder(octopusLogFactory, halibutTimeoutsAndLimits); diff --git a/source/Halibut.Tests/Support/PendingRequestQueueFactories/CancelWhenRequestDequeuedPendingRequestQueueFactory.cs b/source/Halibut.Tests/Support/PendingRequestQueueFactories/CancelWhenRequestDequeuedPendingRequestQueueFactory.cs index 1cb9371d9..8f313a69a 100644 --- a/source/Halibut.Tests/Support/PendingRequestQueueFactories/CancelWhenRequestDequeuedPendingRequestQueueFactory.cs +++ b/source/Halibut.Tests/Support/PendingRequestQueueFactories/CancelWhenRequestDequeuedPendingRequestQueueFactory.cs @@ -47,10 +47,10 @@ public Decorator(IPendingRequestQueue inner, CancellationTokenSource cancellatio public bool IsEmpty => inner.IsEmpty; public int Count => inner.Count; - public async Task ApplyResponse(ResponseMessage response, ServiceEndPoint destination) + public async Task ApplyResponse(ResponseMessage response, Guid requestActivityId) { onResponseApplied?.Invoke(response); - await inner.ApplyResponse(response, destination); + await inner.ApplyResponse(response, requestActivityId); } public async Task DequeueAsync(CancellationToken cancellationToken) diff --git a/source/Halibut.Tests/Support/PendingRequestQueueFactories/CancelWhenRequestQueuedPendingRequestQueueFactory.cs b/source/Halibut.Tests/Support/PendingRequestQueueFactories/CancelWhenRequestQueuedPendingRequestQueueFactory.cs index 63e0ca75b..0681bcf8a 100644 --- a/source/Halibut.Tests/Support/PendingRequestQueueFactories/CancelWhenRequestQueuedPendingRequestQueueFactory.cs +++ b/source/Halibut.Tests/Support/PendingRequestQueueFactories/CancelWhenRequestQueuedPendingRequestQueueFactory.cs @@ -41,7 +41,7 @@ public Decorator(IPendingRequestQueue inner, CancellationTokenSource[] cancellat public bool IsEmpty => inner.IsEmpty; public int Count => inner.Count; - public async Task ApplyResponse(ResponseMessage response, ServiceEndPoint destination) => await inner.ApplyResponse(response, destination); + public async Task ApplyResponse(ResponseMessage response, Guid requestActivityId) => await inner.ApplyResponse(response, requestActivityId); public async Task DequeueAsync(CancellationToken cancellationToken) => await inner.DequeueAsync(cancellationToken); public async Task QueueAndWaitAsync(RequestMessage request, CancellationToken cancellationTokens) diff --git a/source/Halibut/Halibut.csproj b/source/Halibut/Halibut.csproj index 1163b6c28..383684887 100644 --- a/source/Halibut/Halibut.csproj +++ b/source/Halibut/Halibut.csproj @@ -30,7 +30,7 @@ - + diff --git a/source/Halibut/HalibutRuntimeBuilder.cs b/source/Halibut/HalibutRuntimeBuilder.cs index 4f0ad832b..83f2fd8aa 100644 --- a/source/Halibut/HalibutRuntimeBuilder.cs +++ b/source/Halibut/HalibutRuntimeBuilder.cs @@ -2,6 +2,7 @@ using System.Linq; using System.Security.Cryptography.X509Certificates; using Halibut.Diagnostics; +using Halibut.Queue; using Halibut.ServiceModel; using Halibut.Transport.Observability; using Halibut.Transport.Protocol; @@ -13,7 +14,7 @@ namespace Halibut public class HalibutRuntimeBuilder { ILogFactory? logFactory; - IPendingRequestQueueFactory? queueFactory; + Func? queueFactoryFactory; X509Certificate2? serverCertificate; IServiceFactory? serviceFactory; ITrustProvider? trustProvider; @@ -66,7 +67,13 @@ public HalibutRuntimeBuilder WithTrustProvider(ITrustProvider trustProvider) public HalibutRuntimeBuilder WithPendingRequestQueueFactory(IPendingRequestQueueFactory queueFactory) { - this.queueFactory = queueFactory; + this.queueFactoryFactory = _ => queueFactory; + return this; + } + + public HalibutRuntimeBuilder WithPendingRequestQueueFactory(Func queueFactory) + { + this.queueFactoryFactory = queueFactory; return this; } @@ -133,7 +140,7 @@ public HalibutRuntime Build() var serviceFactory = this.serviceFactory ?? new NullServiceFactory(); if (serverCertificate == null) throw new ArgumentException($"Set a server certificate with {nameof(WithServerCertificate)} before calling {nameof(Build)}", nameof(serverCertificate)); var logFactory = this.logFactory ?? new LogFactory(); - var queueFactory = this.queueFactory ?? new PendingRequestQueueFactoryAsync(halibutTimeoutsAndLimits, logFactory); + var trustProvider = this.trustProvider ?? new DefaultTrustProvider(); //use either the supplied type registry, or configure the default one @@ -153,6 +160,11 @@ public HalibutRuntime Build() var builder = new MessageSerializerBuilder(logFactory); configureMessageSerializerBuilder?.Invoke(builder); var messageSerializer = builder.WithTypeRegistry(typeRegistry).Build(); + + + var queueMessageSerializer = new QueueMessageSerializer(messageSerializer.CreateStreamCapturingSerializer); + var queueFactory = this.queueFactoryFactory?.Invoke(queueMessageSerializer)??new PendingRequestQueueFactoryAsync(halibutTimeoutsAndLimits, logFactory); + var streamFactory = this.streamFactory ?? new StreamFactory(); var connectionsObserver = this.connectionsObserver ?? NoOpConnectionsObserver.Instance; var rpcObserver = this.rpcObserver ?? new NoRpcObserver(); diff --git a/source/Halibut/Queue/QueueMessageSerializer.cs b/source/Halibut/Queue/QueueMessageSerializer.cs index 37b9d2164..96bb5b491 100644 --- a/source/Halibut/Queue/QueueMessageSerializer.cs +++ b/source/Halibut/Queue/QueueMessageSerializer.cs @@ -22,6 +22,7 @@ namespace Halibut.Queue { + // TODO make an interface public class QueueMessageSerializer { readonly Func createStreamCapturingSerializer; @@ -33,7 +34,7 @@ public QueueMessageSerializer(Func createStreamCa public (string, IReadOnlyList) WriteMessage(T message) { - IReadOnlyList datatStreams; + IReadOnlyList dataStreams; var sb = new StringBuilder(); using var sw = new StringWriter(sb, CultureInfo.InvariantCulture); @@ -41,10 +42,10 @@ public QueueMessageSerializer(Func createStreamCa { var streamCapturingSerializer = createStreamCapturingSerializer(); streamCapturingSerializer.Serializer.Serialize(jsonTextWriter, new MessageEnvelope { Message = message! }); - datatStreams = streamCapturingSerializer.DataStreams; + dataStreams = streamCapturingSerializer.DataStreams; } - return (sb.ToString(), datatStreams); + return (sb.ToString(), dataStreams); } public (T Message, IReadOnlyList DataStreams) ReadMessage(string json) diff --git a/source/Halibut/Queue/Redis/HalibutHalibutRedisTransport.cs b/source/Halibut/Queue/Redis/HalibutHalibutRedisTransport.cs index 8fe55af6e..923bacfec 100644 --- a/source/Halibut/Queue/Redis/HalibutHalibutRedisTransport.cs +++ b/source/Halibut/Queue/Redis/HalibutHalibutRedisTransport.cs @@ -14,158 +14,139 @@ using System; using System.Diagnostics; +using System.Threading; using System.Threading.Tasks; +using Halibut.Util; using Newtonsoft.Json; +using Nito.AsyncEx; +using StackExchange.Redis; namespace Halibut.Queue.Redis { - class HalibutHalibutRedisTransport : IHalibutRedisTransport + public class HalibutHalibutRedisTransport { const string Namespace = "octopus:server:halibut"; - static readonly string RequestChannel = $"{Namespace}:request"; - static readonly string ResponseChannel = $"{Namespace}:response"; - static readonly string RequestPoppedChannel = $"{Namespace}:requestpopped"; - - readonly IRedisConnection connection; - //TODO: Rethink .net event - public event EventHandler? NewRequestEvent; - public event EventHandler? NewResponseEvent; - public event EventHandler? RequestPoppedEvent; + readonly RedisFacade facade; - public HalibutHalibutRedisTransport(IRedisConnection connection) + public HalibutHalibutRedisTransport(RedisFacade facade) { - this.connection = connection; - - SubscribeToChannels(); + this.facade = facade; } - void SubscribeToChannels() + // Request Pulse + static string RequestMessagesPulseChannelName(Uri endpoint) { - - connection.SubscribeToChannel( - RequestChannel, - message => - { - - var queueItem = JsonConvert.DeserializeObject(message.Message!); - NewRequestEvent?.Invoke(this, queueItem!); - }); - - connection.SubscribeToChannel( - ResponseChannel, - message => - { - var queueItem = JsonConvert.DeserializeObject(message.Message!); - NewResponseEvent?.Invoke(this, queueItem!); - }); - - connection.SubscribeToChannel( - RequestPoppedChannel, - message => - { - var queueItem = JsonConvert.DeserializeObject(message.Message!); - RequestPoppedEvent?.Invoke(this, queueItem!); - }); + return $"{Namespace}::requestpulsechannel::{endpoint}"; } - - static string ResponseKey(string requestId) + + public async Task SubscribeToRequestMessagePulseChannel(Uri endpoint, Action onRequestMessagePulse) { - var validRequestId = requestId.Replace(":", "_"); - return $"{Namespace}:response:{validRequestId}"; + var channelName = RequestMessagesPulseChannelName(endpoint); + return await facade.SubscribeToChannel(channelName, async message => + { + await Task.CompletedTask; + onRequestMessagePulse(message); + }); } - - static string RequestKey(Uri endpoint) + + public async Task PulseRequestPushedToEndpoint(Uri endpoint, CancellationToken cancellationToken) { - return $"{Namespace}:request:{endpoint}"; + var channelName = RequestMessagesPulseChannelName(endpoint); + string emptyJson = "{}"; // Maybe we will actually want to share data in the future, empty json means we can add stuff later. + await facade.PublishToChannel(channelName, emptyJson); } - - public async Task SetResponse(Uri endpoint, string requestId, string payload) + + // Request IDs list + + static string KeyForNextRequestGuidInListForEndpoint(Uri endpoint) { - var responseKey = ResponseKey(requestId); - var responseExpiry = TimeSpan.FromMinutes(10); - - await connection.StringSet(responseKey, payload, responseExpiry); - await PublishQueueItem(ResponseChannel, endpoint, requestId); + return $"{Namespace}::NextRequestInListForEndpoint::{endpoint}"; } - public async Task GetDeleteResponse(string requestId) + public async Task PushRequestGuidOnToQueue(Uri endpoint, Guid guid, CancellationToken cancellationToken) + { + // TODO should we overcomplicate this with json? + // TODO TTL + await facade.ListRightPushAsync(KeyForNextRequestGuidInListForEndpoint(endpoint), guid.ToString()); + } + + public async Task TryPopNextRequestGuid(Uri endpoint, CancellationToken cancellationToken) { - var responseKey = ResponseKey(requestId); - var response = await connection.StringGetDelete(responseKey); - return response; + var result = await facade.ListLeftPopAsync(KeyForNextRequestGuidInListForEndpoint(endpoint)); + return result.ToGuid(); + } + + // Request Message + + static string RequestMessageKey(Uri endpoint, Guid requestId) + { + return $"{Namespace}::requestpulsechannel::{endpoint}::{requestId}"; } - public async Task PushRequest(Uri endpoint, string requestId, string payload) + static string RequestField = "RequestField"; + + public async Task PutRequest(Uri endpoint, Guid requestId, string payload, CancellationToken cancellationToken) { - var redisQueueItem = new RedisHalibuteQueueItem(requestId,payload); + var redisQueueItem = new RedisHalibutQueueItem2(requestId,payload); var serialisedQueueItem = JsonConvert.SerializeObject(redisQueueItem); - var requestKey = RequestKey(endpoint); - - await connection.ListRightPush(requestKey, serialisedQueueItem); + var requestKey = RequestMessageKey(endpoint, requestId); - await PublishQueueItem(RequestChannel, endpoint, requestId); + await facade.SetInHash(requestKey, RequestField, serialisedQueueItem); } - public async Task PopRequest(Uri endpoint) + public async Task TryGetAndRemoveRequest(Uri endpoint, Guid requestId, CancellationToken cancellationToken) { - var requestKey = RequestKey(endpoint); + var requestKey = RequestMessageKey(endpoint, requestId); + var requestMessage = await facade.TryGetAndDeleteFromHash(requestKey, RequestField); + if (requestMessage == null) return null; - var requestMessage = await connection.ListLeftPopAsync(requestKey); - if (requestMessage is null) return null; - - var redisQueueItem = JsonConvert.DeserializeObject(requestMessage); + var redisQueueItem = JsonConvert.DeserializeObject(requestMessage); if (redisQueueItem is null) return null; - await PublishQueueItem(RequestPoppedChannel, endpoint, redisQueueItem.RequestId); - return redisQueueItem; + return redisQueueItem.PayloadJson; } - - async Task PublishQueueItem(string channel, Uri endpoint, string requestId) + + + // Response channel + static string ResponseMessagesChannelName(Uri endpoint, Guid requestId) { - var queueItem = new RedisHalibutQueueItem( - endpoint, - requestId); - - await connection.PublishToChannel(channel, queueItem); + return $"{Namespace}::ResponseMessagesChannelName::{endpoint}::{requestId}"; + } + + public async Task SubScribeToResponses(Uri endpoint, Guid requestOfResponseToWaitFor, + Func onResponse, + CancellationToken cancellationToken) + { + var channelName = ResponseMessagesChannelName(endpoint, requestOfResponseToWaitFor); + var responseReceivedEvent = new AsyncManualResetEvent(false); + return await facade.SubscribeToChannel(channelName, async foo => + { + string? response = foo.Message; + if(response is not null) await onResponse(response, cancellationToken); + }); + } + + public async Task PublishResponse(Uri endpoint, Guid requestId, string payload, CancellationToken cancellationToken) + { + var channelName = ResponseMessagesChannelName(endpoint, requestId); + await facade.PublishToChannel(channelName, payload); } + } - interface IHalibutRedisTransport - { - event EventHandler NewRequestEvent; - event EventHandler NewResponseEvent; - event EventHandler RequestPoppedEvent; - - Task SetResponse(Uri endpoint, string requestId, string payload); - Task PushRequest(Uri endpoint, string requestId, string payload); - Task GetDeleteResponse(string requestId); - Task PopRequest(Uri endpoint); - } - public class RedisHalibuteQueueItem + public class RedisHalibutQueueItem2 { - public RedisHalibuteQueueItem(string requestId, string payloadJson) + public RedisHalibutQueueItem2(Guid requestId, string payloadJson) { RequestId = requestId; PayloadJson = payloadJson; } - public string RequestId { get; protected set; } + public Guid RequestId { get; protected set; } public string PayloadJson { get; protected set; } } - - public class RedisHalibutQueueItem - { - public Uri Endpoint { get; protected set; } - public string RequestId { get; protected set; } - - public RedisHalibutQueueItem(Uri endpoint, string requestId) - { - Endpoint = endpoint; - RequestId = requestId; - } - } } \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/MessageReaderWriter.cs b/source/Halibut/Queue/Redis/MessageReaderWriter.cs new file mode 100644 index 000000000..f2e956068 --- /dev/null +++ b/source/Halibut/Queue/Redis/MessageReaderWriter.cs @@ -0,0 +1,60 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System.Threading; +using System.Threading.Tasks; +using Halibut.Transport.Protocol; + +namespace Halibut.Queue.Redis +{ + public class MessageReaderWriter + { + readonly QueueMessageSerializer queueMessageSerializer; + readonly IStoreDataStreamsForDistributedQueues storeDataStreamsForDistributedQueues; + + public MessageReaderWriter(QueueMessageSerializer queueMessageSerializer, IStoreDataStreamsForDistributedQueues storeDataStreamsForDistributedQueues) + { + this.queueMessageSerializer = queueMessageSerializer; + this.storeDataStreamsForDistributedQueues = storeDataStreamsForDistributedQueues; + } + + public async Task PrepareRequest(RequestMessage request, CancellationToken cancellationToken) + { + var (payload, dataStreams) = queueMessageSerializer.WriteMessage(request); + await storeDataStreamsForDistributedQueues.StoreDataStreams(dataStreams, cancellationToken); + return payload; + } + + public async Task ReadRequest(string jsonRequest, CancellationToken cancellationToken) + { + var (request, dataStreams) = queueMessageSerializer.ReadMessage(jsonRequest); + await storeDataStreamsForDistributedQueues.ReHydrateDataStreams(dataStreams, cancellationToken); + return request; + } + + public async Task PrepareResponse(ResponseMessage response, CancellationToken cancellationToken) + { + var (payload, dataStreams) = queueMessageSerializer.WriteMessage(response); + await storeDataStreamsForDistributedQueues.StoreDataStreams(dataStreams, cancellationToken); + return payload; + } + + public async Task ReadResponse(string jsonResponse, CancellationToken cancellationToken) + { + var (response, dataStreams) = queueMessageSerializer.ReadMessage(jsonResponse); + await storeDataStreamsForDistributedQueues.ReHydrateDataStreams(dataStreams, cancellationToken); + return response; + } + } +} \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/PendingRequestQueue.cs b/source/Halibut/Queue/Redis/PendingRequestQueue.cs new file mode 100644 index 000000000..f77b95d3f --- /dev/null +++ b/source/Halibut/Queue/Redis/PendingRequestQueue.cs @@ -0,0 +1,192 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; +using System.Threading; +using System.Threading.Tasks; +using Halibut.Diagnostics; +using Halibut.ServiceModel; +using Halibut.Transport.Protocol; +using Nito.AsyncEx; + +namespace Halibut.Queue.Redis +{ + class RedisPendingRequestQueue : IPendingRequestQueue, IDisposable + { + readonly Uri endpoint; + readonly ILog log; + readonly HalibutHalibutRedisTransport halibutRedisTransport; + readonly HalibutTimeoutsAndLimits halibutTimeoutsAndLimits; + readonly MessageReaderWriter messageReaderWriter; + readonly AsyncManualResetEvent hasItemsForEndpoint = new(); + + IAsyncDisposable PulseChannelSubDisposer { get; } + + public RedisPendingRequestQueue( + Uri endpoint, + ILog log, + HalibutHalibutRedisTransport halibutRedisTransport, + MessageReaderWriter messageReaderWriter, + HalibutTimeoutsAndLimits halibutTimeoutsAndLimits) + { + this.endpoint = endpoint; + this.log = log; + this.messageReaderWriter = messageReaderWriter; + this.halibutRedisTransport = halibutRedisTransport; + this.halibutTimeoutsAndLimits = halibutTimeoutsAndLimits; + + // TODO: can we unsub if no tentacle is asking for a work for an extended period of time? + // and also NOT sub if the queue is being created to send work. + // The advice is many channels with few subscribers is better than a single channel with many subscribers. + // If we end up with too many channels, we could shared the channels based on modulo of the hash of the endpoint, + // which means we might have only 1000 channels and num_tentacles/1000 subscribers to each channel. For 300K tentacles. + PulseChannelSubDisposer = this.halibutRedisTransport.SubscribeToRequestMessagePulseChannel(endpoint, _ => hasItemsForEndpoint.Set()) + .GetAwaiter().GetResult(); + } + + public async ValueTask DisposeAsync() + { + await PulseChannelSubDisposer.DisposeAsync(); + } + + public async Task QueueAndWaitAsync(RequestMessage request, CancellationToken cancellationToken) + { + // TODO: redis goes down + // TODO: Other node goes down. + // TODO: Respect cancellation token + var pending = new RedisPendingRequest(request, log, halibutTimeoutsAndLimits.PollingRequestQueueTimeout); + + var payload = await messageReaderWriter.PrepareRequest(request, cancellationToken); + + // Start listening for a response to the request, we don't want to miss the response. + await using var _ = await SubscribeToResponse(request.ActivityId, pending.SetResponse, cancellationToken); + + // Make the request available before tell people it is available. + await halibutRedisTransport.PutRequest(endpoint, request.ActivityId, payload, cancellationToken); + await halibutRedisTransport.PushRequestGuidOnToQueue(endpoint, request.ActivityId, cancellationToken); + await halibutRedisTransport.PulseRequestPushedToEndpoint(endpoint, cancellationToken); + + await pending.WaitUntilComplete(cancellationToken, async () => + { + // The time the message is allowed to sit on the queue for has elapsed. + // Let's try to pop if from the queue, either: + // - We pop it, which means it was never collected so let pending deal with the timeout. + // - We could not pop it, which means it was collected. + var requestJson = await halibutRedisTransport.TryGetAndRemoveRequest(endpoint, request.ActivityId, cancellationToken); + if (requestJson != null) + { + pending.BeginTransfer(); + } + }); + + return pending.Response!; + } + + async Task SubscribeToResponse(Guid activityId, + Action onResponse, + CancellationToken cancellationToken) + { + return await halibutRedisTransport.SubScribeToResponses(endpoint, activityId, async (responseJson, ct) => + { + var response = await messageReaderWriter.ReadResponse(responseJson, ct); + onResponse(response); + }, cancellationToken); + } + + public bool IsEmpty => throw new NotImplementedException(); + public int Count => throw new NotImplementedException(); + + public async Task DequeueAsync(CancellationToken cancellationToken) + { + var pending = await DequeueNextAsync(); + if (pending == null) return null; + return new RequestMessageWithCancellationToken(pending, CancellationToken.None); + } + + public async Task ApplyResponse(ResponseMessage response, Guid requestActivityId) + { + if (response == null) return; + + var cancellationToken = CancellationToken.None; + + // This node has now completed the RPC, and so the response must be sent + // back to the node which sent the response + + var payload = await messageReaderWriter.PrepareResponse(response, cancellationToken); + await halibutRedisTransport.PublishResponse(endpoint, requestActivityId, payload, cancellationToken); + } + + async Task DequeueNextAsync() + { + var cancellationToken = CancellationToken.None; + + using var cancellationTokenSource = CancellationTokenSource.CreateLinkedTokenSource(cancellationToken); + try + { + // TODO can we avoid going to redis here? + var first = await TryRemoveNextItemFromQueue(cancellationToken); + if (first != null) return first; + + await Task.WhenAny(hasItemsForEndpoint.WaitAsync(cancellationTokenSource.Token), Task.Delay(new HalibutTimeoutsAndLimits().PollingQueueWaitTimeout, cancellationTokenSource.Token)); + + if (!hasItemsForEndpoint.IsSet) + { + // Timed out waiting for something to go on the queue, send back a null to tentacle + // to keep the connection healthy. + return null; + } + + hasItemsForEndpoint.Reset(); + return await TryRemoveNextItemFromQueue(cancellationToken); + } + finally + { + await cancellationTokenSource.CancelAsync(); + } + } + + async Task TryRemoveNextItemFromQueue(CancellationToken cancellationToken) + { + while (true) + { + var activityId = await halibutRedisTransport.TryPopNextRequestGuid(endpoint, cancellationToken); + + if (activityId is null) + { + // Nothing is on the queue. + return null; + } + + var jsonRequest = await halibutRedisTransport.TryGetAndRemoveRequest(endpoint, activityId.Value, cancellationToken); + + if (jsonRequest == null) + { + // This request has been picked up by someone else, go around the loop and look for something else to do. + continue; + } + + var request = await messageReaderWriter.ReadRequest(jsonRequest, cancellationToken); + + return request; + } + } + + + + public void Dispose() + { + DisposeAsync().GetAwaiter().GetResult(); + } + } +} \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/RedisConnection.cs b/source/Halibut/Queue/Redis/RedisFacade.cs similarity index 53% rename from source/Halibut/Queue/Redis/RedisConnection.cs rename to source/Halibut/Queue/Redis/RedisFacade.cs index 0e688dd3a..e340c8706 100644 --- a/source/Halibut/Queue/Redis/RedisConnection.cs +++ b/source/Halibut/Queue/Redis/RedisFacade.cs @@ -13,19 +13,21 @@ // limitations under the License. using System; +using System.Threading; using System.Threading.Tasks; +using Halibut.Util; using Newtonsoft.Json; using StackExchange.Redis; namespace Halibut.Queue.Redis { - class RedisConnection : IRedisConnection, IDisposable + public class RedisFacade : IDisposable { readonly Lazy connection; ConnectionMultiplexer Connection => connection.Value; - public RedisConnection(string redisHost) + public RedisFacade(string redisHost) { connection = new Lazy(() => @@ -42,43 +44,53 @@ public void Dispose() } } - public void SubscribeToChannel(string channel, Action onMessage) + public async Task SubscribeToChannel(string channelName, Func onMessage) { - Connection.GetSubscriber() - .Subscribe(channel) - .OnMessage(onMessage); + // TODO ever call needs to respect the cancellation token + var channel = await Connection.GetSubscriber() + .SubscribeAsync(new RedisChannel(channelName, RedisChannel.PatternMode.Literal)); + + channel.OnMessage(onMessage); + + return new FuncAsyncDisposable(() => channel.UnsubscribeAsync()); } - - public async Task PublishToChannel(string channel, T payload) - { - var serialized = JsonConvert.SerializeObject(payload); - await PublishToChannel(channel, serialized); - - } - - public async Task PublishToChannel(string channel, string payload) + + public async Task SetInHash(string key, string field, string payload) { - var subscriber = Connection.GetSubscriber(); - await subscriber.PublishAsync(channel, payload); + key = "hash::" + key; + // TODO: TTL + // TODO ever call needs to respect the cancellation token + var ttl = new TimeSpan(9, 9, 9); + var database = Connection.GetDatabase(); + await database.HashSetAsync(key, new RedisValue(field), new RedisValue(payload)); + await database.KeyExpireAsync(key, ttl); } - public async Task StringSet(string key, string payload, TimeSpan expiry) + public async Task TryGetAndDeleteFromHash(string key, string field) { + // TODO ever call needs to respect the cancellation token + key = "hash::" + key; var database = Connection.GetDatabase(); - await database.StringSetAsync(key, payload, expiry:expiry); + var value = await database.HashGetAsync(key, new RedisValue(field)); + var res = await database.KeyDeleteAsync(key); + if (!res) + { + // Someone else deleted this, so return nothing to make the get and delete appear to be atomic. + return null; + } + return value; } - public async Task ListRightPush(string key, string payload) + public async Task PublishToChannel(string channel, string payload) { - var database = Connection.GetDatabase(); - await database.ListRightPushAsync(key, payload); + var subscriber = Connection.GetSubscriber(); + await subscriber.PublishAsync(new RedisChannel(channel, RedisChannel.PatternMode.Literal), payload); } - public async Task StringGetDelete(string key) + public async Task ListRightPushAsync(string key, string payload) { var database = Connection.GetDatabase(); - var payload = await database.StringGetDeleteAsync(key); - return payload; + await database.ListRightPushAsync(key, payload); } public async Task ListLeftPopAsync(string key) @@ -94,17 +106,4 @@ public async Task ListRightPush(string key, string payload) } } - public interface IRedisConnection - { - void SubscribeToChannel(string channel, Action onMessage); - - Task PublishToChannel(string channel, string payload); - Task PublishToChannel(string channel, T payload); - - Task StringSet(string key, string payload, TimeSpan expiry); - Task ListRightPush(string key, string payload); - Task StringGetDelete(string key); - - Task ListLeftPopAsync(string key); - } } \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/RedisPendingRequest.cs b/source/Halibut/Queue/Redis/RedisPendingRequest.cs new file mode 100644 index 000000000..772483948 --- /dev/null +++ b/source/Halibut/Queue/Redis/RedisPendingRequest.cs @@ -0,0 +1,114 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; +using System.Threading; +using System.Threading.Tasks; +using Halibut.Diagnostics; +using Halibut.Transport.Protocol; + +namespace Halibut.Queue.Redis +{ + // TODO this code is mostly generic, so it could be shared + public class RedisPendingRequest + { + readonly RequestMessage request; + readonly ILog log; + readonly ManualResetEventSlim waiter; + readonly object sync = new(); + bool transferBegun; + bool completed; + + readonly TimeSpan pollingRequestMaximumMessageProcessingTimeout; + + public RedisPendingRequest(RequestMessage request, ILog log, TimeSpan pollingRequestMaximumMessageProcessingTimeout) + { + this.request = request; + this.log = log; + this.pollingRequestMaximumMessageProcessingTimeout = pollingRequestMaximumMessageProcessingTimeout; + waiter = new ManualResetEventSlim(false); + } + + public async Task WaitUntilComplete(CancellationToken cancellationToken, Func pollingRequestQueueTimeElapsed) + { + await Task.CompletedTask; + log.Write(EventType.MessageExchange, "Request {0} was queued", request); + + var success = waiter.Wait(request.Destination.PollingRequestQueueTimeout, cancellationToken); + if (success) + { + log.Write(EventType.MessageExchange, "Request {0} was collected by the polling endpoint", request); + return; + } + + try + { + // Let em know the amount of time the request can sit on the queue for has elapsed. + await pollingRequestQueueTimeElapsed(); + } + catch + { + + } + var waitForTransferToComplete = false; + lock (sync) + { + if (transferBegun) + waitForTransferToComplete = true; + else + completed = true; + } + + if (waitForTransferToComplete) + { + success = waiter.Wait(pollingRequestMaximumMessageProcessingTimeout); + if (success) + log.Write(EventType.MessageExchange, "Request {0} was eventually collected by the polling endpoint", request); + else + SetResponse(ResponseMessage.FromException(request, new TimeoutException(string.Format("A request was sent to a polling endpoint, the polling endpoint collected it but did not respond in the allowed time ({0}), so the request timed out.", pollingRequestMaximumMessageProcessingTimeout)))); + } + else + { + log.Write(EventType.MessageExchange, "Request {0} timed out before it could be collected by the polling endpoint", request); + SetResponse(ResponseMessage.FromException(request, new TimeoutException(string.Format("A request was sent to a polling endpoint, but the polling endpoint did not collect the request within the allowed time ({0}), so the request timed out.", request.Destination.PollingRequestQueueTimeout)))); + } + } + + public bool BeginTransfer() + { + lock (sync) + { + if (completed) + return false; + + transferBegun = true; + return true; + } + } + + public ResponseMessage? Response { get; private set; } + + public void SetResponse(ResponseMessage response) + { + lock (sync) + { + if (Response == null) + { + Response = response; + waiter.Set(); + } + } + } + } +} \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs deleted file mode 100644 index afa1ca2b6..000000000 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ /dev/null @@ -1,278 +0,0 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -using System; -using System.Collections.Generic; -using System.Threading; -using System.Threading.Tasks; -using Halibut.Diagnostics; -using Halibut.ServiceModel; -using Halibut.Transport.Protocol; -using Nito.AsyncEx; - -namespace Halibut.Queue.Redis -{ - class RedisPendingRequestQueue : IPendingRequestQueue, IDisposable - { - readonly public static TimeSpan PollingRequestMaximumMessageProcessingTimeout = TimeSpan.FromHours(33); - - readonly Dictionary inProgress = new(); - //TODO: Do we really need the lock (which came from original implementation) - readonly object sync = new(); - readonly AsyncManualResetEvent hasItemsForEndpoint = new(); - readonly Uri endpoint; - readonly ILog log; - readonly IHalibutRedisTransport halibutRedisTransport; - - readonly QueueMessageSerializer queueMessageSerializer; - readonly IStoreDataStreamsForDistributedQueues storeDataStreamsForDistributedQueues; - - public delegate RedisPendingRequestQueue Factory(Uri endpoint, ILog log); - - public RedisPendingRequestQueue( - Uri endpoint, - ILog log, - IHalibutRedisTransport halibutRedisTransport, - QueueMessageSerializer queueMessageSerializer, - IStoreDataStreamsForDistributedQueues storeDataStreamsForDistributedQueues) - { - this.endpoint = endpoint; - this.log = log; - this.halibutRedisTransport = halibutRedisTransport; - this.queueMessageSerializer = queueMessageSerializer; - this.storeDataStreamsForDistributedQueues = storeDataStreamsForDistributedQueues; - - halibutRedisTransport.NewRequestEvent += HandleNewRequest; - halibutRedisTransport.NewResponseEvent += HandleNewResponse; - halibutRedisTransport.RequestPoppedEvent += HandleRequestPopped; - } - - public void Dispose() - { - halibutRedisTransport.NewRequestEvent += HandleNewRequest; - halibutRedisTransport.NewResponseEvent += HandleNewResponse; - } - - void HandleNewRequest(object? sender, RedisHalibutQueueItem e) - { - lock (sync) - { - if (e.Endpoint == endpoint) - { - hasItemsForEndpoint.Set(); - } - } - } - - void HandleNewResponse(object? sender, RedisHalibutQueueItem e) - { - lock (sync) - { - if (inProgress.TryGetValue(e.RequestId, out var pendingRequest)) - { - var responseMessage = halibutRedisTransport.GetDeleteResponse(e.RequestId).GetAwaiter().GetResult(); - - var (response, dataStreams) = this.queueMessageSerializer.ReadMessage(responseMessage!); - storeDataStreamsForDistributedQueues.ReHydrateDataStreams(dataStreams, CancellationToken.None).GetAwaiter().GetResult(); - - pendingRequest.SetResponse(response!); - } - } - } - - void HandleRequestPopped(object? sender, RedisHalibutQueueItem e) - { - lock (sync) - { - if (inProgress.TryGetValue(e.RequestId, out var pendingRequest)) - { - pendingRequest.BeginTransfer(); - } - } - } - - public Task ApplyResponse(ResponseMessage response, ServiceEndPoint destination) - { - throw new NotImplementedException(); - } - - public async Task QueueAndWaitAsync(RequestMessage request, CancellationToken cancellationToken) - { - var pending = new PendingRequest(request, log); - - lock (sync) - { - inProgress.Add(request.Id, pending); - } - - var (payload, dataStreams) = queueMessageSerializer.WriteMessage(request); - await storeDataStreamsForDistributedQueues.StoreDataStreams(dataStreams, cancellationToken); - - await halibutRedisTransport.PushRequest(endpoint, request.Id, payload); - - pending.WaitUntilComplete(cancellationToken); - - lock (sync) - { - inProgress.Remove(request.Id); - } - - return pending.Response!; - } - - public bool IsEmpty => throw new NotImplementedException(); - public int Count => throw new NotImplementedException(); - - public async Task DequeueAsync(CancellationToken cancellationToken) - { - var pending = await DequeueNextAsync(); - if (pending == null) return null; - return new RequestMessageWithCancellationToken(pending, CancellationToken.None); - } - - async Task DequeueNextAsync() - { - var first = await TakeFirst(); - if (first != null) return first; - - await Task.WhenAny(hasItemsForEndpoint.WaitAsync(), Task.Delay(new HalibutTimeoutsAndLimits().PollingQueueWaitTimeout)); - hasItemsForEndpoint.Reset(); - return await TakeFirst(); - } - - async Task TakeFirst() - { - var redisQueueItem = await halibutRedisTransport.PopRequest(endpoint); - - if (redisQueueItem is null) return null; - - var (request, dataStreams) = queueMessageSerializer.ReadMessage(redisQueueItem.PayloadJson); - storeDataStreamsForDistributedQueues.ReHydrateDataStreams(dataStreams, CancellationToken.None).GetAwaiter().GetResult(); - - return request; - } - - public void ApplyResponse(ResponseMessage response) - { - if (response == null) return; - - lock (sync) - { - var (payload, dataStreams) = queueMessageSerializer.WriteMessage(response); - storeDataStreamsForDistributedQueues.StoreDataStreams(dataStreams, CancellationToken.None).GetAwaiter().GetResult(); - - halibutRedisTransport.SetResponse(endpoint, response.Id, payload).GetAwaiter().GetResult(); - } - } - - - class PendingRequest - { - readonly RequestMessage request; - readonly ILog log; - readonly ManualResetEventSlim waiter; - readonly object sync = new object(); - bool transferBegun; - bool completed; - - public PendingRequest(RequestMessage request, ILog log) - { - this.request = request; - this.log = log; - waiter = new ManualResetEventSlim(false); - } - - - public void WaitUntilComplete(CancellationToken cancellationToken) - { - log.Write(EventType.MessageExchange, "Request {0} was queued", request); - - var success = waiter.Wait(request.Destination.PollingRequestQueueTimeout, cancellationToken); - if (success) - { - log.Write(EventType.MessageExchange, "Request {0} was collected by the polling endpoint", request); - return; - } - - var waitForTransferToComplete = false; - lock (sync) - { - if (transferBegun) - { - waitForTransferToComplete = true; - } - else - { - completed = true; - } - } - - if (waitForTransferToComplete) - { - success = waiter.Wait(PollingRequestMaximumMessageProcessingTimeout); - if (success) - { - log.Write(EventType.MessageExchange, "Request {0} was eventually collected by the polling endpoint", request); - } - else - { - SetResponse(ResponseMessage.FromException(request, new TimeoutException(string.Format("A request was sent to a polling endpoint, the polling endpoint collected it but did not respond in the allowed time ({0}), so the request timed out.", PollingRequestMaximumMessageProcessingTimeout)))); - } - } - else - { - log.Write(EventType.MessageExchange, "Request {0} timed out before it could be collected by the polling endpoint", request); - SetResponse(ResponseMessage.FromException(request, new TimeoutException(string.Format("A request was sent to a polling endpoint, but the polling endpoint did not collect the request within the allowed time ({0}), so the request timed out.", request.Destination.PollingRequestQueueTimeout)))); - } - } - - public bool BeginTransfer() - { - lock (sync) - { - if (completed) - return false; - - transferBegun = true; - return true; - } - } - - public ResponseMessage? Response { get; private set; } - - public void SetResponse(ResponseMessage response) - { - Response = response; - waiter.Set(); - } - } - - public ValueTask DisposeAsync() - { - throw new NotImplementedException(); - } - } - - interface IHalibutMessageQueueItemSerializer - { - T FromJson(string responseMessage); - string ToJson(object message); - } - - public enum MessageType - { - RequestMessage, - ResponseMessage - } -} \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueueFactory.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueueFactory.cs new file mode 100644 index 000000000..ca9414f78 --- /dev/null +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueueFactory.cs @@ -0,0 +1,52 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; +using Halibut.Diagnostics; +using Halibut.ServiceModel; + +namespace Halibut.Queue.Redis +{ + public class RedisPendingRequestQueueFactory : IPendingRequestQueueFactory + { + readonly QueueMessageSerializer queueMessageSerializer; + readonly IStoreDataStreamsForDistributedQueues dataStreamStorage; + readonly HalibutHalibutRedisTransport halibutRedisTransport; + readonly ILogFactory logFactory; + readonly HalibutTimeoutsAndLimits halibutTimeoutsAndLimits; + + public RedisPendingRequestQueueFactory( + QueueMessageSerializer queueMessageSerializer, + IStoreDataStreamsForDistributedQueues dataStreamStorage, + HalibutHalibutRedisTransport halibutRedisTransport, + HalibutTimeoutsAndLimits halibutTimeoutsAndLimits, + ILogFactory logFactory) + { + this.queueMessageSerializer = queueMessageSerializer; + this.dataStreamStorage = dataStreamStorage; + this.halibutRedisTransport = halibutRedisTransport; + this.logFactory = logFactory; + this.halibutTimeoutsAndLimits = halibutTimeoutsAndLimits; + } + + public IPendingRequestQueue CreateQueue(Uri endpoint) + { + return new RedisPendingRequestQueue(endpoint, + logFactory.ForEndpoint(endpoint), + halibutRedisTransport, + new MessageReaderWriter(queueMessageSerializer, dataStreamStorage), + halibutTimeoutsAndLimits); + } + } +} \ No newline at end of file diff --git a/source/Halibut/ServiceModel/IPendingRequestQueue.cs b/source/Halibut/ServiceModel/IPendingRequestQueue.cs index a7af25c50..5d2ed930f 100644 --- a/source/Halibut/ServiceModel/IPendingRequestQueue.cs +++ b/source/Halibut/ServiceModel/IPendingRequestQueue.cs @@ -9,7 +9,7 @@ public interface IPendingRequestQueue : IAsyncDisposable { bool IsEmpty { get; } int Count { get; } - Task ApplyResponse(ResponseMessage response, ServiceEndPoint destination); + Task ApplyResponse(ResponseMessage response, Guid requestActivityId); Task DequeueAsync(CancellationToken cancellationToken); Task QueueAndWaitAsync(RequestMessage request, CancellationToken cancellationToken); } diff --git a/source/Halibut/ServiceModel/PendingRequestQueueAsync.cs b/source/Halibut/ServiceModel/PendingRequestQueueAsync.cs index b95bf3279..c33655bea 100644 --- a/source/Halibut/ServiceModel/PendingRequestQueueAsync.cs +++ b/source/Halibut/ServiceModel/PendingRequestQueueAsync.cs @@ -161,7 +161,7 @@ await Task.WhenAny( } } - public async Task ApplyResponse(ResponseMessage response, ServiceEndPoint destination) + public async Task ApplyResponse(ResponseMessage response, Guid requestActivityId) { if (response == null) { diff --git a/source/Halibut/Transport/Protocol/MessageExchangeProtocol.cs b/source/Halibut/Transport/Protocol/MessageExchangeProtocol.cs index 3f156705d..777b7e268 100644 --- a/source/Halibut/Transport/Protocol/MessageExchangeProtocol.cs +++ b/source/Halibut/Transport/Protocol/MessageExchangeProtocol.cs @@ -232,7 +232,7 @@ async Task ProcessReceiverInternalAsync(IPendingRequestQueue pendingReques var linkedCancellationToken = linkedTokenSource.Token; var response = await SendAndReceiveRequest(nextRequest.RequestMessage, linkedCancellationToken); - await pendingRequests.ApplyResponse(response, nextRequest.RequestMessage.Destination); + await pendingRequests.ApplyResponse(response, nextRequest.RequestMessage.ActivityId); } else { @@ -246,7 +246,7 @@ async Task ProcessReceiverInternalAsync(IPendingRequestQueue pendingReques var cancellationException = nextRequest.CancellationToken.IsCancellationRequested ? new TransferringRequestCancelledException(ex) : ex; var response = ResponseMessage.FromException(nextRequest.RequestMessage, cancellationException); - await pendingRequests.ApplyResponse(response, nextRequest.RequestMessage.Destination); + await pendingRequests.ApplyResponse(response, nextRequest.RequestMessage.ActivityId); if (nextRequest.CancellationToken.IsCancellationRequested) { diff --git a/source/Halibut/Transport/Protocol/MessageSerializer.cs b/source/Halibut/Transport/Protocol/MessageSerializer.cs index aedf9dbba..201b182d6 100644 --- a/source/Halibut/Transport/Protocol/MessageSerializer.cs +++ b/source/Halibut/Transport/Protocol/MessageSerializer.cs @@ -14,7 +14,7 @@ namespace Halibut.Transport.Protocol { public class MessageSerializer : IMessageSerializer { - readonly Func createStreamCapturingSerializer; + internal readonly Func CreateStreamCapturingSerializer; readonly IMessageSerializerObserver observer; readonly long readIntoMemoryLimitBytes; readonly long writeIntoMemoryLimitBytes; @@ -28,7 +28,7 @@ internal MessageSerializer( long writeIntoMemoryLimitBytes, ILogFactory logFactory) { - this.createStreamCapturingSerializer = createStreamCapturingSerializer; + this.CreateStreamCapturingSerializer = createStreamCapturingSerializer; this.observer = observer; this.readIntoMemoryLimitBytes = readIntoMemoryLimitBytes; this.writeIntoMemoryLimitBytes = writeIntoMemoryLimitBytes; @@ -51,7 +51,7 @@ public async Task> WriteMessageAsync(Stream stream, // for the moment this MUST be object so that the $type property is included // If it is not, then an old receiver (eg, old tentacle) will not be able to understand messages from a new sender (server) // Once ALL sources and targets are deserializing to MessageEnvelope, (ReadBsonMessage) then this can be changed to T - var streamCapturingSerializer = createStreamCapturingSerializer(); + var streamCapturingSerializer = CreateStreamCapturingSerializer(); streamCapturingSerializer.Serializer.Serialize(bson, new MessageEnvelope { Message = message! }); serializedStreams = streamCapturingSerializer.DataStreams; @@ -152,7 +152,7 @@ public async Task> WriteMessageAsync(Stream stream, (MessageEnvelope MessageEnvelope, IReadOnlyList DataStreams) DeserializeMessageAndDataStreams(JsonReader reader) { - var streamCapturingSerializer = createStreamCapturingSerializer(); + var streamCapturingSerializer = CreateStreamCapturingSerializer(); var result = streamCapturingSerializer.Serializer.Deserialize>(reader); if (result == null) diff --git a/source/Halibut/Util/FuncAsyncDisposable.cs b/source/Halibut/Util/FuncAsyncDisposable.cs new file mode 100644 index 000000000..0f1e752b5 --- /dev/null +++ b/source/Halibut/Util/FuncAsyncDisposable.cs @@ -0,0 +1,34 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; +using System.Threading.Tasks; + +namespace Halibut.Util +{ + public class FuncAsyncDisposable : IAsyncDisposable + { + readonly Func disposer; + + public FuncAsyncDisposable(Func disposer) + { + this.disposer = disposer; + } + + public async ValueTask DisposeAsync() + { + await this.disposer(); + } + } +} \ No newline at end of file diff --git a/source/Halibut/Util/StringExtensionMethods.cs b/source/Halibut/Util/StringExtensionMethods.cs new file mode 100644 index 000000000..1e77ef3f6 --- /dev/null +++ b/source/Halibut/Util/StringExtensionMethods.cs @@ -0,0 +1,29 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; +using System.Diagnostics.CodeAnalysis; + +namespace Halibut.Util +{ + public static class StringExtensionMethods + { + [return: NotNullIfNotNull("str")] + public static Guid? ToGuid(this string? str) + { + if (str == null) return null; + return Guid.Parse(str); + } + } +} \ No newline at end of file From 05759990031858d9f1ebea9194c9d702da0f5f8b Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Mon, 28 Jul 2025 12:26:10 +1000 Subject: [PATCH 003/137] The test --- .../Queue/QueueMessageSerializerFixture.cs | 101 ++++++++++++++++++ 1 file changed, 101 insertions(+) create mode 100644 source/Halibut.Tests/Queue/QueueMessageSerializerFixture.cs diff --git a/source/Halibut.Tests/Queue/QueueMessageSerializerFixture.cs b/source/Halibut.Tests/Queue/QueueMessageSerializerFixture.cs new file mode 100644 index 000000000..dae8aa61f --- /dev/null +++ b/source/Halibut.Tests/Queue/QueueMessageSerializerFixture.cs @@ -0,0 +1,101 @@ +using System; +using System.Linq; +using FluentAssertions; +using Halibut.Diagnostics; +using Halibut.Queue; +using Halibut.Tests.Support; +using Halibut.Transport.Protocol; +using Newtonsoft.Json; +using NUnit.Framework; + +namespace Halibut.Tests.Queue +{ + public class QueueMessageSerializerFixture : BaseTest + { + [Test] + public void SerializeAndDeserializeMessage_ShouldRoundTrip() + { + // Arrange + var sut = new QueueMessageSerializerBuilder(new LogFactory()) + .Build(); + + const string testMessage = "Hello, Queue!"; + + // Act + var (json, dataStreams) = sut.WriteMessage(testMessage); + var (deserializedMessage, deserializedDataStreams) = sut.ReadMessage(json); + + // Assert + deserializedMessage.Should().Be(testMessage); + dataStreams.Should().BeEmpty(); + deserializedDataStreams.Should().BeEmpty(); + } + + [Test] + public void SerializeAndDeserializeMessage_ShouldRoundTrip_RequestMessage() + { + // Arrange + var sut = new QueueMessageSerializerBuilder(new LogFactory()) + .Build(); + + var request = new RequestMessage() + { + Id = "hello", + ActivityId = Guid.NewGuid(), + Destination = new ServiceEndPoint(new Uri("poll://bob"), "n", new HalibutTimeoutsAndLimits()), + ServiceName = "service", + MethodName = "Echo", + Params = new object[] {"hello"} + }; + + // Act + var (json, dataStreams) = sut.WriteMessage(request); + var (deserializedMessage, deserializedDataStreams) = sut.ReadMessage(json); + + // Assert + deserializedMessage.Should().BeEquivalentTo(request); + dataStreams.Should().BeEmpty(); + deserializedDataStreams.Should().BeEmpty(); + } + + public class QueueMessageSerializerBuilder + { + readonly ILogFactory logFactory; + ITypeRegistry? typeRegistry; + Action? configureSerializer; + + public QueueMessageSerializerBuilder(ILogFactory logFactory) + { + this.logFactory = logFactory; + } + + public QueueMessageSerializerBuilder WithTypeRegistry(ITypeRegistry typeRegistry) + { + this.typeRegistry = typeRegistry; + return this; + } + + public QueueMessageSerializerBuilder WithSerializerSettings(Action configure) + { + configureSerializer = configure; + return this; + } + + public QueueMessageSerializer Build() + { + var typeRegistry = this.typeRegistry ?? new TypeRegistry(); + + StreamCapturingJsonSerializer StreamCapturingSerializer() + { + var settings = MessageSerializerBuilder.CreateSerializer(); + var binder = new RegisteredSerializationBinder(typeRegistry); + settings.SerializationBinder = binder; + configureSerializer?.Invoke(settings); + return new StreamCapturingJsonSerializer(settings); + } + + return new QueueMessageSerializer(StreamCapturingSerializer); + } + } + } +} \ No newline at end of file From 178cb7704c3b4e50f860c0f3c4c19ee633b5ca19 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Mon, 28 Jul 2025 15:23:05 +1000 Subject: [PATCH 004/137] . --- .../Redis/RedisPendingRequestQueueFixture.cs | 50 +++++++++- .../IStoreDataStreamsForDistributedQueues.cs | 1 + ...sTransport.cs => HalibutRedisTransport.cs} | 8 +- source/Halibut/Queue/Redis/RedisFacade.cs | 22 +++-- .../Queue/Redis/RedisPendingRequest.cs | 20 ++-- ...stQueue.cs => RedisPendingRequestQueue.cs} | 28 ++++-- .../Redis/RedisPendingRequestQueueFactory.cs | 4 +- source/Halibut/Util/DisposableCollection.cs | 99 +++++++++++++++++++ 8 files changed, 195 insertions(+), 37 deletions(-) rename source/Halibut/Queue/Redis/{HalibutHalibutRedisTransport.cs => HalibutRedisTransport.cs} (95%) rename source/Halibut/Queue/Redis/{PendingRequestQueue.cs => RedisPendingRequestQueue.cs} (88%) create mode 100644 source/Halibut/Util/DisposableCollection.cs diff --git a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs index 3f4daf4ae..340e4a328 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs @@ -1,4 +1,5 @@ using System; +using System.IO.Compression; using System.Threading; using System.Threading.Tasks; using FluentAssertions; @@ -15,6 +16,7 @@ using Halibut.Transport.Protocol; using NSubstitute; using NUnit.Framework; +using DisposableCollection = Halibut.Util.DisposableCollection; namespace Halibut.Tests.Queue.Redis { @@ -27,7 +29,7 @@ public async Task DequeueAsync_ShouldReturnRequestFromRedis() // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid()); var log = Substitute.For(); - var redisTransport = new HalibutHalibutRedisTransport(new RedisFacade("localhost")); + var redisTransport = new HalibutRedisTransport(new RedisFacade("localhost")); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); @@ -48,13 +50,53 @@ public async Task DequeueAsync_ShouldReturnRequestFromRedis() result.RequestMessage.ServiceName.Should().Be(request.ServiceName); } + [Test] + public async Task When100kTentaclesAreSubscribed_TheQueueStillWorks() + { + // Arrange + var endpoint = new Uri("poll://" + Guid.NewGuid()); + var log = Substitute.For(); + var redisTransport = new HalibutRedisTransport(new RedisFacade("localhost")); + + var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); + var messageSerializer = new QueueMessageSerializerBuilder().Build(); + var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); + + var halibutTimeoutsAndLimits = new HalibutTimeoutsAndLimits(); + + await using var disposableCollection = new DisposableCollection(); + for (int i = 0; i < 100000; i++) + { + disposableCollection.Add(new RedisPendingRequestQueue(new Uri("poll://" + Guid.NewGuid()), log, redisTransport, messageReaderWriter, halibutTimeoutsAndLimits)); + } + + this.Logger.Fatal("Waiting"); + await Task.Delay(30000); + this.Logger.Fatal("Done"); + + var request = new RequestMessageBuilder(endpoint.ToString()).Build(); + + await using var sut = new RedisPendingRequestQueue(endpoint, log, new HalibutRedisTransport(new RedisFacade("localhost")), messageReaderWriter, halibutTimeoutsAndLimits); + + var task = sut.QueueAndWaitAsync(request, CancellationToken.None); + + // Act + var result = await sut.DequeueAsync(CancellationToken); + + // Assert + result.Should().NotBeNull(); + result!.RequestMessage.Id.Should().Be(request.Id); + result.RequestMessage.MethodName.Should().Be(request.MethodName); + result.RequestMessage.ServiceName.Should().Be(request.ServiceName); + } + [Test] public async Task FullSendAndReceiveShouldWork() { // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); var log = Substitute.For(); - var redisTransport = new HalibutHalibutRedisTransport(new RedisFacade("localhost")); + var redisTransport = new HalibutRedisTransport(new RedisFacade("localhost")); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); @@ -87,7 +129,7 @@ public async Task FullSendAndReceiveWithDataStreamShouldWork() // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); var log = Substitute.For(); - var redisTransport = new HalibutHalibutRedisTransport(new RedisFacade("localhost")); + var redisTransport = new HalibutRedisTransport(new RedisFacade("localhost")); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); @@ -123,7 +165,7 @@ public async Task FullSendAndReceiveWithDataStreamShouldWork() [LatestClientAndLatestServiceTestCases(testNetworkConditions: false, testListening: false, testWebSocket: false)] public async Task OctopusCanSendMessagesToTentacle_WithEchoService(ClientAndServiceTestCase clientAndServiceTestCase) { - var redisTransport = new HalibutHalibutRedisTransport(new RedisFacade("localhost")); + var redisTransport = new HalibutRedisTransport(new RedisFacade("localhost")); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); await using (var clientAndService = await clientAndServiceTestCase.CreateTestCaseBuilder() diff --git a/source/Halibut/Queue/IStoreDataStreamsForDistributedQueues.cs b/source/Halibut/Queue/IStoreDataStreamsForDistributedQueues.cs index c5e19cacd..dcee12e87 100644 --- a/source/Halibut/Queue/IStoreDataStreamsForDistributedQueues.cs +++ b/source/Halibut/Queue/IStoreDataStreamsForDistributedQueues.cs @@ -33,6 +33,7 @@ public class InMemoryStoreDataStreamsForDistributedQueues : IStoreDataStreamsFor public async Task StoreDataStreams(IReadOnlyList dataStreams, CancellationToken cancellationToken) { await Task.CompletedTask; + foreach (var dataStream in dataStreams) { using var memoryStream = new MemoryStream(); diff --git a/source/Halibut/Queue/Redis/HalibutHalibutRedisTransport.cs b/source/Halibut/Queue/Redis/HalibutRedisTransport.cs similarity index 95% rename from source/Halibut/Queue/Redis/HalibutHalibutRedisTransport.cs rename to source/Halibut/Queue/Redis/HalibutRedisTransport.cs index 923bacfec..4ab9c96f1 100644 --- a/source/Halibut/Queue/Redis/HalibutHalibutRedisTransport.cs +++ b/source/Halibut/Queue/Redis/HalibutRedisTransport.cs @@ -23,13 +23,13 @@ namespace Halibut.Queue.Redis { - public class HalibutHalibutRedisTransport + public class HalibutRedisTransport { const string Namespace = "octopus:server:halibut"; readonly RedisFacade facade; - public HalibutHalibutRedisTransport(RedisFacade facade) + public HalibutRedisTransport(RedisFacade facade) { this.facade = facade; } @@ -37,7 +37,7 @@ public HalibutHalibutRedisTransport(RedisFacade facade) // Request Pulse static string RequestMessagesPulseChannelName(Uri endpoint) { - return $"{Namespace}::requestpulsechannel::{endpoint}"; + return $"{Namespace}::RequestMessagesPulseChannelName::{endpoint}"; } public async Task SubscribeToRequestMessagePulseChannel(Uri endpoint, Action onRequestMessagePulse) @@ -81,7 +81,7 @@ public async Task PushRequestGuidOnToQueue(Uri endpoint, Guid guid, Cancellation static string RequestMessageKey(Uri endpoint, Guid requestId) { - return $"{Namespace}::requestpulsechannel::{endpoint}::{requestId}"; + return $"{Namespace}::RequestMessageKey::{endpoint}::{requestId}"; } static string RequestField = "RequestField"; diff --git a/source/Halibut/Queue/Redis/RedisFacade.cs b/source/Halibut/Queue/Redis/RedisFacade.cs index e340c8706..d32017cc4 100644 --- a/source/Halibut/Queue/Redis/RedisFacade.cs +++ b/source/Halibut/Queue/Redis/RedisFacade.cs @@ -18,6 +18,7 @@ using Halibut.Util; using Newtonsoft.Json; using StackExchange.Redis; +using StackExchange.Redis.KeyspaceIsolation; namespace Halibut.Queue.Redis { @@ -46,6 +47,7 @@ public void Dispose() public async Task SubscribeToChannel(string channelName, Func onMessage) { + channelName = "channel:" + channelName; // TODO ever call needs to respect the cancellation token var channel = await Connection.GetSubscriber() .SubscribeAsync(new RedisChannel(channelName, RedisChannel.PatternMode.Literal)); @@ -55,9 +57,16 @@ public async Task SubscribeToChannel(string channelName, Func< return new FuncAsyncDisposable(() => channel.UnsubscribeAsync()); } + public async Task PublishToChannel(string channelName, string payload) + { + channelName = "channel:" + channelName; + var subscriber = Connection.GetSubscriber(); + await subscriber.PublishAsync(new RedisChannel(channelName, RedisChannel.PatternMode.Literal), payload); + } + public async Task SetInHash(string key, string field, string payload) { - key = "hash::" + key; + key = "hash:" + key; // TODO: TTL // TODO ever call needs to respect the cancellation token var ttl = new TimeSpan(9, 9, 9); @@ -69,7 +78,7 @@ public async Task SetInHash(string key, string field, string payload) public async Task TryGetAndDeleteFromHash(string key, string field) { // TODO ever call needs to respect the cancellation token - key = "hash::" + key; + key = "hash:" + key; var database = Connection.GetDatabase(); var value = await database.HashGetAsync(key, new RedisValue(field)); var res = await database.KeyDeleteAsync(key); @@ -81,20 +90,17 @@ public async Task SetInHash(string key, string field, string payload) return value; } - public async Task PublishToChannel(string channel, string payload) - { - var subscriber = Connection.GetSubscriber(); - await subscriber.PublishAsync(new RedisChannel(channel, RedisChannel.PatternMode.Literal), payload); - } - public async Task ListRightPushAsync(string key, string payload) { + key = "list:" + key; var database = Connection.GetDatabase(); + // TODO can we set TTL on this? await database.ListRightPushAsync(key, payload); } public async Task ListLeftPopAsync(string key) { + key = "list:" + key; var database = Connection.GetDatabase(); var value = await database.ListLeftPopAsync(key); if (value.IsNull) diff --git a/source/Halibut/Queue/Redis/RedisPendingRequest.cs b/source/Halibut/Queue/Redis/RedisPendingRequest.cs index 772483948..8d6f4b49e 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequest.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequest.cs @@ -25,9 +25,9 @@ public class RedisPendingRequest { readonly RequestMessage request; readonly ILog log; - readonly ManualResetEventSlim waiter; + readonly ManualResetEventSlim responseRecievedEvent; readonly object sync = new(); - bool transferBegun; + bool requestCollected; bool completed; readonly TimeSpan pollingRequestMaximumMessageProcessingTimeout; @@ -37,15 +37,16 @@ public RedisPendingRequest(RequestMessage request, ILog log, TimeSpan pollingReq this.request = request; this.log = log; this.pollingRequestMaximumMessageProcessingTimeout = pollingRequestMaximumMessageProcessingTimeout; - waiter = new ManualResetEventSlim(false); + responseRecievedEvent = new ManualResetEventSlim(false); } + // Waits for the request to be collected and response to come back public async Task WaitUntilComplete(CancellationToken cancellationToken, Func pollingRequestQueueTimeElapsed) { await Task.CompletedTask; log.Write(EventType.MessageExchange, "Request {0} was queued", request); - var success = waiter.Wait(request.Destination.PollingRequestQueueTimeout, cancellationToken); + var success = responseRecievedEvent.Wait(request.Destination.PollingRequestQueueTimeout, cancellationToken); if (success) { log.Write(EventType.MessageExchange, "Request {0} was collected by the polling endpoint", request); @@ -59,12 +60,11 @@ public async Task WaitUntilComplete(CancellationToken cancellationToken, Func QueueAndWaitAsync(RequestMessage request, Can // TODO: Respect cancellation token var pending = new RedisPendingRequest(request, log, halibutTimeoutsAndLimits.PollingRequestQueueTimeout); + // TODO: What if this payload was gigantic + // TODO: Do we need to encrypt this? var payload = await messageReaderWriter.PrepareRequest(request, cancellationToken); // Start listening for a response to the request, we don't want to miss the response. await using var _ = await SubscribeToResponse(request.ActivityId, pending.SetResponse, cancellationToken); - // Make the request available before tell people it is available. + // Make the request available before we tell people it is available. await halibutRedisTransport.PutRequest(endpoint, request.ActivityId, payload, cancellationToken); + // --- await halibutRedisTransport.PushRequestGuidOnToQueue(endpoint, request.ActivityId, cancellationToken); await halibutRedisTransport.PulseRequestPushedToEndpoint(endpoint, cancellationToken); - await pending.WaitUntilComplete(cancellationToken, async () => + await pending.WaitUntilComplete(cancellationToken, ClearRequestFromQueueWhenPickupTimeoutExpired(request, cancellationToken, pending)); + + return pending.Response!; + } + + Func ClearRequestFromQueueWhenPickupTimeoutExpired(RequestMessage request, CancellationToken cancellationToken, RedisPendingRequest pending) + { + return async () => { // The time the message is allowed to sit on the queue for has elapsed. // Let's try to pop if from the queue, either: @@ -86,11 +96,9 @@ await pending.WaitUntilComplete(cancellationToken, async () => var requestJson = await halibutRedisTransport.TryGetAndRemoveRequest(endpoint, request.ActivityId, cancellationToken); if (requestJson != null) { - pending.BeginTransfer(); + pending.FYITheRequestHasBeenCollected(); } - }); - - return pending.Response!; + }; } async Task SubscribeToResponse(Guid activityId, @@ -138,7 +146,9 @@ public async Task ApplyResponse(ResponseMessage response, Guid requestActivityId var first = await TryRemoveNextItemFromQueue(cancellationToken); if (first != null) return first; - await Task.WhenAny(hasItemsForEndpoint.WaitAsync(cancellationTokenSource.Token), Task.Delay(new HalibutTimeoutsAndLimits().PollingQueueWaitTimeout, cancellationTokenSource.Token)); + await Task.WhenAny( + hasItemsForEndpoint.WaitAsync(cancellationTokenSource.Token), + Task.Delay(new HalibutTimeoutsAndLimits().PollingQueueWaitTimeout, cancellationTokenSource.Token)); if (!hasItemsForEndpoint.IsSet) { diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueueFactory.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueueFactory.cs index ca9414f78..4b8c6fd3a 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueueFactory.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueueFactory.cs @@ -22,14 +22,14 @@ public class RedisPendingRequestQueueFactory : IPendingRequestQueueFactory { readonly QueueMessageSerializer queueMessageSerializer; readonly IStoreDataStreamsForDistributedQueues dataStreamStorage; - readonly HalibutHalibutRedisTransport halibutRedisTransport; + readonly HalibutRedisTransport halibutRedisTransport; readonly ILogFactory logFactory; readonly HalibutTimeoutsAndLimits halibutTimeoutsAndLimits; public RedisPendingRequestQueueFactory( QueueMessageSerializer queueMessageSerializer, IStoreDataStreamsForDistributedQueues dataStreamStorage, - HalibutHalibutRedisTransport halibutRedisTransport, + HalibutRedisTransport halibutRedisTransport, HalibutTimeoutsAndLimits halibutTimeoutsAndLimits, ILogFactory logFactory) { diff --git a/source/Halibut/Util/DisposableCollection.cs b/source/Halibut/Util/DisposableCollection.cs new file mode 100644 index 000000000..3bbb96e66 --- /dev/null +++ b/source/Halibut/Util/DisposableCollection.cs @@ -0,0 +1,99 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; +using System.Collections.Concurrent; +using System.Collections.Generic; +using System.Linq; +using System.Threading.Tasks; + +namespace Halibut.Util +{ + public class DisposableCollection : IDisposable, IAsyncDisposable + { + + //Dispose in the reverse order of when they were added so we deal with nested objects correctly. + readonly ConcurrentStack disposables = new(); + bool isDisposed; + + public T Add(T disposable) where T : IDisposable + { + if (isDisposed) throw new ObjectDisposedException("Cannot add item for disposal. This collection has already been disposed."); + + if (disposable is IAsyncDisposable asyncDisposable) + { + disposables.Push(asyncDisposable); + } + else + { + disposables.Push(new AsyncDisposer(disposable)); + } + + return disposable; + } + + public IAsyncDisposable AddAsyncDisposable(T asyncDisposable) where T : IAsyncDisposable + { + if (isDisposed) throw new ObjectDisposedException("Cannot add item for disposal. This collection has already been disposed."); + + disposables.Push(asyncDisposable); + + return asyncDisposable; + } + + public void Dispose() + { + DisposeAsync().AsTask().GetAwaiter().GetResult(); + } + + public async ValueTask DisposeAsync() + { + if (isDisposed) return; + isDisposed = true; + + var exceptions = new List(); + while (!disposables.IsEmpty) + try + { + if (disposables.TryPop(out var disposable)) + { + await disposable.DisposeAsync(); + } + } + catch (Exception e) + { + exceptions.Add(e); + } + + if (exceptions.Any()) throw new AggregateException(exceptions); + } + + class AsyncDisposer : IAsyncDisposable + { + readonly IDisposable disposableItem; + + public AsyncDisposer(IDisposable disposableItem) + { + this.disposableItem = disposableItem; + } + + public async ValueTask DisposeAsync() + { + await Task.CompletedTask; + + disposableItem.Dispose(); + } + } + } +} \ No newline at end of file From e268a675656184390a8887fd297d69b0600a52fe Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Mon, 28 Jul 2025 15:44:44 +1000 Subject: [PATCH 005/137] . --- Redis.md | 2 + redis-conf/redis.conf | 598 ++++++++++++++++++ .../Redis/RedisPendingRequestQueueFixture.cs | 34 +- 3 files changed, 623 insertions(+), 11 deletions(-) create mode 100644 Redis.md create mode 100644 redis-conf/redis.conf diff --git a/Redis.md b/Redis.md new file mode 100644 index 000000000..f439e1862 --- /dev/null +++ b/Redis.md @@ -0,0 +1,2 @@ +docker run -v redis-conf:/usr/local/etc/redis -p 6379:6379 --name redis -d redis redis-server /usr/local/etc/redis/redis.conf + diff --git a/redis-conf/redis.conf b/redis-conf/redis.conf new file mode 100644 index 000000000..90a3e5edf --- /dev/null +++ b/redis-conf/redis.conf @@ -0,0 +1,598 @@ +# Redis configuration file example + +# Note on units: when memory size is needed, it is possible to specify +# it in the usual form of 1k 5GB 4M and so forth: +# +# 1k => 1000 bytes +# 1kb => 1024 bytes +# 1m => 1000000 bytes +# 1mb => 1024*1024 bytes +# 1g => 1000000000 bytes +# 1gb => 1024*1024*1024 bytes +# +# units are case insensitive so 1GB 1Gb 1gB are all the same. + +# By default Redis does not run as a daemon. Use 'yes' if you need it. +# Note that Redis will write a pid file in /var/run/redis.pid when daemonized. +daemonize no + +# When running daemonized, Redis writes a pid file in /var/run/redis.pid by +# default. You can specify a custom pid file location here. +pidfile /var/run/redis.pid + +# Accept connections on the specified port, default is 6379. +# If port 0 is specified Redis will not listen on a TCP socket. +port 6379 + +# If you want you can bind a single interface, if the bind option is not +# specified all the interfaces will listen for incoming connections. +# +# bind 127.0.0.1 + +# Specify the path for the unix socket that will be used to listen for +# incoming connections. There is no default, so Redis will not listen +# on a unix socket when not specified. +# +# unixsocket /tmp/redis.sock +# unixsocketperm 755 + +# Close the connection after a client is idle for N seconds (0 to disable) +timeout 0 + +# TCP keepalive. +# +# If non-zero, use SO_KEEPALIVE to send TCP ACKs to clients in absence +# of communication. This is useful for two reasons: +# +# 1) Detect dead peers. +# 2) Take the connection alive from the point of view of network +# equipment in the middle. +# +# On Linux, the specified value (in seconds) is the period used to send ACKs. +# Note that to close the connection the double of the time is needed. +# On other kernels the period depends on the kernel configuration. +# +# A reasonable value for this option is 60 seconds. +tcp-keepalive 0 + +# Specify the server verbosity level. +# This can be one of: +# debug (a lot of information, useful for development/testing) +# verbose (many rarely useful info, but not a mess like the debug level) +# notice (moderately verbose, what you want in production probably) +# warning (only very important / critical messages are logged) +loglevel notice + +# Specify the log file name. Also 'stdout' can be used to force +# Redis to log on the standard output. Note that if you use standard +# output for logging but daemonize, logs will be sent to /dev/null +logfile stdout + +# To enable logging to the system logger, just set 'syslog-enabled' to yes, +# and optionally update the other syslog parameters to suit your needs. +# syslog-enabled no + +# Specify the syslog identity. +# syslog-ident redis + +# Specify the syslog facility. Must be USER or between LOCAL0-LOCAL7. +# syslog-facility local0 + +# Set the number of databases. The default database is DB 0, you can select +# a different one on a per-connection basis using SELECT where +# dbid is a number between 0 and 'databases'-1 +databases 16 + +################################ SNAPSHOTTING ################################# +# +# Save the DB on disk: +# +# save +# +# Will save the DB if both the given number of seconds and the given +# number of write operations against the DB occurred. +# +# In the example below the behaviour will be to save: +# after 900 sec (15 min) if at least 1 key changed +# after 300 sec (5 min) if at least 10 keys changed +# after 60 sec if at least 10000 keys changed +# +# Note: you can disable saving at all commenting all the "save" lines. +# +# It is also possible to remove all the previously configured save +# points by adding a save directive with a single empty string argument +# like in the following example: +# +# save "" + +# save 900 1 +# save 300 10 +# save 60 10000 +save "" + +# By default Redis will stop accepting writes if RDB snapshots are enabled +# (at least one save point) and the latest background save failed. +# This will make the user aware (in an hard way) that data is not persisting +# on disk properly, otherwise chances are that no one will notice and some +# distater will happen. +# +# If the background saving process will start working again Redis will +# automatically allow writes again. +# +# However if you have setup your proper monitoring of the Redis server +# and persistence, you may want to disable this feature so that Redis will +# continue to work as usually even if there are problems with disk, +# permissions, and so forth. +stop-writes-on-bgsave-error yes + +# Compress string objects using LZF when dump .rdb databases? +# For default that's set to 'yes' as it's almost always a win. +# If you want to save some CPU in the saving child set it to 'no' but +# the dataset will likely be bigger if you have compressible values or keys. +rdbcompression yes + +# Since version 5 of RDB a CRC64 checksum is placed at the end of the file. +# This makes the format more resistant to corruption but there is a performance +# hit to pay (around 10%) when saving and loading RDB files, so you can disable it +# for maximum performances. +# +# RDB files created with checksum disabled have a checksum of zero that will +# tell the loading code to skip the check. +rdbchecksum yes + +# The filename where to dump the DB +dbfilename dump.rdb + +# The working directory. +# +# The DB will be written inside this directory, with the filename specified +# above using the 'dbfilename' configuration directive. +# +# The Append Only File will also be created inside this directory. +# +# Note that you must specify a directory here, not a file name. +dir ./ + +################################# REPLICATION ################################# + +# Master-Slave replication. Use slaveof to make a Redis instance a copy of +# another Redis server. Note that the configuration is local to the slave +# so for example it is possible to configure the slave to save the DB with a +# different interval, or to listen to another port, and so on. +# +# slaveof + +# If the master is password protected (using the "requirepass" configuration +# directive below) it is possible to tell the slave to authenticate before +# starting the replication synchronization process, otherwise the master will +# refuse the slave request. +# +# masterauth + +# When a slave loses its connection with the master, or when the replication +# is still in progress, the slave can act in two different ways: +# +# 1) if slave-serve-stale-data is set to 'yes' (the default) the slave will +# still reply to client requests, possibly with out of date data, or the +# data set may just be empty if this is the first synchronization. +# +# 2) if slave-serve-stale-data is set to 'no' the slave will reply with +# an error "SYNC with master in progress" to all the kind of commands +# but to INFO and SLAVEOF. +# +slave-serve-stale-data yes + +# You can configure a slave instance to accept writes or not. Writing against +# a slave instance may be useful to store some ephemeral data (because data +# written on a slave will be easily deleted after resync with the master) but +# may also cause problems if clients are writing to it because of a +# misconfiguration. +# +# Since Redis 2.6 by default slaves are read-only. +# +# Note: read only slaves are not designed to be exposed to untrusted clients +# on the internet. It's just a protection layer against misuse of the instance. +# Still a read only slave exports by default all the administrative commands +# such as CONFIG, DEBUG, and so forth. To a limited extend you can improve +# security of read only slaves using 'rename-command' to shadow all the +# administrative / dangerous commands. +slave-read-only yes + +# Slaves send PINGs to server in a predefined interval. It's possible to change +# this interval with the repl_ping_slave_period option. The default value is 10 +# seconds. +# +# repl-ping-slave-period 10 + +# The following option sets a timeout for both Bulk transfer I/O timeout and +# master data or ping response timeout. The default value is 60 seconds. +# +# It is important to make sure that this value is greater than the value +# specified for repl-ping-slave-period otherwise a timeout will be detected +# every time there is low traffic between the master and the slave. +# +# repl-timeout 60 + +# Disable TCP_NODELAY on the slave socket after SYNC? +# +# If you select "yes" Redis will use a smaller number of TCP packets and +# less bandwidth to send data to slaves. But this can add a delay for +# the data to appear on the slave side, up to 40 milliseconds with +# Linux kernels using a default configuration. +# +# If you select "no" the delay for data to appear on the slave side will +# be reduced but more bandwidth will be used for replication. +# +# By default we optimize for low latency, but in very high traffic conditions +# or when the master and slaves are many hops away, turning this to "yes" may +# be a good idea. +repl-disable-tcp-nodelay no + +# The slave priority is an integer number published by Redis in the INFO output. +# It is used by Redis Sentinel in order to select a slave to promote into a +# master if the master is no longer working correctly. +# +# A slave with a low priority number is considered better for promotion, so +# for instance if there are three slaves with priority 10, 100, 25 Sentinel will +# pick the one wtih priority 10, that is the lowest. +# +# However a special priority of 0 marks the slave as not able to perform the +# role of master, so a slave with priority of 0 will never be selected by +# Redis Sentinel for promotion. +# +# By default the priority is 100. +slave-priority 100 + +################################## SECURITY ################################### + +# Require clients to issue AUTH before processing any other +# commands. This might be useful in environments in which you do not trust +# others with access to the host running redis-server. +# +# This should stay commented out for backward compatibility and because most +# people do not need auth (e.g. they run their own servers). +# +# Warning: since Redis is pretty fast an outside user can try up to +# 150k passwords per second against a good box. This means that you should +# use a very strong password otherwise it will be very easy to break. +# +# requirepass foobared + +# Command renaming. +# +# It is possible to change the name of dangerous commands in a shared +# environment. For instance the CONFIG command may be renamed into something +# hard to guess so that it will still be available for internal-use tools +# but not available for general clients. +# +# Example: +# +# rename-command CONFIG b840fc02d524045429941cc15f59e41cb7be6c52 +# +# It is also possible to completely kill a command by renaming it into +# an empty string: +# +# rename-command CONFIG "" +# +# Please note that changing the name of commands that are logged into the +# AOF file or transmitted to slaves may cause problems. + +################################### LIMITS #################################### + +# Set the max number of connected clients at the same time. By default +# this limit is set to 10000 clients, however if the Redis server is not +# able to configure the process file limit to allow for the specified limit +# the max number of allowed clients is set to the current file limit +# minus 32 (as Redis reserves a few file descriptors for internal uses). +# +# Once the limit is reached Redis will close all the new connections sending +# an error 'max number of clients reached'. +# +maxclients 10000 + +# Don't use more memory than the specified amount of bytes. +# When the memory limit is reached Redis will try to remove keys +# accordingly to the eviction policy selected (see maxmemmory-policy). +# +# If Redis can't remove keys according to the policy, or if the policy is +# set to 'noeviction', Redis will start to reply with errors to commands +# that would use more memory, like SET, LPUSH, and so on, and will continue +# to reply to read-only commands like GET. +# +# This option is usually useful when using Redis as an LRU cache, or to set +# an hard memory limit for an instance (using the 'noeviction' policy). +# +# WARNING: If you have slaves attached to an instance with maxmemory on, +# the size of the output buffers needed to feed the slaves are subtracted +# from the used memory count, so that network problems / resyncs will +# not trigger a loop where keys are evicted, and in turn the output +# buffer of slaves is full with DELs of keys evicted triggering the deletion +# of more keys, and so forth until the database is completely emptied. +# +# In short... if you have slaves attached it is suggested that you set a lower +# limit for maxmemory so that there is some free RAM on the system for slave +# output buffers (but this is not needed if the policy is 'noeviction'). +# +# maxmemory + +# MAXMEMORY POLICY: how Redis will select what to remove when maxmemory +# is reached. You can select among five behaviors: +# +# volatile-lru -> remove the key with an expire set using an LRU algorithm +# allkeys-lru -> remove any key accordingly to the LRU algorithm +# volatile-random -> remove a random key with an expire set +# allkeys-random -> remove a random key, any key +# volatile-ttl -> remove the key with the nearest expire time (minor TTL) +# noeviction -> don't expire at all, just return an error on write operations +# +# Note: with any of the above policies, Redis will return an error on write +# operations, when there are not suitable keys for eviction. +# +# At the date of writing this commands are: set setnx setex append +# incr decr rpush lpush rpushx lpushx linsert lset rpoplpush sadd +# sinter sinterstore sunion sunionstore sdiff sdiffstore zadd zincrby +# zunionstore zinterstore hset hsetnx hmset hincrby incrby decrby +# getset mset msetnx exec sort +# +# The default is: +# +# maxmemory-policy volatile-lru + +# LRU and minimal TTL algorithms are not precise algorithms but approximated +# algorithms (in order to save memory), so you can select as well the sample +# size to check. For instance for default Redis will check three keys and +# pick the one that was used less recently, you can change the sample size +# using the following configuration directive. +# +# maxmemory-samples 3 + +############################## APPEND ONLY MODE ############################### + +# By default Redis asynchronously dumps the dataset on disk. This mode is +# good enough in many applications, but an issue with the Redis process or +# a power outage may result into a few minutes of writes lost (depending on +# the configured save points). +# +# The Append Only File is an alternative persistence mode that provides +# much better durability. For instance using the default data fsync policy +# (see later in the config file) Redis can lose just one second of writes in a +# dramatic event like a server power outage, or a single write if something +# wrong with the Redis process itself happens, but the operating system is +# still running correctly. +# +# AOF and RDB persistence can be enabled at the same time without problems. +# If the AOF is enabled on startup Redis will load the AOF, that is the file +# with the better durability guarantees. +# +# Please check http://redis.io/topics/persistence for more information. + +appendonly no + +# The name of the append only file (default: "appendonly.aof") +# appendfilename appendonly.aof + +# The fsync() call tells the Operating System to actually write data on disk +# instead to wait for more data in the output buffer. Some OS will really flush +# data on disk, some other OS will just try to do it ASAP. +# +# Redis supports three different modes: +# +# no: don't fsync, just let the OS flush the data when it wants. Faster. +# always: fsync after every write to the append only log . Slow, Safest. +# everysec: fsync only one time every second. Compromise. +# +# The default is "everysec", as that's usually the right compromise between +# speed and data safety. It's up to you to understand if you can relax this to +# "no" that will let the operating system flush the output buffer when +# it wants, for better performances (but if you can live with the idea of +# some data loss consider the default persistence mode that's snapshotting), +# or on the contrary, use "always" that's very slow but a bit safer than +# everysec. +# +# More details please check the following article: +# http://antirez.com/post/redis-persistence-demystified.html +# +# If unsure, use "everysec". + +# appendfsync always +appendfsync everysec +# appendfsync no + +# When the AOF fsync policy is set to always or everysec, and a background +# saving process (a background save or AOF log background rewriting) is +# performing a lot of I/O against the disk, in some Linux configurations +# Redis may block too long on the fsync() call. Note that there is no fix for +# this currently, as even performing fsync in a different thread will block +# our synchronous write(2) call. +# +# In order to mitigate this problem it's possible to use the following option +# that will prevent fsync() from being called in the main process while a +# BGSAVE or BGREWRITEAOF is in progress. +# +# This means that while another child is saving, the durability of Redis is +# the same as "appendfsync none". In practical terms, this means that it is +# possible to lose up to 30 seconds of log in the worst scenario (with the +# default Linux settings). +# +# If you have latency problems turn this to "yes". Otherwise leave it as +# "no" that is the safest pick from the point of view of durability. +no-appendfsync-on-rewrite no + +# Automatic rewrite of the append only file. +# Redis is able to automatically rewrite the log file implicitly calling +# BGREWRITEAOF when the AOF log size grows by the specified percentage. +# +# This is how it works: Redis remembers the size of the AOF file after the +# latest rewrite (if no rewrite has happened since the restart, the size of +# the AOF at startup is used). +# +# This base size is compared to the current size. If the current size is +# bigger than the specified percentage, the rewrite is triggered. Also +# you need to specify a minimal size for the AOF file to be rewritten, this +# is useful to avoid rewriting the AOF file even if the percentage increase +# is reached but it is still pretty small. +# +# Specify a percentage of zero in order to disable the automatic AOF +# rewrite feature. + +auto-aof-rewrite-percentage 100 +auto-aof-rewrite-min-size 64mb + +################################ LUA SCRIPTING ############################### + +# Max execution time of a Lua script in milliseconds. +# +# If the maximum execution time is reached Redis will log that a script is +# still in execution after the maximum allowed time and will start to +# reply to queries with an error. +# +# When a long running script exceed the maximum execution time only the +# SCRIPT KILL and SHUTDOWN NOSAVE commands are available. The first can be +# used to stop a script that did not yet called write commands. The second +# is the only way to shut down the server in the case a write commands was +# already issue by the script but the user don't want to wait for the natural +# termination of the script. +# +# Set it to 0 or a negative value for unlimited execution without warnings. +lua-time-limit 5000 + +################################## SLOW LOG ################################### + +# The Redis Slow Log is a system to log queries that exceeded a specified +# execution time. The execution time does not include the I/O operations +# like talking with the client, sending the reply and so forth, +# but just the time needed to actually execute the command (this is the only +# stage of command execution where the thread is blocked and can not serve +# other requests in the meantime). +# +# You can configure the slow log with two parameters: one tells Redis +# what is the execution time, in microseconds, to exceed in order for the +# command to get logged, and the other parameter is the length of the +# slow log. When a new command is logged the oldest one is removed from the +# queue of logged commands. + +# The following time is expressed in microseconds, so 1000000 is equivalent +# to one second. Note that a negative number disables the slow log, while +# a value of zero forces the logging of every command. +slowlog-log-slower-than 10000 + +# There is no limit to this length. Just be aware that it will consume memory. +# You can reclaim memory used by the slow log with SLOWLOG RESET. +slowlog-max-len 128 + +############################### ADVANCED CONFIG ############################### + +# Hashes are encoded using a memory efficient data structure when they have a +# small number of entries, and the biggest entry does not exceed a given +# threshold. These thresholds can be configured using the following directives. +hash-max-ziplist-entries 512 +hash-max-ziplist-value 64 + +# Similarly to hashes, small lists are also encoded in a special way in order +# to save a lot of space. The special representation is only used when +# you are under the following limits: +list-max-ziplist-entries 512 +list-max-ziplist-value 64 + +# Sets have a special encoding in just one case: when a set is composed +# of just strings that happens to be integers in radix 10 in the range +# of 64 bit signed integers. +# The following configuration setting sets the limit in the size of the +# set in order to use this special memory saving encoding. +set-max-intset-entries 512 + +# Similarly to hashes and lists, sorted sets are also specially encoded in +# order to save a lot of space. This encoding is only used when the length and +# elements of a sorted set are below the following limits: +zset-max-ziplist-entries 128 +zset-max-ziplist-value 64 + +# Active rehashing uses 1 millisecond every 100 milliseconds of CPU time in +# order to help rehashing the main Redis hash table (the one mapping top-level +# keys to values). The hash table implementation Redis uses (see dict.c) +# performs a lazy rehashing: the more operation you run into an hash table +# that is rehashing, the more rehashing "steps" are performed, so if the +# server is idle the rehashing is never complete and some more memory is used +# by the hash table. +# +# The default is to use this millisecond 10 times every second in order to +# active rehashing the main dictionaries, freeing memory when possible. +# +# If unsure: +# use "activerehashing no" if you have hard latency requirements and it is +# not a good thing in your environment that Redis can reply form time to time +# to queries with 2 milliseconds delay. +# +# use "activerehashing yes" if you don't have such hard requirements but +# want to free memory asap when possible. +activerehashing yes + +# The client output buffer limits can be used to force disconnection of clients +# that are not reading data from the server fast enough for some reason (a +# common reason is that a Pub/Sub client can't consume messages as fast as the +# publisher can produce them). +# +# The limit can be set differently for the three different classes of clients: +# +# normal -> normal clients +# slave -> slave clients and MONITOR clients +# pubsub -> clients subcribed to at least one pubsub channel or pattern +# +# The syntax of every client-output-buffer-limit directive is the following: +# +# client-output-buffer-limit +# +# A client is immediately disconnected once the hard limit is reached, or if +# the soft limit is reached and remains reached for the specified number of +# seconds (continuously). +# So for instance if the hard limit is 32 megabytes and the soft limit is +# 16 megabytes / 10 seconds, the client will get disconnected immediately +# if the size of the output buffers reach 32 megabytes, but will also get +# disconnected if the client reaches 16 megabytes and continuously overcomes +# the limit for 10 seconds. +# +# By default normal clients are not limited because they don't receive data +# without asking (in a push way), but just after a request, so only +# asynchronous clients may create a scenario where data is requested faster +# than it can read. +# +# Instead there is a default limit for pubsub and slave clients, since +# subscribers and slaves receive data in a push fashion. +# +# Both the hard or the soft limit can be disabled by setting them to zero. +client-output-buffer-limit normal 0 0 0 +client-output-buffer-limit slave 256mb 64mb 60 +client-output-buffer-limit pubsub 32mb 8mb 60 + +# Redis calls an internal function to perform many background tasks, like +# closing connections of clients in timeot, purging expired keys that are +# never requested, and so forth. +# +# Not all tasks are perforemd with the same frequency, but Redis checks for +# tasks to perform accordingly to the specified "hz" value. +# +# By default "hz" is set to 10. Raising the value will use more CPU when +# Redis is idle, but at the same time will make Redis more responsive when +# there are many keys expiring at the same time, and timeouts may be +# handled with more precision. +# +# The range is between 1 and 500, however a value over 100 is usually not +# a good idea. Most users should use the default of 10 and raise this up to +# 100 only in environments where very low latency is required. +hz 10 + +# When a child rewrites the AOF file, if the following option is enabled +# the file will be fsync-ed every 32 MB of data generated. This is useful +# in order to commit the file to the disk more incrementally and avoid +# big latency spikes. +aof-rewrite-incremental-fsync yes + +################################## INCLUDES ################################### + +# Include one or more other config files here. This is useful if you +# have a standard template that goes to all Redis server but also need +# to customize a few per-server settings. Include files can include +# other files, so use this wisely. +# +# include /path/to/local.conf +# include /path/to/other.conf \ No newline at end of file diff --git a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs index 340e4a328..69d4a212d 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs @@ -1,4 +1,5 @@ using System; +using System.Diagnostics; using System.IO.Compression; using System.Threading; using System.Threading.Tasks; @@ -16,6 +17,7 @@ using Halibut.Transport.Protocol; using NSubstitute; using NUnit.Framework; +using Serilog; using DisposableCollection = Halibut.Util.DisposableCollection; namespace Halibut.Tests.Queue.Redis @@ -68,26 +70,36 @@ public async Task When100kTentaclesAreSubscribed_TheQueueStillWorks() for (int i = 0; i < 100000; i++) { disposableCollection.Add(new RedisPendingRequestQueue(new Uri("poll://" + Guid.NewGuid()), log, redisTransport, messageReaderWriter, halibutTimeoutsAndLimits)); + if(i % 10000 == 0) + { + Logger.Information("Up to: {i}", i); + } } this.Logger.Fatal("Waiting"); await Task.Delay(30000); this.Logger.Fatal("Done"); + + while (true) + { + var sw = Stopwatch.StartNew(); + var request = new RequestMessageBuilder(endpoint.ToString()).Build(); - var request = new RequestMessageBuilder(endpoint.ToString()).Build(); - - await using var sut = new RedisPendingRequestQueue(endpoint, log, new HalibutRedisTransport(new RedisFacade("localhost")), messageReaderWriter, halibutTimeoutsAndLimits); + await using var sut = new RedisPendingRequestQueue(endpoint, log, new HalibutRedisTransport(new RedisFacade("localhost")), messageReaderWriter, halibutTimeoutsAndLimits); - var task = sut.QueueAndWaitAsync(request, CancellationToken.None); + var task = sut.QueueAndWaitAsync(request, CancellationToken.None); - // Act - var result = await sut.DequeueAsync(CancellationToken); + // Act + var result = await sut.DequeueAsync(CancellationToken); - // Assert - result.Should().NotBeNull(); - result!.RequestMessage.Id.Should().Be(request.Id); - result.RequestMessage.MethodName.Should().Be(request.MethodName); - result.RequestMessage.ServiceName.Should().Be(request.ServiceName); + // Assert + result.Should().NotBeNull(); + result!.RequestMessage.Id.Should().Be(request.Id); + result.RequestMessage.MethodName.Should().Be(request.MethodName); + result.RequestMessage.ServiceName.Should().Be(request.ServiceName); + Logger.Information("It took {}", sw.Elapsed.TotalSeconds.ToString("0.00")); + } + } [Test] From 9ca97e75bdf27d5f01b46fdc2a89325014d55005 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Tue, 29 Jul 2025 13:27:45 +1000 Subject: [PATCH 006/137] This works with octopus --- .../Queue/QueueMessageSerializerFixture.cs | 81 +++++++++++++++++++ .../Redis/RedisPendingRequestQueueFixture.cs | 21 +++-- source/Halibut/DataStream.cs | 2 +- .../IStoreDataStreamsForDistributedQueues.cs | 28 +++++++ ...ryStoreDataStreamsForDistributedQueues.cs} | 9 +-- .../Queue/Redis/MessageReaderWriter.cs | 1 + .../Redis/RedisPendingRequestQueueFactory.cs | 1 + .../Protocol/HalibutContractResolver.cs | 4 +- 8 files changed, 130 insertions(+), 17 deletions(-) create mode 100644 source/Halibut/Queue/QueuedDataStreams/IStoreDataStreamsForDistributedQueues.cs rename source/Halibut/Queue/{IStoreDataStreamsForDistributedQueues.cs => QueuedDataStreams/InMemoryStoreDataStreamsForDistributedQueues.cs} (85%) diff --git a/source/Halibut.Tests/Queue/QueueMessageSerializerFixture.cs b/source/Halibut.Tests/Queue/QueueMessageSerializerFixture.cs index dae8aa61f..880dcfa4e 100644 --- a/source/Halibut.Tests/Queue/QueueMessageSerializerFixture.cs +++ b/source/Halibut.Tests/Queue/QueueMessageSerializerFixture.cs @@ -1,5 +1,8 @@ using System; +using System.IO; using System.Linq; +using System.Threading; +using System.Threading.Tasks; using FluentAssertions; using Halibut.Diagnostics; using Halibut.Queue; @@ -57,6 +60,84 @@ public void SerializeAndDeserializeMessage_ShouldRoundTrip_RequestMessage() dataStreams.Should().BeEmpty(); deserializedDataStreams.Should().BeEmpty(); } + + [Test] + public void SerializeAndDeserializeMessageWithDataStream_ShouldRoundTrip_RequestMessage() + { + var typeRegistry = new TypeRegistry(); + typeRegistry.Register(typeof(IHaveTypeWithDataStreamsService)); + // Arrange + var sut = new QueueMessageSerializerBuilder(new LogFactory()) + .WithTypeRegistry(typeRegistry) + .Build(); + + var request = new RequestMessage() + { + Id = "hello", + ActivityId = Guid.NewGuid(), + Destination = new ServiceEndPoint(new Uri("poll://bob"), "n", new HalibutTimeoutsAndLimits()), + ServiceName = "service", + MethodName = "Echo", + Params = new object[] {"hello", + DataStream.FromString("yo") + ,new TypeWithDataStreams(new RepeatingStringDataStream("bob", 10)) + + } + }; + + // Act + var (json, dataStreams) = sut.WriteMessage(request); + + dataStreams[1].Should().BeOfType(); + + json.Should().Contain("TypeWithDataStreams"); + json.Should().NotContain("RepeatingStringDataStream"); + + var (deserializedMessage, deserializedDataStreams) = sut.ReadMessage(json); + + // Assert + //deserializedMessage.Should().BeEquivalentTo(request); + deserializedDataStreams.Count.Should().Be(2); + } + + public interface IHaveTypeWithDataStreamsService + { + public void Do(TypeWithDataStreams typeWithDataStreams); + } + public class TypeWithDataStreams + { + public TypeWithDataStreams(DataStream dataStream) + { + DataStream = dataStream; + } + + public DataStream DataStream { get; set; } + } + + + public class RepeatingStringDataStream : DataStream + { + string toRepeat; + int HowManyTimes; + + public RepeatingStringDataStream(string toRepeat, int howManyTimes) + : base(toRepeat.GetUTF8Bytes().Length * howManyTimes, WriteRepeatedStringsAsync(toRepeat, howManyTimes)) + { + this.toRepeat = toRepeat; + HowManyTimes = howManyTimes; + } + + static Func WriteRepeatedStringsAsync(string toRepeat, int howManyTimes) + { + return (async (stream, token) => + { + for (int i = 0; i < howManyTimes; i++) + { + await stream.WriteAsync(toRepeat.GetUTF8Bytes(), token); + } + }); + } + } public class QueueMessageSerializerBuilder { diff --git a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs index 69d4a212d..2adc47821 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs @@ -1,11 +1,13 @@ using System; using System.Diagnostics; +using System.Globalization; using System.IO.Compression; using System.Threading; using System.Threading.Tasks; using FluentAssertions; using Halibut.Diagnostics; using Halibut.Queue; +using Halibut.Queue.QueuedDataStreams; using Halibut.Queue.Redis; using Halibut.Tests.Builders; using Halibut.Tests.Support; @@ -52,7 +54,7 @@ public async Task DequeueAsync_ShouldReturnRequestFromRedis() result.RequestMessage.ServiceName.Should().Be(request.ServiceName); } - [Test] + //[Test] public async Task When100kTentaclesAreSubscribed_TheQueueStillWorks() { // Arrange @@ -67,7 +69,7 @@ public async Task When100kTentaclesAreSubscribed_TheQueueStillWorks() var halibutTimeoutsAndLimits = new HalibutTimeoutsAndLimits(); await using var disposableCollection = new DisposableCollection(); - for (int i = 0; i < 100000; i++) + for (int i = 0; i < 300000; i++) { disposableCollection.Add(new RedisPendingRequestQueue(new Uri("poll://" + Guid.NewGuid()), log, redisTransport, messageReaderWriter, halibutTimeoutsAndLimits)); if(i % 10000 == 0) @@ -80,24 +82,31 @@ public async Task When100kTentaclesAreSubscribed_TheQueueStillWorks() await Task.Delay(30000); this.Logger.Fatal("Done"); - while (true) + for(int i = 0; i < 10; i++) { - var sw = Stopwatch.StartNew(); + var request = new RequestMessageBuilder(endpoint.ToString()).Build(); await using var sut = new RedisPendingRequestQueue(endpoint, log, new HalibutRedisTransport(new RedisFacade("localhost")), messageReaderWriter, halibutTimeoutsAndLimits); + + var resultTask = sut.DequeueAsync(CancellationToken); + + await Task.Delay(100); + + var sw = Stopwatch.StartNew(); var task = sut.QueueAndWaitAsync(request, CancellationToken.None); + var result = await resultTask; // Act - var result = await sut.DequeueAsync(CancellationToken); + // Assert result.Should().NotBeNull(); result!.RequestMessage.Id.Should().Be(request.Id); result.RequestMessage.MethodName.Should().Be(request.MethodName); result.RequestMessage.ServiceName.Should().Be(request.ServiceName); - Logger.Information("It took {}", sw.Elapsed.TotalSeconds.ToString("0.00")); + Logger.Information("It took {F}", sw.Elapsed.TotalSeconds.ToString("0.00")); } } diff --git a/source/Halibut/DataStream.cs b/source/Halibut/DataStream.cs index 9e413ef40..591ac23ef 100644 --- a/source/Halibut/DataStream.cs +++ b/source/Halibut/DataStream.cs @@ -10,7 +10,7 @@ namespace Halibut { public class DataStream : IEquatable, IDataStreamInternal { - Func writerAsync; + protected Func writerAsync; IDataStreamReceiver? receiver; [JsonConstructor] diff --git a/source/Halibut/Queue/QueuedDataStreams/IStoreDataStreamsForDistributedQueues.cs b/source/Halibut/Queue/QueuedDataStreams/IStoreDataStreamsForDistributedQueues.cs new file mode 100644 index 000000000..950e61cc9 --- /dev/null +++ b/source/Halibut/Queue/QueuedDataStreams/IStoreDataStreamsForDistributedQueues.cs @@ -0,0 +1,28 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; +using System.Collections.Generic; +using System.Threading; +using System.Threading.Tasks; + +namespace Halibut.Queue.QueuedDataStreams +{ + public interface IStoreDataStreamsForDistributedQueues + { + public Task StoreDataStreams(IReadOnlyList dataStreams, CancellationToken cancellationToken); + + public Task ReHydrateDataStreams(IReadOnlyList dataStreams, CancellationToken cancellationToken); + } +} \ No newline at end of file diff --git a/source/Halibut/Queue/IStoreDataStreamsForDistributedQueues.cs b/source/Halibut/Queue/QueuedDataStreams/InMemoryStoreDataStreamsForDistributedQueues.cs similarity index 85% rename from source/Halibut/Queue/IStoreDataStreamsForDistributedQueues.cs rename to source/Halibut/Queue/QueuedDataStreams/InMemoryStoreDataStreamsForDistributedQueues.cs index dcee12e87..b09413336 100644 --- a/source/Halibut/Queue/IStoreDataStreamsForDistributedQueues.cs +++ b/source/Halibut/Queue/QueuedDataStreams/InMemoryStoreDataStreamsForDistributedQueues.cs @@ -18,15 +18,8 @@ using System.Threading; using System.Threading.Tasks; -namespace Halibut.Queue +namespace Halibut.Queue.QueuedDataStreams { - public interface IStoreDataStreamsForDistributedQueues - { - public Task StoreDataStreams(IReadOnlyList dataStreams, CancellationToken cancellationToken); - - public Task ReHydrateDataStreams(IReadOnlyList dataStreams, CancellationToken cancellationToken); - } - public class InMemoryStoreDataStreamsForDistributedQueues : IStoreDataStreamsForDistributedQueues { public IDictionary dataStreamsStored = new Dictionary(); diff --git a/source/Halibut/Queue/Redis/MessageReaderWriter.cs b/source/Halibut/Queue/Redis/MessageReaderWriter.cs index f2e956068..8b88062d4 100644 --- a/source/Halibut/Queue/Redis/MessageReaderWriter.cs +++ b/source/Halibut/Queue/Redis/MessageReaderWriter.cs @@ -14,6 +14,7 @@ using System.Threading; using System.Threading.Tasks; +using Halibut.Queue.QueuedDataStreams; using Halibut.Transport.Protocol; namespace Halibut.Queue.Redis diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueueFactory.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueueFactory.cs index 4b8c6fd3a..1d100120a 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueueFactory.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueueFactory.cs @@ -14,6 +14,7 @@ using System; using Halibut.Diagnostics; +using Halibut.Queue.QueuedDataStreams; using Halibut.ServiceModel; namespace Halibut.Queue.Redis diff --git a/source/Halibut/Transport/Protocol/HalibutContractResolver.cs b/source/Halibut/Transport/Protocol/HalibutContractResolver.cs index 8b975bd56..75aec0cbe 100644 --- a/source/Halibut/Transport/Protocol/HalibutContractResolver.cs +++ b/source/Halibut/Transport/Protocol/HalibutContractResolver.cs @@ -12,9 +12,9 @@ public class HalibutContractResolver : DefaultContractResolver public override JsonContract ResolveContract(Type type) { - if (type == typeof(DataStream)) + if (typeof(DataStream).IsAssignableFrom(type)) { - var contract = base.ResolveContract(type); + var contract = base.ResolveContract(typeof(DataStream)); // The contract is shared, so we need to make sure multiple threads don't try to edit it at the same time. if (!HaveAddedCaptureOnSerializeCallback) { From 28e0fa30a00477b78797343d97a94f9e243cf44e Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Wed, 30 Jul 2025 08:28:16 +1000 Subject: [PATCH 007/137] Cancel works --- .../Redis/RedisPendingRequestQueueFixture.cs | 103 +++++--- .../Queue/Redis/HalibutRedisTransport.cs | 80 ++++-- source/Halibut/Queue/Redis/RedisFacade.cs | 14 + .../Queue/Redis/RedisPendingRequest.cs | 114 --------- .../Queue/Redis/RedisPendingRequestQueue.cs | 60 +++-- .../Redis/WatchForRequestCancellation.cs | 82 ++++++ source/Halibut/ServiceModel/PendingRequest.cs | 241 ++++++++++++++++++ .../ServiceModel/PendingRequestQueueAsync.cs | 209 +-------------- source/Halibut/Util/DisposableCollection.cs | 10 + source/Halibut/Util/Try.cs | 24 ++ 10 files changed, 542 insertions(+), 395 deletions(-) delete mode 100644 source/Halibut/Queue/Redis/RedisPendingRequest.cs create mode 100644 source/Halibut/Queue/Redis/WatchForRequestCancellation.cs create mode 100644 source/Halibut/ServiceModel/PendingRequest.cs diff --git a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs index 2adc47821..825577789 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs @@ -1,14 +1,18 @@ using System; +using System.Collections.Generic; using System.Diagnostics; using System.Globalization; +using System.IO; using System.IO.Compression; using System.Threading; using System.Threading.Tasks; using FluentAssertions; using Halibut.Diagnostics; +using Halibut.Exceptions; using Halibut.Queue; using Halibut.Queue.QueuedDataStreams; using Halibut.Queue.Redis; +using Halibut.ServiceModel; using Halibut.Tests.Builders; using Halibut.Tests.Support; using Halibut.Tests.Support.TestAttributes; @@ -17,6 +21,7 @@ using Halibut.Tests.Util; using Halibut.TestUtils.Contracts; using Halibut.Transport.Protocol; +using Nito.AsyncEx; using NSubstitute; using NUnit.Framework; using Serilog; @@ -26,7 +31,6 @@ namespace Halibut.Tests.Queue.Redis { public class RedisPendingRequestQueueFixture : BaseTest { - [Test] public async Task DequeueAsync_ShouldReturnRequestFromRedis() { @@ -41,7 +45,7 @@ public async Task DequeueAsync_ShouldReturnRequestFromRedis() var request = new RequestMessageBuilder("poll://test-endpoint").Build(); var sut = new RedisPendingRequestQueue(endpoint, log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); - + var task = sut.QueueAndWaitAsync(request, CancellationToken.None); // Act @@ -53,7 +57,7 @@ public async Task DequeueAsync_ShouldReturnRequestFromRedis() result.RequestMessage.MethodName.Should().Be(request.MethodName); result.RequestMessage.ServiceName.Should().Be(request.ServiceName); } - + //[Test] public async Task When100kTentaclesAreSubscribed_TheQueueStillWorks() { @@ -61,45 +65,43 @@ public async Task When100kTentaclesAreSubscribed_TheQueueStillWorks() var endpoint = new Uri("poll://" + Guid.NewGuid()); var log = Substitute.For(); var redisTransport = new HalibutRedisTransport(new RedisFacade("localhost")); - + var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); - + var halibutTimeoutsAndLimits = new HalibutTimeoutsAndLimits(); await using var disposableCollection = new DisposableCollection(); for (int i = 0; i < 300000; i++) { disposableCollection.Add(new RedisPendingRequestQueue(new Uri("poll://" + Guid.NewGuid()), log, redisTransport, messageReaderWriter, halibutTimeoutsAndLimits)); - if(i % 10000 == 0) + if (i % 10000 == 0) { Logger.Information("Up to: {i}", i); } } - + this.Logger.Fatal("Waiting"); await Task.Delay(30000); this.Logger.Fatal("Done"); - for(int i = 0; i < 10; i++) + for (int i = 0; i < 10; i++) { - var request = new RequestMessageBuilder(endpoint.ToString()).Build(); - + await using var sut = new RedisPendingRequestQueue(endpoint, log, new HalibutRedisTransport(new RedisFacade("localhost")), messageReaderWriter, halibutTimeoutsAndLimits); - + var resultTask = sut.DequeueAsync(CancellationToken); await Task.Delay(100); - + var sw = Stopwatch.StartNew(); - + var task = sut.QueueAndWaitAsync(request, CancellationToken.None); var result = await resultTask; // Act - // Assert result.Should().NotBeNull(); @@ -108,9 +110,8 @@ public async Task When100kTentaclesAreSubscribed_TheQueueStillWorks() result.RequestMessage.ServiceName.Should().Be(request.ServiceName); Logger.Information("It took {F}", sw.Elapsed.TotalSeconds.ToString("0.00")); } - } - + [Test] public async Task FullSendAndReceiveShouldWork() { @@ -126,11 +127,11 @@ public async Task FullSendAndReceiveShouldWork() var node1Sender = new RedisPendingRequestQueue(endpoint, log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); var node2Reciever = new RedisPendingRequestQueue(endpoint, log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); - + var queueAndWaitAsync = node1Sender.QueueAndWaitAsync(request, CancellationToken.None); - + var requestMessageWithCancellationToken = await node2Reciever.DequeueAsync(CancellationToken); - + requestMessageWithCancellationToken.Should().NotBeNull(); requestMessageWithCancellationToken!.RequestMessage.Id.Should().Be(request.Id); requestMessageWithCancellationToken.RequestMessage.MethodName.Should().Be(request.MethodName); @@ -143,7 +144,7 @@ public async Task FullSendAndReceiveShouldWork() responseMessage.Result.Should().Be("Yay"); } - + [Test] public async Task FullSendAndReceiveWithDataStreamShouldWork() { @@ -157,44 +158,43 @@ public async Task FullSendAndReceiveWithDataStreamShouldWork() var request = new RequestMessageBuilder("poll://test-endpoint").Build(); request.Params = new[] { new ComplexObjectMultipleDataStreams(DataStream.FromString("hello"), DataStream.FromString("world")) }; - var node1Sender = new RedisPendingRequestQueue(endpoint, log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); var node2Reciever = new RedisPendingRequestQueue(endpoint, log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); - + var queueAndWaitAsync = node1Sender.QueueAndWaitAsync(request, CancellationToken.None); - + var requestMessageWithCancellationToken = await node2Reciever.DequeueAsync(CancellationToken); - - var objWithDataStreams = (ComplexObjectMultipleDataStreams) requestMessageWithCancellationToken!.RequestMessage.Params[0]; + + var objWithDataStreams = (ComplexObjectMultipleDataStreams)requestMessageWithCancellationToken!.RequestMessage.Params[0]; (await objWithDataStreams.Payload1!.ReadAsString(CancellationToken)).Should().Be("hello"); (await objWithDataStreams.Payload2!.ReadAsString(CancellationToken)).Should().Be("world"); - var response = ResponseMessage.FromResult(requestMessageWithCancellationToken.RequestMessage, + var response = ResponseMessage.FromResult(requestMessageWithCancellationToken.RequestMessage, new ComplexObjectMultipleDataStreams(DataStream.FromString("good"), DataStream.FromString("bye"))); - + await node2Reciever.ApplyResponse(response, requestMessageWithCancellationToken.RequestMessage.ActivityId); var responseMessage = await queueAndWaitAsync; - var returnObject = (ComplexObjectMultipleDataStreams) responseMessage.Result!; + var returnObject = (ComplexObjectMultipleDataStreams)responseMessage.Result!; (await returnObject.Payload1!.ReadAsString(CancellationToken)).Should().Be("good"); (await returnObject.Payload2!.ReadAsString(CancellationToken)).Should().Be("bye"); } - + [Test] [LatestClientAndLatestServiceTestCases(testNetworkConditions: false, testListening: false, testWebSocket: false)] public async Task OctopusCanSendMessagesToTentacle_WithEchoService(ClientAndServiceTestCase clientAndServiceTestCase) { var redisTransport = new HalibutRedisTransport(new RedisFacade("localhost")); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); - + await using (var clientAndService = await clientAndServiceTestCase.CreateTestCaseBuilder() .WithStandardServices() .AsLatestClientAndLatestServiceBuilder() - .WithPendingRequestQueueFactory((queueMessageSerializer, logFactory) => + .WithPendingRequestQueueFactory((queueMessageSerializer, logFactory) => new RedisPendingRequestQueueFactory( - queueMessageSerializer, + queueMessageSerializer, dataStreamStore, redisTransport, new HalibutTimeoutsAndLimits(), @@ -210,7 +210,44 @@ public async Task OctopusCanSendMessagesToTentacle_WithEchoService(ClientAndServ } } } + + [Test] + public async Task CancellingARequestShouldResultInTheDequeuedResponseTokenBeingCanelled() + { + // Arrange + var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); + var log = Substitute.For(); + var redisTransport = new HalibutRedisTransport(new RedisFacade("localhost")); + var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); + var messageSerializer = new QueueMessageSerializerBuilder().Build(); + var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); + + var request = new RequestMessageBuilder("poll://test-endpoint").Build(); + request.Params = new[] { new ComplexObjectMultipleDataStreams(DataStream.FromString("hello"), DataStream.FromString("world")) }; + + var node1Sender = new RedisPendingRequestQueue(endpoint, log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + + using var cts = new CancellationTokenSource(); + + var queueAndWaitAsync = node1Sender.QueueAndWaitAsync(request, cts.Token); + + var requestMessageWithCancellationToken = await node1Sender.DequeueAsync(CancellationToken); + + requestMessageWithCancellationToken!.CancellationToken.IsCancellationRequested.Should().BeFalse(); + + await cts.CancelAsync(); + + try + { + await Task.Delay(TimeSpan.FromMinutes(1), requestMessageWithCancellationToken.CancellationToken); + } catch (TaskCanceledException){} + + + + requestMessageWithCancellationToken!.CancellationToken.IsCancellationRequested.Should().BeTrue(); + } + public class QueueMessageSerializerBuilder { public QueueMessageSerializer Build() @@ -230,4 +267,4 @@ StreamCapturingJsonSerializer StreamCapturingSerializer() } } } -} \ No newline at end of file +} \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/HalibutRedisTransport.cs b/source/Halibut/Queue/Redis/HalibutRedisTransport.cs index 4ab9c96f1..afa78ce69 100644 --- a/source/Halibut/Queue/Redis/HalibutRedisTransport.cs +++ b/source/Halibut/Queue/Redis/HalibutRedisTransport.cs @@ -26,7 +26,7 @@ namespace Halibut.Queue.Redis public class HalibutRedisTransport { const string Namespace = "octopus:server:halibut"; - + readonly RedisFacade facade; public HalibutRedisTransport(RedisFacade facade) @@ -39,7 +39,7 @@ static string RequestMessagesPulseChannelName(Uri endpoint) { return $"{Namespace}::RequestMessagesPulseChannelName::{endpoint}"; } - + public async Task SubscribeToRequestMessagePulseChannel(Uri endpoint, Action onRequestMessagePulse) { var channelName = RequestMessagesPulseChannelName(endpoint); @@ -49,16 +49,16 @@ public async Task SubscribeToRequestMessagePulseChannel(Uri en onRequestMessagePulse(message); }); } - + public async Task PulseRequestPushedToEndpoint(Uri endpoint, CancellationToken cancellationToken) { var channelName = RequestMessagesPulseChannelName(endpoint); string emptyJson = "{}"; // Maybe we will actually want to share data in the future, empty json means we can add stuff later. await facade.PublishToChannel(channelName, emptyJson); } - + // Request IDs list - + static string KeyForNextRequestGuidInListForEndpoint(Uri endpoint) { return $"{Namespace}::NextRequestInListForEndpoint::{endpoint}"; @@ -70,25 +70,25 @@ public async Task PushRequestGuidOnToQueue(Uri endpoint, Guid guid, Cancellation // TODO TTL await facade.ListRightPushAsync(KeyForNextRequestGuidInListForEndpoint(endpoint), guid.ToString()); } - + public async Task TryPopNextRequestGuid(Uri endpoint, CancellationToken cancellationToken) { var result = await facade.ListLeftPopAsync(KeyForNextRequestGuidInListForEndpoint(endpoint)); return result.ToGuid(); } - + // Request Message - + static string RequestMessageKey(Uri endpoint, Guid requestId) { return $"{Namespace}::RequestMessageKey::{endpoint}::{requestId}"; } static string RequestField = "RequestField"; - + public async Task PutRequest(Uri endpoint, Guid requestId, string payload, CancellationToken cancellationToken) { - var redisQueueItem = new RedisHalibutQueueItem2(requestId,payload); + var redisQueueItem = new RedisHalibutQueueItem2(requestId, payload); var serialisedQueueItem = JsonConvert.SerializeObject(redisQueueItem); @@ -96,46 +96,88 @@ public async Task PutRequest(Uri endpoint, Guid requestId, string payload, Cance await facade.SetInHash(requestKey, RequestField, serialisedQueueItem); } - + public async Task TryGetAndRemoveRequest(Uri endpoint, Guid requestId, CancellationToken cancellationToken) { var requestKey = RequestMessageKey(endpoint, requestId); var requestMessage = await facade.TryGetAndDeleteFromHash(requestKey, RequestField); - if (requestMessage == null) return null; + if (requestMessage == null) return null; var redisQueueItem = JsonConvert.DeserializeObject(requestMessage); if (redisQueueItem is null) return null; return redisQueueItem.PayloadJson; } - - + + // Response channel static string ResponseMessagesChannelName(Uri endpoint, Guid requestId) { return $"{Namespace}::ResponseMessagesChannelName::{endpoint}::{requestId}"; } - + public async Task SubScribeToResponses(Uri endpoint, Guid requestOfResponseToWaitFor, Func onResponse, CancellationToken cancellationToken) { var channelName = ResponseMessagesChannelName(endpoint, requestOfResponseToWaitFor); - var responseReceivedEvent = new AsyncManualResetEvent(false); return await facade.SubscribeToChannel(channelName, async foo => { string? response = foo.Message; - if(response is not null) await onResponse(response, cancellationToken); + if (response is not null) await onResponse(response, cancellationToken); }); } - + public async Task PublishResponse(Uri endpoint, Guid requestId, string payload, CancellationToken cancellationToken) { var channelName = ResponseMessagesChannelName(endpoint, requestId); await facade.PublishToChannel(channelName, payload); } + + + // Cancellation channel + static string RequestCancelledChannel(Uri endpoint, Guid requestId) + { + return $"{Namespace}::RequestCancelledChannel::{endpoint}::{requestId}"; + } + + public async Task SubscribeToRequestCancellation(Uri endpoint, Guid request, + Func onCancellationReceived, + CancellationToken cancellationToken) + { + var channelName = RequestCancelledChannel(endpoint, request); + return await facade.SubscribeToChannel(channelName, async foo => + { + string? response = foo.Message; + if (response is not null) await onCancellationReceived(cancellationToken); + }); + } + + public async Task PublishCancellation(Uri endpoint, Guid requestId, CancellationToken cancellationToken) + { + var channelName = RequestCancelledChannel(endpoint, requestId); + await facade.PublishToChannel(channelName, "{}"); + } + + // Cancellation Notification + + public string RequestCancelledMarkerKey(Uri endpoint, Guid requestId) + { + return $"{Namespace}::RequestCancelledMarkerKey::{endpoint}::{requestId}"; + } + + public async Task MarkRequestAsCancelled(Uri endpoint, Guid requestId, CancellationToken cancellationToken) + { + var key = RequestCancelledMarkerKey(endpoint, requestId); + await facade.SetString(key, "{}"); + } - } + public async Task IsRequestMarkedAsCancelled(Uri endpoint, Guid requestId, CancellationToken cancellationToken) + { + var key = RequestCancelledMarkerKey(endpoint, requestId); + return (await facade.GetString(key)) != null; + } +} public class RedisHalibutQueueItem2 diff --git a/source/Halibut/Queue/Redis/RedisFacade.cs b/source/Halibut/Queue/Redis/RedisFacade.cs index d32017cc4..f7cdbab05 100644 --- a/source/Halibut/Queue/Redis/RedisFacade.cs +++ b/source/Halibut/Queue/Redis/RedisFacade.cs @@ -110,6 +110,20 @@ public async Task ListRightPushAsync(string key, string payload) return value; } + + public async Task SetString(string key, string value) + { + key = "string:" + key; + var database = Connection.GetDatabase(); + await database.StringSetAsync(key, value); + } + + public async Task GetString(string key) + { + key = "string:" + key; + var database = Connection.GetDatabase(); + return await database.StringGetAsync(key); + } } } \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/RedisPendingRequest.cs b/source/Halibut/Queue/Redis/RedisPendingRequest.cs deleted file mode 100644 index 8d6f4b49e..000000000 --- a/source/Halibut/Queue/Redis/RedisPendingRequest.cs +++ /dev/null @@ -1,114 +0,0 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -using System; -using System.Threading; -using System.Threading.Tasks; -using Halibut.Diagnostics; -using Halibut.Transport.Protocol; - -namespace Halibut.Queue.Redis -{ - // TODO this code is mostly generic, so it could be shared - public class RedisPendingRequest - { - readonly RequestMessage request; - readonly ILog log; - readonly ManualResetEventSlim responseRecievedEvent; - readonly object sync = new(); - bool requestCollected; - bool completed; - - readonly TimeSpan pollingRequestMaximumMessageProcessingTimeout; - - public RedisPendingRequest(RequestMessage request, ILog log, TimeSpan pollingRequestMaximumMessageProcessingTimeout) - { - this.request = request; - this.log = log; - this.pollingRequestMaximumMessageProcessingTimeout = pollingRequestMaximumMessageProcessingTimeout; - responseRecievedEvent = new ManualResetEventSlim(false); - } - - // Waits for the request to be collected and response to come back - public async Task WaitUntilComplete(CancellationToken cancellationToken, Func pollingRequestQueueTimeElapsed) - { - await Task.CompletedTask; - log.Write(EventType.MessageExchange, "Request {0} was queued", request); - - var success = responseRecievedEvent.Wait(request.Destination.PollingRequestQueueTimeout, cancellationToken); - if (success) - { - log.Write(EventType.MessageExchange, "Request {0} was collected by the polling endpoint", request); - return; - } - - try - { - // Let em know the amount of time the request can sit on the queue for has elapsed. - await pollingRequestQueueTimeElapsed(); - } - catch - { - } - var waitForTransferToComplete = false; - lock (sync) - { - if (requestCollected) - waitForTransferToComplete = true; - else - completed = true; - } - - if (waitForTransferToComplete) - { - success = responseRecievedEvent.Wait(pollingRequestMaximumMessageProcessingTimeout); - if (success) - log.Write(EventType.MessageExchange, "Request {0} was eventually collected by the polling endpoint", request); - else - SetResponse(ResponseMessage.FromException(request, new TimeoutException(string.Format("A request was sent to a polling endpoint, the polling endpoint collected it but did not respond in the allowed time ({0}), so the request timed out.", pollingRequestMaximumMessageProcessingTimeout)))); - } - else - { - log.Write(EventType.MessageExchange, "Request {0} timed out before it could be collected by the polling endpoint", request); - SetResponse(ResponseMessage.FromException(request, new TimeoutException(string.Format("A request was sent to a polling endpoint, but the polling endpoint did not collect the request within the allowed time ({0}), so the request timed out.", request.Destination.PollingRequestQueueTimeout)))); - } - } - - public bool FYITheRequestHasBeenCollected() - { - lock (sync) - { - if (completed) - return false; - - requestCollected = true; - return true; - } - } - - public ResponseMessage? Response { get; private set; } - - public void SetResponse(ResponseMessage response) - { - lock (sync) - { - if (Response == null) - { - Response = response; - responseRecievedEvent.Set(); - } - } - } - } -} \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index 19bd92ef5..b62d3b5ac 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -13,11 +13,13 @@ // limitations under the License. using System; +using System.Collections.Concurrent; using System.Threading; using System.Threading.Tasks; using Halibut.Diagnostics; using Halibut.ServiceModel; using Halibut.Transport.Protocol; +using Halibut.Util; using Nito.AsyncEx; namespace Halibut.Queue.Redis @@ -60,47 +62,52 @@ public async ValueTask DisposeAsync() await PulseChannelSubDisposer.DisposeAsync(); } - public async Task QueueAndWaitAsync(RequestMessage request, CancellationToken cancellationToken) + public async Task QueueAndWaitAsync(RequestMessage request, CancellationToken requestCancellationToken) { // TODO: redis goes down // TODO: Other node goes down. // TODO: Respect cancellation token - var pending = new RedisPendingRequest(request, log, halibutTimeoutsAndLimits.PollingRequestQueueTimeout); + using var pending = new PendingRequest(request, log); // TODO: What if this payload was gigantic // TODO: Do we need to encrypt this? - var payload = await messageReaderWriter.PrepareRequest(request, cancellationToken); + var payload = await messageReaderWriter.PrepareRequest(request, requestCancellationToken); // Start listening for a response to the request, we don't want to miss the response. - await using var _ = await SubscribeToResponse(request.ActivityId, pending.SetResponse, cancellationToken); + await using var _ = await SubscribeToResponse(request.ActivityId, pending.SetResponse, requestCancellationToken); + + await using var tryClearRequestFromQueueWhenRequestIsCancelled + = pending.PendingRequestCancellationToken.Register(async () => await TryClearRequestFromQueue(request, pending)); + await using var trySendCancelWhenRequestIsCancelled + = pending.PendingRequestCancellationToken.Register(async () => await WatchForRequestCancellation.TrySendCancellation(halibutRedisTransport, endpoint, request)); // Make the request available before we tell people it is available. - await halibutRedisTransport.PutRequest(endpoint, request.ActivityId, payload, cancellationToken); - // --- - await halibutRedisTransport.PushRequestGuidOnToQueue(endpoint, request.ActivityId, cancellationToken); - await halibutRedisTransport.PulseRequestPushedToEndpoint(endpoint, cancellationToken); + await halibutRedisTransport.PutRequest(endpoint, request.ActivityId, payload, requestCancellationToken); + await halibutRedisTransport.PushRequestGuidOnToQueue(endpoint, request.ActivityId, requestCancellationToken); + await halibutRedisTransport.PulseRequestPushedToEndpoint(endpoint, requestCancellationToken); - await pending.WaitUntilComplete(cancellationToken, ClearRequestFromQueueWhenPickupTimeoutExpired(request, cancellationToken, pending)); + await pending.WaitUntilComplete(() => TryClearRequestFromQueue(request, pending), requestCancellationToken); return pending.Response!; } - Func ClearRequestFromQueueWhenPickupTimeoutExpired(RequestMessage request, CancellationToken cancellationToken, RedisPendingRequest pending) + async Task TryClearRequestFromQueue(RequestMessage request, PendingRequest pending) { - return async () => + // The time the message is allowed to sit on the queue for has elapsed. + // Let's try to pop if from the queue, either: + // - We pop it, which means it was never collected so let pending deal with the timeout. + // - We could not pop it, which means it was collected. + using var cts = new CancellationTokenSource(); + cts.CancelAfter(TimeSpan.FromMinutes(2)); // Best efforts. + var requestJson = await halibutRedisTransport.TryGetAndRemoveRequest(endpoint, request.ActivityId, cts.Token); + if (requestJson != null) { - // The time the message is allowed to sit on the queue for has elapsed. - // Let's try to pop if from the queue, either: - // - We pop it, which means it was never collected so let pending deal with the timeout. - // - We could not pop it, which means it was collected. - var requestJson = await halibutRedisTransport.TryGetAndRemoveRequest(endpoint, request.ActivityId, cancellationToken); - if (requestJson != null) - { - pending.FYITheRequestHasBeenCollected(); - } - }; + await pending.RequestHasBeenCollectedAndWillBeTransfered(); + } } + + async Task SubscribeToResponse(Guid activityId, Action onResponse, CancellationToken cancellationToken) @@ -115,12 +122,21 @@ async Task SubscribeToResponse(Guid activityId, public bool IsEmpty => throw new NotImplementedException(); public int Count => throw new NotImplementedException(); + ConcurrentDictionary disposablesForInFlightRequests = new(); + public async Task DequeueAsync(CancellationToken cancellationToken) { var pending = await DequeueNextAsync(); if (pending == null) return null; - return new RequestMessageWithCancellationToken(pending, CancellationToken.None); + + var watchForRequestCancellation = new WatchForRequestCancellation(endpoint, pending.ActivityId, halibutRedisTransport); + disposablesForInFlightRequests[pending.ActivityId] = new DisposableCollection(watchForRequestCancellation); + + + return new RequestMessageWithCancellationToken(pending, watchForRequestCancellation.RequestCancelledCancellationToken); } + + public async Task ApplyResponse(ResponseMessage response, Guid requestActivityId) { diff --git a/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs b/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs new file mode 100644 index 000000000..6c4a13a41 --- /dev/null +++ b/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs @@ -0,0 +1,82 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; +using System.Threading; +using System.Threading.Tasks; +using Halibut.Transport.Protocol; +using Halibut.Util; + +namespace Halibut.Queue.Redis +{ + public class WatchForRequestCancellation : IAsyncDisposable + { + + public static async Task TrySendCancellation( + HalibutRedisTransport halibutRedisTransport, + Uri endpoint, + RequestMessage request) + { + using var cts = new CancellationTokenSource(); + cts.CancelAfter(TimeSpan.FromMinutes(2)); // Best efforts. + await halibutRedisTransport.PublishCancellation(endpoint, request.ActivityId, cts.Token); + await halibutRedisTransport.MarkRequestAsCancelled(endpoint, request.ActivityId, cts.Token); + } + readonly CancellationTokenSource requestCancelledCts = new(); + public CancellationToken RequestCancelledCancellationToken => requestCancelledCts.Token; + + readonly CancellationTokenSource watchForCancellationTokenSource = new(); + + Task watchTask; + + public WatchForRequestCancellation(Uri endpoint, Guid requestActivityId, HalibutRedisTransport halibutRedisTransport) + { + var token = watchForCancellationTokenSource.Token; + watchTask = Task.Run(async () => await WatchForExceptions(endpoint, requestActivityId, halibutRedisTransport, token)); + } + + async Task WatchForExceptions(Uri endpoint, Guid requestActivityId, HalibutRedisTransport halibutRedisTransport, CancellationToken token) + { + try + { + await using var _ = await halibutRedisTransport.SubscribeToRequestCancellation(endpoint, requestActivityId, + async _ => + { + await requestCancelledCts.CancelAsync(); + await watchForCancellationTokenSource.CancelAsync(); + }, + token); + if (await halibutRedisTransport.IsRequestMarkedAsCancelled(endpoint, requestActivityId, token)) + { + await requestCancelledCts.CancelAsync(); + await watchForCancellationTokenSource.CancelAsync(); + } + + await Task.Delay(Timeout.Infinite, token); + } + catch + { + // TODO log when we get an exception we don't expect. + } + } + + public async ValueTask DisposeAsync() + { + await Try.IgnoringError(async () => await watchForCancellationTokenSource.CancelAsync()); + Try.IgnoringError(() => watchForCancellationTokenSource.Dispose()); + await Try.IgnoringError(async () => await requestCancelledCts.CancelAsync()); + Try.IgnoringError(() => requestCancelledCts.Dispose()); + } + } +} \ No newline at end of file diff --git a/source/Halibut/ServiceModel/PendingRequest.cs b/source/Halibut/ServiceModel/PendingRequest.cs new file mode 100644 index 000000000..f8db96f48 --- /dev/null +++ b/source/Halibut/ServiceModel/PendingRequest.cs @@ -0,0 +1,241 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; +using System.Threading; +using System.Threading.Tasks; +using Halibut.Diagnostics; +using Halibut.Exceptions; +using Halibut.Transport; +using Halibut.Transport.Protocol; +using Nito.AsyncEx; + +namespace Halibut.ServiceModel +{ + + class PendingRequest : IDisposable + { + readonly RequestMessage request; + readonly ILog log; + readonly AsyncManualResetEvent responseWaiter = new(false); + readonly SemaphoreSlim transferLock = new(1, 1); + bool transferBegun; + bool completed; + readonly CancellationTokenSource pendingRequestCancellationTokenSource; + ResponseMessage? response; + + public PendingRequest(RequestMessage request, ILog log) + { + this.request = request; + this.log = log; + + pendingRequestCancellationTokenSource = new CancellationTokenSource(); + PendingRequestCancellationToken = pendingRequestCancellationTokenSource.Token; + } + + public RequestMessage Request => request; + + /// + /// + /// + /// + /// This will be called either when the pick-up timeout has elapsed OR if the Cancellation Token has been triggered. + /// This gives the user an opportunity to remove the pending request from shared places and optionally + /// call BeginTransfer + /// + /// + public async Task WaitUntilComplete(Func timePendingRequestCanBeOnTheQueueHasElapsed, CancellationToken cancellationToken) + { + log.Write(EventType.MessageExchange, "Request {0} was queued", request); + + bool responseSet; + var cancelled = false; + + try + { + responseSet = await WaitForResponseToBeSet( + request.Destination.PollingRequestQueueTimeout, + // Don't cancel a dequeued request as we need to wait PollingRequestMaximumMessageProcessingTimeout for it to complete + cancelTheRequestWhenTransferHasBegun: false, + cancellationToken); + + if (responseSet) + { + log.Write(EventType.MessageExchange, "Request {0} was collected by the polling endpoint", request); + return; + } + } + catch (RequestCancelledException) + { + cancelled = true; + if(!transferBegun) await timePendingRequestCanBeOnTheQueueHasElapsed(); + using (await transferLock.LockAsync(CancellationToken.None)) + { + if (!transferBegun) + { + completed = true; + log.Write(EventType.MessageExchange, "Request {0} was cancelled before it could be collected by the polling endpoint", request); + throw; + } + } + } + + if(!transferBegun) await timePendingRequestCanBeOnTheQueueHasElapsed(); + var waitForTransferToComplete = false; + using (await transferLock.LockAsync(CancellationToken.None)) + { + if (transferBegun) + { + waitForTransferToComplete = true; + } + else + { + completed = true; + } + } + + if (waitForTransferToComplete) + { + responseSet = await WaitForResponseToBeSet( + null, + // Cancel the dequeued request to force Reads and Writes to be cancelled + cancelTheRequestWhenTransferHasBegun: true, + cancellationToken); + + if (responseSet) + { + // We end up here when the request is cancelled but already being transferred so we need to adjust the log message accordingly + if (cancelled) + { + log.Write(EventType.MessageExchange, "Request {0} was collected by the polling endpoint", request); + } + else + { + log.Write(EventType.MessageExchange, "Request {0} was eventually collected by the polling endpoint", request); + } + } + else + { + if (cancellationToken.IsCancellationRequested) + { + log.Write(EventType.MessageExchange, "Request {0} was cancelled before a response was received", request); + SetResponse(ResponseMessage.FromException( + request, + new TimeoutException($"A request was sent to a polling endpoint, the polling endpoint collected it but the request was cancelled before the polling endpoint responded."), + ConnectionState.Connecting)); + } + else + { + // This should never happen. + log.Write(EventType.MessageExchange, "Request {0} had an internal error, unexpectedly stopped waiting for the response.", request); + SetResponse(ResponseMessage.FromException( + request, + new PendingRequestQueueInternalException($"Request {request.Id} had an internal error, unexpectedly stopped waiting for the response."))); + } + } + } + else + { + log.Write(EventType.MessageExchange, "Request {0} timed out before it could be collected by the polling endpoint", request); + SetResponse(ResponseMessage.FromException( + request, + new TimeoutException($"A request was sent to a polling endpoint, but the polling endpoint did not collect the request within the allowed time ({request.Destination.PollingRequestQueueTimeout}), so the request timed out."), + ConnectionState.Connecting)); + } + } + + async Task WaitForResponseToBeSet( + TimeSpan? timeout, + bool cancelTheRequestWhenTransferHasBegun, + CancellationToken cancellationToken) + { + using var timeoutCancellationTokenSource = timeout.HasValue ? new CancellationTokenSource(timeout.Value) : new CancellationTokenSource(); + using var linkedTokenSource = CancellationTokenSource.CreateLinkedTokenSource(timeoutCancellationTokenSource.Token, cancellationToken); + + try + { + await responseWaiter.WaitAsync(linkedTokenSource.Token); + } + catch (OperationCanceledException ex) + { + using (await transferLock.LockAsync(CancellationToken.None)) + { + if (transferBegun && cancelTheRequestWhenTransferHasBegun) + { + // Cancel the dequeued request. This will cause co-operative cancellation on the thread dequeuing the request + pendingRequestCancellationTokenSource.Cancel(); + } + else if (!transferBegun) + { + // Cancel the queued request. This will flag the request as cancelled to stop it being dequeued + pendingRequestCancellationTokenSource.Cancel(); + } + + if (timeoutCancellationTokenSource.IsCancellationRequested) + { + return false; + } + + throw transferBegun ? new TransferringRequestCancelledException(ex) : new ConnectingRequestCancelledException(ex); + } + } + + return true; + } + + public async Task RequestHasBeenCollectedAndWillBeTransfered() + { + // The PendingRequest is Disposed at the end of QueueAndWaitAsync but a race condition + // exists in the current approach that means DequeueAsync could pick this request up after + // it has been disposed. At that point we are no longer interested in the PendingRequest so + // this is "ok" and wrapping BeginTransfer in a try..catch.. ensures we don't error if the + // race condition occurs and also stops the polling tentacle dequeuing the request successfully. + try + { + using (await transferLock.LockAsync(CancellationToken.None)) + { + // Check if the request has already been completed or if the request has been cancelled + // to ensure we don't dequeue an already completed or already cancelled request + if (completed || pendingRequestCancellationTokenSource.IsCancellationRequested) + { + return false; + } + + transferBegun = true; + return true; + } + } + catch (ObjectDisposedException) + { + return false; + } + } + + public ResponseMessage Response => response ?? throw new InvalidOperationException("Response has not been set."); + public CancellationToken PendingRequestCancellationToken { get; } + + public void SetResponse(ResponseMessage response) + { + this.response = response; + responseWaiter.Set(); + } + + public void Dispose() + { + pendingRequestCancellationTokenSource?.Dispose(); + transferLock?.Dispose(); + } + } + +} \ No newline at end of file diff --git a/source/Halibut/ServiceModel/PendingRequestQueueAsync.cs b/source/Halibut/ServiceModel/PendingRequestQueueAsync.cs index c33655bea..f106ea08e 100644 --- a/source/Halibut/ServiceModel/PendingRequestQueueAsync.cs +++ b/source/Halibut/ServiceModel/PendingRequestQueueAsync.cs @@ -5,7 +5,6 @@ using System.Threading.Tasks; using Halibut.Diagnostics; using Halibut.Exceptions; -using Halibut.Transport; using Halibut.Transport.Protocol; using Nito.AsyncEx; @@ -59,7 +58,7 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can try { - await pending.WaitUntilComplete(cancellationToken); + await pending.WaitUntilComplete(() => Task.CompletedTask, cancellationToken); } finally { @@ -109,7 +108,7 @@ public int Count return null; } - var result = await pending.BeginTransfer(); + var result = await pending.RequestHasBeenCollectedAndWillBeTransfered(); if (result) { return new (pending.Request, pending.PendingRequestCancellationToken); @@ -177,210 +176,6 @@ public async Task ApplyResponse(ResponseMessage response, Guid requestActivityId } } - class PendingRequest : IDisposable - { - readonly RequestMessage request; - readonly ILog log; - readonly AsyncManualResetEvent responseWaiter = new(false); - readonly SemaphoreSlim transferLock = new(1, 1); - bool transferBegun; - bool completed; - readonly CancellationTokenSource pendingRequestCancellationTokenSource; - ResponseMessage? response; - - public PendingRequest(RequestMessage request, ILog log) - { - this.request = request; - this.log = log; - - pendingRequestCancellationTokenSource = new CancellationTokenSource(); - PendingRequestCancellationToken = pendingRequestCancellationTokenSource.Token; - } - - public RequestMessage Request => request; - - public async Task WaitUntilComplete(CancellationToken cancellationToken) - { - log.Write(EventType.MessageExchange, "Request {0} was queued", request); - - bool responseSet; - var cancelled = false; - - try - { - responseSet = await WaitForResponseToBeSet( - request.Destination.PollingRequestQueueTimeout, - // Don't cancel a dequeued request as we need to wait PollingRequestMaximumMessageProcessingTimeout for it to complete - cancelTheRequestWhenTransferHasBegun: false, - cancellationToken); - - if (responseSet) - { - log.Write(EventType.MessageExchange, "Request {0} was collected by the polling endpoint", request); - return; - } - } - catch (RequestCancelledException) - { - cancelled = true; - - using (await transferLock.LockAsync(CancellationToken.None)) - { - if (!transferBegun) - { - completed = true; - log.Write(EventType.MessageExchange, "Request {0} was cancelled before it could be collected by the polling endpoint", request); - throw; - } - } - } - - var waitForTransferToComplete = false; - using (await transferLock.LockAsync(CancellationToken.None)) - { - if (transferBegun) - { - waitForTransferToComplete = true; - } - else - { - completed = true; - } - } - - if (waitForTransferToComplete) - { - responseSet = await WaitForResponseToBeSet( - null, - // Cancel the dequeued request to force Reads and Writes to be cancelled - cancelTheRequestWhenTransferHasBegun: true, - cancellationToken); - - if (responseSet) - { - // We end up here when the request is cancelled but already being transferred so we need to adjust the log message accordingly - if (cancelled) - { - log.Write(EventType.MessageExchange, "Request {0} was collected by the polling endpoint", request); - } - else - { - log.Write(EventType.MessageExchange, "Request {0} was eventually collected by the polling endpoint", request); - } - } - else - { - if (cancellationToken.IsCancellationRequested) - { - log.Write(EventType.MessageExchange, "Request {0} was cancelled before a response was received", request); - SetResponse(ResponseMessage.FromException( - request, - new TimeoutException($"A request was sent to a polling endpoint, the polling endpoint collected it but the request was cancelled before the polling endpoint responded."), - ConnectionState.Connecting)); - } - else - { - // This should never happen. - log.Write(EventType.MessageExchange, "Request {0} had an internal error, unexpectedly stopped waiting for the response.", request); - SetResponse(ResponseMessage.FromException( - request, - new PendingRequestQueueInternalException($"Request {request.Id} had an internal error, unexpectedly stopped waiting for the response."))); - } - } - } - else - { - log.Write(EventType.MessageExchange, "Request {0} timed out before it could be collected by the polling endpoint", request); - SetResponse(ResponseMessage.FromException( - request, - new TimeoutException($"A request was sent to a polling endpoint, but the polling endpoint did not collect the request within the allowed time ({request.Destination.PollingRequestQueueTimeout}), so the request timed out."), - ConnectionState.Connecting)); - } - } - - async Task WaitForResponseToBeSet( - TimeSpan? timeout, - bool cancelTheRequestWhenTransferHasBegun, - CancellationToken cancellationToken) - { - using var timeoutCancellationTokenSource = timeout.HasValue ? new CancellationTokenSource(timeout.Value) : new CancellationTokenSource(); - using var linkedTokenSource = CancellationTokenSource.CreateLinkedTokenSource(timeoutCancellationTokenSource.Token, cancellationToken); - - try - { - await responseWaiter.WaitAsync(linkedTokenSource.Token); - } - catch (OperationCanceledException ex) - { - using (await transferLock.LockAsync(CancellationToken.None)) - { - if (transferBegun && cancelTheRequestWhenTransferHasBegun) - { - // Cancel the dequeued request. This will cause co-operative cancellation on the thread dequeuing the request - pendingRequestCancellationTokenSource.Cancel(); - } - else if (!transferBegun) - { - // Cancel the queued request. This will flag the request as cancelled to stop it being dequeued - pendingRequestCancellationTokenSource.Cancel(); - } - - if (timeoutCancellationTokenSource.IsCancellationRequested) - { - return false; - } - - throw transferBegun ? new TransferringRequestCancelledException(ex) : new ConnectingRequestCancelledException(ex); - } - } - - return true; - } - - public async Task BeginTransfer() - { - // The PendingRequest is Disposed at the end of QueueAndWaitAsync but a race condition - // exists in the current approach that means DequeueAsync could pick this request up after - // it has been disposed. At that point we are no longer interested in the PendingRequest so - // this is "ok" and wrapping BeginTransfer in a try..catch.. ensures we don't error if the - // race condition occurs and also stops the polling tentacle dequeuing the request successfully. - try - { - using (await transferLock.LockAsync(CancellationToken.None)) - { - // Check if the request has already been completed or if the request has been cancelled - // to ensure we don't dequeue an already completed or already cancelled request - if (completed || pendingRequestCancellationTokenSource.IsCancellationRequested) - { - return false; - } - - transferBegun = true; - return true; - } - } - catch (ObjectDisposedException) - { - return false; - } - } - - public ResponseMessage Response => response ?? throw new InvalidOperationException("Response has not been set."); - public CancellationToken PendingRequestCancellationToken { get; } - - public void SetResponse(ResponseMessage response) - { - this.response = response; - responseWaiter.Set(); - } - - public void Dispose() - { - pendingRequestCancellationTokenSource?.Dispose(); - transferLock?.Dispose(); - } - } - public ValueTask DisposeAsync() { entireQueueCancellationTokenSource.Cancel(); diff --git a/source/Halibut/Util/DisposableCollection.cs b/source/Halibut/Util/DisposableCollection.cs index 3bbb96e66..f79ea1a06 100644 --- a/source/Halibut/Util/DisposableCollection.cs +++ b/source/Halibut/Util/DisposableCollection.cs @@ -27,6 +27,16 @@ public class DisposableCollection : IDisposable, IAsyncDisposable readonly ConcurrentStack disposables = new(); bool isDisposed; + public DisposableCollection() + { + + } + + public DisposableCollection(params IAsyncDisposable[] disposables) + { + this.disposables.PushRange(disposables); + } + public T Add(T disposable) where T : IDisposable { if (isDisposed) throw new ObjectDisposedException("Cannot add item for disposal. This collection has already been disposed."); diff --git a/source/Halibut/Util/Try.cs b/source/Halibut/Util/Try.cs index 4c01d353c..0aba28cb5 100644 --- a/source/Halibut/Util/Try.cs +++ b/source/Halibut/Util/Try.cs @@ -34,6 +34,30 @@ public static SilentStreamDisposer CatchingErrorOnDisposal(Stream streamToDispos { return new SilentStreamDisposer(streamToDispose, onFailure); } + + public static void IgnoringError(Action tryThisAction) + { + try + { + tryThisAction(); + } + catch + { + // ignored + } + } + + public static async Task IgnoringError(Func tryThisAction) + { + try + { + await tryThisAction(); + } + catch + { + // ignored + } + } } } \ No newline at end of file From b084ae84abe4a0939b00fbfa5feda30d1d4637f5 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Wed, 30 Jul 2025 13:04:47 +1000 Subject: [PATCH 008/137] pick up test --- .../Redis/RedisPendingRequestQueueFixture.cs | 45 ++++++++++++++++--- source/Halibut/Queue/Redis/RedisFacade.cs | 22 +++++---- 2 files changed, 51 insertions(+), 16 deletions(-) diff --git a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs index 825577789..711e0529a 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs @@ -31,13 +31,17 @@ namespace Halibut.Tests.Queue.Redis { public class RedisPendingRequestQueueFixture : BaseTest { + private static RedisFacade CreateRedisFacade() + { + return new RedisFacade("localhost", Guid.NewGuid().ToString()); + } [Test] public async Task DequeueAsync_ShouldReturnRequestFromRedis() { // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid()); var log = Substitute.For(); - var redisTransport = new HalibutRedisTransport(new RedisFacade("localhost")); + var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); @@ -57,6 +61,33 @@ public async Task DequeueAsync_ShouldReturnRequestFromRedis() result.RequestMessage.MethodName.Should().Be(request.MethodName); result.RequestMessage.ServiceName.Should().Be(request.ServiceName); } + + [Test] + public async Task WhenThePickupTimeoutExpires_AnErrorsReturned_AndTheRequestCanNotBeCollected() + { + // Arrange + var endpoint = new Uri("poll://" + Guid.NewGuid()); + var log = Substitute.For(); + var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); + var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); + var messageSerializer = new QueueMessageSerializerBuilder().Build(); + var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); + + var request = new RequestMessageBuilder("poll://test-endpoint") + .WithServiceEndpoint(b => b.WithPollingRequestQueueTimeout(TimeSpan.FromMilliseconds(100))) + .Build(); + + var sut = new RedisPendingRequestQueue(endpoint, log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + + var response = await sut.QueueAndWaitAsync(request, CancellationToken.None); + + response.Error!.Message.Should().Contain("A request was sent to a polling endpoint, but the polling endpoint did not collect the request within the allowed time"); + + // Act + var result = await sut.DequeueAsync(CancellationToken); + + result.Should().BeNull(); + } //[Test] public async Task When100kTentaclesAreSubscribed_TheQueueStillWorks() @@ -64,7 +95,7 @@ public async Task When100kTentaclesAreSubscribed_TheQueueStillWorks() // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid()); var log = Substitute.For(); - var redisTransport = new HalibutRedisTransport(new RedisFacade("localhost")); + var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); @@ -90,7 +121,7 @@ public async Task When100kTentaclesAreSubscribed_TheQueueStillWorks() { var request = new RequestMessageBuilder(endpoint.ToString()).Build(); - await using var sut = new RedisPendingRequestQueue(endpoint, log, new HalibutRedisTransport(new RedisFacade("localhost")), messageReaderWriter, halibutTimeoutsAndLimits); + await using var sut = new RedisPendingRequestQueue(endpoint, log, new HalibutRedisTransport(CreateRedisFacade()), messageReaderWriter, halibutTimeoutsAndLimits); var resultTask = sut.DequeueAsync(CancellationToken); @@ -118,7 +149,7 @@ public async Task FullSendAndReceiveShouldWork() // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); var log = Substitute.For(); - var redisTransport = new HalibutRedisTransport(new RedisFacade("localhost")); + var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); @@ -151,7 +182,7 @@ public async Task FullSendAndReceiveWithDataStreamShouldWork() // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); var log = Substitute.For(); - var redisTransport = new HalibutRedisTransport(new RedisFacade("localhost")); + var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); @@ -186,7 +217,7 @@ public async Task FullSendAndReceiveWithDataStreamShouldWork() [LatestClientAndLatestServiceTestCases(testNetworkConditions: false, testListening: false, testWebSocket: false)] public async Task OctopusCanSendMessagesToTentacle_WithEchoService(ClientAndServiceTestCase clientAndServiceTestCase) { - var redisTransport = new HalibutRedisTransport(new RedisFacade("localhost")); + var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); await using (var clientAndService = await clientAndServiceTestCase.CreateTestCaseBuilder() @@ -218,7 +249,7 @@ public async Task CancellingARequestShouldResultInTheDequeuedResponseTokenBeingC // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); var log = Substitute.For(); - var redisTransport = new HalibutRedisTransport(new RedisFacade("localhost")); + var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); diff --git a/source/Halibut/Queue/Redis/RedisFacade.cs b/source/Halibut/Queue/Redis/RedisFacade.cs index f7cdbab05..2941e2699 100644 --- a/source/Halibut/Queue/Redis/RedisFacade.cs +++ b/source/Halibut/Queue/Redis/RedisFacade.cs @@ -27,10 +27,14 @@ public class RedisFacade : IDisposable readonly Lazy connection; ConnectionMultiplexer Connection => connection.Value; + + string keyPrefix; - public RedisFacade(string redisHost) + public RedisFacade(string redisHost, string? keyPrefix) { + this.keyPrefix = keyPrefix ?? "halibut"; + connection = new Lazy(() => { return ConnectionMultiplexer.Connect(redisHost); @@ -47,7 +51,7 @@ public void Dispose() public async Task SubscribeToChannel(string channelName, Func onMessage) { - channelName = "channel:" + channelName; + channelName = "channel:" + keyPrefix + ":" + channelName; // TODO ever call needs to respect the cancellation token var channel = await Connection.GetSubscriber() .SubscribeAsync(new RedisChannel(channelName, RedisChannel.PatternMode.Literal)); @@ -59,14 +63,14 @@ public async Task SubscribeToChannel(string channelName, Func< public async Task PublishToChannel(string channelName, string payload) { - channelName = "channel:" + channelName; + channelName = "channel:" + keyPrefix + ":" + channelName; var subscriber = Connection.GetSubscriber(); await subscriber.PublishAsync(new RedisChannel(channelName, RedisChannel.PatternMode.Literal), payload); } public async Task SetInHash(string key, string field, string payload) { - key = "hash:" + key; + key = "hash:" + keyPrefix + ":" + key; // TODO: TTL // TODO ever call needs to respect the cancellation token var ttl = new TimeSpan(9, 9, 9); @@ -78,7 +82,7 @@ public async Task SetInHash(string key, string field, string payload) public async Task TryGetAndDeleteFromHash(string key, string field) { // TODO ever call needs to respect the cancellation token - key = "hash:" + key; + key = "hash:" + keyPrefix + ":" + key; var database = Connection.GetDatabase(); var value = await database.HashGetAsync(key, new RedisValue(field)); var res = await database.KeyDeleteAsync(key); @@ -92,7 +96,7 @@ public async Task SetInHash(string key, string field, string payload) public async Task ListRightPushAsync(string key, string payload) { - key = "list:" + key; + key = "list:" + keyPrefix + ":" + key; var database = Connection.GetDatabase(); // TODO can we set TTL on this? await database.ListRightPushAsync(key, payload); @@ -100,7 +104,7 @@ public async Task ListRightPushAsync(string key, string payload) public async Task ListLeftPopAsync(string key) { - key = "list:" + key; + key = "list:" + keyPrefix + ":" + key; var database = Connection.GetDatabase(); var value = await database.ListLeftPopAsync(key); if (value.IsNull) @@ -113,14 +117,14 @@ public async Task ListRightPushAsync(string key, string payload) public async Task SetString(string key, string value) { - key = "string:" + key; + key = "string:" + keyPrefix + ":" + key; var database = Connection.GetDatabase(); await database.StringSetAsync(key, value); } public async Task GetString(string key) { - key = "string:" + key; + key = "string:" + keyPrefix + ":" + key; var database = Connection.GetDatabase(); return await database.StringGetAsync(key); } From 2ba97f8e359c8391e4eecd14b276f9fc5cb20f3c Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Thu, 31 Jul 2025 09:27:44 +1000 Subject: [PATCH 009/137] Redis sub retry --- .../RedisFacadeWhenRedisGoesDownAwayTests.cs | 134 ++++++++++++++++ .../Redis/RedisPendingRequestQueueFixture.cs | 9 +- .../Queue/Redis/HalibutRedisTransport.cs | 2 + source/Halibut/Queue/Redis/RedisFacade.cs | 151 ++++++++++++++++-- .../Queue/Redis/RedisPendingRequestQueue.cs | 1 + .../Queue/Redis/ResilientSubscriber.cs | 120 ++++++++++++++ .../Redis/WatchForRequestCancellation.cs | 15 +- 7 files changed, 412 insertions(+), 20 deletions(-) create mode 100644 source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs create mode 100644 source/Halibut/Queue/Redis/ResilientSubscriber.cs diff --git a/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs b/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs new file mode 100644 index 000000000..ed5727875 --- /dev/null +++ b/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs @@ -0,0 +1,134 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; +using System.Collections.Generic; +using System.Globalization; +using System.Threading.Tasks; +using FluentAssertions; +using Halibut.Diagnostics; +using Halibut.Logging; +using Halibut.Queue.Redis; +using Halibut.Tests.Support.Logging; +using NUnit.Framework; +using Octopus.TestPortForwarder; +using StackExchange.Redis; + +namespace Halibut.Tests.Queue.Redis +{ + public class RedisFacadeWhenRedisGoesDownAwayTests : BaseTest + { + private static RedisFacade CreateRedisFacade(int port) => new("localhost:" + port, Guid.NewGuid().ToString(), new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix("")); + + + int redisPort = 6379; + + [Test] + public async Task WhenTheConnectionToRedisBrieflyGoesDown_FutureRequestsAShortTimeLaterCanBeExecuted() + { + + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); + + var configurationOptions = new ConfigurationOptions(); + configurationOptions.EndPoints.Add("localhost:" + redisPort); + + await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); + + await redisFacade.SetString("foo", "bar"); + + (await redisFacade.GetString("foo")).Should().Be("bar"); + + portForwarder.EnterKillNewAndExistingConnectionsMode(); + portForwarder.ReturnToNormalMode(); + + // After a short delay it does seem to work again. + await Task.Delay(100); + + await redisFacade.GetString("foo"); + + + await Task.CompletedTask; + } + + [Test] + public async Task WhenSubWhenRedisCanNotBeReached() + { + + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); + + var configurationOptions = new ConfigurationOptions(); + configurationOptions.EndPoints.Add("localhost:" + portForwarder.ListeningPort); + configurationOptions.AbortOnConnectFail = false; + + var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix("Unstable"); + log.Write(EventType.Diagnostic, "Hello from log"); + + await using var redisViaPortForwarder = new RedisFacade(configurationOptions, Guid.NewGuid().ToString(), log); + + portForwarder.EnterKillNewAndExistingConnectionsMode(); + + + await using var channel = await redisViaPortForwarder.SubscribeToChannel("bob", async message => + { + await Task.CompletedTask; + }); + } + + [Test] + public async Task WhenSubScribedToAChannelAndRedisConnectGoesAwayWhatHappens() + { + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); + + var redisLogCreator = new TestContextLogCreator("Redis", LogLevel.Trace); + + var guid = Guid.NewGuid().ToString(); + await using var redisViaPortForwarder = new RedisFacade("localhost:" + portForwarder.ListeningPort, guid, redisLogCreator.CreateNewForPrefix("Unstable")); + + await using var redisStableConnection = new RedisFacade("localhost:" + redisPort, guid, redisLogCreator.CreateNewForPrefix("Stable")); + + var msgs = new List(); + + await using var channel = await redisViaPortForwarder.SubscribeToChannel("bob", async message => + { + await Task.CompletedTask; + msgs.Add(message.Message!); + }); + + await Task.Delay(1000); + await redisViaPortForwarder.PublishToChannel("bob", "hello"); + await redisStableConnection.PublishToChannel("bob", "hello stable"); + msgs.Should().BeEquivalentTo("hello", "hello stable"); + + portForwarder.EnterKillNewAndExistingConnectionsMode(); + await Task.Delay(5000); + portForwarder.ReturnToNormalMode(); + + + while (msgs.Count <= 2) + { + Logger.Information("Trying again"); + await redisStableConnection.PublishToChannel("bob", "hello"); + await Task.Delay(5000); + } + + + + + + await Task.CompletedTask; + } + + + } +} \ No newline at end of file diff --git a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs index 711e0529a..31881daa9 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs @@ -9,12 +9,14 @@ using FluentAssertions; using Halibut.Diagnostics; using Halibut.Exceptions; +using Halibut.Logging; using Halibut.Queue; using Halibut.Queue.QueuedDataStreams; using Halibut.Queue.Redis; using Halibut.ServiceModel; using Halibut.Tests.Builders; using Halibut.Tests.Support; +using Halibut.Tests.Support.Logging; using Halibut.Tests.Support.TestAttributes; using Halibut.Tests.Support.TestCases; using Halibut.Tests.TestServices.Async; @@ -26,15 +28,14 @@ using NUnit.Framework; using Serilog; using DisposableCollection = Halibut.Util.DisposableCollection; +using ILog = Halibut.Diagnostics.ILog; namespace Halibut.Tests.Queue.Redis { public class RedisPendingRequestQueueFixture : BaseTest { - private static RedisFacade CreateRedisFacade() - { - return new RedisFacade("localhost", Guid.NewGuid().ToString()); - } + private static RedisFacade CreateRedisFacade() => new("localhost", Guid.NewGuid().ToString(), new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix("")); + [Test] public async Task DequeueAsync_ShouldReturnRequestFromRedis() { diff --git a/source/Halibut/Queue/Redis/HalibutRedisTransport.cs b/source/Halibut/Queue/Redis/HalibutRedisTransport.cs index afa78ce69..ed8211cd0 100644 --- a/source/Halibut/Queue/Redis/HalibutRedisTransport.cs +++ b/source/Halibut/Queue/Redis/HalibutRedisTransport.cs @@ -160,6 +160,8 @@ public async Task PublishCancellation(Uri endpoint, Guid requestId, Cancellation } // Cancellation Notification + // TODO: I think this wants to become some sort of: is the sender still interested in the results + // e.g. if the sender of the request is gone we should treat that as a cancellation. public string RequestCancelledMarkerKey(Uri endpoint, Guid requestId) { diff --git a/source/Halibut/Queue/Redis/RedisFacade.cs b/source/Halibut/Queue/Redis/RedisFacade.cs index 2941e2699..8948e51d6 100644 --- a/source/Halibut/Queue/Redis/RedisFacade.cs +++ b/source/Halibut/Queue/Redis/RedisFacade.cs @@ -16,49 +16,178 @@ using System.Threading; using System.Threading.Tasks; using Halibut.Util; +using Halibut.Diagnostics; // Add logging support using Newtonsoft.Json; +using Nito.AsyncEx; using StackExchange.Redis; using StackExchange.Redis.KeyspaceIsolation; namespace Halibut.Queue.Redis { - public class RedisFacade : IDisposable + /// + /// Facade for Redis operations with built-in connection monitoring and disconnect detection. + /// + /// Usage example for connection monitoring: + /// + /// var facade = new RedisFacade("localhost:6379", "myapp", logger); + /// + /// // Monitor overall connection events + /// facade.ConnectionFailed += message => Console.WriteLine($"Connection failed: {message}"); + /// facade.ConnectionRestored += message => Console.WriteLine($"Connection restored: {message}"); + /// facade.ErrorOccurred += message => Console.WriteLine($"Redis error: {message}"); + /// + /// // Subscribe with per-subscription monitoring + /// var subscription = await facade.SubscribeToChannel("my-channel", async message => { + /// Console.WriteLine($"Received: {message}"); + /// }); + /// + /// // Monitor individual subscription disconnects + /// if (subscription is RedisSubscriptionWrapper wrapper) + /// { + /// wrapper.SubscriptionDisconnected += message => Console.WriteLine($"Subscription lost: {message}"); + /// wrapper.SubscriptionReconnected += message => Console.WriteLine($"Subscription restored: {message}"); + /// } + /// + /// // Check connection status + /// if (!facade.IsConnected) + /// { + /// Console.WriteLine("Redis is not connected!"); + /// } + /// + /// + public class RedisFacade : IAsyncDisposable { readonly Lazy connection; + readonly ILog log; ConnectionMultiplexer Connection => connection.Value; string keyPrefix; - public RedisFacade(string redisHost, string? keyPrefix) - { + CancellationTokenSource cts; + CancellationToken facadeCancellationToken; + + public RedisFacade(string configuration, string? keyPrefix, ILog log) : this(ConfigurationOptions.Parse(configuration), keyPrefix, log) + { + + } + public RedisFacade(ConfigurationOptions redisOptions, string? keyPrefix, ILog log) + { this.keyPrefix = keyPrefix ?? "halibut"; + this.log = log; + this.cts = new CancellationTokenSource(); + this.facadeCancellationToken = cts.Token; connection = new Lazy(() => { - return ConnectionMultiplexer.Connect(redisHost); + var multiplexer = ConnectionMultiplexer.Connect(redisOptions); + + //redisOptions.ReconnectRetryPolicy = new LinearRetry() + + // Subscribe to connection events + multiplexer.ConnectionFailed += OnConnectionFailed; + multiplexer.ConnectionRestored += OnConnectionRestored; + multiplexer.ErrorMessage += OnErrorMessage; + + return multiplexer; }); } - public void Dispose() + + public class ConnectionInErrorHelper + { + readonly TaskCompletionSource connectionInError = new TaskCompletionSource(); + + public bool IsConnectionInError => connectionInError.Task.IsCompleted; + + public void SetIsInError() => connectionInError.SetResult(); + + public Task CompletesWhenAConnectionErrorOccurs => connectionInError.Task; + } + + private ConnectionInErrorHelper AConnectionHasErroredOutSinceYouGotThis = new ConnectionInErrorHelper(); + private readonly object errorOccuredLock = new object(); + + private ConnectionInErrorHelper ConnectionInErrorHelperProvider() => AConnectionHasErroredOutSinceYouGotThis; + + + private void RecordConnectionErrorHasOccured() + { + lock (errorOccuredLock) + { + var inErrorConnectionInError = this.AConnectionHasErroredOutSinceYouGotThis; + this.AConnectionHasErroredOutSinceYouGotThis = new (); + inErrorConnectionInError.SetIsInError(); + } + } + private void OnConnectionFailed(object? sender, ConnectionFailedEventArgs e) + { + RecordConnectionErrorHasOccured(); + + var message = $"Redis connection failed - EndPoint: {e.EndPoint}, Failure: {e.FailureType}, Exception: {e.Exception?.Message}"; + + log?.Write(EventType.Error, message); + } + + private void OnErrorMessage(object? sender, RedisErrorEventArgs e) { + var message = $"Redis error - EndPoint: {e.EndPoint}, Message: {e.Message}"; + log?.Write(EventType.Error, message); + } + + private void OnConnectionRestored(object? sender, ConnectionFailedEventArgs e) + { + var message = $"Redis connection restored - EndPoint: {e.EndPoint}"; + log?.Write(EventType.Diagnostic, message); + } + + public bool IsConnected => connection.IsValueCreated && Connection.IsConnected; + + public async ValueTask DisposeAsync() + { + await Try.IgnoringError(async () => await cts.CancelAsync()); + Try.IgnoringError(() => cts.Dispose()); + if (connection.IsValueCreated) { - connection.Value.Dispose(); + var conn = connection.Value; + + Try.IgnoringError(() => + { + // Unsubscribe from events before disposing + conn.ConnectionFailed -= OnConnectionFailed; + conn.ConnectionRestored -= OnConnectionRestored; + conn.ErrorMessage -= OnErrorMessage; + }); + + await Try.IgnoringError(async () => await conn.DisposeAsync()); } } + + public async Task SubscribeToChannel(string channelName, Func onMessage) { + channelName = "channel:" + keyPrefix + ":" + channelName; // TODO ever call needs to respect the cancellation token - var channel = await Connection.GetSubscriber() - .SubscribeAsync(new RedisChannel(channelName, RedisChannel.PatternMode.Literal)); - - channel.OnMessage(onMessage); + // var channel = await Connection.GetSubscriber() + // .SubscribeAsync(new RedisChannel(channelName, RedisChannel.PatternMode.Literal)); + // + // channel.OnMessage(onMessage); + + var resilientSubscriber = new ResilientSubscriber(this.Connection, + channelName, + onMessage, + facadeCancellationToken, + () => ConnectionInErrorHelperProvider(), + log + ); + + await resilientSubscriber.StartSubscribe(); - return new FuncAsyncDisposable(() => channel.UnsubscribeAsync()); + return resilientSubscriber; } public async Task PublishToChannel(string channelName, string payload) diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index b62d3b5ac..cd681ff00 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -67,6 +67,7 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can // TODO: redis goes down // TODO: Other node goes down. // TODO: Respect cancellation token + // TODO RedisConnectionException can be raised out of here, what should the queue do? using var pending = new PendingRequest(request, log); // TODO: What if this payload was gigantic diff --git a/source/Halibut/Queue/Redis/ResilientSubscriber.cs b/source/Halibut/Queue/Redis/ResilientSubscriber.cs new file mode 100644 index 000000000..cfdb7c9d6 --- /dev/null +++ b/source/Halibut/Queue/Redis/ResilientSubscriber.cs @@ -0,0 +1,120 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; +using System.Threading; +using System.Threading.Tasks; +using Halibut.Diagnostics; +using Halibut.Util; +using StackExchange.Redis; + +namespace Halibut.Queue.Redis +{ + public class ResilientSubscriber : IAsyncDisposable + { + readonly ConnectionMultiplexer Connection; + readonly string channelName; + readonly Func onMessage; + readonly Func ConnectionInErrorProvider; + readonly ILog log; + + CancellationTokenSource staySubscribedCancellationTokenSource; + + public ResilientSubscriber(ConnectionMultiplexer connection, + string channelName, + Func onMessage, + CancellationToken cancellationTokenForConnectionMultiplexer, + Func connectionInErrorProvider, ILog log) + { + Connection = connection; + this.channelName = channelName; + this.onMessage = onMessage; + ConnectionInErrorProvider = connectionInErrorProvider; + this.log = log; + this.staySubscribedCancellationTokenSource = CancellationTokenSource.CreateLinkedTokenSource(cancellationTokenForConnectionMultiplexer); + } + + public async Task StartSubscribe() + { + log?.Write(EventType.Diagnostic, $"Starting resilient subscription to channel: {channelName}"); + await Task.CompletedTask; + var _ = Task.Run(async () => + { + await Try.IgnoringError(async () => await KeepResubscribingShouldConnectionFail()); + }); + } + async Task KeepResubscribingShouldConnectionFail() + { + var staySubscribedCancellationToken = staySubscribedCancellationTokenSource.Token; + while (!staySubscribedCancellationToken.IsCancellationRequested) + { + try + { + var (connectionInError, channelMessageQueue) = await subscribeToChannel(staySubscribedCancellationToken); + await using var _ = new FuncAsyncDisposable(() => Try.IgnoringError(() => channelMessageQueue.UnsubscribeAsync())); + if(staySubscribedCancellationTokenSource.IsCancellationRequested) return; + log?.Write(EventType.Diagnostic, $"Waiting for connection error on channel: {channelName}"); + + // Now wait for a connection error to occur since we started to subscribe. + await connectionInError.CompletesWhenAConnectionErrorOccurs.WaitAsync(staySubscribedCancellationToken); + log?.Write(EventType.Diagnostic, $"Connection error detected on channel: {channelName}, resubscribing"); + } + catch (Exception ex) + { + log?.Write(EventType.Error, $"Error in subscription loop for channel {channelName}: {ex.Message}"); + } + } + } + + async Task<(RedisFacade.ConnectionInErrorHelper connectionInError, ChannelMessageQueue channelMessageQueue)> + subscribeToChannel(CancellationToken cancellationToken) + { + while (true) + { + cancellationToken.ThrowIfCancellationRequested(); + try + { + var connectionInError = ConnectionInErrorProvider(); + + if (!connectionInError.IsConnectionInError && Connection.GetSubscriber().IsConnected()) + { + log?.Write(EventType.Diagnostic, $"Successfully subscribing to channel: {channelName}"); + var channelMessageQueue = await Connection.GetSubscriber() + .SubscribeAsync(new RedisChannel(channelName, RedisChannel.PatternMode.Literal)); + channelMessageQueue.OnMessage(onMessage); + log?.Write(EventType.Diagnostic, $"Successfully subscribed to channel: {channelName}"); + return (connectionInError, channelMessageQueue); + } + else + { + log?.Write(EventType.Diagnostic, $"Connection not ready for channel {channelName}, waiting 5 seconds before retry"); + await Task.Delay(5000, cancellationToken); + } + } + catch (Exception ex) + { + log?.Write(EventType.Error, $"Error subscribing to channel {channelName}: {ex.Message}, retrying in 5 seconds"); + await Task.Delay(5000, cancellationToken); + } + } + } + + public async ValueTask DisposeAsync() + { + log?.Write(EventType.Diagnostic, $"Disposing resilient subscriber for channel: {channelName}"); + await Try.IgnoringError(async () => await staySubscribedCancellationTokenSource.CancelAsync()); + Try.IgnoringError(() => staySubscribedCancellationTokenSource.Dispose()); + } + } +} \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs b/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs index 6c4a13a41..c2d7769a5 100644 --- a/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs +++ b/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs @@ -57,13 +57,18 @@ async Task WatchForExceptions(Uri endpoint, Guid requestActivityId, HalibutRedis await watchForCancellationTokenSource.CancelAsync(); }, token); - if (await halibutRedisTransport.IsRequestMarkedAsCancelled(endpoint, requestActivityId, token)) + // Also poll to see if the request is cancelled since we can miss + // the publication. + while (!token.IsCancellationRequested) { - await requestCancelledCts.CancelAsync(); - await watchForCancellationTokenSource.CancelAsync(); + // TODO: What happens if this throws? + if (await halibutRedisTransport.IsRequestMarkedAsCancelled(endpoint, requestActivityId, token)) + { + await requestCancelledCts.CancelAsync(); + await watchForCancellationTokenSource.CancelAsync(); + } + await Task.Delay(TimeSpan.FromMinutes(1), token); } - - await Task.Delay(Timeout.Infinite, token); } catch { From 0e0461de8e79d7eb037a25b0154db6d942ad9187 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Thu, 31 Jul 2025 12:05:44 +1000 Subject: [PATCH 010/137] Ok this is better --- .../RedisFacadeWhenRedisGoesDownAwayTests.cs | 146 ++++++++++++++---- .../Queue/Redis/HalibutRedisTransport.cs | 18 ++- source/Halibut/Queue/Redis/RedisFacade.cs | 70 +++++---- .../Queue/Redis/RedisPendingRequestQueue.cs | 20 ++- .../Queue/Redis/ResilientSubscriber.cs | 120 -------------- .../Redis/WatchForRequestCancellation.cs | 28 ++-- 6 files changed, 190 insertions(+), 212 deletions(-) delete mode 100644 source/Halibut/Queue/Redis/ResilientSubscriber.cs diff --git a/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs b/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs index ed5727875..173d5ecc2 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs @@ -13,7 +13,9 @@ // limitations under the License. using System; +using System.Collections.Concurrent; using System.Collections.Generic; +using System.Diagnostics.CodeAnalysis; using System.Globalization; using System.Threading.Tasks; using FluentAssertions; @@ -21,12 +23,14 @@ using Halibut.Logging; using Halibut.Queue.Redis; using Halibut.Tests.Support.Logging; +using Halibut.Util; using NUnit.Framework; using Octopus.TestPortForwarder; using StackExchange.Redis; namespace Halibut.Tests.Queue.Redis { + [SuppressMessage("Usage", "VSTHRD003:Avoid awaiting foreign Tasks")] public class RedisFacadeWhenRedisGoesDownAwayTests : BaseTest { private static RedisFacade CreateRedisFacade(int port) => new("localhost:" + port, Guid.NewGuid().ToString(), new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix("")); @@ -35,14 +39,11 @@ public class RedisFacadeWhenRedisGoesDownAwayTests : BaseTest int redisPort = 6379; [Test] - public async Task WhenTheConnectionToRedisBrieflyGoesDown_FutureRequestsAShortTimeLaterCanBeExecuted() + public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_SometimeLaterOnWeCanDoBasicCalls() { using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); - var configurationOptions = new ConfigurationOptions(); - configurationOptions.EndPoints.Add("localhost:" + redisPort); - await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); await redisFacade.SetString("foo", "bar"); @@ -53,40 +54,117 @@ public async Task WhenTheConnectionToRedisBrieflyGoesDown_FutureRequestsAShortTi portForwarder.ReturnToNormalMode(); // After a short delay it does seem to work again. - await Task.Delay(100); + await Task.Delay(1000); await redisFacade.GetString("foo"); + } + + [Test] + public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelyDoBasicCalls() + { + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); + await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); - await Task.CompletedTask; + await redisFacade.SetString("foo", "bar"); + + (await redisFacade.GetString("foo")).Should().Be("bar"); + + portForwarder.EnterKillNewAndExistingConnectionsMode(); + portForwarder.ReturnToNormalMode(); + + // No delay here + + await redisFacade.GetString("foo"); } [Test] - public async Task WhenSubWhenRedisCanNotBeReached() + public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndWeTryToSubscribe_WhenTheConnectionIsRestored_WeCanReceiveMessages() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); + var redisLogCreator = new TestContextLogCreator("Redis", LogLevel.Trace); + var guid = Guid.NewGuid().ToString(); + await using var redisViaPortForwarder = new RedisFacade("localhost:" + portForwarder.ListeningPort, guid, redisLogCreator.CreateNewForPrefix("Unstable")); + await using var redisStableConnection = new RedisFacade("localhost:" + redisPort, guid, redisLogCreator.CreateNewForPrefix("Stable")); - var configurationOptions = new ConfigurationOptions(); - configurationOptions.EndPoints.Add("localhost:" + portForwarder.ListeningPort); - configurationOptions.AbortOnConnectFail = false; + await redisViaPortForwarder.SetString("Establish connection", "before we subscribe"); + + portForwarder.EnterKillNewAndExistingConnectionsMode(); + + var msgs = new ConcurrentBag(); + var subscribeToChannelTask = redisViaPortForwarder.SubscribeToChannel("bob", async message => + { + await Task.CompletedTask; + msgs.Add(message.Message!); + }, CancellationToken); + - var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix("Unstable"); - log.Write(EventType.Diagnostic, "Hello from log"); + // Give everything enough time to have a crack at trying to subscribe to messages. + await Task.Delay(2000); + await redisStableConnection.PublishToChannel("bob", "MISSED"); - await using var redisViaPortForwarder = new RedisFacade(configurationOptions, Guid.NewGuid().ToString(), log); + // Just in case the subscriber reconnects faster than the publish call. + await Task.Delay(2000); - portForwarder.EnterKillNewAndExistingConnectionsMode(); + portForwarder.ReturnToNormalMode(); + // Keep going around the loop until we recieve something + while (msgs.Count == 0) + { + Logger.Information("Trying again"); + await redisStableConnection.PublishToChannel("bob", "RECONNECT"); + await Task.Delay(1000); + } - await using var channel = await redisViaPortForwarder.SubscribeToChannel("bob", async message => + msgs.Should().Contain("RECONNECT", "Since the subscriber should eventually connect up"); + msgs.Should().NotContain("MISSED", "Since this was sent when the subscriber could not have been connected. " + + "If this is seen maybe the test itself has a bug."); + } + + [Test] + public async Task WhenTheConnectionIsNeverEstablished_AndWeTryToSubscribe_WhenTheConnectionIsRestored_WeCanReceiveMessages() + { + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); + var redisLogCreator = new TestContextLogCreator("Redis", LogLevel.Trace); + var guid = Guid.NewGuid().ToString(); + await using var redisViaPortForwarder = new RedisFacade("localhost:" + portForwarder.ListeningPort, guid, redisLogCreator.CreateNewForPrefix("Unstable")); + await using var redisStableConnection = new RedisFacade("localhost:" + redisPort, guid, redisLogCreator.CreateNewForPrefix("Stable")); + + portForwarder.EnterKillNewAndExistingConnectionsMode(); + + var msgs = new ConcurrentBag(); + var subscribeToChannelTask = redisViaPortForwarder.SubscribeToChannel("bob", async message => { await Task.CompletedTask; - }); + msgs.Add(message.Message!); + }, CancellationToken); + + await using var _ = new FuncAsyncDisposable(() => Try.IgnoringError(async () => await (await subscribeToChannelTask).DisposeAsync())); + + // Give everything enough time to have a crack at trying to subscribe to messages. + await Task.Delay(2000); + await redisStableConnection.PublishToChannel("bob", "MISSED"); + + // Just in case the subscriber reconnects faster than the publish call. + await Task.Delay(2000); + + portForwarder.ReturnToNormalMode(); + + // Keep going around the loop until we recieve something + while (msgs.Count == 0) + { + Logger.Information("Trying again"); + await redisStableConnection.PublishToChannel("bob", "RECONNECT"); + await Task.Delay(1000); + } + + msgs.Should().Contain("RECONNECT", "Since the subscriber should eventually connect up"); + msgs.Should().NotContain("MISSED", "Since this was sent when the subscriber could not have been connected. " + + "If this is seen maybe the test itself has a bug."); } [Test] - public async Task WhenSubScribedToAChannelAndRedisConnectGoesAwayWhatHappens() + public async Task WhenSubscribedAndTheConnectionGoesDown_WhenTheConnectionIsRestored_MessagesCanEventuallyBeSentToTheSubscriberAgain() { using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); @@ -97,36 +175,36 @@ public async Task WhenSubScribedToAChannelAndRedisConnectGoesAwayWhatHappens() await using var redisStableConnection = new RedisFacade("localhost:" + redisPort, guid, redisLogCreator.CreateNewForPrefix("Stable")); - var msgs = new List(); - + var msgs = new ConcurrentBag(); await using var channel = await redisViaPortForwarder.SubscribeToChannel("bob", async message => { await Task.CompletedTask; msgs.Add(message.Message!); - }); - - await Task.Delay(1000); - await redisViaPortForwarder.PublishToChannel("bob", "hello"); + }, CancellationToken); + + // Check both sides can publish. + await redisViaPortForwarder.PublishToChannel("bob", "hello unstable"); await redisStableConnection.PublishToChannel("bob", "hello stable"); - msgs.Should().BeEquivalentTo("hello", "hello stable"); + await Task.Delay(1000); // TODO better + msgs.Should().BeEquivalentTo("hello unstable", "hello stable"); portForwarder.EnterKillNewAndExistingConnectionsMode(); - await Task.Delay(5000); + // The stable connection should still be able to publish to redis. + // But the subscriber on the unstable connection will not got the message. + await redisStableConnection.PublishToChannel("bob", "MISSED"); + await Task.Delay(1111); portForwarder.ReturnToNormalMode(); - while (msgs.Count <= 2) { Logger.Information("Trying again"); - await redisStableConnection.PublishToChannel("bob", "hello"); - await Task.Delay(5000); + await redisStableConnection.PublishToChannel("bob", "RECONNECT"); + await Task.Delay(1000); } - - - - - await Task.CompletedTask; + msgs.Should().Contain("RECONNECT", "Since the subscriber should eventually be re-connected"); + msgs.Should().NotContain("MISSED", "Since this was sent when the subscriber could not have been connected. " + + "If this is seen maybe the test itself has a bug."); } diff --git a/source/Halibut/Queue/Redis/HalibutRedisTransport.cs b/source/Halibut/Queue/Redis/HalibutRedisTransport.cs index ed8211cd0..e764a0ab2 100644 --- a/source/Halibut/Queue/Redis/HalibutRedisTransport.cs +++ b/source/Halibut/Queue/Redis/HalibutRedisTransport.cs @@ -40,14 +40,15 @@ static string RequestMessagesPulseChannelName(Uri endpoint) return $"{Namespace}::RequestMessagesPulseChannelName::{endpoint}"; } - public async Task SubscribeToRequestMessagePulseChannel(Uri endpoint, Action onRequestMessagePulse) + public async Task SubscribeToRequestMessagePulseChannel(Uri endpoint, Action onRequestMessagePulse, CancellationToken cancellationToken) { var channelName = RequestMessagesPulseChannelName(endpoint); return await facade.SubscribeToChannel(channelName, async message => { await Task.CompletedTask; onRequestMessagePulse(message); - }); + }, + cancellationToken); } public async Task PulseRequestPushedToEndpoint(Uri endpoint, CancellationToken cancellationToken) @@ -117,15 +118,16 @@ static string ResponseMessagesChannelName(Uri endpoint, Guid requestId) } public async Task SubScribeToResponses(Uri endpoint, Guid requestOfResponseToWaitFor, - Func onResponse, + Func onResponse, CancellationToken cancellationToken) { var channelName = ResponseMessagesChannelName(endpoint, requestOfResponseToWaitFor); return await facade.SubscribeToChannel(channelName, async foo => { string? response = foo.Message; - if (response is not null) await onResponse(response, cancellationToken); - }); + if (response is not null) await onResponse(response); + }, + cancellationToken); } public async Task PublishResponse(Uri endpoint, Guid requestId, string payload, CancellationToken cancellationToken) @@ -142,15 +144,15 @@ static string RequestCancelledChannel(Uri endpoint, Guid requestId) } public async Task SubscribeToRequestCancellation(Uri endpoint, Guid request, - Func onCancellationReceived, + Func onCancellationReceived, CancellationToken cancellationToken) { var channelName = RequestCancelledChannel(endpoint, request); return await facade.SubscribeToChannel(channelName, async foo => { string? response = foo.Message; - if (response is not null) await onCancellationReceived(cancellationToken); - }); + if (response is not null) await onCancellationReceived(); + }, cancellationToken); } public async Task PublishCancellation(Uri endpoint, Guid requestId, CancellationToken cancellationToken) diff --git a/source/Halibut/Queue/Redis/RedisFacade.cs b/source/Halibut/Queue/Redis/RedisFacade.cs index 8948e51d6..be136d306 100644 --- a/source/Halibut/Queue/Redis/RedisFacade.cs +++ b/source/Halibut/Queue/Redis/RedisFacade.cs @@ -78,6 +78,9 @@ public RedisFacade(ConfigurationOptions redisOptions, string? keyPrefix, ILog lo this.log = log; this.cts = new CancellationTokenSource(); this.facadeCancellationToken = cts.Token; + + // aka have more goes at connecting. + redisOptions.AbortOnConnectFail = false; connection = new Lazy(() => { @@ -93,40 +96,38 @@ public RedisFacade(ConfigurationOptions redisOptions, string? keyPrefix, ILog lo return multiplexer; }); } - - - public class ConnectionInErrorHelper + + public class ShouldAbandonAndReconnectHelper { readonly TaskCompletionSource connectionInError = new TaskCompletionSource(); public bool IsConnectionInError => connectionInError.Task.IsCompleted; - public void SetIsInError() => connectionInError.SetResult(); + public void SetReconnectionIsAdvised() => connectionInError.SetResult(); - public Task CompletesWhenAConnectionErrorOccurs => connectionInError.Task; + public Task WaitUntilShouldReSubscribeTask => connectionInError.Task; } - private ConnectionInErrorHelper AConnectionHasErroredOutSinceYouGotThis = new ConnectionInErrorHelper(); + private ShouldAbandonAndReconnectHelper ShouldAbandonAndReconnect = new ShouldAbandonAndReconnectHelper(); private readonly object errorOccuredLock = new object(); - private ConnectionInErrorHelper ConnectionInErrorHelperProvider() => AConnectionHasErroredOutSinceYouGotThis; + private ShouldAbandonAndReconnectHelper ConnectionInErrorHelperProvider() => ShouldAbandonAndReconnect; - private void RecordConnectionErrorHasOccured() + private void AdviseThatClientsShouldStartReconnecting() { lock (errorOccuredLock) { - var inErrorConnectionInError = this.AConnectionHasErroredOutSinceYouGotThis; - this.AConnectionHasErroredOutSinceYouGotThis = new (); - inErrorConnectionInError.SetIsInError(); + var shouldAbandonAndReconnect = this.ShouldAbandonAndReconnect; + this.ShouldAbandonAndReconnect = new (); + shouldAbandonAndReconnect.SetReconnectionIsAdvised(); } } private void OnConnectionFailed(object? sender, ConnectionFailedEventArgs e) { - RecordConnectionErrorHasOccured(); - + AdviseThatClientsShouldStartReconnecting(); + var message = $"Redis connection failed - EndPoint: {e.EndPoint}, Failure: {e.FailureType}, Exception: {e.Exception?.Message}"; - log?.Write(EventType.Error, message); } @@ -167,27 +168,31 @@ public async ValueTask DisposeAsync() - public async Task SubscribeToChannel(string channelName, Func onMessage) + public async Task SubscribeToChannel(string channelName, Func onMessage, CancellationToken cancellationToken) { channelName = "channel:" + keyPrefix + ":" + channelName; - // TODO ever call needs to respect the cancellation token - // var channel = await Connection.GetSubscriber() - // .SubscribeAsync(new RedisChannel(channelName, RedisChannel.PatternMode.Literal)); - // - // channel.OnMessage(onMessage); - - var resilientSubscriber = new ResilientSubscriber(this.Connection, - channelName, - onMessage, - facadeCancellationToken, - () => ConnectionInErrorHelperProvider(), - log - ); - - await resilientSubscriber.StartSubscribe(); - - return resilientSubscriber; + while (true) + { + cancellationToken.ThrowIfCancellationRequested(); + try + { + // This can throw if we are unable to connect to redis. + var channel = await Connection.GetSubscriber() + .SubscribeAsync(new RedisChannel(channelName, RedisChannel.PatternMode.Literal)); + + // Once we are connected to redis, it seems even if the connection to redis dies. + // The client will take care of re-connecting to redis. + channel.OnMessage(onMessage); + + return new FuncAsyncDisposable(async () => await channel.UnsubscribeAsync()); + } + catch + { + // TODO: Get AI to log. + await Try.IgnoringError(async () => await Task.Delay(2000, cancellationToken)); + } + } } public async Task PublishToChannel(string channelName, string payload) @@ -214,6 +219,7 @@ public async Task SetInHash(string key, string field, string payload) key = "hash:" + keyPrefix + ":" + key; var database = Connection.GetDatabase(); var value = await database.HashGetAsync(key, new RedisValue(field)); + // TODO: If we retry this is not idempotent. var res = await database.KeyDeleteAsync(key); if (!res) { diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index cd681ff00..6eb4b94a0 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -32,8 +32,12 @@ class RedisPendingRequestQueue : IPendingRequestQueue, IDisposable readonly HalibutTimeoutsAndLimits halibutTimeoutsAndLimits; readonly MessageReaderWriter messageReaderWriter; readonly AsyncManualResetEvent hasItemsForEndpoint = new(); + + readonly CancellationTokenSource queueCts = new (); + ConcurrentDictionary disposablesForInFlightRequests = new(); + readonly CancellationToken queueToken; - IAsyncDisposable PulseChannelSubDisposer { get; } + Task PulseChannelSubDisposer { get; } public RedisPendingRequestQueue( Uri endpoint, @@ -47,19 +51,21 @@ public RedisPendingRequestQueue( this.messageReaderWriter = messageReaderWriter; this.halibutRedisTransport = halibutRedisTransport; this.halibutTimeoutsAndLimits = halibutTimeoutsAndLimits; + this.queueToken = queueCts.Token; // TODO: can we unsub if no tentacle is asking for a work for an extended period of time? // and also NOT sub if the queue is being created to send work. // The advice is many channels with few subscribers is better than a single channel with many subscribers. // If we end up with too many channels, we could shared the channels based on modulo of the hash of the endpoint, // which means we might have only 1000 channels and num_tentacles/1000 subscribers to each channel. For 300K tentacles. - PulseChannelSubDisposer = this.halibutRedisTransport.SubscribeToRequestMessagePulseChannel(endpoint, _ => hasItemsForEndpoint.Set()) - .GetAwaiter().GetResult(); + PulseChannelSubDisposer = Task.Run(() => this.halibutRedisTransport.SubscribeToRequestMessagePulseChannel(endpoint, _ => hasItemsForEndpoint.Set(), queueToken)); } public async ValueTask DisposeAsync() { - await PulseChannelSubDisposer.DisposeAsync(); + await Try.IgnoringError(async () => await queueCts.CancelAsync()); + Try.IgnoringError(() => queueCts.Dispose()); + await Try.IgnoringError(async () => await (await PulseChannelSubDisposer).DisposeAsync()); } public async Task QueueAndWaitAsync(RequestMessage request, CancellationToken requestCancellationToken) @@ -113,9 +119,9 @@ async Task SubscribeToResponse(Guid activityId, Action onResponse, CancellationToken cancellationToken) { - return await halibutRedisTransport.SubScribeToResponses(endpoint, activityId, async (responseJson, ct) => + return await halibutRedisTransport.SubScribeToResponses(endpoint, activityId, async (responseJson) => { - var response = await messageReaderWriter.ReadResponse(responseJson, ct); + var response = await messageReaderWriter.ReadResponse(responseJson, cancellationToken); onResponse(response); }, cancellationToken); } @@ -123,8 +129,8 @@ async Task SubscribeToResponse(Guid activityId, public bool IsEmpty => throw new NotImplementedException(); public int Count => throw new NotImplementedException(); - ConcurrentDictionary disposablesForInFlightRequests = new(); + public async Task DequeueAsync(CancellationToken cancellationToken) { var pending = await DequeueNextAsync(); diff --git a/source/Halibut/Queue/Redis/ResilientSubscriber.cs b/source/Halibut/Queue/Redis/ResilientSubscriber.cs deleted file mode 100644 index cfdb7c9d6..000000000 --- a/source/Halibut/Queue/Redis/ResilientSubscriber.cs +++ /dev/null @@ -1,120 +0,0 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -using System; -using System.Threading; -using System.Threading.Tasks; -using Halibut.Diagnostics; -using Halibut.Util; -using StackExchange.Redis; - -namespace Halibut.Queue.Redis -{ - public class ResilientSubscriber : IAsyncDisposable - { - readonly ConnectionMultiplexer Connection; - readonly string channelName; - readonly Func onMessage; - readonly Func ConnectionInErrorProvider; - readonly ILog log; - - CancellationTokenSource staySubscribedCancellationTokenSource; - - public ResilientSubscriber(ConnectionMultiplexer connection, - string channelName, - Func onMessage, - CancellationToken cancellationTokenForConnectionMultiplexer, - Func connectionInErrorProvider, ILog log) - { - Connection = connection; - this.channelName = channelName; - this.onMessage = onMessage; - ConnectionInErrorProvider = connectionInErrorProvider; - this.log = log; - this.staySubscribedCancellationTokenSource = CancellationTokenSource.CreateLinkedTokenSource(cancellationTokenForConnectionMultiplexer); - } - - public async Task StartSubscribe() - { - log?.Write(EventType.Diagnostic, $"Starting resilient subscription to channel: {channelName}"); - await Task.CompletedTask; - var _ = Task.Run(async () => - { - await Try.IgnoringError(async () => await KeepResubscribingShouldConnectionFail()); - }); - } - async Task KeepResubscribingShouldConnectionFail() - { - var staySubscribedCancellationToken = staySubscribedCancellationTokenSource.Token; - while (!staySubscribedCancellationToken.IsCancellationRequested) - { - try - { - var (connectionInError, channelMessageQueue) = await subscribeToChannel(staySubscribedCancellationToken); - await using var _ = new FuncAsyncDisposable(() => Try.IgnoringError(() => channelMessageQueue.UnsubscribeAsync())); - if(staySubscribedCancellationTokenSource.IsCancellationRequested) return; - log?.Write(EventType.Diagnostic, $"Waiting for connection error on channel: {channelName}"); - - // Now wait for a connection error to occur since we started to subscribe. - await connectionInError.CompletesWhenAConnectionErrorOccurs.WaitAsync(staySubscribedCancellationToken); - log?.Write(EventType.Diagnostic, $"Connection error detected on channel: {channelName}, resubscribing"); - } - catch (Exception ex) - { - log?.Write(EventType.Error, $"Error in subscription loop for channel {channelName}: {ex.Message}"); - } - } - } - - async Task<(RedisFacade.ConnectionInErrorHelper connectionInError, ChannelMessageQueue channelMessageQueue)> - subscribeToChannel(CancellationToken cancellationToken) - { - while (true) - { - cancellationToken.ThrowIfCancellationRequested(); - try - { - var connectionInError = ConnectionInErrorProvider(); - - if (!connectionInError.IsConnectionInError && Connection.GetSubscriber().IsConnected()) - { - log?.Write(EventType.Diagnostic, $"Successfully subscribing to channel: {channelName}"); - var channelMessageQueue = await Connection.GetSubscriber() - .SubscribeAsync(new RedisChannel(channelName, RedisChannel.PatternMode.Literal)); - channelMessageQueue.OnMessage(onMessage); - log?.Write(EventType.Diagnostic, $"Successfully subscribed to channel: {channelName}"); - return (connectionInError, channelMessageQueue); - } - else - { - log?.Write(EventType.Diagnostic, $"Connection not ready for channel {channelName}, waiting 5 seconds before retry"); - await Task.Delay(5000, cancellationToken); - } - } - catch (Exception ex) - { - log?.Write(EventType.Error, $"Error subscribing to channel {channelName}: {ex.Message}, retrying in 5 seconds"); - await Task.Delay(5000, cancellationToken); - } - } - } - - public async ValueTask DisposeAsync() - { - log?.Write(EventType.Diagnostic, $"Disposing resilient subscriber for channel: {channelName}"); - await Try.IgnoringError(async () => await staySubscribedCancellationTokenSource.CancelAsync()); - Try.IgnoringError(() => staySubscribedCancellationTokenSource.Dispose()); - } - } -} \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs b/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs index c2d7769a5..088d4cca9 100644 --- a/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs +++ b/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs @@ -38,12 +38,10 @@ public static async Task TrySendCancellation( readonly CancellationTokenSource watchForCancellationTokenSource = new(); - Task watchTask; - public WatchForRequestCancellation(Uri endpoint, Guid requestActivityId, HalibutRedisTransport halibutRedisTransport) { var token = watchForCancellationTokenSource.Token; - watchTask = Task.Run(async () => await WatchForExceptions(endpoint, requestActivityId, halibutRedisTransport, token)); + var _ = Task.Run(async () => await WatchForExceptions(endpoint, requestActivityId, halibutRedisTransport, token)); } async Task WatchForExceptions(Uri endpoint, Guid requestActivityId, HalibutRedisTransport halibutRedisTransport, CancellationToken token) @@ -51,23 +49,31 @@ async Task WatchForExceptions(Uri endpoint, Guid requestActivityId, HalibutRedis try { await using var _ = await halibutRedisTransport.SubscribeToRequestCancellation(endpoint, requestActivityId, - async _ => + async () => { - await requestCancelledCts.CancelAsync(); - await watchForCancellationTokenSource.CancelAsync(); + await Task.CompletedTask; + //await requestCancelledCts.CancelAsync(); + //await watchForCancellationTokenSource.CancelAsync(); }, token); + // Also poll to see if the request is cancelled since we can miss // the publication. while (!token.IsCancellationRequested) { - // TODO: What happens if this throws? - if (await halibutRedisTransport.IsRequestMarkedAsCancelled(endpoint, requestActivityId, token)) + try + { + if (await halibutRedisTransport.IsRequestMarkedAsCancelled(endpoint, requestActivityId, token)) + { + await requestCancelledCts.CancelAsync(); + await watchForCancellationTokenSource.CancelAsync(); + } + } + catch { - await requestCancelledCts.CancelAsync(); - await watchForCancellationTokenSource.CancelAsync(); + // TODO: Ask the ai to log. } - await Task.Delay(TimeSpan.FromMinutes(1), token); + await Task.Delay(TimeSpan.FromSeconds(60), token); } } catch From a245561b843705161f7e7ef3682a8b94a4e4dd72 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Sun, 3 Aug 2025 21:01:44 +1000 Subject: [PATCH 011/137] Add simple retries --- .../RedisFacadeWhenRedisGoesDownAwayTests.cs | 122 +++++++++++++++ source/Halibut/Queue/Redis/RedisFacade.cs | 140 +++++++++++++++--- .../Queue/Redis/RedisPendingRequestQueue.cs | 2 + 3 files changed, 241 insertions(+), 23 deletions(-) diff --git a/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs b/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs index 173d5ecc2..8727f6690 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs @@ -78,6 +78,128 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe await redisFacade.GetString("foo"); } + [Test] + public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelyPublishToChannel() + { + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); + + await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); + + // Establish connection first + await redisFacade.SetString("connection", "established"); + + portForwarder.EnterKillNewAndExistingConnectionsMode(); + portForwarder.ReturnToNormalMode(); + + // No delay here - should retry and succeed + await redisFacade.PublishToChannel("test-channel", "test-message"); + } + + [Test] + public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelySetInHash() + { + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); + + await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); + + // Establish connection first + await redisFacade.SetString("connection", "established"); + + portForwarder.EnterKillNewAndExistingConnectionsMode(); + portForwarder.ReturnToNormalMode(); + + // No delay here - should retry and succeed + await redisFacade.SetInHash("test-hash", "test-field", "test-value"); + } + + [Test] + public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelyTryGetAndDeleteFromHash() + { + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); + + await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); + + // Establish connection and set up test data + await redisFacade.SetInHash("test-hash", "test-field", "test-value"); + + portForwarder.EnterKillNewAndExistingConnectionsMode(); + portForwarder.ReturnToNormalMode(); + + // No delay here - should retry and succeed + var result = await redisFacade.TryGetAndDeleteFromHash("test-hash", "test-field"); + result.Should().Be("test-value"); + } + + [Test] + public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelyListRightPush() + { + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); + + await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); + + // Establish connection first + await redisFacade.SetString("connection", "established"); + + portForwarder.EnterKillNewAndExistingConnectionsMode(); + portForwarder.ReturnToNormalMode(); + + // No delay here - should retry and succeed + await redisFacade.ListRightPushAsync("test-list", "test-item"); + } + + [Test] + public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelyListLeftPop() + { + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); + + await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); + + // Establish connection and set up test data + await redisFacade.ListRightPushAsync("test-list", "test-item"); + + portForwarder.EnterKillNewAndExistingConnectionsMode(); + portForwarder.ReturnToNormalMode(); + + // No delay here - should retry and succeed + var result = await redisFacade.ListLeftPopAsync("test-list"); + result.Should().Be("test-item"); + } + + [Test] + public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelySetString() + { + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); + + await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); + + // Establish connection first + await redisFacade.SetString("connection", "established"); + + portForwarder.EnterKillNewAndExistingConnectionsMode(); + portForwarder.ReturnToNormalMode(); + + // No delay here - should retry and succeed + await redisFacade.SetString("test-key", "test-value"); + } + + [Test] + public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelyGetString() + { + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); + + await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); + + // Establish connection and set up test data + await redisFacade.SetString("test-key", "test-value"); + + portForwarder.EnterKillNewAndExistingConnectionsMode(); + portForwarder.ReturnToNormalMode(); + + // No delay here - should retry and succeed + var result = await redisFacade.GetString("test-key"); + result.Should().Be("test-value"); + } + [Test] public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndWeTryToSubscribe_WhenTheConnectionIsRestored_WeCanReceiveMessages() { diff --git a/source/Halibut/Queue/Redis/RedisFacade.cs b/source/Halibut/Queue/Redis/RedisFacade.cs index be136d306..e7136a3f6 100644 --- a/source/Halibut/Queue/Redis/RedisFacade.cs +++ b/source/Halibut/Queue/Redis/RedisFacade.cs @@ -13,6 +13,7 @@ // limitations under the License. using System; +using System.Diagnostics; using System.Threading; using System.Threading.Tasks; using Halibut.Util; @@ -143,6 +144,63 @@ private void OnConnectionRestored(object? sender, ConnectionFailedEventArgs e) log?.Write(EventType.Diagnostic, message); } + /// + /// Executes an operation with retry logic. Retries for up to 12 seconds with 1-second intervals. + /// + private async Task ExecuteWithRetry(Func> operation, CancellationToken cancellationToken) + { + using var linkedTokenSource = CancellationTokenSource.CreateLinkedTokenSource(cancellationToken, facadeCancellationToken); + var combinedToken = linkedTokenSource.Token; + + var totalDuration = TimeSpan.FromSeconds(12); + var retryDelay = TimeSpan.FromSeconds(1); + var stopwatch = Stopwatch.StartNew(); + + while (true) + { + combinedToken.ThrowIfCancellationRequested(); + + try + { + return await operation(); + } + catch (Exception ex) when (stopwatch.Elapsed < totalDuration && !combinedToken.IsCancellationRequested) + { + log?.Write(EventType.Diagnostic, $"Redis operation failed, retrying in {retryDelay.TotalSeconds}s: {ex.Message}"); + await Task.Delay(retryDelay, combinedToken); + } + } + } + + /// + /// Executes an operation with retry logic. Retries for up to 12 seconds with 1-second intervals. + /// + private async Task ExecuteWithRetry(Func operation, CancellationToken cancellationToken) + { + using var linkedTokenSource = CancellationTokenSource.CreateLinkedTokenSource(cancellationToken, facadeCancellationToken); + var combinedToken = linkedTokenSource.Token; + + var totalDuration = TimeSpan.FromSeconds(12); + var retryDelay = TimeSpan.FromSeconds(1); + var stopwatch = Stopwatch.StartNew(); + + while (true) + { + combinedToken.ThrowIfCancellationRequested(); + + try + { + await operation(); + return; + } + catch (Exception ex) when (stopwatch.Elapsed < totalDuration && !combinedToken.IsCancellationRequested) + { + log?.Write(EventType.Diagnostic, $"Redis operation failed, retrying in {retryDelay.TotalSeconds}s: {ex.Message}"); + await Task.Delay(retryDelay, combinedToken); + } + } + } + public bool IsConnected => connection.IsValueCreated && Connection.IsConnected; public async ValueTask DisposeAsync() @@ -198,70 +256,106 @@ public async Task SubscribeToChannel(string channelName, Func< public async Task PublishToChannel(string channelName, string payload) { channelName = "channel:" + keyPrefix + ":" + channelName; - var subscriber = Connection.GetSubscriber(); - await subscriber.PublishAsync(new RedisChannel(channelName, RedisChannel.PatternMode.Literal), payload); + await ExecuteWithRetry(async () => + { + var subscriber = Connection.GetSubscriber(); + await subscriber.PublishAsync(new RedisChannel(channelName, RedisChannel.PatternMode.Literal), payload); + }, CancellationToken.None); } public async Task SetInHash(string key, string field, string payload) { key = "hash:" + keyPrefix + ":" + key; + // TODO: TTL // TODO ever call needs to respect the cancellation token var ttl = new TimeSpan(9, 9, 9); - var database = Connection.GetDatabase(); - await database.HashSetAsync(key, new RedisValue(field), new RedisValue(payload)); - await database.KeyExpireAsync(key, ttl); + + // Retry each operation independently + await ExecuteWithRetry(async () => + { + var database = Connection.GetDatabase(); + await database.HashSetAsync(key, new RedisValue(field), new RedisValue(payload)); + }, CancellationToken.None); + + await ExecuteWithRetry(async () => + { + var database = Connection.GetDatabase(); + await database.KeyExpireAsync(key, ttl); + }, CancellationToken.None); } public async Task TryGetAndDeleteFromHash(string key, string field) { - // TODO ever call needs to respect the cancellation token key = "hash:" + keyPrefix + ":" + key; - var database = Connection.GetDatabase(); - var value = await database.HashGetAsync(key, new RedisValue(field)); + + // Retry each operation independently + var value = await ExecuteWithRetry(async () => + { + var database = Connection.GetDatabase(); + return await database.HashGetAsync(key, new RedisValue(field)); + }, CancellationToken.None); + // TODO: If we retry this is not idempotent. - var res = await database.KeyDeleteAsync(key); + var res = await ExecuteWithRetry(async () => + { + var database = Connection.GetDatabase(); + return await database.KeyDeleteAsync(key); + }, CancellationToken.None); + if (!res) { // Someone else deleted this, so return nothing to make the get and delete appear to be atomic. return null; } - return value; + return (string?)value; } public async Task ListRightPushAsync(string key, string payload) { key = "list:" + keyPrefix + ":" + key; - var database = Connection.GetDatabase(); - // TODO can we set TTL on this? - await database.ListRightPushAsync(key, payload); + await ExecuteWithRetry(async () => + { + var database = Connection.GetDatabase(); + // TODO can we set TTL on this? + await database.ListRightPushAsync(key, payload); + }, CancellationToken.None); } public async Task ListLeftPopAsync(string key) { key = "list:" + keyPrefix + ":" + key; - var database = Connection.GetDatabase(); - var value = await database.ListLeftPopAsync(key); - if (value.IsNull) + return await ExecuteWithRetry(async () => { - return null; - } + var database = Connection.GetDatabase(); + var value = await database.ListLeftPopAsync(key); + if (value.IsNull) + { + return null; + } - return value; + return (string?)value; + }, CancellationToken.None); } public async Task SetString(string key, string value) { key = "string:" + keyPrefix + ":" + key; - var database = Connection.GetDatabase(); - await database.StringSetAsync(key, value); + await ExecuteWithRetry(async () => + { + var database = Connection.GetDatabase(); + await database.StringSetAsync(key, value); + }, CancellationToken.None); } public async Task GetString(string key) { key = "string:" + keyPrefix + ":" + key; - var database = Connection.GetDatabase(); - return await database.StringGetAsync(key); + return await ExecuteWithRetry(async () => + { + var database = Connection.GetDatabase(); + return await database.StringGetAsync(key); + }, CancellationToken.None); } } diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index 6eb4b94a0..4cce89e04 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -180,6 +180,8 @@ await Task.WhenAny( return null; } + // TODO: Does this work well for multiple clients? We might go round before we collect work. + // TODO: test this. hasItemsForEndpoint.Reset(); return await TryRemoveNextItemFromQueue(cancellationToken); } From 6cdec4ffbc295868213b2e0f9bf4f4a03d6dbe62 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Sun, 3 Aug 2025 21:34:10 +1000 Subject: [PATCH 012/137] Show that the reciever can re-connect itself --- .../Redis/RedisPendingRequestQueueFixture.cs | 63 ++++++++++++++++++- .../Queue/Redis/RedisPendingRequestQueue.cs | 2 +- 2 files changed, 61 insertions(+), 4 deletions(-) diff --git a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs index 31881daa9..80ebc22d5 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs @@ -26,6 +26,7 @@ using Nito.AsyncEx; using NSubstitute; using NUnit.Framework; +using Octopus.TestPortForwarder; using Serilog; using DisposableCollection = Halibut.Util.DisposableCollection; using ILog = Halibut.Diagnostics.ILog; @@ -34,7 +35,8 @@ namespace Halibut.Tests.Queue.Redis { public class RedisPendingRequestQueueFixture : BaseTest { - private static RedisFacade CreateRedisFacade() => new("localhost", Guid.NewGuid().ToString(), new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix("")); + const int redisPort = 6379; + private static RedisFacade CreateRedisFacade(int? port = redisPort, Guid? guid = null) => new("localhost:" + port, (guid ?? Guid.NewGuid()).ToString(), new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix("")); [Test] public async Task DequeueAsync_ShouldReturnRequestFromRedis() @@ -213,10 +215,65 @@ public async Task FullSendAndReceiveWithDataStreamShouldWork() (await returnObject.Payload1!.ReadAsString(CancellationToken)).Should().Be("good"); (await returnObject.Payload2!.ReadAsString(CancellationToken)).Should().Be("bye"); } + + // TODO when receiver (dequeue) can not connect to redis it should not throw an exception. + // Or should it to try to encourage a new TCP connection which could go to a different node. + + [Test] + public async Task WhenTheReceiverConnectionToRedisIsInterruptedAndRestoredBeforeWorkIsPublished_TheRecieverShouldBeAbleToCollectThatWorkQuickly() + { + // Arrange + var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); + var log = Substitute.For(); + var guid = Guid.NewGuid(); + await using var redisFacadeSender = CreateRedisFacade(guid: guid); + + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); + await using var redisFacadeReceiver = CreateRedisFacade(portForwarder.ListeningPort, guid: guid); + + var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); + var messageSerializer = new QueueMessageSerializerBuilder().Build(); + var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); + + var request = new RequestMessageBuilder("poll://test-endpoint").Build(); + + var halibutTimeoutAndLimits = new HalibutTimeoutsAndLimits(); + halibutTimeoutAndLimits.PollingQueueWaitTimeout = TimeSpan.FromDays(1); // We should not need to rely on the timeout working for very short disconnects. + + var node1Sender = new RedisPendingRequestQueue(endpoint, log, new HalibutRedisTransport(redisFacadeSender), messageReaderWriter, halibutTimeoutAndLimits); + var node2Receiver = new RedisPendingRequestQueue(endpoint, log, new HalibutRedisTransport(redisFacadeReceiver), messageReaderWriter, halibutTimeoutAndLimits); + var dequeueTask = node2Receiver.DequeueAsync(CancellationToken); + + await Task.Delay(5000, CancellationToken); // Allow some time for the receiver to subscribe to work. + dequeueTask.IsCompleted.Should().BeFalse("Dequeue should not have "); + + portForwarder.EnterKillNewAndExistingConnectionsMode(); + await Task.Delay(1000, CancellationToken); // The network outage continues! + + portForwarder.ReturnToNormalMode(); // The network outage gets all fixed up :D + Logger.Information("Network restored!"); + + // The receiver should be able to get itself back into a state where it can collect messages quickly, within this time. + await Task.Delay(TimeSpan.FromSeconds(30), CancellationToken); + + var queueAndWaitAsync = node1Sender.QueueAndWaitAsync(request, CancellationToken.None); + + // Surely it will be done in 25s, it should take less than 1s. + await Task.WhenAny(Task.Delay(TimeSpan.FromSeconds(20), CancellationToken), dequeueTask); + + dequeueTask.IsCompleted.Should().BeTrue("The queue did not app"); + + var requestReceived = await dequeueTask; + requestReceived.Should().NotBeNull(); + requestReceived!.RequestMessage.ActivityId.Should().Be(request.ActivityId); + } + + + [Test] [LatestClientAndLatestServiceTestCases(testNetworkConditions: false, testListening: false, testWebSocket: false)] - public async Task OctopusCanSendMessagesToTentacle_WithEchoService(ClientAndServiceTestCase clientAndServiceTestCase) + public async Task WhenUsingTheRedisQueue_ASimpleEchoServiceCanBeCalled(ClientAndServiceTestCase clientAndServiceTestCase) { var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); @@ -245,7 +302,7 @@ public async Task OctopusCanSendMessagesToTentacle_WithEchoService(ClientAndServ [Test] - public async Task CancellingARequestShouldResultInTheDequeuedResponseTokenBeingCanelled() + public async Task CancellingARequestShouldResultInTheDequeuedResponseTokenBeingCancelled() { // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index 4cce89e04..b745c0ad0 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -171,7 +171,7 @@ public async Task ApplyResponse(ResponseMessage response, Guid requestActivityId await Task.WhenAny( hasItemsForEndpoint.WaitAsync(cancellationTokenSource.Token), - Task.Delay(new HalibutTimeoutsAndLimits().PollingQueueWaitTimeout, cancellationTokenSource.Token)); + Task.Delay(halibutTimeoutsAndLimits.PollingQueueWaitTimeout, cancellationTokenSource.Token)); if (!hasItemsForEndpoint.IsSet) { From 760c315f1dcbd18655107aa9494fb65c8a322645 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Mon, 4 Aug 2025 09:36:29 +1000 Subject: [PATCH 013/137] Add failing test for dequeuer disconnect --- .../Redis/RedisPendingRequestQueueFixture.cs | 83 +++++++++++++++++++ .../Queue/Redis/RedisPendingRequestQueue.cs | 5 +- .../Redis/WatchForRequestCancellation.cs | 61 +++++++++++--- 3 files changed, 137 insertions(+), 12 deletions(-) diff --git a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs index 80ebc22d5..4a81e2402 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs @@ -269,6 +269,89 @@ public async Task WhenTheReceiverConnectionToRedisIsInterruptedAndRestoredBefore requestReceived!.RequestMessage.ActivityId.Should().Be(request.ActivityId); } + [Test] + public async Task WhenTheSendersConnectionToRedisIsBrieflyInterruptedWhileSendingWork_TheWorkIsStillSent() + { + // Arrange + var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); + var log = Substitute.For(); + var guid = Guid.NewGuid(); + await using var redisFacadeReceiver = CreateRedisFacade(guid: guid); + + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); + await using var redisFacadeSender = CreateRedisFacade(portForwarder.ListeningPort, guid: guid); + + var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); + var messageSerializer = new QueueMessageSerializerBuilder().Build(); + var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); + + var request = new RequestMessageBuilder("poll://test-endpoint").Build(); + + var halibutTimeoutAndLimits = new HalibutTimeoutsAndLimits(); + halibutTimeoutAndLimits.PollingQueueWaitTimeout = TimeSpan.FromDays(1); // We should not need to rely on the timeout working for very short disconnects. + + + var node1Sender = new RedisPendingRequestQueue(endpoint, log, new HalibutRedisTransport(redisFacadeSender), messageReaderWriter, halibutTimeoutAndLimits); + var node2Receiver = new RedisPendingRequestQueue(endpoint, log, new HalibutRedisTransport(redisFacadeSender), messageReaderWriter, halibutTimeoutAndLimits); + + portForwarder.EnterKillNewAndExistingConnectionsMode(); + + var networkRestoreTask = Task.Run(async () => + { + + await Task.Delay(TimeSpan.FromSeconds(3), CancellationToken); + portForwarder.ReturnToNormalMode(); + }); + + var queueAndWaitAsync = node1Sender.QueueAndWaitAsync(request, CancellationToken.None); + + var dequeuedRequest = await node2Receiver.DequeueAsync(CancellationToken); + + dequeuedRequest.Should().NotBeNull(); + dequeuedRequest!.RequestMessage.ActivityId.Should().Be(request.ActivityId); + } + + [Test] + public async Task WhenTheReceiverDequeuesWorkAndThenDisconnectsFromRedisForEver_TheSenderEventuallyTimesOut() + { + // Arrange + var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); + var log = Substitute.For(); + var guid = Guid.NewGuid(); + await using var redisFacadeSender = CreateRedisFacade(guid: guid); + + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); + await using var redisFacadeReceiver = CreateRedisFacade(portForwarder.ListeningPort, guid: guid); + + var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); + var messageSerializer = new QueueMessageSerializerBuilder().Build(); + var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); + + var request = new RequestMessageBuilder("poll://test-endpoint").Build(); + + var halibutTimeoutAndLimits = new HalibutTimeoutsAndLimits(); + halibutTimeoutAndLimits.PollingQueueWaitTimeout = TimeSpan.FromDays(1); // We should not need to rely on the timeout working for very short disconnects. + + + var node1Sender = new RedisPendingRequestQueue(endpoint, log, new HalibutRedisTransport(redisFacadeSender), messageReaderWriter, halibutTimeoutAndLimits); + var node2Receiver = new RedisPendingRequestQueue(endpoint, log, new HalibutRedisTransport(redisFacadeSender), messageReaderWriter, halibutTimeoutAndLimits); + + + var queueAndWaitTask = node1Sender.QueueAndWaitAsync(request, CancellationToken.None); + + var dequeuedRequest = await node2Receiver.DequeueAsync(CancellationToken); + + // Now disconnect the receiver from redis. + portForwarder.EnterKillNewAndExistingConnectionsMode(); + + await Task.WhenAny(Task.Delay(TimeSpan.FromMinutes(1), CancellationToken), queueAndWaitTask); + + queueAndWaitTask.IsCompleted.Should().BeTrue(); + + var response = await queueAndWaitTask; + response.Error.Should().NotBeNull(); + response.Error!.Message.Should().Contain("Timed out waiting for the collectors heart beat"); + } [Test] diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index b745c0ad0..530e039b7 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -86,7 +86,8 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can await using var tryClearRequestFromQueueWhenRequestIsCancelled = pending.PendingRequestCancellationToken.Register(async () => await TryClearRequestFromQueue(request, pending)); await using var trySendCancelWhenRequestIsCancelled - = pending.PendingRequestCancellationToken.Register(async () => await WatchForRequestCancellation.TrySendCancellation(halibutRedisTransport, endpoint, request)); + = pending.PendingRequestCancellationToken.Register(async () => + await WatchForRequestCancellation.TrySendCancellation(halibutRedisTransport, endpoint, request, log)); // Make the request available before we tell people it is available. await halibutRedisTransport.PutRequest(endpoint, request.ActivityId, payload, requestCancellationToken); @@ -136,7 +137,7 @@ async Task SubscribeToResponse(Guid activityId, var pending = await DequeueNextAsync(); if (pending == null) return null; - var watchForRequestCancellation = new WatchForRequestCancellation(endpoint, pending.ActivityId, halibutRedisTransport); + var watchForRequestCancellation = new WatchForRequestCancellation(endpoint, pending.ActivityId, halibutRedisTransport, log); disposablesForInFlightRequests[pending.ActivityId] = new DisposableCollection(watchForRequestCancellation); diff --git a/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs b/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs index 088d4cca9..3d5fb75ab 100644 --- a/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs +++ b/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs @@ -15,6 +15,7 @@ using System; using System.Threading; using System.Threading.Tasks; +using Halibut.Diagnostics; using Halibut.Transport.Protocol; using Halibut.Util; @@ -26,20 +27,47 @@ public class WatchForRequestCancellation : IAsyncDisposable public static async Task TrySendCancellation( HalibutRedisTransport halibutRedisTransport, Uri endpoint, - RequestMessage request) + RequestMessage request, + ILog log) { + log.Write(EventType.Diagnostic, $"Attempting to send cancellation for request - Endpoint: {endpoint}, ActivityId: {request.ActivityId}"); + using var cts = new CancellationTokenSource(); cts.CancelAfter(TimeSpan.FromMinutes(2)); // Best efforts. - await halibutRedisTransport.PublishCancellation(endpoint, request.ActivityId, cts.Token); - await halibutRedisTransport.MarkRequestAsCancelled(endpoint, request.ActivityId, cts.Token); + + try + { + log.Write(EventType.Diagnostic, $"Publishing cancellation notification - Endpoint: {endpoint}, ActivityId: {request.ActivityId}"); + await halibutRedisTransport.PublishCancellation(endpoint, request.ActivityId, cts.Token); + + log.Write(EventType.Diagnostic, $"Marking request as cancelled - Endpoint: {endpoint}, ActivityId: {request.ActivityId}"); + await halibutRedisTransport.MarkRequestAsCancelled(endpoint, request.ActivityId, cts.Token); + + log.Write(EventType.Diagnostic, $"Successfully sent cancellation for request - Endpoint: {endpoint}, ActivityId: {request.ActivityId}"); + } + catch (OperationCanceledException ex) + { + log.Write(EventType.Error, $"Cancellation send operation timed out after 2 minutes - Endpoint: {endpoint}, ActivityId: {request.ActivityId}, Error: {ex.Message}"); + throw; + } + catch (Exception ex) + { + log.Write(EventType.Error, $"Failed to send cancellation for request - Endpoint: {endpoint}, ActivityId: {request.ActivityId}, Error: {ex.Message}"); + throw; + } } readonly CancellationTokenSource requestCancelledCts = new(); public CancellationToken RequestCancelledCancellationToken => requestCancelledCts.Token; readonly CancellationTokenSource watchForCancellationTokenSource = new(); - public WatchForRequestCancellation(Uri endpoint, Guid requestActivityId, HalibutRedisTransport halibutRedisTransport) + readonly ILog log; + + public WatchForRequestCancellation(Uri endpoint, Guid requestActivityId, HalibutRedisTransport halibutRedisTransport, ILog log) { + this.log = log; + log.Write(EventType.Diagnostic, $"Starting to watch for request cancellation - Endpoint: {endpoint}, ActivityId: {requestActivityId}"); + var token = watchForCancellationTokenSource.Token; var _ = Task.Run(async () => await WatchForExceptions(endpoint, requestActivityId, halibutRedisTransport, token)); } @@ -48,15 +76,20 @@ async Task WatchForExceptions(Uri endpoint, Guid requestActivityId, HalibutRedis { try { + log.Write(EventType.Diagnostic, $"Subscribing to request cancellation notifications - Endpoint: {endpoint}, ActivityId: {requestActivityId}"); + await using var _ = await halibutRedisTransport.SubscribeToRequestCancellation(endpoint, requestActivityId, async () => { await Task.CompletedTask; - //await requestCancelledCts.CancelAsync(); - //await watchForCancellationTokenSource.CancelAsync(); + log.Write(EventType.Diagnostic, $"Received cancellation notification via subscription - Endpoint: {endpoint}, ActivityId: {requestActivityId}"); + await requestCancelledCts.CancelAsync(); + await watchForCancellationTokenSource.CancelAsync(); }, token); + log.Write(EventType.Diagnostic, $"Starting polling loop for request cancellation - Endpoint: {endpoint}, ActivityId: {requestActivityId}"); + // Also poll to see if the request is cancelled since we can miss // the publication. while (!token.IsCancellationRequested) @@ -65,29 +98,37 @@ async Task WatchForExceptions(Uri endpoint, Guid requestActivityId, HalibutRedis { if (await halibutRedisTransport.IsRequestMarkedAsCancelled(endpoint, requestActivityId, token)) { + log.Write(EventType.Diagnostic, $"Request cancellation detected via polling - Endpoint: {endpoint}, ActivityId: {requestActivityId}"); await requestCancelledCts.CancelAsync(); await watchForCancellationTokenSource.CancelAsync(); + break; } } - catch + catch (Exception ex) { - // TODO: Ask the ai to log. + log.Write(EventType.Diagnostic, $"Error while polling for request cancellation - Endpoint: {endpoint}, ActivityId: {requestActivityId}, Error: {ex.Message}"); } await Task.Delay(TimeSpan.FromSeconds(60), token); } + + log.Write(EventType.Diagnostic, $"Exiting watch loop for request cancellation - Endpoint: {endpoint}, ActivityId: {requestActivityId}"); } - catch + catch (Exception ex) { - // TODO log when we get an exception we don't expect. + log.Write(EventType.Error, $"Unexpected error in request cancellation watcher - Endpoint: {endpoint}, ActivityId: {requestActivityId}, Error: {ex.Message}"); } } public async ValueTask DisposeAsync() { + log.Write(EventType.Diagnostic, "Disposing WatchForRequestCancellation"); + await Try.IgnoringError(async () => await watchForCancellationTokenSource.CancelAsync()); Try.IgnoringError(() => watchForCancellationTokenSource.Dispose()); await Try.IgnoringError(async () => await requestCancelledCts.CancelAsync()); Try.IgnoringError(() => requestCancelledCts.Dispose()); + + log.Write(EventType.Diagnostic, "WatchForRequestCancellation disposed"); } } } \ No newline at end of file From 285a16193a3759efd065f2a31bb6fa0fe505b362 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Tue, 5 Aug 2025 09:44:11 +1000 Subject: [PATCH 014/137] If the node processing the request disconnects we can detect that now --- .../Builders/RequestMessageBuilder.cs | 3 +- .../RedisFacadeWhenRedisGoesDownAwayTests.cs | 18 ++ .../Redis/RedisPendingRequestQueueFixture.cs | 27 +-- .../Queue/Redis/HalibutRedisTransport.cs | 34 +++- .../Redis/ProcessingNodeHeartBeatSender.cs | 186 ++++++++++++++++++ source/Halibut/Queue/Redis/RedisFacade.cs | 10 + .../Queue/Redis/RedisPendingRequestQueue.cs | 64 ++++-- .../Redis/WatchForRequestCancellation.cs | 30 ++- source/Halibut/ServiceModel/PendingRequest.cs | 36 ++-- .../ServiceModel/PendingRequestQueueAsync.cs | 2 +- 10 files changed, 350 insertions(+), 60 deletions(-) create mode 100644 source/Halibut/Queue/Redis/ProcessingNodeHeartBeatSender.cs diff --git a/source/Halibut.Tests/Builders/RequestMessageBuilder.cs b/source/Halibut.Tests/Builders/RequestMessageBuilder.cs index d9f685696..9e3731738 100644 --- a/source/Halibut.Tests/Builders/RequestMessageBuilder.cs +++ b/source/Halibut.Tests/Builders/RequestMessageBuilder.cs @@ -25,7 +25,8 @@ public RequestMessage Build() var request = new RequestMessage { Id = Guid.NewGuid().ToString(), - Destination = serviceEndPoint + Destination = serviceEndPoint, + ActivityId = Guid.NewGuid(), }; diff --git a/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs b/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs index 8727f6690..2196b68fa 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs @@ -200,6 +200,24 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe result.Should().Be("test-value"); } + [Test] + public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelyHashContainsKey() + { + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); + + await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); + + // Establish connection and set up test data + await redisFacade.SetInHash("test-hash", "test-field", "test-value"); + + portForwarder.EnterKillNewAndExistingConnectionsMode(); + portForwarder.ReturnToNormalMode(); + + // No delay here - should retry and succeed + var exists = await redisFacade.HashContainsKey("test-hash", "test-field"); + exists.Should().BeTrue(); + } + [Test] public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndWeTryToSubscribe_WhenTheConnectionIsRestored_WeCanReceiveMessages() { diff --git a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs index 4a81e2402..62d2206f7 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs @@ -43,7 +43,7 @@ public async Task DequeueAsync_ShouldReturnRequestFromRedis() { // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid()); - var log = Substitute.For(); + var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); @@ -70,7 +70,7 @@ public async Task WhenThePickupTimeoutExpires_AnErrorsReturned_AndTheRequestCanN { // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid()); - var log = Substitute.For(); + var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); @@ -97,7 +97,7 @@ public async Task When100kTentaclesAreSubscribed_TheQueueStillWorks() { // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid()); - var log = Substitute.For(); + var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); @@ -151,7 +151,7 @@ public async Task FullSendAndReceiveShouldWork() { // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); - var log = Substitute.For(); + var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); @@ -184,7 +184,7 @@ public async Task FullSendAndReceiveWithDataStreamShouldWork() { // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); - var log = Substitute.For(); + var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); @@ -224,7 +224,7 @@ public async Task WhenTheReceiverConnectionToRedisIsInterruptedAndRestoredBefore { // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); - var log = Substitute.For(); + var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); var guid = Guid.NewGuid(); await using var redisFacadeSender = CreateRedisFacade(guid: guid); @@ -274,7 +274,7 @@ public async Task WhenTheSendersConnectionToRedisIsBrieflyInterruptedWhileSendin { // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); - var log = Substitute.For(); + var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); var guid = Guid.NewGuid(); await using var redisFacadeReceiver = CreateRedisFacade(guid: guid); @@ -316,7 +316,7 @@ public async Task WhenTheReceiverDequeuesWorkAndThenDisconnectsFromRedisForEver_ { // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); - var log = Substitute.For(); + var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); var guid = Guid.NewGuid(); await using var redisFacadeSender = CreateRedisFacade(guid: guid); @@ -328,13 +328,16 @@ public async Task WhenTheReceiverDequeuesWorkAndThenDisconnectsFromRedisForEver_ var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); var request = new RequestMessageBuilder("poll://test-endpoint").Build(); + + request.Destination.PollingRequestQueueTimeout = TimeSpan.FromDays(1); var halibutTimeoutAndLimits = new HalibutTimeoutsAndLimits(); + halibutTimeoutAndLimits.PollingRequestQueueTimeout = TimeSpan.FromDays(1); halibutTimeoutAndLimits.PollingQueueWaitTimeout = TimeSpan.FromDays(1); // We should not need to rely on the timeout working for very short disconnects. var node1Sender = new RedisPendingRequestQueue(endpoint, log, new HalibutRedisTransport(redisFacadeSender), messageReaderWriter, halibutTimeoutAndLimits); - var node2Receiver = new RedisPendingRequestQueue(endpoint, log, new HalibutRedisTransport(redisFacadeSender), messageReaderWriter, halibutTimeoutAndLimits); + var node2Receiver = new RedisPendingRequestQueue(endpoint, log, new HalibutRedisTransport(redisFacadeReceiver), messageReaderWriter, halibutTimeoutAndLimits); var queueAndWaitTask = node1Sender.QueueAndWaitAsync(request, CancellationToken.None); @@ -344,13 +347,13 @@ public async Task WhenTheReceiverDequeuesWorkAndThenDisconnectsFromRedisForEver_ // Now disconnect the receiver from redis. portForwarder.EnterKillNewAndExistingConnectionsMode(); - await Task.WhenAny(Task.Delay(TimeSpan.FromMinutes(1), CancellationToken), queueAndWaitTask); + await Task.WhenAny(Task.Delay(TimeSpan.FromMinutes(2), CancellationToken), queueAndWaitTask); queueAndWaitTask.IsCompleted.Should().BeTrue(); var response = await queueAndWaitTask; response.Error.Should().NotBeNull(); - response.Error!.Message.Should().Contain("Timed out waiting for the collectors heart beat"); + response.Error!.Message.Should().Contain("The node processing the request did not send a heartbeat for long enough, and so the node is now assumed to be offline."); } @@ -389,7 +392,7 @@ public async Task CancellingARequestShouldResultInTheDequeuedResponseTokenBeingC { // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); - var log = Substitute.For(); + var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); diff --git a/source/Halibut/Queue/Redis/HalibutRedisTransport.cs b/source/Halibut/Queue/Redis/HalibutRedisTransport.cs index e764a0ab2..2088facd0 100644 --- a/source/Halibut/Queue/Redis/HalibutRedisTransport.cs +++ b/source/Halibut/Queue/Redis/HalibutRedisTransport.cs @@ -110,6 +110,11 @@ public async Task PutRequest(Uri endpoint, Guid requestId, string payload, Cance return redisQueueItem.PayloadJson; } + public async Task IsRequestStillOnQueue(Uri endpoint, Guid requestId, CancellationToken cancellationToken) + { + var requestKey = RequestMessageKey(endpoint, requestId); + return await facade.HashContainsKey(requestKey, RequestField); + } // Response channel static string ResponseMessagesChannelName(Uri endpoint, Guid requestId) @@ -181,7 +186,34 @@ public async Task IsRequestMarkedAsCancelled(Uri endpoint, Guid requestId, var key = RequestCancelledMarkerKey(endpoint, requestId); return (await facade.GetString(key)) != null; } -} + + + // Node Processing the request heart beat channel + static string NodeProcessingTheRequestHeartBeatChannel(Uri endpoint, Guid requestId) + { + return $"{Namespace}::NodeProcessingTheRequestHeartBeatChannel::{endpoint}::{requestId}"; + } + + public async Task SubscribeToNodeProcessingTheRequestHeartBeatChannel( + Uri endpoint, + Guid request, + Func onHeartBeat, + CancellationToken cancellationToken) + { + var channelName = NodeProcessingTheRequestHeartBeatChannel(endpoint, request); + return await facade.SubscribeToChannel(channelName, async foo => + { + string? response = foo.Message; + if (response is not null) await onHeartBeat(); + }, cancellationToken); + } + + public async Task SendHeartBeatFromNodeProcessingTheRequest(Uri endpoint, Guid requestId, CancellationToken cancellationToken) + { + var channelName = NodeProcessingTheRequestHeartBeatChannel(endpoint, requestId); + await facade.PublishToChannel(channelName, "{}"); + } + } public class RedisHalibutQueueItem2 diff --git a/source/Halibut/Queue/Redis/ProcessingNodeHeartBeatSender.cs b/source/Halibut/Queue/Redis/ProcessingNodeHeartBeatSender.cs new file mode 100644 index 000000000..b8de9c781 --- /dev/null +++ b/source/Halibut/Queue/Redis/ProcessingNodeHeartBeatSender.cs @@ -0,0 +1,186 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; +using System.Threading; +using System.Threading.Tasks; +using Halibut.Diagnostics; +using Halibut.ServiceModel; +using Halibut.Transport.Protocol; +using Halibut.Util; + +namespace Halibut.Queue.Redis +{ + public class ProcessingNodeHeartBeatSender : IAsyncDisposable + { + public enum NodeProcessingRequestWatcherResult + { + ProcessingNodeIsLikelyDisconnected, + NoIssue // TODO: this name is so bad the reviewer will be forced to think of a better one. + } + + private readonly Uri endpoint; + private readonly Guid requestActivityId; + private readonly HalibutRedisTransport halibutRedisTransport; + private readonly CancellationTokenSource cancellationTokenSource; + private readonly ILog log; + + public ProcessingNodeHeartBeatSender(Uri endpoint, + Guid requestActivityId, + HalibutRedisTransport halibutRedisTransport, + ILog log) + { + this.endpoint = endpoint; + this.requestActivityId = requestActivityId; + this.halibutRedisTransport = halibutRedisTransport; + this.cancellationTokenSource = new CancellationTokenSource(); + this.log = log; + + log.Write(EventType.Diagnostic, "Starting ProcessingNodeHeartBeatSender for request {0} to endpoint {1}", requestActivityId, endpoint); + Task.Run(() => SendPulsesWhileProcessingRequest(cancellationTokenSource.Token)); + } + + private async Task SendPulsesWhileProcessingRequest(CancellationToken cancellationToken) + { + log.Write(EventType.Diagnostic, "Starting heartbeat pulse loop for request {0}", requestActivityId); + + TimeSpan delayBetweenPulse; + while (!cancellationToken.IsCancellationRequested) + { + try + { + await halibutRedisTransport.SendHeartBeatFromNodeProcessingTheRequest(endpoint, requestActivityId, cancellationToken); + delayBetweenPulse = TimeSpan.FromSeconds(15); + log.Write(EventType.Diagnostic, "Successfully sent heartbeat for request {0}, next pulse in {1} seconds", requestActivityId, delayBetweenPulse.TotalSeconds); + } + catch (Exception ex) + { + if(cancellationToken.IsCancellationRequested) + { + log.Write(EventType.Diagnostic, "Heartbeat pulse loop cancelled for request {0}", requestActivityId); + return; + } + // Panic send pulses. + delayBetweenPulse = TimeSpan.FromSeconds(7); + log.WriteException(EventType.Diagnostic, "Failed to send heartbeat for request {0}, switching to panic mode with {1} second intervals", ex, requestActivityId, delayBetweenPulse.TotalSeconds); + } + + await Try.IgnoringError(async () => await Task.Delay(delayBetweenPulse, cancellationToken)); + } + + log.Write(EventType.Diagnostic, "Heartbeat pulse loop ended for request {0}", requestActivityId); + } + + public static async Task WaitUntilNodeProcessingRequestFlatLines( + Uri endpoint, + RequestMessage request, + PendingRequest pending, + HalibutRedisTransport halibutRedisTransport, + ILog log, + CancellationToken watchCancellationToken) + { + log.Write(EventType.Diagnostic, "Starting to watch for processing node flatline for request {0} to endpoint {1}", request.ActivityId, endpoint); + + // Once the pending's CT has been cancelled we no longer care to keep observing + using var cts = CancellationTokenSource.CreateLinkedTokenSource(watchCancellationToken, pending.PendingRequestCancellationToken); + + DateTimeOffset? lastHeartBeat = null; + + try + { + await using var subscription = await halibutRedisTransport.SubscribeToNodeProcessingTheRequestHeartBeatChannel( + endpoint, + request.ActivityId, async () => + { + await Task.CompletedTask; + lastHeartBeat = DateTimeOffset.Now; + log.Write(EventType.Diagnostic, "Received heartbeat for request {0} from processing node", request.ActivityId); + }, cts.Token); + + await WaitForRequestToBeCollected(endpoint, request, pending, halibutRedisTransport, log, cts.Token); + + // When the request was collected is a good enough heart beat. + if (lastHeartBeat == null || lastHeartBeat.Value < DateTimeOffset.Now) + { + lastHeartBeat = DateTimeOffset.Now; + log.Write(EventType.Diagnostic, "Using request collection time as heartbeat for request {0}", request.ActivityId); + } + + while (!cts.Token.IsCancellationRequested) + { + // TODO: I am sure a fancy pants delay could be done here calculated from the now and last heart beat etc. + await Try.IgnoringError(async () => await Task.Delay(TimeSpan.FromSeconds(10), cts.Token)); + var timeSinceLastHeartBeat = DateTimeOffset.Now - lastHeartBeat.Value; + if (timeSinceLastHeartBeat > TimeSpan.FromSeconds(60)) + { + log.Write(EventType.Diagnostic, "Processing node appears disconnected for request {0}, last heartbeat was {1} seconds ago", request.ActivityId, timeSinceLastHeartBeat.TotalSeconds); + return NodeProcessingRequestWatcherResult.ProcessingNodeIsLikelyDisconnected; + } + } + + log.Write(EventType.Diagnostic, "Processing node watcher cancelled for request {0}", request.ActivityId); + return NodeProcessingRequestWatcherResult.NoIssue; + } + catch (Exception ex) + { + log.WriteException(EventType.Diagnostic, "Error while watching processing node for request {0}", ex, request.ActivityId); + throw; + } + } + + static async Task WaitForRequestToBeCollected(Uri endpoint, RequestMessage request, PendingRequest pending, HalibutRedisTransport halibutRedisTransport, ILog log, CancellationToken cancellationToken) + { + log.Write(EventType.Diagnostic, "Waiting for request {0} to be collected from queue", request.ActivityId); + + while (!cancellationToken.IsCancellationRequested) + { + try + { + // This might only be set when the queue timeout is reached. + if(pending.HasRequestBeenMarkedAsCollected) + { + log.Write(EventType.Diagnostic, "Request {0} has been marked as collected", request.ActivityId); + return; + } + + // So check ourselves if the request has been collected. + var requestIsStillOnQueue = await halibutRedisTransport.IsRequestStillOnQueue(endpoint, request.ActivityId, cancellationToken); + if(!requestIsStillOnQueue) + { + log.Write(EventType.Diagnostic, "Request {0} is no longer on queue", request.ActivityId); + return; + } + } + catch (Exception ex) + { + log.WriteException(EventType.Diagnostic, "Error checking if request {0} is still on queue", ex, request.ActivityId); + } + + await Try.IgnoringError(async () => await Task.WhenAny(Task.Delay(TimeSpan.FromSeconds(30), cancellationToken), pending.WaitForRequestToBeMarkedAsCollected(cancellationToken))); + } + + log.Write(EventType.Diagnostic, "Stopped waiting for request {0} to be collected (cancelled)", request.ActivityId); + } + + public async ValueTask DisposeAsync() + { + log.Write(EventType.Diagnostic, "Disposing ProcessingNodeHeartBeatSender for request {0}", requestActivityId); + + await Try.IgnoringError(async () => await cancellationTokenSource.CancelAsync()); + Try.IgnoringError(() => cancellationTokenSource.Dispose()); + + log.Write(EventType.Diagnostic, "ProcessingNodeHeartBeatSender disposed for request {0}", requestActivityId); + } + } +} \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/RedisFacade.cs b/source/Halibut/Queue/Redis/RedisFacade.cs index e7136a3f6..aeeb2ed49 100644 --- a/source/Halibut/Queue/Redis/RedisFacade.cs +++ b/source/Halibut/Queue/Redis/RedisFacade.cs @@ -285,6 +285,16 @@ await ExecuteWithRetry(async () => }, CancellationToken.None); } + public async Task HashContainsKey(string key, string field) + { + key = "hash:" + keyPrefix + ":" + key; + return await ExecuteWithRetry(async () => + { + var database = Connection.GetDatabase(); + return await database.HashExistsAsync(key, new RedisValue(field)); + }, CancellationToken.None); + } + public async Task TryGetAndDeleteFromHash(string key, string field) { key = "hash:" + keyPrefix + ":" + key; diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index 530e039b7..30e18f6bf 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -70,6 +70,8 @@ public async ValueTask DisposeAsync() public async Task QueueAndWaitAsync(RequestMessage request, CancellationToken requestCancellationToken) { + using var cts = CancellationTokenSource.CreateLinkedTokenSource(queueCts.Token, requestCancellationToken); + var cancellationToken = cts.Token; // TODO: redis goes down // TODO: Other node goes down. // TODO: Respect cancellation token @@ -78,10 +80,11 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can // TODO: What if this payload was gigantic // TODO: Do we need to encrypt this? - var payload = await messageReaderWriter.PrepareRequest(request, requestCancellationToken); + var payload = await messageReaderWriter.PrepareRequest(request, cancellationToken); // Start listening for a response to the request, we don't want to miss the response. - await using var _ = await SubscribeToResponse(request.ActivityId, pending.SetResponse, requestCancellationToken); + // TODO: subscribing to the response is not reliable, we also need to account for missed publications. + await using var _ = await SubscribeToResponse(request.ActivityId, pending.SetResponse, cancellationToken); await using var tryClearRequestFromQueueWhenRequestIsCancelled = pending.PendingRequestCancellationToken.Register(async () => await TryClearRequestFromQueue(request, pending)); @@ -90,11 +93,24 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can await WatchForRequestCancellation.TrySendCancellation(halibutRedisTransport, endpoint, request, log)); // Make the request available before we tell people it is available. - await halibutRedisTransport.PutRequest(endpoint, request.ActivityId, payload, requestCancellationToken); - await halibutRedisTransport.PushRequestGuidOnToQueue(endpoint, request.ActivityId, requestCancellationToken); - await halibutRedisTransport.PulseRequestPushedToEndpoint(endpoint, requestCancellationToken); + await halibutRedisTransport.PutRequest(endpoint, request.ActivityId, payload, cancellationToken); + await halibutRedisTransport.PushRequestGuidOnToQueue(endpoint, request.ActivityId, cancellationToken); + await halibutRedisTransport.PulseRequestPushedToEndpoint(endpoint, cancellationToken); - await pending.WaitUntilComplete(() => TryClearRequestFromQueue(request, pending), requestCancellationToken); + await using var watcherCts = CancellationTokenSource.CreateLinkedTokenSource(cts.Token).CancelOnDispose(); + + var watchProcessingNodeTask = Task.Run(async () => + { + var disconnected = await ProcessingNodeHeartBeatSender.WaitUntilNodeProcessingRequestFlatLines(endpoint, request, pending, halibutRedisTransport, log, watcherCts.CancellationToken); + if (!watcherCts.CancellationToken.IsCancellationRequested && disconnected == ProcessingNodeHeartBeatSender.NodeProcessingRequestWatcherResult.ProcessingNodeIsLikelyDisconnected) + { + // TODO: if(responseWatcher.CheckForResponseNow() == ResponseNotFound) { + pending.SetResponse(ResponseMessage.FromError(request, "The node processing the request did not send a heartbeat for long enough, and so the node is now assumed to be offline.")); + //} + } + }); + + await pending.WaitUntilComplete(() => TryClearRequestFromQueue(request, pending), cancellationToken); return pending.Response!; } @@ -105,12 +121,19 @@ async Task TryClearRequestFromQueue(RequestMessage request, PendingRequest pendi // Let's try to pop if from the queue, either: // - We pop it, which means it was never collected so let pending deal with the timeout. // - We could not pop it, which means it was collected. - using var cts = new CancellationTokenSource(); - cts.CancelAfter(TimeSpan.FromMinutes(2)); // Best efforts. - var requestJson = await halibutRedisTransport.TryGetAndRemoveRequest(endpoint, request.ActivityId, cts.Token); - if (requestJson != null) + try + { + using var cts = new CancellationTokenSource(); + cts.CancelAfter(TimeSpan.FromMinutes(2)); // Best efforts. + var requestJson = await halibutRedisTransport.TryGetAndRemoveRequest(endpoint, request.ActivityId, cts.Token); + if (requestJson != null) + { + await pending.RequestHasBeenCollectedAndWillBeTransferred(); + } + } + catch (Exception) { - await pending.RequestHasBeenCollectedAndWillBeTransfered(); + // TODO log the exception here. } } @@ -137,11 +160,22 @@ async Task SubscribeToResponse(Guid activityId, var pending = await DequeueNextAsync(); if (pending == null) return null; - var watchForRequestCancellation = new WatchForRequestCancellation(endpoint, pending.ActivityId, halibutRedisTransport, log); - disposablesForInFlightRequests[pending.ActivityId] = new DisposableCollection(watchForRequestCancellation); - - return new RequestMessageWithCancellationToken(pending, watchForRequestCancellation.RequestCancelledCancellationToken); + var disposables = new DisposableCollection(); + try + { + var watchForRequestCancellation = new WatchForRequestCancellation(endpoint, pending.ActivityId, halibutRedisTransport, log); + disposables.AddAsyncDisposable(watchForRequestCancellation); + disposables.AddAsyncDisposable(new ProcessingNodeHeartBeatSender(endpoint, pending.ActivityId, halibutRedisTransport, log)); + var response = new RequestMessageWithCancellationToken(pending, watchForRequestCancellation.RequestCancelledCancellationToken); + disposablesForInFlightRequests[pending.ActivityId] = disposables; + return response; + } + catch (Exception) + { + await Try.IgnoringError(async () => await disposables.DisposeAsync()); + throw; + } } diff --git a/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs b/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs index 3d5fb75ab..cfb8a5838 100644 --- a/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs +++ b/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs @@ -30,30 +30,28 @@ public static async Task TrySendCancellation( RequestMessage request, ILog log) { - log.Write(EventType.Diagnostic, $"Attempting to send cancellation for request - Endpoint: {endpoint}, ActivityId: {request.ActivityId}"); + log.Write(EventType.Diagnostic, "Attempting to send cancellation for request - Endpoint: {0}, ActivityId: {1}", endpoint, request.ActivityId); using var cts = new CancellationTokenSource(); cts.CancelAfter(TimeSpan.FromMinutes(2)); // Best efforts. try { - log.Write(EventType.Diagnostic, $"Publishing cancellation notification - Endpoint: {endpoint}, ActivityId: {request.ActivityId}"); + log.Write(EventType.Diagnostic, "Publishing cancellation notification - Endpoint: {0}, ActivityId: {1}", endpoint, request.ActivityId); await halibutRedisTransport.PublishCancellation(endpoint, request.ActivityId, cts.Token); - log.Write(EventType.Diagnostic, $"Marking request as cancelled - Endpoint: {endpoint}, ActivityId: {request.ActivityId}"); + log.Write(EventType.Diagnostic, "Marking request as cancelled - Endpoint: {0}, ActivityId: {1}", endpoint, request.ActivityId); await halibutRedisTransport.MarkRequestAsCancelled(endpoint, request.ActivityId, cts.Token); - log.Write(EventType.Diagnostic, $"Successfully sent cancellation for request - Endpoint: {endpoint}, ActivityId: {request.ActivityId}"); + log.Write(EventType.Diagnostic, "Successfully sent cancellation for request - Endpoint: {0}, ActivityId: {1}", endpoint, request.ActivityId); } catch (OperationCanceledException ex) { - log.Write(EventType.Error, $"Cancellation send operation timed out after 2 minutes - Endpoint: {endpoint}, ActivityId: {request.ActivityId}, Error: {ex.Message}"); - throw; + log.Write(EventType.Error, "Cancellation send operation timed out after 2 minutes - Endpoint: {0}, ActivityId: {1}, Error: {2}", endpoint, request.ActivityId, ex.Message); } catch (Exception ex) { - log.Write(EventType.Error, $"Failed to send cancellation for request - Endpoint: {endpoint}, ActivityId: {request.ActivityId}, Error: {ex.Message}"); - throw; + log.Write(EventType.Error, "Failed to send cancellation for request - Endpoint: {0}, ActivityId: {1}, Error: {2}", endpoint, request.ActivityId, ex.Message); } } readonly CancellationTokenSource requestCancelledCts = new(); @@ -66,7 +64,7 @@ public static async Task TrySendCancellation( public WatchForRequestCancellation(Uri endpoint, Guid requestActivityId, HalibutRedisTransport halibutRedisTransport, ILog log) { this.log = log; - log.Write(EventType.Diagnostic, $"Starting to watch for request cancellation - Endpoint: {endpoint}, ActivityId: {requestActivityId}"); + log.Write(EventType.Diagnostic, "Starting to watch for request cancellation - Endpoint: {0}, ActivityId: {1}", endpoint, requestActivityId); var token = watchForCancellationTokenSource.Token; var _ = Task.Run(async () => await WatchForExceptions(endpoint, requestActivityId, halibutRedisTransport, token)); @@ -76,19 +74,19 @@ async Task WatchForExceptions(Uri endpoint, Guid requestActivityId, HalibutRedis { try { - log.Write(EventType.Diagnostic, $"Subscribing to request cancellation notifications - Endpoint: {endpoint}, ActivityId: {requestActivityId}"); + log.Write(EventType.Diagnostic, "Subscribing to request cancellation notifications - Endpoint: {0}, ActivityId: {1}", endpoint, requestActivityId); await using var _ = await halibutRedisTransport.SubscribeToRequestCancellation(endpoint, requestActivityId, async () => { await Task.CompletedTask; - log.Write(EventType.Diagnostic, $"Received cancellation notification via subscription - Endpoint: {endpoint}, ActivityId: {requestActivityId}"); + log.Write(EventType.Diagnostic, "Received cancellation notification via subscription - Endpoint: {0}, ActivityId: {1}", endpoint, requestActivityId); await requestCancelledCts.CancelAsync(); await watchForCancellationTokenSource.CancelAsync(); }, token); - log.Write(EventType.Diagnostic, $"Starting polling loop for request cancellation - Endpoint: {endpoint}, ActivityId: {requestActivityId}"); + log.Write(EventType.Diagnostic, "Starting polling loop for request cancellation - Endpoint: {0}, ActivityId: {1}", endpoint, requestActivityId); // Also poll to see if the request is cancelled since we can miss // the publication. @@ -98,7 +96,7 @@ async Task WatchForExceptions(Uri endpoint, Guid requestActivityId, HalibutRedis { if (await halibutRedisTransport.IsRequestMarkedAsCancelled(endpoint, requestActivityId, token)) { - log.Write(EventType.Diagnostic, $"Request cancellation detected via polling - Endpoint: {endpoint}, ActivityId: {requestActivityId}"); + log.Write(EventType.Diagnostic, "Request cancellation detected via polling - Endpoint: {0}, ActivityId: {1}", endpoint, requestActivityId); await requestCancelledCts.CancelAsync(); await watchForCancellationTokenSource.CancelAsync(); break; @@ -106,16 +104,16 @@ async Task WatchForExceptions(Uri endpoint, Guid requestActivityId, HalibutRedis } catch (Exception ex) { - log.Write(EventType.Diagnostic, $"Error while polling for request cancellation - Endpoint: {endpoint}, ActivityId: {requestActivityId}, Error: {ex.Message}"); + log.Write(EventType.Diagnostic, "Error while polling for request cancellation - Endpoint: {0}, ActivityId: {1}, Error: {2}", endpoint, requestActivityId, ex.Message); } await Task.Delay(TimeSpan.FromSeconds(60), token); } - log.Write(EventType.Diagnostic, $"Exiting watch loop for request cancellation - Endpoint: {endpoint}, ActivityId: {requestActivityId}"); + log.Write(EventType.Diagnostic, "Exiting watch loop for request cancellation - Endpoint: {0}, ActivityId: {1}", endpoint, requestActivityId); } catch (Exception ex) { - log.Write(EventType.Error, $"Unexpected error in request cancellation watcher - Endpoint: {endpoint}, ActivityId: {requestActivityId}, Error: {ex.Message}"); + log.Write(EventType.Error, "Unexpected error in request cancellation watcher - Endpoint: {0}, ActivityId: {1}, Error: {2}", endpoint, requestActivityId, ex.Message); } } diff --git a/source/Halibut/ServiceModel/PendingRequest.cs b/source/Halibut/ServiceModel/PendingRequest.cs index f8db96f48..def14dd53 100644 --- a/source/Halibut/ServiceModel/PendingRequest.cs +++ b/source/Halibut/ServiceModel/PendingRequest.cs @@ -23,14 +23,14 @@ namespace Halibut.ServiceModel { - - class PendingRequest : IDisposable + public class PendingRequest : IDisposable { readonly RequestMessage request; readonly ILog log; readonly AsyncManualResetEvent responseWaiter = new(false); readonly SemaphoreSlim transferLock = new(1, 1); - bool transferBegun; + //bool transferBegun; + AsyncManualResetEvent requestCollected = new(false); bool completed; readonly CancellationTokenSource pendingRequestCancellationTokenSource; ResponseMessage? response; @@ -44,6 +44,10 @@ public PendingRequest(RequestMessage request, ILog log) PendingRequestCancellationToken = pendingRequestCancellationTokenSource.Token; } + public Task WaitForRequestToBeMarkedAsCollected(CancellationToken cancellationToken) => requestCollected.WaitAsync(cancellationToken); + + public bool HasRequestBeenMarkedAsCollected => requestCollected.IsSet; + public RequestMessage Request => request; /// @@ -79,10 +83,10 @@ public async Task WaitUntilComplete(Func timePendingRequestCanBeOnTheQueue catch (RequestCancelledException) { cancelled = true; - if(!transferBegun) await timePendingRequestCanBeOnTheQueueHasElapsed(); + if(!requestCollected.IsSet) await timePendingRequestCanBeOnTheQueueHasElapsed(); using (await transferLock.LockAsync(CancellationToken.None)) { - if (!transferBegun) + if (!requestCollected.IsSet) { completed = true; log.Write(EventType.MessageExchange, "Request {0} was cancelled before it could be collected by the polling endpoint", request); @@ -91,11 +95,11 @@ public async Task WaitUntilComplete(Func timePendingRequestCanBeOnTheQueue } } - if(!transferBegun) await timePendingRequestCanBeOnTheQueueHasElapsed(); + if(!requestCollected.IsSet) await timePendingRequestCanBeOnTheQueueHasElapsed(); var waitForTransferToComplete = false; using (await transferLock.LockAsync(CancellationToken.None)) { - if (transferBegun) + if (requestCollected.IsSet) { waitForTransferToComplete = true; } @@ -171,12 +175,12 @@ async Task WaitForResponseToBeSet( { using (await transferLock.LockAsync(CancellationToken.None)) { - if (transferBegun && cancelTheRequestWhenTransferHasBegun) + if (requestCollected.IsSet && cancelTheRequestWhenTransferHasBegun) { // Cancel the dequeued request. This will cause co-operative cancellation on the thread dequeuing the request pendingRequestCancellationTokenSource.Cancel(); } - else if (!transferBegun) + else if (!requestCollected.IsSet) { // Cancel the queued request. This will flag the request as cancelled to stop it being dequeued pendingRequestCancellationTokenSource.Cancel(); @@ -187,14 +191,14 @@ async Task WaitForResponseToBeSet( return false; } - throw transferBegun ? new TransferringRequestCancelledException(ex) : new ConnectingRequestCancelledException(ex); + throw requestCollected.IsSet ? new TransferringRequestCancelledException(ex) : new ConnectingRequestCancelledException(ex); } } return true; } - public async Task RequestHasBeenCollectedAndWillBeTransfered() + public async Task RequestHasBeenCollectedAndWillBeTransferred() { // The PendingRequest is Disposed at the end of QueueAndWaitAsync but a race condition // exists in the current approach that means DequeueAsync could pick this request up after @@ -212,7 +216,7 @@ public async Task RequestHasBeenCollectedAndWillBeTransfered() return false; } - transferBegun = true; + requestCollected.Set(); return true; } } @@ -227,8 +231,12 @@ public async Task RequestHasBeenCollectedAndWillBeTransfered() public void SetResponse(ResponseMessage response) { - this.response = response; - responseWaiter.Set(); + lock (responseWaiter) + { + if(this.response != null) return; + this.response = response; + responseWaiter.Set(); + } } public void Dispose() diff --git a/source/Halibut/ServiceModel/PendingRequestQueueAsync.cs b/source/Halibut/ServiceModel/PendingRequestQueueAsync.cs index f106ea08e..9750699b2 100644 --- a/source/Halibut/ServiceModel/PendingRequestQueueAsync.cs +++ b/source/Halibut/ServiceModel/PendingRequestQueueAsync.cs @@ -108,7 +108,7 @@ public int Count return null; } - var result = await pending.RequestHasBeenCollectedAndWillBeTransfered(); + var result = await pending.RequestHasBeenCollectedAndWillBeTransferred(); if (result) { return new (pending.Request, pending.PendingRequestCancellationToken); From b507febf81b7c88299c96da48e96663e8f0a643d Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Tue, 5 Aug 2025 09:50:21 +1000 Subject: [PATCH 015/137] . --- .../Halibut/Queue/Redis/RedisPendingRequestQueue.cs | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index 30e18f6bf..c958f0b29 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -117,6 +117,8 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can async Task TryClearRequestFromQueue(RequestMessage request, PendingRequest pending) { + log.Write(EventType.Diagnostic, "Attempting to clear request {0} from queue for endpoint {1}", request.ActivityId, endpoint); + // The time the message is allowed to sit on the queue for has elapsed. // Let's try to pop if from the queue, either: // - We pop it, which means it was never collected so let pending deal with the timeout. @@ -128,12 +130,17 @@ async Task TryClearRequestFromQueue(RequestMessage request, PendingRequest pendi var requestJson = await halibutRedisTransport.TryGetAndRemoveRequest(endpoint, request.ActivityId, cts.Token); if (requestJson != null) { + log.Write(EventType.Diagnostic, "Successfully removed request {0} from queue - request was never collected by a processing node", request.ActivityId); await pending.RequestHasBeenCollectedAndWillBeTransferred(); } + else + { + log.Write(EventType.Diagnostic, "Request {0} was not found in queue - it was already collected by a processing node", request.ActivityId); + } } - catch (Exception) + catch (Exception ex) { - // TODO log the exception here. + log.WriteException(EventType.Error, "Failed to clear request {0} from queue for endpoint {1}", ex, request.ActivityId, endpoint); } } From 80a6d48039c3a2fb1a6f465f35d025f60ce33fdd Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Tue, 5 Aug 2025 12:16:59 +1000 Subject: [PATCH 016/137] The sender can breifly disconnect and still recieve the response --- .../Redis/RedisPendingRequestQueueFixture.cs | 44 +++++++++++ .../Queue/Redis/HalibutRedisTransport.cs | 75 ++++++++++++------- source/Halibut/Queue/Redis/RedisFacade.cs | 11 +++ .../Queue/Redis/RedisPendingRequestQueue.cs | 33 ++++++-- 4 files changed, 132 insertions(+), 31 deletions(-) diff --git a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs index 62d2206f7..ffa79cb93 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs @@ -329,6 +329,7 @@ public async Task WhenTheReceiverDequeuesWorkAndThenDisconnectsFromRedisForEver_ var request = new RequestMessageBuilder("poll://test-endpoint").Build(); + // TODO: This only needs to be set because we do not detect the work was collected as soon as it has been collected. request.Destination.PollingRequestQueueTimeout = TimeSpan.FromDays(1); var halibutTimeoutAndLimits = new HalibutTimeoutsAndLimits(); @@ -355,6 +356,49 @@ public async Task WhenTheReceiverDequeuesWorkAndThenDisconnectsFromRedisForEver_ response.Error.Should().NotBeNull(); response.Error!.Message.Should().Contain("The node processing the request did not send a heartbeat for long enough, and so the node is now assumed to be offline."); } + + [Test] + public async Task WhenTheSenderDisconnectsFromRedisRightWhenTheReceiverSendsTheResponseBack_TheSenderStillGetsTheResponse() + { + // Arrange + var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); + var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); + var guid = Guid.NewGuid(); + + var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); + var messageSerializer = new QueueMessageSerializerBuilder().Build(); + var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); + + await using var stableConnection = CreateRedisFacade(guid: guid); + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); + await using var unreliableConnection = CreateRedisFacade(portForwarder.ListeningPort, guid: guid); + + var node1Sender = new RedisPendingRequestQueue(endpoint, log, new HalibutRedisTransport(unreliableConnection), messageReaderWriter, new HalibutTimeoutsAndLimits()); + var node2Receiver = new RedisPendingRequestQueue(endpoint, log, new HalibutRedisTransport(stableConnection), messageReaderWriter, new HalibutTimeoutsAndLimits()); + + var request = new RequestMessageBuilder("poll://test-endpoint").Build(); + // TODO: This only needs to be set because we do not detect the work was collected as soon as it has been collected. + request.Destination.PollingRequestQueueTimeout = TimeSpan.FromDays(1); + var queueAndWaitTask = node1Sender.QueueAndWaitAsync(request, CancellationToken.None); + + var dequeuedRequest = await node2Receiver.DequeueAsync(CancellationToken); + + // Just before we send the response, disconnect the sender. + portForwarder.EnterKillNewAndExistingConnectionsMode(); + await node2Receiver.ApplyResponse(ResponseMessage.FromResult(dequeuedRequest!.RequestMessage, "Yay"), dequeuedRequest!.RequestMessage.ActivityId); + + await Task.Delay(TimeSpan.FromSeconds(2), CancellationToken); + portForwarder.ReturnToNormalMode(); + + await Task.WhenAny(Task.Delay(TimeSpan.FromMinutes(2), CancellationToken), queueAndWaitTask); + + queueAndWaitTask.IsCompleted.Should().BeTrue(); + + var response = await queueAndWaitTask; + response.Error.Should().BeNull(); + response.Result.Should().Be("Yay"); + + } [Test] diff --git a/source/Halibut/Queue/Redis/HalibutRedisTransport.cs b/source/Halibut/Queue/Redis/HalibutRedisTransport.cs index 2088facd0..1c561a0d9 100644 --- a/source/Halibut/Queue/Redis/HalibutRedisTransport.cs +++ b/source/Halibut/Queue/Redis/HalibutRedisTransport.cs @@ -116,32 +116,6 @@ public async Task IsRequestStillOnQueue(Uri endpoint, Guid requestId, Canc return await facade.HashContainsKey(requestKey, RequestField); } - // Response channel - static string ResponseMessagesChannelName(Uri endpoint, Guid requestId) - { - return $"{Namespace}::ResponseMessagesChannelName::{endpoint}::{requestId}"; - } - - public async Task SubScribeToResponses(Uri endpoint, Guid requestOfResponseToWaitFor, - Func onResponse, - CancellationToken cancellationToken) - { - var channelName = ResponseMessagesChannelName(endpoint, requestOfResponseToWaitFor); - return await facade.SubscribeToChannel(channelName, async foo => - { - string? response = foo.Message; - if (response is not null) await onResponse(response); - }, - cancellationToken); - } - - public async Task PublishResponse(Uri endpoint, Guid requestId, string payload, CancellationToken cancellationToken) - { - var channelName = ResponseMessagesChannelName(endpoint, requestId); - await facade.PublishToChannel(channelName, payload); - } - - // Cancellation channel static string RequestCancelledChannel(Uri endpoint, Guid requestId) { @@ -213,6 +187,55 @@ public async Task SendHeartBeatFromNodeProcessingTheRequest(Uri endpoint, Guid r var channelName = NodeProcessingTheRequestHeartBeatChannel(endpoint, requestId); await facade.PublishToChannel(channelName, "{}"); } + + // Generic methods for watching for any string value being set + + string GenericChannelName(string thingToWatchFor, Uri endpoint, Guid identifier) + { + return $"{Namespace}::GenericChannel::{thingToWatchFor}::{endpoint}::{identifier}"; + } + + public async Task SubscribeToGenericNotification(string thingToWatchFor, Uri endpoint, Guid identifier, + Func onValueReceived, + CancellationToken cancellationToken) + { + var channelName = GenericChannelName(thingToWatchFor, endpoint, identifier); + return await facade.SubscribeToChannel(channelName, async foo => + { + string? value = foo.Message; + if (value is not null) await onValueReceived(value); + }, cancellationToken); + } + + public async Task PublishThatValueIsAvailable(string thingToWatchFor, Uri endpoint, Guid identifier, string value, CancellationToken cancellationToken) + { + var channelName = GenericChannelName(thingToWatchFor, endpoint, identifier); + await facade.PublishToChannel(channelName, value); + } + + string GenericMarkerKey(string thingToWatchFor, Uri endpoint, Guid identifier) + { + return $"{Namespace}::GenericMarker::{thingToWatchFor}::{endpoint}::{identifier}"; + } + + public async Task SendValue(string thingToWatchFor, Uri endpoint, Guid identifier, string value, CancellationToken cancellationToken) + { + // TODO: really must have a TTL + var key = GenericMarkerKey(thingToWatchFor, endpoint, identifier); + await facade.SetString(key, value); + } + + public async Task GetGenericMarker(string thingToWatchFor, Uri endpoint, Guid identifier, CancellationToken cancellationToken) + { + var key = GenericMarkerKey(thingToWatchFor, endpoint, identifier); + return await facade.GetString(key); + } + + public async Task DeleteGenericMarker(string thingToWatchFor, Uri endpoint, Guid identifier, CancellationToken cancellationToken) + { + var key = GenericMarkerKey(thingToWatchFor, endpoint, identifier); + return await facade.DeleteString(key); + } } diff --git a/source/Halibut/Queue/Redis/RedisFacade.cs b/source/Halibut/Queue/Redis/RedisFacade.cs index aeeb2ed49..c92cf6441 100644 --- a/source/Halibut/Queue/Redis/RedisFacade.cs +++ b/source/Halibut/Queue/Redis/RedisFacade.cs @@ -350,6 +350,7 @@ await ExecuteWithRetry(async () => public async Task SetString(string key, string value) { + // TODO TTL key = "string:" + keyPrefix + ":" + key; await ExecuteWithRetry(async () => { @@ -367,6 +368,16 @@ await ExecuteWithRetry(async () => return await database.StringGetAsync(key); }, CancellationToken.None); } + + public async Task DeleteString(string key) + { + key = "string:" + keyPrefix + ":" + key; + return await ExecuteWithRetry(async () => + { + var database = Connection.GetDatabase(); + return await database.KeyDeleteAsync(key); + }, CancellationToken.None); + } } } \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index c958f0b29..1a607a0e6 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -14,6 +14,7 @@ using System; using System.Collections.Concurrent; +using System.Security.Cryptography; using System.Threading; using System.Threading.Tasks; using Halibut.Diagnostics; @@ -146,15 +147,37 @@ async Task TryClearRequestFromQueue(RequestMessage request, PendingRequest pendi + const string ResponseMessageSubscriptionName = "ResponseMessage"; + async Task SubscribeToResponse(Guid activityId, Action onResponse, CancellationToken cancellationToken) { - return await halibutRedisTransport.SubScribeToResponses(endpoint, activityId, async (responseJson) => + await Task.CompletedTask; + var sub = new PollAndSubscribeForSingleMessage(ResponseMessageSubscriptionName, endpoint, activityId, halibutRedisTransport, log); + var _ = Task.Run(async () => { - var response = await messageReaderWriter.ReadResponse(responseJson, cancellationToken); - onResponse(response); - }, cancellationToken); + try + { + var responseJson = await sub.ResultTask; + var response = await messageReaderWriter.ReadResponse(responseJson, cancellationToken); + onResponse(response); + } + catch (OperationCanceledException) + { + // TODO ignore + } + catch (Exception) + { + // TODO log + } + }); + return sub; + // return await halibutRedisTransport.SubScribeToResponses(endpoint, activityId, async (responseJson) => + // { + // var response = await messageReaderWriter.ReadResponse(responseJson, cancellationToken); + // onResponse(response); + // }, cancellationToken); } public bool IsEmpty => throw new NotImplementedException(); @@ -197,7 +220,7 @@ public async Task ApplyResponse(ResponseMessage response, Guid requestActivityId // back to the node which sent the response var payload = await messageReaderWriter.PrepareResponse(response, cancellationToken); - await halibutRedisTransport.PublishResponse(endpoint, requestActivityId, payload, cancellationToken); + await PollAndSubscribeForSingleMessage.TrySendMessage(ResponseMessageSubscriptionName, halibutRedisTransport, endpoint, requestActivityId, payload, log); } async Task DequeueNextAsync() From 7051ab23b2e0124198a7262825a60f4e33c103f9 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Tue, 5 Aug 2025 16:06:43 +1000 Subject: [PATCH 017/137] well well that was hard --- .../Redis/RedisPendingRequestQueueFixture.cs | 23 +- .../PendingRequestQueueFixture.cs | 2 + .../Redis/ProcessingNodeHeartBeatSender.cs | 4 +- source/Halibut/Queue/Redis/RedisFacade.cs | 9 +- .../Queue/Redis/RedisPendingRequestQueue.cs | 4 +- source/Halibut/ServiceModel/PendingRequest.cs | 6 +- .../Halibut/ServiceModel/PendingRequestV2.cs | 204 ++++++++++++++++++ 7 files changed, 230 insertions(+), 22 deletions(-) create mode 100644 source/Halibut/ServiceModel/PendingRequestV2.cs diff --git a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs index ffa79cb93..3eeeebca1 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs @@ -310,7 +310,7 @@ public async Task WhenTheSendersConnectionToRedisIsBrieflyInterruptedWhileSendin dequeuedRequest.Should().NotBeNull(); dequeuedRequest!.RequestMessage.ActivityId.Should().Be(request.ActivityId); } - + [Test] public async Task WhenTheReceiverDequeuesWorkAndThenDisconnectsFromRedisForEver_TheSenderEventuallyTimesOut() { @@ -318,29 +318,28 @@ public async Task WhenTheReceiverDequeuesWorkAndThenDisconnectsFromRedisForEver_ var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); var guid = Guid.NewGuid(); - await using var redisFacadeSender = CreateRedisFacade(guid: guid); - - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); - await using var redisFacadeReceiver = CreateRedisFacade(portForwarder.ListeningPort, guid: guid); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); - var request = new RequestMessageBuilder("poll://test-endpoint").Build(); - - // TODO: This only needs to be set because we do not detect the work was collected as soon as it has been collected. - request.Destination.PollingRequestQueueTimeout = TimeSpan.FromDays(1); - var halibutTimeoutAndLimits = new HalibutTimeoutsAndLimits(); halibutTimeoutAndLimits.PollingRequestQueueTimeout = TimeSpan.FromDays(1); halibutTimeoutAndLimits.PollingQueueWaitTimeout = TimeSpan.FromDays(1); // We should not need to rely on the timeout working for very short disconnects. + await using var stableRedisConnection = CreateRedisFacade(guid: guid); + + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); + await using var unstableRedisConnection = CreateRedisFacade(portForwarder.ListeningPort, guid: guid); - var node1Sender = new RedisPendingRequestQueue(endpoint, log, new HalibutRedisTransport(redisFacadeSender), messageReaderWriter, halibutTimeoutAndLimits); - var node2Receiver = new RedisPendingRequestQueue(endpoint, log, new HalibutRedisTransport(redisFacadeReceiver), messageReaderWriter, halibutTimeoutAndLimits); + var node1Sender = new RedisPendingRequestQueue(endpoint, log, new HalibutRedisTransport(stableRedisConnection), messageReaderWriter, halibutTimeoutAndLimits); + var node2Receiver = new RedisPendingRequestQueue(endpoint, log, new HalibutRedisTransport(unstableRedisConnection), messageReaderWriter, halibutTimeoutAndLimits); + var request = new RequestMessageBuilder("poll://test-endpoint").Build(); + + // TODO: Setting this low shows we don't timeout because the request was not picked up in time. + request.Destination.PollingRequestQueueTimeout = TimeSpan.FromSeconds(5); var queueAndWaitTask = node1Sender.QueueAndWaitAsync(request, CancellationToken.None); var dequeuedRequest = await node2Receiver.DequeueAsync(CancellationToken); diff --git a/source/Halibut.Tests/ServiceModel/PendingRequestQueueFixture.cs b/source/Halibut.Tests/ServiceModel/PendingRequestQueueFixture.cs index 0a886f012..86936736b 100644 --- a/source/Halibut.Tests/ServiceModel/PendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/ServiceModel/PendingRequestQueueFixture.cs @@ -418,11 +418,13 @@ public async Task QueueAndWait_CancellingAPendingRequestAfterItIsDequeued_Should // Cancel, and give the queue time to start waiting for a response #if NET8_0_OR_GREATER + await Task.Delay(1000); await cancellationTokenSource.CancelAsync(); #else cancellationTokenSource.Cancel(); #endif await Task.Delay(1000, CancellationToken); + dequeued!.CancellationToken.IsCancellationRequested.Should().BeTrue("Should have cancelled the request"); await AssertException.Throws(queueAndWaitTask); diff --git a/source/Halibut/Queue/Redis/ProcessingNodeHeartBeatSender.cs b/source/Halibut/Queue/Redis/ProcessingNodeHeartBeatSender.cs index b8de9c781..8ae764ca3 100644 --- a/source/Halibut/Queue/Redis/ProcessingNodeHeartBeatSender.cs +++ b/source/Halibut/Queue/Redis/ProcessingNodeHeartBeatSender.cs @@ -122,7 +122,9 @@ public static async Task WaitUntilNodeProces // TODO: I am sure a fancy pants delay could be done here calculated from the now and last heart beat etc. await Try.IgnoringError(async () => await Task.Delay(TimeSpan.FromSeconds(10), cts.Token)); var timeSinceLastHeartBeat = DateTimeOffset.Now - lastHeartBeat.Value; - if (timeSinceLastHeartBeat > TimeSpan.FromSeconds(60)) + // TODO 60? + var maxTimeBetweenHeartBeetsBeforeProcessingNodeIsAssumedToBeOffline = TimeSpan.FromSeconds(30); + if (timeSinceLastHeartBeat > maxTimeBetweenHeartBeetsBeforeProcessingNodeIsAssumedToBeOffline) { log.Write(EventType.Diagnostic, "Processing node appears disconnected for request {0}, last heartbeat was {1} seconds ago", request.ActivityId, timeSinceLastHeartBeat.TotalSeconds); return NodeProcessingRequestWatcherResult.ProcessingNodeIsLikelyDisconnected; diff --git a/source/Halibut/Queue/Redis/RedisFacade.cs b/source/Halibut/Queue/Redis/RedisFacade.cs index c92cf6441..f7f532c5d 100644 --- a/source/Halibut/Queue/Redis/RedisFacade.cs +++ b/source/Halibut/Queue/Redis/RedisFacade.cs @@ -143,6 +143,9 @@ private void OnConnectionRestored(object? sender, ConnectionFailedEventArgs e) var message = $"Redis connection restored - EndPoint: {e.EndPoint}"; log?.Write(EventType.Diagnostic, message); } + + // We can survive redis being unavailable for this amount of time. + static readonly TimeSpan MaxDurationToRetryFor = TimeSpan.FromSeconds(30); /// /// Executes an operation with retry logic. Retries for up to 12 seconds with 1-second intervals. @@ -152,7 +155,6 @@ private async Task ExecuteWithRetry(Func> operation, CancellationT using var linkedTokenSource = CancellationTokenSource.CreateLinkedTokenSource(cancellationToken, facadeCancellationToken); var combinedToken = linkedTokenSource.Token; - var totalDuration = TimeSpan.FromSeconds(12); var retryDelay = TimeSpan.FromSeconds(1); var stopwatch = Stopwatch.StartNew(); @@ -164,7 +166,7 @@ private async Task ExecuteWithRetry(Func> operation, CancellationT { return await operation(); } - catch (Exception ex) when (stopwatch.Elapsed < totalDuration && !combinedToken.IsCancellationRequested) + catch (Exception ex) when (stopwatch.Elapsed < MaxDurationToRetryFor && !combinedToken.IsCancellationRequested) { log?.Write(EventType.Diagnostic, $"Redis operation failed, retrying in {retryDelay.TotalSeconds}s: {ex.Message}"); await Task.Delay(retryDelay, combinedToken); @@ -180,7 +182,6 @@ private async Task ExecuteWithRetry(Func operation, CancellationToken canc using var linkedTokenSource = CancellationTokenSource.CreateLinkedTokenSource(cancellationToken, facadeCancellationToken); var combinedToken = linkedTokenSource.Token; - var totalDuration = TimeSpan.FromSeconds(12); var retryDelay = TimeSpan.FromSeconds(1); var stopwatch = Stopwatch.StartNew(); @@ -193,7 +194,7 @@ private async Task ExecuteWithRetry(Func operation, CancellationToken canc await operation(); return; } - catch (Exception ex) when (stopwatch.Elapsed < totalDuration && !combinedToken.IsCancellationRequested) + catch (Exception ex) when (stopwatch.Elapsed < MaxDurationToRetryFor && !combinedToken.IsCancellationRequested) { log?.Write(EventType.Diagnostic, $"Redis operation failed, retrying in {retryDelay.TotalSeconds}s: {ex.Message}"); await Task.Delay(retryDelay, combinedToken); diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index 1a607a0e6..cbb6342d4 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -117,7 +117,7 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can } async Task TryClearRequestFromQueue(RequestMessage request, PendingRequest pending) - { + { log.Write(EventType.Diagnostic, "Attempting to clear request {0} from queue for endpoint {1}", request.ActivityId, endpoint); // The time the message is allowed to sit on the queue for has elapsed. @@ -132,10 +132,10 @@ async Task TryClearRequestFromQueue(RequestMessage request, PendingRequest pendi if (requestJson != null) { log.Write(EventType.Diagnostic, "Successfully removed request {0} from queue - request was never collected by a processing node", request.ActivityId); - await pending.RequestHasBeenCollectedAndWillBeTransferred(); } else { + await pending.RequestHasBeenCollectedAndWillBeTransferred(); log.Write(EventType.Diagnostic, "Request {0} was not found in queue - it was already collected by a processing node", request.ActivityId); } } diff --git a/source/Halibut/ServiceModel/PendingRequest.cs b/source/Halibut/ServiceModel/PendingRequest.cs index def14dd53..be9ec5baa 100644 --- a/source/Halibut/ServiceModel/PendingRequest.cs +++ b/source/Halibut/ServiceModel/PendingRequest.cs @@ -23,7 +23,7 @@ namespace Halibut.ServiceModel { - public class PendingRequest : IDisposable + public class PendingRequestOld : IDisposable { readonly RequestMessage request; readonly ILog log; @@ -35,7 +35,7 @@ public class PendingRequest : IDisposable readonly CancellationTokenSource pendingRequestCancellationTokenSource; ResponseMessage? response; - public PendingRequest(RequestMessage request, ILog log) + public PendingRequestOld(RequestMessage request, ILog log) { this.request = request; this.log = log; @@ -183,7 +183,7 @@ async Task WaitForResponseToBeSet( else if (!requestCollected.IsSet) { // Cancel the queued request. This will flag the request as cancelled to stop it being dequeued - pendingRequestCancellationTokenSource.Cancel(); + //pendingRequestCancellationTokenSource.Cancel(); } if (timeoutCancellationTokenSource.IsCancellationRequested) diff --git a/source/Halibut/ServiceModel/PendingRequestV2.cs b/source/Halibut/ServiceModel/PendingRequestV2.cs new file mode 100644 index 000000000..f41d8bafe --- /dev/null +++ b/source/Halibut/ServiceModel/PendingRequestV2.cs @@ -0,0 +1,204 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; +using System.Threading; +using System.Threading.Tasks; +using Halibut.Diagnostics; +using Halibut.Exceptions; +using Halibut.Transport; +using Halibut.Transport.Protocol; +using Halibut.Util; +using Nito.AsyncEx; + +namespace Halibut.ServiceModel +{ + public class PendingRequest : IDisposable + { + readonly RequestMessage request; + readonly ILog log; + readonly AsyncManualResetEvent responseWaiter = new(false); + readonly SemaphoreSlim transferLock = new(1, 1); + //bool transferBegun; + AsyncManualResetEvent requestCollected = new(false); + readonly CancellationTokenSource pendingRequestCancellationTokenSource; + ResponseMessage? response; + + public PendingRequest(RequestMessage request, ILog log) + { + this.request = request; + this.log = log; + + pendingRequestCancellationTokenSource = new CancellationTokenSource(); + PendingRequestCancellationToken = pendingRequestCancellationTokenSource.Token; + } + + public Task WaitForRequestToBeMarkedAsCollected(CancellationToken cancellationToken) => requestCollected.WaitAsync(cancellationToken); + + public bool HasRequestBeenMarkedAsCollected => requestCollected.IsSet; + + public RequestMessage Request => request; + + /// + /// + /// + /// + /// This will be called either when the pick-up timeout has elapsed OR if the Cancellation Token has been triggered. + /// This gives the user an opportunity to remove the pending request from shared places and optionally + /// call BeginTransfer + /// + /// + public async Task WaitUntilComplete(Func timePendingRequestCanBeOnTheQueueHasElapsed, CancellationToken cancellationToken) + { + log.Write(EventType.MessageExchange, "Request {0} was queued", request); + + await using var cts = CancellationTokenSource.CreateLinkedTokenSource(cancellationToken).CancelOnDispose(); + + var pendingRequestPickupTimeout = Try.IgnoringError(async () => await Task.Delay(request.Destination.PollingRequestQueueTimeout, cts.CancellationToken)); + var responseWaiterTask = responseWaiter.WaitAsync(cts.CancellationToken); + + await Task.WhenAny(pendingRequestPickupTimeout, responseWaiterTask); + + using (await transferLock.LockAsync(CancellationToken.None)) + { + // Response has been returned so just say we are done. + if (responseWaiter.IsSet) + { + log.Write(EventType.MessageExchange, "Request {0} was collected by the polling endpoint", request); + return; + } + + + if (cancellationToken.IsCancellationRequested) + { + if (!requestCollected.IsSet) log.Write(EventType.MessageExchange, "Request {0} was cancelled before it could be collected by the polling endpoint", request); + else log.Write(EventType.MessageExchange, "Request {0} was collected by the polling endpoint, will try to cancel the request", request); + + await Try.IgnoringError(async () => await pendingRequestCancellationTokenSource.CancelAsync()); + cancellationToken.ThrowIfCancellationRequested(); + } + } + + if (!requestCollected.IsSet) await timePendingRequestCanBeOnTheQueueHasElapsed(); + + using (await transferLock.LockAsync(CancellationToken.None)) { + if (!requestCollected.IsSet) + { + // Request was not collected within the pickup time. + // Prevent anyone from processing the request further. + await Try.IgnoringError(async () => await pendingRequestCancellationTokenSource.CancelAsync()); + + log.Write(EventType.MessageExchange, "Request {0} timed out before it could be collected by the polling endpoint", request); + SetResponseNoLock(ResponseMessage.FromException( + request, + new TimeoutException($"A request was sent to a polling endpoint, but the polling endpoint did not collect the request within the allowed time ({request.Destination.PollingRequestQueueTimeout}), so the request timed out."), + ConnectionState.Connecting)); + return; + } + } + + + // The request has been collected so now wait patiently for a response + log.Write(EventType.MessageExchange, "Request {0} was eventually collected by the polling endpoint", request); + try + { + await responseWaiterTask; + } + catch (Exception) when (!cts.CancellationToken.IsCancellationRequested) + { + using (await transferLock.LockAsync(CancellationToken.None)) + { + if (!responseWaiter.IsSet) + { + log.Write(EventType.MessageExchange, "Request {0} was cancelled before a response was received", request); + SetResponseNoLock(ResponseMessage.FromException( + request, + new TimeoutException($"A request was sent to a polling endpoint, the polling endpoint collected it but the request was cancelled before the polling endpoint responded."), + ConnectionState.Connecting)); + await Try.IgnoringError(async () => await pendingRequestCancellationTokenSource.CancelAsync()); + cancellationToken.ThrowIfCancellationRequested(); + } + } + } + catch (Exception) + { + // This should never happen. + log.Write(EventType.MessageExchange, "Request {0} had an internal error, unexpectedly stopped waiting for the response.", request); + await SetResponseAsync(ResponseMessage.FromException( + request, + new PendingRequestQueueInternalException($"Request {request.Id} had an internal error, unexpectedly stopped waiting for the response."))); + } + } + + + public async Task RequestHasBeenCollectedAndWillBeTransferred() + { + // The PendingRequest is Disposed at the end of QueueAndWaitAsync but a race condition + // exists in the current approach that means DequeueAsync could pick this request up after + // it has been disposed. At that point we are no longer interested in the PendingRequest so + // this is "ok" and wrapping BeginTransfer in a try..catch.. ensures we don't error if the + // race condition occurs and also stops the polling tentacle dequeuing the request successfully. + try + { + using (await transferLock.LockAsync(CancellationToken.None)) + { + // Check if the request has already been completed or if the request has been cancelled + // to ensure we don't dequeue an already completed or already cancelled request + if (requestCollected.IsSet || pendingRequestCancellationTokenSource.IsCancellationRequested) + { + return false; + } + + requestCollected.Set(); + return true; + } + } + catch (ObjectDisposedException) + { + return false; + } + } + + public ResponseMessage Response => response ?? throw new InvalidOperationException("Response has not been set."); + public CancellationToken PendingRequestCancellationToken { get; } + + public void SetResponse(ResponseMessage response) + { + this.SetResponseAsync(response).GetAwaiter().GetResult(); + } + + public async Task SetResponseAsync(ResponseMessage response) + { + using (await transferLock.LockAsync(CancellationToken.None)) + { + SetResponseNoLock(response); + } + } + + void SetResponseNoLock(ResponseMessage response) + { + if(this.response != null) return; + this.response = response; + responseWaiter.Set(); + requestCollected.Set(); // Also the request has been collected, if we have a response. + } + + public void Dispose() + { + pendingRequestCancellationTokenSource?.Dispose(); + transferLock?.Dispose(); + } + } + +} \ No newline at end of file From 0e8637899ca38e795fae334b64b9bc657b1a48a5 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Wed, 6 Aug 2025 06:26:19 +1000 Subject: [PATCH 018/137] . --- .../Redis/ProcessingNodeHeartBeatSender.cs | 5 +++- .../Halibut/ServiceModel/PendingRequestV2.cs | 25 ++++++++++++------- 2 files changed, 20 insertions(+), 10 deletions(-) diff --git a/source/Halibut/Queue/Redis/ProcessingNodeHeartBeatSender.cs b/source/Halibut/Queue/Redis/ProcessingNodeHeartBeatSender.cs index 8ae764ca3..138ac9643 100644 --- a/source/Halibut/Queue/Redis/ProcessingNodeHeartBeatSender.cs +++ b/source/Halibut/Queue/Redis/ProcessingNodeHeartBeatSender.cs @@ -149,18 +149,21 @@ static async Task WaitForRequestToBeCollected(Uri endpoint, RequestMessage reque { try { - // This might only be set when the queue timeout is reached. + // Has something else determined the request was collected? + // TODO should we bail out of here if the PendingRequest is complete? if(pending.HasRequestBeenMarkedAsCollected) { log.Write(EventType.Diagnostic, "Request {0} has been marked as collected", request.ActivityId); return; } + // So check ourselves if the request has been collected. var requestIsStillOnQueue = await halibutRedisTransport.IsRequestStillOnQueue(endpoint, request.ActivityId, cancellationToken); if(!requestIsStillOnQueue) { log.Write(EventType.Diagnostic, "Request {0} is no longer on queue", request.ActivityId); + await pending.RequestHasBeenCollectedAndWillBeTransferred(); return; } } diff --git a/source/Halibut/ServiceModel/PendingRequestV2.cs b/source/Halibut/ServiceModel/PendingRequestV2.cs index f41d8bafe..84154160e 100644 --- a/source/Halibut/ServiceModel/PendingRequestV2.cs +++ b/source/Halibut/ServiceModel/PendingRequestV2.cs @@ -103,7 +103,8 @@ public async Task WaitUntilComplete(Func timePendingRequestCanBeOnTheQueue SetResponseNoLock(ResponseMessage.FromException( request, new TimeoutException($"A request was sent to a polling endpoint, but the polling endpoint did not collect the request within the allowed time ({request.Destination.PollingRequestQueueTimeout}), so the request timed out."), - ConnectionState.Connecting)); + ConnectionState.Connecting), + false); return; } } @@ -125,7 +126,8 @@ public async Task WaitUntilComplete(Func timePendingRequestCanBeOnTheQueue SetResponseNoLock(ResponseMessage.FromException( request, new TimeoutException($"A request was sent to a polling endpoint, the polling endpoint collected it but the request was cancelled before the polling endpoint responded."), - ConnectionState.Connecting)); + ConnectionState.Connecting), + false); await Try.IgnoringError(async () => await pendingRequestCancellationTokenSource.CancelAsync()); cancellationToken.ThrowIfCancellationRequested(); } @@ -137,7 +139,8 @@ public async Task WaitUntilComplete(Func timePendingRequestCanBeOnTheQueue log.Write(EventType.MessageExchange, "Request {0} had an internal error, unexpectedly stopped waiting for the response.", request); await SetResponseAsync(ResponseMessage.FromException( request, - new PendingRequestQueueInternalException($"Request {request.Id} had an internal error, unexpectedly stopped waiting for the response."))); + new PendingRequestQueueInternalException($"Request {request.Id} had an internal error, unexpectedly stopped waiting for the response.")), + false); } } @@ -155,7 +158,10 @@ public async Task RequestHasBeenCollectedAndWillBeTransferred() { // Check if the request has already been completed or if the request has been cancelled // to ensure we don't dequeue an already completed or already cancelled request - if (requestCollected.IsSet || pendingRequestCancellationTokenSource.IsCancellationRequested) + if (requestCollected.IsSet + || pendingRequestCancellationTokenSource.IsCancellationRequested + || responseWaiter.IsSet) + { return false; } @@ -175,23 +181,24 @@ public async Task RequestHasBeenCollectedAndWillBeTransferred() public void SetResponse(ResponseMessage response) { - this.SetResponseAsync(response).GetAwaiter().GetResult(); + // If someone is calling this then we know for sure they collected the request + this.SetResponseAsync(response, true).GetAwaiter().GetResult(); } - public async Task SetResponseAsync(ResponseMessage response) + async Task SetResponseAsync(ResponseMessage response, bool requestWasCollected) { using (await transferLock.LockAsync(CancellationToken.None)) { - SetResponseNoLock(response); + SetResponseNoLock(response, requestWasCollected); } } - void SetResponseNoLock(ResponseMessage response) + void SetResponseNoLock(ResponseMessage response, bool requestWasCollected) { if(this.response != null) return; this.response = response; responseWaiter.Set(); - requestCollected.Set(); // Also the request has been collected, if we have a response. + if(requestWasCollected) requestCollected.Set(); // Also the request has been collected, if we have a response. } public void Dispose() From edc2454166bafc2e762a3030ab1f1dcd20469a30 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Wed, 6 Aug 2025 06:27:03 +1000 Subject: [PATCH 019/137] . --- .../Util/LinearBackoffStrategyFixture.cs | 191 ++++++++++++++++++ .../Redis/PollAndSubscribeForSingleMessage.cs | 186 +++++++++++++++++ source/Halibut/Util/LinearBackoffStrategy.cs | 91 +++++++++ 3 files changed, 468 insertions(+) create mode 100644 source/Halibut.Tests/Util/LinearBackoffStrategyFixture.cs create mode 100644 source/Halibut/Queue/Redis/PollAndSubscribeForSingleMessage.cs create mode 100644 source/Halibut/Util/LinearBackoffStrategy.cs diff --git a/source/Halibut.Tests/Util/LinearBackoffStrategyFixture.cs b/source/Halibut.Tests/Util/LinearBackoffStrategyFixture.cs new file mode 100644 index 000000000..1a268259c --- /dev/null +++ b/source/Halibut.Tests/Util/LinearBackoffStrategyFixture.cs @@ -0,0 +1,191 @@ +using System; +using Halibut.Util; +using NUnit.Framework; + +namespace Halibut.Tests.Util +{ + public class LinearBackoffStrategyFixture + { + [Test] + [TestCase(1, 1, 30, 1, ExpectedResult = 1)] + [TestCase(1, 1, 30, 2, ExpectedResult = 2)] + [TestCase(1, 1, 30, 3, ExpectedResult = 3)] + [TestCase(1, 1, 30, 10, ExpectedResult = 10)] + [TestCase(1, 1, 30, 30, ExpectedResult = 30)] + [TestCase(1, 1, 30, 31, ExpectedResult = 30)] // Should cap at maximum + [TestCase(1, 1, 30, 50, ExpectedResult = 30)] // Should cap at maximum + [TestCase(2, 3, 20, 1, ExpectedResult = 2)] // initialDelay=2, increment=3 + [TestCase(2, 3, 20, 2, ExpectedResult = 5)] // 2 + (2-1)*3 = 5 + [TestCase(2, 3, 20, 3, ExpectedResult = 8)] // 2 + (3-1)*3 = 8 + [TestCase(2, 3, 20, 7, ExpectedResult = 20)] // 2 + (7-1)*3 = 20 (at max) + [TestCase(2, 3, 20, 8, ExpectedResult = 20)] // Should cap at maximum + [TestCase(0, 2, 10, 1, ExpectedResult = 0)] // Zero initial delay + [TestCase(0, 2, 10, 2, ExpectedResult = 2)] // 0 + (2-1)*2 = 2 + [TestCase(0, 2, 10, 6, ExpectedResult = 10)] // 0 + (6-1)*2 = 10 (at max) + public int CalculateDelayForAttemptShouldBeCorrect(int initialDelaySeconds, int incrementSeconds, int maxDelaySeconds, int attemptNumber) + { + var strategy = new LinearBackoffStrategy( + TimeSpan.FromSeconds(initialDelaySeconds), + TimeSpan.FromSeconds(incrementSeconds), + TimeSpan.FromSeconds(maxDelaySeconds) + ); + + var delay = strategy.CalculateDelayForAttempt(attemptNumber); + + return (int)delay.TotalSeconds; + } + + [Test] + public void GetSleepPeriodShouldReturnZeroWhenNoAttemptsMade() + { + var strategy = LinearBackoffStrategy.Create(); + + var delay = strategy.GetSleepPeriod(); + + Assert.That(delay, Is.EqualTo(TimeSpan.Zero)); + } + + [Test] + public void GetSleepPeriodShouldIncreaseLinearlyWithAttempts() + { + var strategy = new LinearBackoffStrategy( + TimeSpan.FromSeconds(2), + TimeSpan.FromSeconds(3), + TimeSpan.FromSeconds(20) + ); + + // First attempt + strategy.Try(); + Assert.That(strategy.GetSleepPeriod(), Is.EqualTo(TimeSpan.FromSeconds(2))); + Assert.That(strategy.AttemptCount, Is.EqualTo(1)); + + // Second attempt + strategy.Try(); + Assert.That(strategy.GetSleepPeriod(), Is.EqualTo(TimeSpan.FromSeconds(5))); // 2 + (2-1)*3 = 5 + Assert.That(strategy.AttemptCount, Is.EqualTo(2)); + + // Third attempt + strategy.Try(); + Assert.That(strategy.GetSleepPeriod(), Is.EqualTo(TimeSpan.FromSeconds(8))); // 2 + (3-1)*3 = 8 + Assert.That(strategy.AttemptCount, Is.EqualTo(3)); + } + + [Test] + public void GetSleepPeriodShouldCapAtMaximumDelay() + { + var strategy = new LinearBackoffStrategy( + TimeSpan.FromSeconds(5), + TimeSpan.FromSeconds(10), + TimeSpan.FromSeconds(20) + ); + + strategy.Try(); // Attempt 1: 5 seconds + Assert.That(strategy.GetSleepPeriod(), Is.EqualTo(TimeSpan.FromSeconds(5))); + + strategy.Try(); // Attempt 2: 15 seconds + Assert.That(strategy.GetSleepPeriod(), Is.EqualTo(TimeSpan.FromSeconds(15))); + + strategy.Try(); // Attempt 3: would be 25, but capped at 20 + Assert.That(strategy.GetSleepPeriod(), Is.EqualTo(TimeSpan.FromSeconds(20))); + + strategy.Try(); // Attempt 4: still capped at 20 + Assert.That(strategy.GetSleepPeriod(), Is.EqualTo(TimeSpan.FromSeconds(20))); + } + + [Test] + public void SuccessShouldResetAttemptCount() + { + var strategy = LinearBackoffStrategy.Create(); + + strategy.Try(); + strategy.Try(); + strategy.Try(); + Assert.That(strategy.AttemptCount, Is.EqualTo(3)); + Assert.That(strategy.GetSleepPeriod(), Is.EqualTo(TimeSpan.FromSeconds(3))); + + strategy.Success(); + Assert.That(strategy.AttemptCount, Is.EqualTo(0)); + Assert.That(strategy.GetSleepPeriod(), Is.EqualTo(TimeSpan.Zero)); + + // After reset, should start from the beginning again + strategy.Try(); + Assert.That(strategy.AttemptCount, Is.EqualTo(1)); + Assert.That(strategy.GetSleepPeriod(), Is.EqualTo(TimeSpan.FromSeconds(1))); + } + + [Test] + public void CreateShouldReturnStrategyWithDefaultValues() + { + var strategy = LinearBackoffStrategy.Create(); + + Assert.That(strategy.InitialDelay, Is.EqualTo(TimeSpan.FromSeconds(1))); + Assert.That(strategy.Increment, Is.EqualTo(TimeSpan.FromSeconds(1))); + Assert.That(strategy.MaximumDelay, Is.EqualTo(TimeSpan.FromSeconds(30))); + Assert.That(strategy.AttemptCount, Is.EqualTo(0)); + } + + [Test] + [TestCase(-1)] // Negative initial delay + public void InvalidInitialDelayShouldThrow(int initialDelaySeconds) + { + Assert.Throws(() => + new LinearBackoffStrategy( + TimeSpan.FromSeconds(initialDelaySeconds), + TimeSpan.FromSeconds(1), + TimeSpan.FromSeconds(10) + ) + ); + } + + [Test] + [TestCase(0)] // Zero increment + [TestCase(-1)] // Negative increment + public void InvalidIncrementShouldThrow(int incrementSeconds) + { + Assert.Throws(() => + new LinearBackoffStrategy( + TimeSpan.FromSeconds(1), + TimeSpan.FromSeconds(incrementSeconds), + TimeSpan.FromSeconds(10) + ) + ); + } + + [Test] + public void MaximumDelayLessThanInitialDelayShouldThrow() + { + Assert.Throws(() => + new LinearBackoffStrategy( + TimeSpan.FromSeconds(10), // Initial delay + TimeSpan.FromSeconds(1), // Increment + TimeSpan.FromSeconds(5) // Maximum delay (less than initial) + ) + ); + } + + [Test] + [TestCase(0, ExpectedResult = 0)] + [TestCase(-1, ExpectedResult = 0)] + [TestCase(-10, ExpectedResult = 0)] + public int CalculateDelayForAttemptShouldReturnZeroForInvalidAttemptNumbers(int attemptNumber) + { + var strategy = LinearBackoffStrategy.Create(); + var delay = strategy.CalculateDelayForAttempt(attemptNumber); + return (int)delay.TotalSeconds; + } + + [Test] + public void PropertiesShouldReturnConstructorValues() + { + var initialDelay = TimeSpan.FromSeconds(3); + var increment = TimeSpan.FromSeconds(2); + var maximumDelay = TimeSpan.FromSeconds(15); + + var strategy = new LinearBackoffStrategy(initialDelay, increment, maximumDelay); + + Assert.That(strategy.InitialDelay, Is.EqualTo(initialDelay)); + Assert.That(strategy.Increment, Is.EqualTo(increment)); + Assert.That(strategy.MaximumDelay, Is.EqualTo(maximumDelay)); + } + } +} \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/PollAndSubscribeForSingleMessage.cs b/source/Halibut/Queue/Redis/PollAndSubscribeForSingleMessage.cs new file mode 100644 index 000000000..ff8af3ef5 --- /dev/null +++ b/source/Halibut/Queue/Redis/PollAndSubscribeForSingleMessage.cs @@ -0,0 +1,186 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; +using System.Threading; +using System.Threading.Tasks; +using Halibut.Diagnostics; +using Halibut.Util; +using Nito.AsyncEx; + +namespace Halibut.Queue.Redis +{ + public class PollAndSubscribeForSingleMessage : IAsyncDisposable + { + public static async Task TrySendMessage( + string messageTypeName, + HalibutRedisTransport halibutRedisTransport, + Uri endpoint, + Guid activityId, + string value, + ILog log) + { + log.Write(EventType.Diagnostic, "Attempting to set {0} for - Endpoint: {1}, ActivityId: {2}", messageTypeName, endpoint, activityId); + + using var cts = new CancellationTokenSource(); + cts.CancelAfter(TimeSpan.FromMinutes(2)); // Best efforts. + + try + { + log.Write(EventType.Diagnostic, "Marking {0} as set - Endpoint: {1}, ActivityId: {2}", messageTypeName, endpoint, activityId); + await halibutRedisTransport.SendValue(messageTypeName, endpoint, activityId, value, cts.Token); + + log.Write(EventType.Diagnostic, "Publishing {0} notification - Endpoint: {1}, ActivityId: {2}", messageTypeName, endpoint, activityId); + await halibutRedisTransport.PublishThatValueIsAvailable(messageTypeName, endpoint, activityId, value, cts.Token); + + log.Write(EventType.Diagnostic, "Successfully set {0} - Endpoint: {1}, ActivityId: {2}", messageTypeName, endpoint, activityId); + } + catch (OperationCanceledException ex) + { + log.Write(EventType.Error, "Set {0} operation timed out after 2 minutes - Endpoint: {1}, ActivityId: {2}, Error: {3}", messageTypeName, endpoint, activityId, ex.Message); + } + catch (Exception ex) + { + log.Write(EventType.Error, "Failed to set {0} - Endpoint: {1}, ActivityId: {2}, Error: {3}", messageTypeName, endpoint, activityId, ex.Message); + } + } + + readonly CancellationTokenSource watcherTokenSource = new(); + + readonly ILog log; + readonly string messageTypeName; + readonly HalibutRedisTransport halibutRedisTransport; + readonly Uri endpoint; + readonly Guid activityId; + readonly LinearBackoffStrategy pollBackoffStrategy; + + TaskCompletionSource message = new(); + + public Task ResultTask => message.Task; + + public PollAndSubscribeForSingleMessage(string messageTypeName, Uri endpoint, Guid activityId, HalibutRedisTransport halibutRedisTransport, ILog log) + { + this.log = log; + this.messageTypeName = messageTypeName; + this.endpoint = endpoint; + this.activityId = activityId; + this.halibutRedisTransport = halibutRedisTransport; + this.pollBackoffStrategy = new LinearBackoffStrategy( + TimeSpan.FromSeconds(15), // Initial delay: 15s + TimeSpan.FromSeconds(15), // Increment: 15s + TimeSpan.FromMinutes(2) // Maximum delay: 2 minutes + ); + log.Write(EventType.Diagnostic, "Starting to watch for {0} - Endpoint: {1}, ActivityId: {2}", messageTypeName, endpoint, activityId); + + var token = watcherTokenSource.Token; + var _ = Task.Run(async () => await WatchAndWaitForMessage(token)); + } + + readonly SemaphoreSlim trySetResultSemaphore = new SemaphoreSlim(1, 1); + + public async Task TrySetResultAndRemoveValueFromRedis(string value, CancellationToken cancellationToken) + { + using var l = await trySetResultSemaphore.LockAsync(cancellationToken); + try + { + if(!message.Task.IsCompleted) message.TrySetResult(value); + } + catch (Exception) + { + // TODO log we could not set result. + } + + try + { + await halibutRedisTransport.DeleteGenericMarker(messageTypeName, endpoint, activityId, cancellationToken); + } + catch (Exception) + { + // TODO log we could not remove value + } + + } + async Task WatchAndWaitForMessage(CancellationToken token) + { + try + { + log.Write(EventType.Diagnostic, "Subscribing to {0} notifications - Endpoint: {1}, ActivityId: {2}", messageTypeName, endpoint, activityId); + + await using var _ = await halibutRedisTransport.SubscribeToGenericNotification(messageTypeName, endpoint, activityId, + async _ => + { + + log.Write(EventType.Diagnostic, "Received {0} notification via subscription - Endpoint: {1}, ActivityId: {2}", messageTypeName, endpoint, activityId); + + var value = await halibutRedisTransport.GetGenericMarker(messageTypeName, endpoint, activityId, token); + if (value != null) + { + await TrySetResultAndRemoveValueFromRedis(value, token); + } + + await watcherTokenSource.CancelAsync(); + }, + token); + + log.Write(EventType.Diagnostic, "Starting polling loop for {0} - Endpoint: {1}, ActivityId: {2}", messageTypeName, endpoint, activityId); + + // Also poll to see if the value is set since we can miss + // the publication. + while (!token.IsCancellationRequested) + { + try + { + var value = await halibutRedisTransport.GetGenericMarker(messageTypeName, endpoint, activityId, token); + if (value != null) + { + log.Write(EventType.Diagnostic, "{0} detected via polling - Endpoint: {1}, ActivityId: {2}", messageTypeName, endpoint, activityId); + pollBackoffStrategy.Success(); // Reset backoff strategy on successful retrieval + await TrySetResultAndRemoveValueFromRedis(value, token); + await watcherTokenSource.CancelAsync(); + break; + } + } + catch (Exception ex) + { + log.Write(EventType.Diagnostic, "Error while polling for {0} - Endpoint: {1}, ActivityId: {2}, Error: {3}", messageTypeName, endpoint, activityId, ex.Message); + } + + pollBackoffStrategy.Try(); + var delay = pollBackoffStrategy.GetSleepPeriod(); + log.Write(EventType.Diagnostic, "Waiting {0} seconds before next poll for {1} - Endpoint: {2}, ActivityId: {3}", delay.TotalSeconds, messageTypeName, endpoint, activityId); + await Task.Delay(delay, token); + } + + log.Write(EventType.Diagnostic, "Exiting watch loop for {0} - Endpoint: {1}, ActivityId: {2}", messageTypeName, endpoint, activityId); + } + catch (Exception ex) + { + log.Write(EventType.Error, "Unexpected error in {0} watcher - Endpoint: {1}, ActivityId: {2}, Error: {3}", messageTypeName, endpoint, activityId, ex.Message); + } + } + + public async ValueTask DisposeAsync() + { + log.Write(EventType.Diagnostic, "Disposing GenericWatcher for {0}", messageTypeName); + + await Try.IgnoringError(async () => await watcherTokenSource.CancelAsync()); + Try.IgnoringError(() => watcherTokenSource.Dispose()); + + // If the message task is not yet complete, then complete if now with null since we have nothing for it. + Try.IgnoringError(() => message.TrySetCanceled()); + + log.Write(EventType.Diagnostic, "GenericWatcher for {0} disposed", messageTypeName); + } + } +} \ No newline at end of file diff --git a/source/Halibut/Util/LinearBackoffStrategy.cs b/source/Halibut/Util/LinearBackoffStrategy.cs new file mode 100644 index 000000000..9ca036d0c --- /dev/null +++ b/source/Halibut/Util/LinearBackoffStrategy.cs @@ -0,0 +1,91 @@ +using System; + +namespace Halibut.Util +{ + /// + /// A simple linear delay backoff strategy that increases the delay by a fixed increment + /// on each retry attempt (e.g., 1s, 2s, 3s, 4s, etc.). + /// + public class LinearBackoffStrategy + { + int attemptCount; + + public LinearBackoffStrategy(TimeSpan initialDelay, TimeSpan increment, TimeSpan maximumDelay) + { + if (initialDelay < TimeSpan.Zero) + throw new ArgumentOutOfRangeException(nameof(initialDelay), "Initial delay must be non-negative"); + if (increment <= TimeSpan.Zero) + throw new ArgumentOutOfRangeException(nameof(increment), "Increment must be greater than zero"); + if (maximumDelay < initialDelay) + throw new ArgumentOutOfRangeException(nameof(maximumDelay), "Maximum delay must be greater than or equal to initial delay"); + + InitialDelay = initialDelay; + Increment = increment; + MaximumDelay = maximumDelay; + attemptCount = 0; + } + + public TimeSpan InitialDelay { get; } + public TimeSpan Increment { get; } + public TimeSpan MaximumDelay { get; } + public int AttemptCount => attemptCount; + + /// + /// Creates a LinearBackoffStrategy with sensible defaults: + /// Initial delay of 1 second, increment of 1 second, maximum delay of 30 seconds. + /// + public static LinearBackoffStrategy Create() + { + return new LinearBackoffStrategy( + initialDelay: TimeSpan.FromSeconds(1), + increment: TimeSpan.FromSeconds(1), + maximumDelay: TimeSpan.FromSeconds(30) + ); + } + + /// + /// Records a retry attempt and increments the internal attempt counter. + /// + public virtual void Try() + { + attemptCount++; + } + + /// + /// Resets the backoff strategy after a successful operation. + /// + public virtual void Success() + { + attemptCount = 0; + } + + /// + /// Gets the delay period for the current attempt number. + /// The delay increases linearly: initialDelay + (attemptCount - 1) * increment. + /// + public virtual TimeSpan GetSleepPeriod() + { + if (attemptCount <= 0) + return TimeSpan.Zero; + + var delay = InitialDelay + TimeSpan.FromTicks((attemptCount - 1) * Increment.Ticks); + + // Cap at maximum delay + return delay > MaximumDelay ? MaximumDelay : delay; + } + + /// + /// Calculates the delay for a specific attempt number without modifying internal state. + /// + public TimeSpan CalculateDelayForAttempt(int attemptNumber) + { + if (attemptNumber <= 0) + return TimeSpan.Zero; + + var delay = InitialDelay + TimeSpan.FromTicks((attemptNumber - 1) * Increment.Ticks); + + // Cap at maximum delay + return delay > MaximumDelay ? MaximumDelay : delay; + } + } +} \ No newline at end of file From 839b65cf74ef9e13d4b696eeac2550c8e8615ac8 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Wed, 6 Aug 2025 07:12:36 +1000 Subject: [PATCH 020/137] some clean up --- .../Redis/RedisPendingRequestQueueFixture.cs | 4 + .../Redis/ProcessingNodeHeartBeatSender.cs | 7 +- .../Queue/Redis/RedisPendingRequestQueue.cs | 78 +++++++++++++------ .../Halibut/ServiceModel/PendingRequestV2.cs | 12 +-- 4 files changed, 67 insertions(+), 34 deletions(-) diff --git a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs index 3eeeebca1..fc0cc55e4 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs @@ -335,10 +335,14 @@ public async Task WhenTheReceiverDequeuesWorkAndThenDisconnectsFromRedisForEver_ var node1Sender = new RedisPendingRequestQueue(endpoint, log, new HalibutRedisTransport(stableRedisConnection), messageReaderWriter, halibutTimeoutAndLimits); var node2Receiver = new RedisPendingRequestQueue(endpoint, log, new HalibutRedisTransport(unstableRedisConnection), messageReaderWriter, halibutTimeoutAndLimits); + // Lower this to complete the test sooner. + node1Sender.MaxTimeBetweenHeartBeetsBeforeProcessingNodeIsAssumedToBeOffline = TimeSpan.FromSeconds(30); + node2Receiver.MaxTimeBetweenHeartBeetsBeforeProcessingNodeIsAssumedToBeOffline = TimeSpan.FromSeconds(30); var request = new RequestMessageBuilder("poll://test-endpoint").Build(); // TODO: Setting this low shows we don't timeout because the request was not picked up in time. + // Could be its own test. request.Destination.PollingRequestQueueTimeout = TimeSpan.FromSeconds(5); var queueAndWaitTask = node1Sender.QueueAndWaitAsync(request, CancellationToken.None); diff --git a/source/Halibut/Queue/Redis/ProcessingNodeHeartBeatSender.cs b/source/Halibut/Queue/Redis/ProcessingNodeHeartBeatSender.cs index 138ac9643..7c3ea291b 100644 --- a/source/Halibut/Queue/Redis/ProcessingNodeHeartBeatSender.cs +++ b/source/Halibut/Queue/Redis/ProcessingNodeHeartBeatSender.cs @@ -35,6 +35,9 @@ public enum NodeProcessingRequestWatcherResult private readonly HalibutRedisTransport halibutRedisTransport; private readonly CancellationTokenSource cancellationTokenSource; private readonly ILog log; + + public static readonly TimeSpan DefaultMaxTimeBetweenHeartBeetsBeforeProcessingNodeIsAssumedToBeOffline = TimeSpan.FromSeconds(60); + public ProcessingNodeHeartBeatSender(Uri endpoint, Guid requestActivityId, @@ -46,7 +49,6 @@ public ProcessingNodeHeartBeatSender(Uri endpoint, this.halibutRedisTransport = halibutRedisTransport; this.cancellationTokenSource = new CancellationTokenSource(); this.log = log; - log.Write(EventType.Diagnostic, "Starting ProcessingNodeHeartBeatSender for request {0} to endpoint {1}", requestActivityId, endpoint); Task.Run(() => SendPulsesWhileProcessingRequest(cancellationTokenSource.Token)); } @@ -88,6 +90,7 @@ public static async Task WaitUntilNodeProces PendingRequest pending, HalibutRedisTransport halibutRedisTransport, ILog log, + TimeSpan maxTimeBetweenHeartBeetsBeforeProcessingNodeIsAssumedToBeOffline, CancellationToken watchCancellationToken) { log.Write(EventType.Diagnostic, "Starting to watch for processing node flatline for request {0} to endpoint {1}", request.ActivityId, endpoint); @@ -122,8 +125,6 @@ public static async Task WaitUntilNodeProces // TODO: I am sure a fancy pants delay could be done here calculated from the now and last heart beat etc. await Try.IgnoringError(async () => await Task.Delay(TimeSpan.FromSeconds(10), cts.Token)); var timeSinceLastHeartBeat = DateTimeOffset.Now - lastHeartBeat.Value; - // TODO 60? - var maxTimeBetweenHeartBeetsBeforeProcessingNodeIsAssumedToBeOffline = TimeSpan.FromSeconds(30); if (timeSinceLastHeartBeat > maxTimeBetweenHeartBeetsBeforeProcessingNodeIsAssumedToBeOffline) { log.Write(EventType.Diagnostic, "Processing node appears disconnected for request {0}, last heartbeat was {1} seconds ago", request.ActivityId, timeSinceLastHeartBeat.TotalSeconds); diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index cbb6342d4..0510825ad 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -75,7 +75,6 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can var cancellationToken = cts.Token; // TODO: redis goes down // TODO: Other node goes down. - // TODO: Respect cancellation token // TODO RedisConnectionException can be raised out of here, what should the queue do? using var pending = new PendingRequest(request, log); @@ -84,39 +83,61 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can var payload = await messageReaderWriter.PrepareRequest(request, cancellationToken); // Start listening for a response to the request, we don't want to miss the response. - // TODO: subscribing to the response is not reliable, we also need to account for missed publications. await using var _ = await SubscribeToResponse(request.ActivityId, pending.SetResponse, cancellationToken); - await using var tryClearRequestFromQueueWhenRequestIsCancelled - = pending.PendingRequestCancellationToken.Register(async () => await TryClearRequestFromQueue(request, pending)); - await using var trySendCancelWhenRequestIsCancelled - = pending.PendingRequestCancellationToken.Register(async () => - await WatchForRequestCancellation.TrySendCancellation(halibutRedisTransport, endpoint, request, log)); + // These are not guaranteed to execute, we should ensure they are executed in a finally block. - // Make the request available before we tell people it is available. - await halibutRedisTransport.PutRequest(endpoint, request.ActivityId, payload, cancellationToken); - await halibutRedisTransport.PushRequestGuidOnToQueue(endpoint, request.ActivityId, cancellationToken); - await halibutRedisTransport.PulseRequestPushedToEndpoint(endpoint, cancellationToken); - - await using var watcherCts = CancellationTokenSource.CreateLinkedTokenSource(cts.Token).CancelOnDispose(); + var tryClearRequestFromQueueAtMostOnce = new AsyncLazy(async () => await TryClearRequestFromQueue(request, pending)); + try + { + // Make the request available before we tell people it is available. + await halibutRedisTransport.PutRequest(endpoint, request.ActivityId, payload, cancellationToken); + await halibutRedisTransport.PushRequestGuidOnToQueue(endpoint, request.ActivityId, cancellationToken); + await halibutRedisTransport.PulseRequestPushedToEndpoint(endpoint, cancellationToken); + + await using var watcherCts = CancellationTokenSource.CreateLinkedTokenSource(cts.Token).CancelOnDispose(); + var watchProcessingNodeTask = WatchProcessingNodeIsStillConnectedInBackground(request, pending, watcherCts); + + await pending.WaitUntilComplete(async () => await tryClearRequestFromQueueAtMostOnce.Task, cancellationToken); + } + finally + { + // Make an attempt to ensure the request is removed from redis. + var background = Task.Run(async () => await Try.IgnoringError(async () => await tryClearRequestFromQueueAtMostOnce.Task)); + var backgroundCancellation = Task.Run(async () => await SendCancellationIfRequestWasCancelled(request, pending)); + } + + return pending.Response!; + } + + async Task SendCancellationIfRequestWasCancelled(RequestMessage request, PendingRequest pending) + { + if (pending.PendingRequestCancellationToken.IsCancellationRequested) + { + // TODO log + await WatchForRequestCancellation.TrySendCancellation(halibutRedisTransport, endpoint, request, log); + } + else + { + // TODO log + } + } - var watchProcessingNodeTask = Task.Run(async () => + Task WatchProcessingNodeIsStillConnectedInBackground(RequestMessage request, PendingRequest pending, CancellationTokenSourceAsyncDisposable watcherCts) + { + return Task.Run(async () => { - var disconnected = await ProcessingNodeHeartBeatSender.WaitUntilNodeProcessingRequestFlatLines(endpoint, request, pending, halibutRedisTransport, log, watcherCts.CancellationToken); + var disconnected = await ProcessingNodeHeartBeatSender.WaitUntilNodeProcessingRequestFlatLines(endpoint, request, pending, halibutRedisTransport, log, MaxTimeBetweenHeartBeetsBeforeProcessingNodeIsAssumedToBeOffline, watcherCts.CancellationToken); if (!watcherCts.CancellationToken.IsCancellationRequested && disconnected == ProcessingNodeHeartBeatSender.NodeProcessingRequestWatcherResult.ProcessingNodeIsLikelyDisconnected) { // TODO: if(responseWatcher.CheckForResponseNow() == ResponseNotFound) { - pending.SetResponse(ResponseMessage.FromError(request, "The node processing the request did not send a heartbeat for long enough, and so the node is now assumed to be offline.")); + pending.SetResponse(ResponseMessage.FromError(request, "The node processing the request did not send a heartbeat for long enough, and so the node is now assumed to be offline.")); //} - } + } }); - - await pending.WaitUntilComplete(() => TryClearRequestFromQueue(request, pending), cancellationToken); - - return pending.Response!; } - async Task TryClearRequestFromQueue(RequestMessage request, PendingRequest pending) + async Task TryClearRequestFromQueue(RequestMessage request, PendingRequest pending) { log.Write(EventType.Diagnostic, "Attempting to clear request {0} from queue for endpoint {1}", request.ActivityId, endpoint); @@ -126,12 +147,18 @@ async Task TryClearRequestFromQueue(RequestMessage request, PendingRequest pendi // - We could not pop it, which means it was collected. try { + if (pending.HasRequestBeenMarkedAsCollected) + { + // TODO: log + return false; + } using var cts = new CancellationTokenSource(); cts.CancelAfter(TimeSpan.FromMinutes(2)); // Best efforts. var requestJson = await halibutRedisTransport.TryGetAndRemoveRequest(endpoint, request.ActivityId, cts.Token); if (requestJson != null) { log.Write(EventType.Diagnostic, "Successfully removed request {0} from queue - request was never collected by a processing node", request.ActivityId); + return true; } else { @@ -143,6 +170,7 @@ async Task TryClearRequestFromQueue(RequestMessage request, PendingRequest pendi { log.WriteException(EventType.Error, "Failed to clear request {0} from queue for endpoint {1}", ex, request.ActivityId, endpoint); } + return false; } @@ -183,10 +211,14 @@ async Task SubscribeToResponse(Guid activityId, public bool IsEmpty => throw new NotImplementedException(); public int Count => throw new NotImplementedException(); - + public TimeSpan MaxTimeBetweenHeartBeetsBeforeProcessingNodeIsAssumedToBeOffline { get; set; } = ProcessingNodeHeartBeatSender.DefaultMaxTimeBetweenHeartBeetsBeforeProcessingNodeIsAssumedToBeOffline; public async Task DequeueAsync(CancellationToken cancellationToken) { + // TODO: is it god or bad that redis exceptions will bubble out of here. + // I think it will kill the TCP connection, which will force re-connect (in perhaps a backoff function) + // This could result in connecting to a node that is actually connected to redis. It could also + // cause a cascade of failure from high load. var pending = await DequeueNextAsync(); if (pending == null) return null; diff --git a/source/Halibut/ServiceModel/PendingRequestV2.cs b/source/Halibut/ServiceModel/PendingRequestV2.cs index 84154160e..77e7c169e 100644 --- a/source/Halibut/ServiceModel/PendingRequestV2.cs +++ b/source/Halibut/ServiceModel/PendingRequestV2.cs @@ -158,16 +158,12 @@ public async Task RequestHasBeenCollectedAndWillBeTransferred() { // Check if the request has already been completed or if the request has been cancelled // to ensure we don't dequeue an already completed or already cancelled request - if (requestCollected.IsSet - || pendingRequestCancellationTokenSource.IsCancellationRequested - || responseWaiter.IsSet) - - { - return false; - } + var requestHasBeenCollected = this.requestCollected.IsSet; requestCollected.Set(); - return true; + return !requestHasBeenCollected + && !responseWaiter.IsSet + && !pendingRequestCancellationTokenSource.IsCancellationRequested; } } catch (ObjectDisposedException) From 6699b9a4cb85fb3e38c9a9a2f84e76cf8001ec2d Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Wed, 6 Aug 2025 07:14:10 +1000 Subject: [PATCH 021/137] . --- source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index 0510825ad..c84016104 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -85,8 +85,6 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can // Start listening for a response to the request, we don't want to miss the response. await using var _ = await SubscribeToResponse(request.ActivityId, pending.SetResponse, cancellationToken); - // These are not guaranteed to execute, we should ensure they are executed in a finally block. - var tryClearRequestFromQueueAtMostOnce = new AsyncLazy(async () => await TryClearRequestFromQueue(request, pending)); try { @@ -96,7 +94,7 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can await halibutRedisTransport.PulseRequestPushedToEndpoint(endpoint, cancellationToken); await using var watcherCts = CancellationTokenSource.CreateLinkedTokenSource(cts.Token).CancelOnDispose(); - var watchProcessingNodeTask = WatchProcessingNodeIsStillConnectedInBackground(request, pending, watcherCts); + WatchProcessingNodeIsStillConnectedInBackground(request, pending, watcherCts); await pending.WaitUntilComplete(async () => await tryClearRequestFromQueueAtMostOnce.Task, cancellationToken); } @@ -123,9 +121,9 @@ async Task SendCancellationIfRequestWasCancelled(RequestMessage request, Pending } } - Task WatchProcessingNodeIsStillConnectedInBackground(RequestMessage request, PendingRequest pending, CancellationTokenSourceAsyncDisposable watcherCts) + void WatchProcessingNodeIsStillConnectedInBackground(RequestMessage request, PendingRequest pending, CancellationTokenSourceAsyncDisposable watcherCts) { - return Task.Run(async () => + Task.Run(async () => { var disconnected = await ProcessingNodeHeartBeatSender.WaitUntilNodeProcessingRequestFlatLines(endpoint, request, pending, halibutRedisTransport, log, MaxTimeBetweenHeartBeetsBeforeProcessingNodeIsAssumedToBeOffline, watcherCts.CancellationToken); if (!watcherCts.CancellationToken.IsCancellationRequested && disconnected == ProcessingNodeHeartBeatSender.NodeProcessingRequestWatcherResult.ProcessingNodeIsLikelyDisconnected) From e4d10772805fb4dcd39b17df9205091440243245 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Wed, 6 Aug 2025 12:35:59 +1000 Subject: [PATCH 022/137] Now support cancelling in flight requests when the sender is offline --- .../Builders/RequestMessageBuilder.cs | 11 +- .../Redis/RedisPendingRequestQueueFixture.cs | 59 ++++++++- .../Queue/Redis/HalibutRedisTransport.cs | 28 +++- ...rtBeatSender.cs => NodeHeartBeatSender.cs} | 125 +++++++++++------- .../Queue/Redis/RedisPendingRequestQueue.cs | 48 +++++-- ...orRequestCancellationOrSenderDisconnect.cs | 91 +++++++++++++ .../Halibut/ServiceModel/PendingRequestV2.cs | 16 +-- 7 files changed, 293 insertions(+), 85 deletions(-) rename source/Halibut/Queue/Redis/{ProcessingNodeHeartBeatSender.cs => NodeHeartBeatSender.cs} (56%) create mode 100644 source/Halibut/Queue/Redis/WatchForRequestCancellationOrSenderDisconnect.cs diff --git a/source/Halibut.Tests/Builders/RequestMessageBuilder.cs b/source/Halibut.Tests/Builders/RequestMessageBuilder.cs index 9e3731738..8b5e08b8e 100644 --- a/source/Halibut.Tests/Builders/RequestMessageBuilder.cs +++ b/source/Halibut.Tests/Builders/RequestMessageBuilder.cs @@ -6,6 +6,7 @@ namespace Halibut.Tests.Builders public class RequestMessageBuilder { readonly ServiceEndPointBuilder serviceEndPointBuilder = new(); + private Guid? activityId; public RequestMessageBuilder(string endpoint) { @@ -18,6 +19,12 @@ public RequestMessageBuilder WithServiceEndpoint(Action return this; } + public RequestMessageBuilder WithActivityId(Guid activityId) + { + this.activityId = activityId; + return this; + } + public RequestMessage Build() { var serviceEndPoint = serviceEndPointBuilder.Build(); @@ -26,11 +33,9 @@ public RequestMessage Build() { Id = Guid.NewGuid().ToString(), Destination = serviceEndPoint, - ActivityId = Guid.NewGuid(), + ActivityId = activityId ?? Guid.NewGuid(), }; - - return request; } } diff --git a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs index fc0cc55e4..cfe370f71 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs @@ -239,7 +239,6 @@ public async Task WhenTheReceiverConnectionToRedisIsInterruptedAndRestoredBefore var halibutTimeoutAndLimits = new HalibutTimeoutsAndLimits(); halibutTimeoutAndLimits.PollingQueueWaitTimeout = TimeSpan.FromDays(1); // We should not need to rely on the timeout working for very short disconnects. - var node1Sender = new RedisPendingRequestQueue(endpoint, log, new HalibutRedisTransport(redisFacadeSender), messageReaderWriter, halibutTimeoutAndLimits); var node2Receiver = new RedisPendingRequestQueue(endpoint, log, new HalibutRedisTransport(redisFacadeReceiver), messageReaderWriter, halibutTimeoutAndLimits); @@ -310,6 +309,7 @@ public async Task WhenTheSendersConnectionToRedisIsBrieflyInterruptedWhileSendin dequeuedRequest.Should().NotBeNull(); dequeuedRequest!.RequestMessage.ActivityId.Should().Be(request.ActivityId); } + [Test] public async Task WhenTheReceiverDequeuesWorkAndThenDisconnectsFromRedisForEver_TheSenderEventuallyTimesOut() @@ -336,8 +336,10 @@ public async Task WhenTheReceiverDequeuesWorkAndThenDisconnectsFromRedisForEver_ var node2Receiver = new RedisPendingRequestQueue(endpoint, log, new HalibutRedisTransport(unstableRedisConnection), messageReaderWriter, halibutTimeoutAndLimits); // Lower this to complete the test sooner. - node1Sender.MaxTimeBetweenHeartBeetsBeforeProcessingNodeIsAssumedToBeOffline = TimeSpan.FromSeconds(30); - node2Receiver.MaxTimeBetweenHeartBeetsBeforeProcessingNodeIsAssumedToBeOffline = TimeSpan.FromSeconds(30); + node1Sender.DelayBetweenHeartBeatsForReceiver = TimeSpan.FromSeconds(1); + node2Receiver.DelayBetweenHeartBeatsForReceiver = TimeSpan.FromSeconds(1); + node1Sender.NodeOfflineTimeoutBetweenHeartBeatsFromReceiver = TimeSpan.FromSeconds(10); + node2Receiver.NodeOfflineTimeoutBetweenHeartBeatsFromReceiver = TimeSpan.FromSeconds(10); var request = new RequestMessageBuilder("poll://test-endpoint").Build(); @@ -351,7 +353,7 @@ public async Task WhenTheReceiverDequeuesWorkAndThenDisconnectsFromRedisForEver_ // Now disconnect the receiver from redis. portForwarder.EnterKillNewAndExistingConnectionsMode(); - await Task.WhenAny(Task.Delay(TimeSpan.FromMinutes(2), CancellationToken), queueAndWaitTask); + await Task.WhenAny(Task.Delay(TimeSpan.FromSeconds(20), CancellationToken), queueAndWaitTask); queueAndWaitTask.IsCompleted.Should().BeTrue(); @@ -360,6 +362,51 @@ public async Task WhenTheReceiverDequeuesWorkAndThenDisconnectsFromRedisForEver_ response.Error!.Message.Should().Contain("The node processing the request did not send a heartbeat for long enough, and so the node is now assumed to be offline."); } + [Test] + public async Task WhenTheReceiverDequeuesWork_AndTheSenderDisconnects_AndNeverReconnects_TheDequeuedWorkIsEventuallyCancelled() + { + // Arrange + var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); + var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); + var guid = Guid.NewGuid(); + + var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); + var messageSerializer = new QueueMessageSerializerBuilder().Build(); + var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); + + var halibutTimeoutAndLimits = new HalibutTimeoutsAndLimits(); + halibutTimeoutAndLimits.PollingRequestQueueTimeout = TimeSpan.FromDays(1); + halibutTimeoutAndLimits.PollingQueueWaitTimeout = TimeSpan.FromDays(1); // We should not need to rely on the timeout working for very short disconnects. + + await using var stableRedisConnection = CreateRedisFacade(guid: guid); + + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); + await using var unstableRedisConnection = CreateRedisFacade(portForwarder.ListeningPort, guid: guid); + + var node1Sender = new RedisPendingRequestQueue(endpoint, log, new HalibutRedisTransport(unstableRedisConnection), messageReaderWriter, halibutTimeoutAndLimits); + var node2Receiver = new RedisPendingRequestQueue(endpoint, log, new HalibutRedisTransport(stableRedisConnection), messageReaderWriter, halibutTimeoutAndLimits); + + node1Sender.DelayBetweenHeartBeatsForSender= TimeSpan.FromSeconds(1); + node2Receiver.DelayBetweenHeartBeatsForSender= TimeSpan.FromSeconds(1); + node1Sender.NodeOfflineTimeoutBetweenHeartBeatsFromSender = TimeSpan.FromSeconds(15); + node2Receiver.NodeOfflineTimeoutBetweenHeartBeatsFromSender = TimeSpan.FromSeconds(15); + + var request = new RequestMessageBuilder("poll://test-endpoint").Build(); + + var queueAndWaitTask = node1Sender.QueueAndWaitAsync(request, CancellationToken.None); + + var dequeuedRequest = await node2Receiver.DequeueAsync(CancellationToken); + dequeuedRequest!.CancellationToken.IsCancellationRequested.Should().BeFalse(); + + // Now disconnect the sender from redis. + portForwarder.EnterKillNewAndExistingConnectionsMode(); + + await Task.WhenAny(Task.Delay(TimeSpan.FromMinutes(2), dequeuedRequest.CancellationToken)); + + + dequeuedRequest.CancellationToken.IsCancellationRequested.Should().BeTrue(); + } + [Test] public async Task WhenTheSenderDisconnectsFromRedisRightWhenTheReceiverSendsTheResponseBack_TheSenderStillGetsTheResponse() { @@ -462,11 +509,9 @@ public async Task CancellingARequestShouldResultInTheDequeuedResponseTokenBeingC try { - await Task.Delay(TimeSpan.FromMinutes(1), requestMessageWithCancellationToken.CancellationToken); + await Task.Delay(TimeSpan.FromSeconds(10), requestMessageWithCancellationToken.CancellationToken); } catch (TaskCanceledException){} - - requestMessageWithCancellationToken!.CancellationToken.IsCancellationRequested.Should().BeTrue(); } diff --git a/source/Halibut/Queue/Redis/HalibutRedisTransport.cs b/source/Halibut/Queue/Redis/HalibutRedisTransport.cs index 1c561a0d9..aaf6b0fb3 100644 --- a/source/Halibut/Queue/Redis/HalibutRedisTransport.cs +++ b/source/Halibut/Queue/Redis/HalibutRedisTransport.cs @@ -163,18 +163,19 @@ public async Task IsRequestMarkedAsCancelled(Uri endpoint, Guid requestId, // Node Processing the request heart beat channel - static string NodeProcessingTheRequestHeartBeatChannel(Uri endpoint, Guid requestId) + static string NodeProcessingTheRequestHeartBeatChannel(Uri endpoint, Guid requestId, HalibutQueueNodeSendingPulses nodeSendingPulsesType) { - return $"{Namespace}::NodeProcessingTheRequestHeartBeatChannel::{endpoint}::{requestId}"; + return $"{Namespace}::NodeProcessingTheRequestHeartBeatChannel::{endpoint}::{requestId}::{nodeSendingPulsesType}"; } - public async Task SubscribeToNodeProcessingTheRequestHeartBeatChannel( + public async Task SubscribeToNodeHeartBeatChannel( Uri endpoint, Guid request, + HalibutQueueNodeSendingPulses nodeSendingPulsesType, Func onHeartBeat, CancellationToken cancellationToken) { - var channelName = NodeProcessingTheRequestHeartBeatChannel(endpoint, request); + var channelName = NodeProcessingTheRequestHeartBeatChannel(endpoint, request, nodeSendingPulsesType); return await facade.SubscribeToChannel(channelName, async foo => { string? response = foo.Message; @@ -182,11 +183,26 @@ public async Task SubscribeToNodeProcessingTheRequestHeartBeat }, cancellationToken); } - public async Task SendHeartBeatFromNodeProcessingTheRequest(Uri endpoint, Guid requestId, CancellationToken cancellationToken) + public async Task SendHeartBeatFromNodeProcessingTheRequest(Uri endpoint, Guid requestId, HalibutQueueNodeSendingPulses nodeSendingPulsesType, CancellationToken cancellationToken) { - var channelName = NodeProcessingTheRequestHeartBeatChannel(endpoint, requestId); + var channelName = NodeProcessingTheRequestHeartBeatChannel(endpoint, requestId, nodeSendingPulsesType); await facade.PublishToChannel(channelName, "{}"); } + + // Backward compatibility methods (defaulting to Receiver for existing code) + public async Task SubscribeToNodeProcessingTheRequestHeartBeatChannel( + Uri endpoint, + Guid request, + Func onHeartBeat, + CancellationToken cancellationToken) + { + return await SubscribeToNodeHeartBeatChannel(endpoint, request, HalibutQueueNodeSendingPulses.Receiver, onHeartBeat, cancellationToken); + } + + public async Task SendHeartBeatFromNodeProcessingTheRequest(Uri endpoint, Guid requestId, CancellationToken cancellationToken) + { + await SendHeartBeatFromNodeProcessingTheRequest(endpoint, requestId, HalibutQueueNodeSendingPulses.Receiver, cancellationToken); + } // Generic methods for watching for any string value being set diff --git a/source/Halibut/Queue/Redis/ProcessingNodeHeartBeatSender.cs b/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs similarity index 56% rename from source/Halibut/Queue/Redis/ProcessingNodeHeartBeatSender.cs rename to source/Halibut/Queue/Redis/NodeHeartBeatSender.cs index 7c3ea291b..5ce7cce37 100644 --- a/source/Halibut/Queue/Redis/ProcessingNodeHeartBeatSender.cs +++ b/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs @@ -13,6 +13,7 @@ // limitations under the License. using System; +using System.Security.Cryptography; using System.Threading; using System.Threading.Tasks; using Halibut.Diagnostics; @@ -22,69 +23,77 @@ namespace Halibut.Queue.Redis { - public class ProcessingNodeHeartBeatSender : IAsyncDisposable + public enum HalibutQueueNodeSendingPulses + { + Sender, + Receiver + } + public class NodeHeartBeatSender : IAsyncDisposable { public enum NodeProcessingRequestWatcherResult { - ProcessingNodeIsLikelyDisconnected, - NoIssue // TODO: this name is so bad the reviewer will be forced to think of a better one. + NodeMayHaveDisconnected, + NoDisconnectSeen } - + private readonly Uri endpoint; private readonly Guid requestActivityId; private readonly HalibutRedisTransport halibutRedisTransport; private readonly CancellationTokenSource cancellationTokenSource; private readonly ILog log; - - public static readonly TimeSpan DefaultMaxTimeBetweenHeartBeetsBeforeProcessingNodeIsAssumedToBeOffline = TimeSpan.FromSeconds(60); - + private readonly HalibutQueueNodeSendingPulses nodeSendingPulsesType; - public ProcessingNodeHeartBeatSender(Uri endpoint, + internal Task TaskSendingPulses; + public NodeHeartBeatSender( + Uri endpoint, Guid requestActivityId, HalibutRedisTransport halibutRedisTransport, - ILog log) + ILog log, + HalibutQueueNodeSendingPulses nodeSendingPulsesType, + TimeSpan defaultDelayBetweenPulses) { this.endpoint = endpoint; this.requestActivityId = requestActivityId; this.halibutRedisTransport = halibutRedisTransport; - this.cancellationTokenSource = new CancellationTokenSource(); + this.nodeSendingPulsesType = nodeSendingPulsesType; + cancellationTokenSource = new CancellationTokenSource(); this.log = log; - log.Write(EventType.Diagnostic, "Starting ProcessingNodeHeartBeatSender for request {0} to endpoint {1}", requestActivityId, endpoint); - Task.Run(() => SendPulsesWhileProcessingRequest(cancellationTokenSource.Token)); + log.Write(EventType.Diagnostic, "Starting NodeHeartBeatSender for {0} node, request {1}, endpoint {2}", nodeSendingPulsesType, requestActivityId, endpoint); + TaskSendingPulses = Task.Run(() => SendPulsesWhileProcessingRequest(defaultDelayBetweenPulses, cancellationTokenSource.Token)); } - private async Task SendPulsesWhileProcessingRequest(CancellationToken cancellationToken) + private async Task SendPulsesWhileProcessingRequest(TimeSpan defaultDelayBetweenPulses, CancellationToken cancellationToken) { - log.Write(EventType.Diagnostic, "Starting heartbeat pulse loop for request {0}", requestActivityId); + log.Write(EventType.Diagnostic, "Starting heartbeat pulse loop for {0} node, request {1}", nodeSendingPulsesType, requestActivityId); TimeSpan delayBetweenPulse; while (!cancellationToken.IsCancellationRequested) { try { - await halibutRedisTransport.SendHeartBeatFromNodeProcessingTheRequest(endpoint, requestActivityId, cancellationToken); - delayBetweenPulse = TimeSpan.FromSeconds(15); - log.Write(EventType.Diagnostic, "Successfully sent heartbeat for request {0}, next pulse in {1} seconds", requestActivityId, delayBetweenPulse.TotalSeconds); + await halibutRedisTransport.SendHeartBeatFromNodeProcessingTheRequest(endpoint, requestActivityId, nodeSendingPulsesType, cancellationToken); + delayBetweenPulse = defaultDelayBetweenPulses; + log.Write(EventType.Diagnostic, "Successfully sent heartbeat for {0} node, request {1}, next pulse in {2} seconds", nodeSendingPulsesType, requestActivityId, delayBetweenPulse.TotalSeconds); } catch (Exception ex) { if(cancellationToken.IsCancellationRequested) { - log.Write(EventType.Diagnostic, "Heartbeat pulse loop cancelled for request {0}", requestActivityId); + log.Write(EventType.Diagnostic, "Heartbeat pulse loop cancelled for {0} node, request {1}", nodeSendingPulsesType, requestActivityId); return; } // Panic send pulses. - delayBetweenPulse = TimeSpan.FromSeconds(7); - log.WriteException(EventType.Diagnostic, "Failed to send heartbeat for request {0}, switching to panic mode with {1} second intervals", ex, requestActivityId, delayBetweenPulse.TotalSeconds); + delayBetweenPulse = defaultDelayBetweenPulses / 2; + log.WriteException(EventType.Diagnostic, "Failed to send heartbeat for {0} node, request {1}, switching to panic mode with {2} second intervals", ex, nodeSendingPulsesType, requestActivityId, delayBetweenPulse.TotalSeconds); } await Try.IgnoringError(async () => await Task.Delay(delayBetweenPulse, cancellationToken)); } - log.Write(EventType.Diagnostic, "Heartbeat pulse loop ended for request {0}", requestActivityId); + log.Write(EventType.Diagnostic, "Heartbeat pulse loop ended for {0} node, request {1}", nodeSendingPulsesType, requestActivityId); } - public static async Task WaitUntilNodeProcessingRequestFlatLines( + public static async Task WatchThatNodeProcessingTheRequestIsStillAlive( Uri endpoint, RequestMessage request, PendingRequest pending, @@ -93,54 +102,72 @@ public static async Task WaitUntilNodeProces TimeSpan maxTimeBetweenHeartBeetsBeforeProcessingNodeIsAssumedToBeOffline, CancellationToken watchCancellationToken) { - log.Write(EventType.Diagnostic, "Starting to watch for processing node flatline for request {0} to endpoint {1}", request.ActivityId, endpoint); - - // Once the pending's CT has been cancelled we no longer care to keep observing + // Once the pending's CT has been cancelled we no longer care to keep observing using var cts = CancellationTokenSource.CreateLinkedTokenSource(watchCancellationToken, pending.PendingRequestCancellationToken); + // TODO: test this is indeed called first. + await WaitForRequestToBeCollected(endpoint, request, pending, halibutRedisTransport, log, cts.Token); + + return await WatchForPulsesFromNode(endpoint, request.ActivityId, halibutRedisTransport, log, maxTimeBetweenHeartBeetsBeforeProcessingNodeIsAssumedToBeOffline, HalibutQueueNodeSendingPulses.Receiver, cts.Token); + } - DateTimeOffset? lastHeartBeat = null; + public static async Task WatchThatNodeWhichSentTheRequestIsStillAlive( + Uri endpoint, + Guid requestActivityId, + HalibutRedisTransport halibutRedisTransport, + ILog log, + TimeSpan maxTimeBetweenSenderHeartBeetsBeforeSenderIsAssumedToBeOffline, + CancellationToken watchCancellationToken) + { + return await WatchForPulsesFromNode(endpoint, requestActivityId, halibutRedisTransport, log, maxTimeBetweenSenderHeartBeetsBeforeSenderIsAssumedToBeOffline, HalibutQueueNodeSendingPulses.Sender, watchCancellationToken); + } + + private static async Task WatchForPulsesFromNode( + Uri endpoint, + Guid requestActivityId, + HalibutRedisTransport halibutRedisTransport, + ILog log, + TimeSpan maxTimeBetweenHeartBeetsBeforeProcessingNodeIsAssumedToBeOffline, + HalibutQueueNodeSendingPulses watchingForPulsesFrom, + CancellationToken watchCancellationToken) + { + log.Write(EventType.Diagnostic, "Starting to watch for {0} node flatline, request {1}, endpoint {2}", watchingForPulsesFrom, requestActivityId, endpoint); + + DateTimeOffset? lastHeartBeat = DateTimeOffset.Now; try { - await using var subscription = await halibutRedisTransport.SubscribeToNodeProcessingTheRequestHeartBeatChannel( + await using var subscription = await halibutRedisTransport.SubscribeToNodeHeartBeatChannel( endpoint, - request.ActivityId, async () => + requestActivityId, + watchingForPulsesFrom, async () => { await Task.CompletedTask; lastHeartBeat = DateTimeOffset.Now; - log.Write(EventType.Diagnostic, "Received heartbeat for request {0} from processing node", request.ActivityId); - }, cts.Token); + log.Write(EventType.Diagnostic, "Received heartbeat from {0} node, request {1}", watchingForPulsesFrom, requestActivityId); + }, watchCancellationToken); - await WaitForRequestToBeCollected(endpoint, request, pending, halibutRedisTransport, log, cts.Token); - - // When the request was collected is a good enough heart beat. - if (lastHeartBeat == null || lastHeartBeat.Value < DateTimeOffset.Now) - { - lastHeartBeat = DateTimeOffset.Now; - log.Write(EventType.Diagnostic, "Using request collection time as heartbeat for request {0}", request.ActivityId); - } - - while (!cts.Token.IsCancellationRequested) + while (!watchCancellationToken.IsCancellationRequested) { // TODO: I am sure a fancy pants delay could be done here calculated from the now and last heart beat etc. - await Try.IgnoringError(async () => await Task.Delay(TimeSpan.FromSeconds(10), cts.Token)); + await Try.IgnoringError(async () => await Task.Delay(TimeSpan.FromSeconds(10), watchCancellationToken)); var timeSinceLastHeartBeat = DateTimeOffset.Now - lastHeartBeat.Value; if (timeSinceLastHeartBeat > maxTimeBetweenHeartBeetsBeforeProcessingNodeIsAssumedToBeOffline) { - log.Write(EventType.Diagnostic, "Processing node appears disconnected for request {0}, last heartbeat was {1} seconds ago", request.ActivityId, timeSinceLastHeartBeat.TotalSeconds); - return NodeProcessingRequestWatcherResult.ProcessingNodeIsLikelyDisconnected; + log.Write(EventType.Diagnostic, "{0} node appears disconnected, request {1}, last heartbeat was {2} seconds ago", watchingForPulsesFrom, requestActivityId, timeSinceLastHeartBeat.TotalSeconds); + return NodeProcessingRequestWatcherResult.NodeMayHaveDisconnected; } } - log.Write(EventType.Diagnostic, "Processing node watcher cancelled for request {0}", request.ActivityId); - return NodeProcessingRequestWatcherResult.NoIssue; + log.Write(EventType.Diagnostic, "{0} node watcher cancelled, request {1}", watchingForPulsesFrom, requestActivityId); + return NodeProcessingRequestWatcherResult.NoDisconnectSeen; } - catch (Exception ex) + catch (Exception ex) when (!watchCancellationToken.IsCancellationRequested) { - log.WriteException(EventType.Diagnostic, "Error while watching processing node for request {0}", ex, request.ActivityId); + log.WriteException(EventType.Diagnostic, "Error while watching {0} node, request {1}", ex, watchingForPulsesFrom, requestActivityId); throw; } } + static async Task WaitForRequestToBeCollected(Uri endpoint, RequestMessage request, PendingRequest pending, HalibutRedisTransport halibutRedisTransport, ILog log, CancellationToken cancellationToken) { @@ -181,12 +208,12 @@ static async Task WaitForRequestToBeCollected(Uri endpoint, RequestMessage reque public async ValueTask DisposeAsync() { - log.Write(EventType.Diagnostic, "Disposing ProcessingNodeHeartBeatSender for request {0}", requestActivityId); + log.Write(EventType.Diagnostic, "Disposing NodeHeartBeatSender for {0} node, request {1}", nodeSendingPulsesType, requestActivityId); await Try.IgnoringError(async () => await cancellationTokenSource.CancelAsync()); Try.IgnoringError(() => cancellationTokenSource.Dispose()); - log.Write(EventType.Diagnostic, "ProcessingNodeHeartBeatSender disposed for request {0}", requestActivityId); + log.Write(EventType.Diagnostic, "NodeHeartBeatSender disposed for {0} node, request {1}", nodeSendingPulsesType, requestActivityId); } } } \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index c84016104..eb61e613d 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -88,6 +88,7 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can var tryClearRequestFromQueueAtMostOnce = new AsyncLazy(async () => await TryClearRequestFromQueue(request, pending)); try { + await using var senderPulse = new NodeHeartBeatSender(endpoint, request.ActivityId, halibutRedisTransport, log, HalibutQueueNodeSendingPulses.Sender, DelayBetweenHeartBeatsForSender); // Make the request available before we tell people it is available. await halibutRedisTransport.PutRequest(endpoint, request.ActivityId, payload, cancellationToken); await halibutRedisTransport.PushRequestGuidOnToQueue(endpoint, request.ActivityId, cancellationToken); @@ -125,12 +126,24 @@ void WatchProcessingNodeIsStillConnectedInBackground(RequestMessage request, Pen { Task.Run(async () => { - var disconnected = await ProcessingNodeHeartBeatSender.WaitUntilNodeProcessingRequestFlatLines(endpoint, request, pending, halibutRedisTransport, log, MaxTimeBetweenHeartBeetsBeforeProcessingNodeIsAssumedToBeOffline, watcherCts.CancellationToken); - if (!watcherCts.CancellationToken.IsCancellationRequested && disconnected == ProcessingNodeHeartBeatSender.NodeProcessingRequestWatcherResult.ProcessingNodeIsLikelyDisconnected) + var watcherCtsCancellationToken = watcherCts.CancellationToken; + try + { + var disconnected = await NodeHeartBeatSender.WatchThatNodeProcessingTheRequestIsStillAlive(endpoint, request, pending, halibutRedisTransport, log, NodeOfflineTimeoutBetweenHeartBeatsFromReceiver, watcherCtsCancellationToken); + if (!watcherCtsCancellationToken.IsCancellationRequested && disconnected == NodeHeartBeatSender.NodeProcessingRequestWatcherResult.NodeMayHaveDisconnected) + { + // TODO: if(responseWatcher.CheckForResponseNow() == ResponseNotFound) { + pending.SetResponse(ResponseMessage.FromError(request, "The node processing the request did not send a heartbeat for long enough, and so the node is now assumed to be offline.")); + //} + } + } + catch (Exception) when (watcherCtsCancellationToken.IsCancellationRequested) { - // TODO: if(responseWatcher.CheckForResponseNow() == ResponseNotFound) { - pending.SetResponse(ResponseMessage.FromError(request, "The node processing the request did not send a heartbeat for long enough, and so the node is now assumed to be offline.")); - //} + log.Write(EventType.Diagnostic, "Processing node watcher cancelled for request {0}, endpoint {1}", request.ActivityId, endpoint); + } + catch (Exception ex) + { + log.WriteException(EventType.Error, "Error watching processing node for request {0}, endpoint {1}", ex, request.ActivityId, endpoint); } }); } @@ -209,8 +222,20 @@ async Task SubscribeToResponse(Guid activityId, public bool IsEmpty => throw new NotImplementedException(); public int Count => throw new NotImplementedException(); - public TimeSpan MaxTimeBetweenHeartBeetsBeforeProcessingNodeIsAssumedToBeOffline { get; set; } = ProcessingNodeHeartBeatSender.DefaultMaxTimeBetweenHeartBeetsBeforeProcessingNodeIsAssumedToBeOffline; + // Setting this too high means things above the RPC might not have time to retry. + public TimeSpan NodeOfflineTimeoutBetweenHeartBeatsFromReceiver { get; set; } = TimeSpan.FromSeconds(60); + // The timespan is more generous for the sender going offline, since if it does go offline, + // since under some cases the request completing is advantageous. That node needs to + // re-do the entire RPC for idempotent RPCs this might mean that the task required is already done. + internal TimeSpan NodeOfflineTimeoutBetweenHeartBeatsFromSender { get; set; } = TimeSpan.FromSeconds(90); + + internal TimeSpan DelayBetweenHeartBeatsForSender { get; set; } = TimeSpan.FromSeconds(15); + + internal TimeSpan DelayBetweenHeartBeatsForReceiver { get; set; } = TimeSpan.FromSeconds(15); + + + public async Task DequeueAsync(CancellationToken cancellationToken) { // TODO: is it god or bad that redis exceptions will bubble out of here. @@ -224,10 +249,10 @@ async Task SubscribeToResponse(Guid activityId, var disposables = new DisposableCollection(); try { - var watchForRequestCancellation = new WatchForRequestCancellation(endpoint, pending.ActivityId, halibutRedisTransport, log); - disposables.AddAsyncDisposable(watchForRequestCancellation); - disposables.AddAsyncDisposable(new ProcessingNodeHeartBeatSender(endpoint, pending.ActivityId, halibutRedisTransport, log)); - var response = new RequestMessageWithCancellationToken(pending, watchForRequestCancellation.RequestCancelledCancellationToken); + disposables.AddAsyncDisposable(new NodeHeartBeatSender(endpoint, pending.ActivityId, halibutRedisTransport, log, HalibutQueueNodeSendingPulses.Receiver, DelayBetweenHeartBeatsForReceiver)); + + var watcher = new WatchForRequestCancellationOrSenderDisconnect(endpoint, pending.ActivityId, halibutRedisTransport, NodeOfflineTimeoutBetweenHeartBeatsFromSender, log); + var response = new RequestMessageWithCancellationToken(pending, watcher.RequestProcessingCancellationToken); disposablesForInFlightRequests[pending.ActivityId] = disposables; return response; } @@ -238,8 +263,6 @@ async Task SubscribeToResponse(Guid activityId, } } - - public async Task ApplyResponse(ResponseMessage response, Guid requestActivityId) { if (response == null) return; @@ -307,6 +330,7 @@ await Task.WhenAny( } var request = await messageReaderWriter.ReadRequest(jsonRequest, cancellationToken); + log.Write(EventType.Diagnostic, "Successfully collected request {0} from queue for endpoint {1}", request.ActivityId, endpoint); return request; } diff --git a/source/Halibut/Queue/Redis/WatchForRequestCancellationOrSenderDisconnect.cs b/source/Halibut/Queue/Redis/WatchForRequestCancellationOrSenderDisconnect.cs new file mode 100644 index 000000000..6d9a3820a --- /dev/null +++ b/source/Halibut/Queue/Redis/WatchForRequestCancellationOrSenderDisconnect.cs @@ -0,0 +1,91 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; +using System.Security.Policy; +using System.Threading; +using System.Threading.Tasks; +using Halibut.Diagnostics; +using Halibut.Util; + +namespace Halibut.Queue.Redis +{ + + public class WatchForRequestCancellationOrSenderDisconnect : IAsyncDisposable + { + readonly CancellationTokenSource RequestCancellationTokenSource; + public CancellationToken RequestProcessingCancellationToken { get; } + readonly WatchForRequestCancellation watchForRequestCancellation; + + readonly CancellationTokenSource KeepWatchingCancellationTokenSource; + + DisposableCollection disposableCollection = new DisposableCollection(); + + public WatchForRequestCancellationOrSenderDisconnect( + Uri endpoint, + Guid requestActivityId, + HalibutRedisTransport halibutRedisTransport, + TimeSpan nodeOfflineTimeoutBetweenHeartBeatsFromSender, + ILog log) + { + try + { + watchForRequestCancellation = new WatchForRequestCancellation(endpoint, requestActivityId, halibutRedisTransport, log); + disposableCollection.AddAsyncDisposable(watchForRequestCancellation); + + RequestCancellationTokenSource = CancellationTokenSource.CreateLinkedTokenSource(watchForRequestCancellation.RequestCancelledCancellationToken); + disposableCollection.Add(RequestCancellationTokenSource); + RequestProcessingCancellationToken = RequestCancellationTokenSource.Token; + + KeepWatchingCancellationTokenSource = new CancellationTokenSource(); + disposableCollection.Add(KeepWatchingCancellationTokenSource); + + Task.Run(() => WatchThatNodeWhichSentTheRequestIsStillAlive(endpoint, requestActivityId, halibutRedisTransport, nodeOfflineTimeoutBetweenHeartBeatsFromSender, log)); + } + catch (Exception) + { + Try.IgnoringError(async () => await disposableCollection.DisposeAsync()).GetAwaiter().GetResult(); + throw; + } + + } + + async Task WatchThatNodeWhichSentTheRequestIsStillAlive(Uri endpoint, Guid requestActivityId, HalibutRedisTransport halibutRedisTransport, TimeSpan nodeOfflineTimeoutBetweenHeartBeatsFromSender, ILog log) + { + var watchCancellationToken = KeepWatchingCancellationTokenSource.Token; + try + { + var res = await NodeHeartBeatSender + .WatchThatNodeWhichSentTheRequestIsStillAlive(endpoint, requestActivityId, halibutRedisTransport, log, nodeOfflineTimeoutBetweenHeartBeatsFromSender, watchCancellationToken); + if (res == NodeHeartBeatSender.NodeProcessingRequestWatcherResult.NodeMayHaveDisconnected) + { + await RequestCancellationTokenSource.CancelAsync(); + } + } + catch (Exception) when (watchCancellationToken.IsCancellationRequested) + { + log.Write(EventType.Diagnostic, "Sender node watcher cancelled for request {0}, endpoint {1}", requestActivityId, endpoint); + } + catch (Exception ex) + { + log.WriteException(EventType.Error, "Error watching sender node for request {0}, endpoint {1}", ex, requestActivityId, endpoint); + } + } + + public async ValueTask DisposeAsync() + { + await disposableCollection.DisposeAsync(); + } + } +} \ No newline at end of file diff --git a/source/Halibut/ServiceModel/PendingRequestV2.cs b/source/Halibut/ServiceModel/PendingRequestV2.cs index 77e7c169e..a88ce3de4 100644 --- a/source/Halibut/ServiceModel/PendingRequestV2.cs +++ b/source/Halibut/ServiceModel/PendingRequestV2.cs @@ -53,13 +53,13 @@ public PendingRequest(RequestMessage request, ILog log) /// /// /// - /// + /// /// This will be called either when the pick-up timeout has elapsed OR if the Cancellation Token has been triggered. /// This gives the user an opportunity to remove the pending request from shared places and optionally /// call BeginTransfer /// /// - public async Task WaitUntilComplete(Func timePendingRequestCanBeOnTheQueueHasElapsed, CancellationToken cancellationToken) + public async Task WaitUntilComplete(Func checkIfPendingRequestWasCollectedOrRemoveIt, CancellationToken cancellationToken) { log.Write(EventType.MessageExchange, "Request {0} was queued", request); @@ -78,8 +78,12 @@ public async Task WaitUntilComplete(Func timePendingRequestCanBeOnTheQueue log.Write(EventType.MessageExchange, "Request {0} was collected by the polling endpoint", request); return; } + } - + if (!requestCollected.IsSet) await checkIfPendingRequestWasCollectedOrRemoveIt(); + + using (await transferLock.LockAsync(CancellationToken.None)) { + if (cancellationToken.IsCancellationRequested) { if (!requestCollected.IsSet) log.Write(EventType.MessageExchange, "Request {0} was cancelled before it could be collected by the polling endpoint", request); @@ -88,11 +92,7 @@ public async Task WaitUntilComplete(Func timePendingRequestCanBeOnTheQueue await Try.IgnoringError(async () => await pendingRequestCancellationTokenSource.CancelAsync()); cancellationToken.ThrowIfCancellationRequested(); } - } - - if (!requestCollected.IsSet) await timePendingRequestCanBeOnTheQueueHasElapsed(); - - using (await transferLock.LockAsync(CancellationToken.None)) { + if (!requestCollected.IsSet) { // Request was not collected within the pickup time. From 0d18bcaa70b53549f2091d74c8f124f212eaf1b0 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Wed, 6 Aug 2025 15:23:40 +1000 Subject: [PATCH 023/137] Fix bug where we kept sending pulses --- Redis.md | 2 +- source/Halibut.Tests/Halibut.Tests.csproj | 1 + .../Redis/RedisPendingRequestQueueFixture.cs | 121 +++++++++++++++++- .../Queue/Redis/NodeHeartBeatSender.cs | 4 +- .../Queue/Redis/RedisPendingRequestQueue.cs | 72 ++++++++--- 5 files changed, 176 insertions(+), 24 deletions(-) diff --git a/Redis.md b/Redis.md index f439e1862..8d621b663 100644 --- a/Redis.md +++ b/Redis.md @@ -1,2 +1,2 @@ -docker run -v redis-conf:/usr/local/etc/redis -p 6379:6379 --name redis -d redis redis-server /usr/local/etc/redis/redis.conf +docker run -v `pwd`/redis-conf:/usr/local/etc/redis -p 6379:6379 --name redis -d redis redis-server /usr/local/etc/redis/redis.conf diff --git a/source/Halibut.Tests/Halibut.Tests.csproj b/source/Halibut.Tests/Halibut.Tests.csproj index 8b2037fc9..c08878e60 100644 --- a/source/Halibut.Tests/Halibut.Tests.csproj +++ b/source/Halibut.Tests/Halibut.Tests.csproj @@ -49,6 +49,7 @@ + diff --git a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs index cfe370f71..dc2b94e56 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs @@ -23,6 +23,7 @@ using Halibut.Tests.Util; using Halibut.TestUtils.Contracts; using Halibut.Transport.Protocol; +using Halibut.Util; using Nito.AsyncEx; using NSubstitute; using NUnit.Framework; @@ -178,6 +179,122 @@ public async Task FullSendAndReceiveShouldWork() responseMessage.Result.Should().Be("Yay"); } + + [Test] + public async Task OnceARequestIsComplete_NoInflightDisposableShouldExist() + { + // Arrange + var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); + var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); + var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); + var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); + var messageSerializer = new QueueMessageSerializerBuilder().Build(); + var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); + + var request = new RequestMessageBuilder("poll://test-endpoint").Build(); + + var queue = new RedisPendingRequestQueue(endpoint, log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + + // Act + var queueAndWaitAsync = queue.QueueAndWaitAsync(request, CancellationToken.None); + + var requestMessageWithCancellationToken = await queue.DequeueAsync(CancellationToken); + requestMessageWithCancellationToken.Should().NotBeNull(); + + var response = ResponseMessage.FromResult(requestMessageWithCancellationToken!.RequestMessage, "Yay"); + await queue.ApplyResponse(response, requestMessageWithCancellationToken.RequestMessage.ActivityId); + + var responseMessage = await queueAndWaitAsync; + responseMessage.Result.Should().Be("Yay"); + + // Assert + queue.disposablesForInFlightRequests.Should().BeEmpty(); + } + + [Test] + public async Task OnceARequestIsComplete_NoSenderHeartBeatsShouldBeSent() + { + // Arrange + var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); + var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); + var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); + var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); + var messageSerializer = new QueueMessageSerializerBuilder().Build(); + var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); + + var request = new RequestMessageBuilder("poll://test-endpoint").Build(); + + var queue = new RedisPendingRequestQueue(endpoint, log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + queue.DelayBetweenHeartBeatsForSender = TimeSpan.FromSeconds(1); + + // Act + var queueAndWaitAsync = queue.QueueAndWaitAsync(request, CancellationToken.None); + + var requestMessageWithCancellationToken = await queue.DequeueAsync(CancellationToken); + requestMessageWithCancellationToken.Should().NotBeNull(); + + var response = ResponseMessage.FromResult(requestMessageWithCancellationToken!.RequestMessage, "Yay"); + await queue.ApplyResponse(response, requestMessageWithCancellationToken.RequestMessage.ActivityId); + + + var responseMessage = await queueAndWaitAsync; + responseMessage.Result.Should().Be("Yay"); + + // Assert + bool heartBeatSent = false; + var cts = new CancellationTokenSource().CancelOnDispose(); + using var _ = redisTransport.SubscribeToNodeHeartBeatChannel(endpoint, request.ActivityId, HalibutQueueNodeSendingPulses.Sender, async () => + { + await Task.CompletedTask; + heartBeatSent = true; + }, + cts.CancellationToken); + + await Task.Delay(5000); + heartBeatSent.Should().BeFalse(); + } + + [Test] + public async Task OnceARequestIsComplete_NoReceiverHeartBeatsShouldBeSent() + { + // Arrange + var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); + var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); + var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); + var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); + var messageSerializer = new QueueMessageSerializerBuilder().Build(); + var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); + + var request = new RequestMessageBuilder("poll://test-endpoint").Build(); + + var queue = new RedisPendingRequestQueue(endpoint, log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + queue.DelayBetweenHeartBeatsForReceiver = TimeSpan.FromSeconds(1); + + // Act + var queueAndWaitAsync = queue.QueueAndWaitAsync(request, CancellationToken.None); + + var requestMessageWithCancellationToken = await queue.DequeueAsync(CancellationToken); + requestMessageWithCancellationToken.Should().NotBeNull(); + + var response = ResponseMessage.FromResult(requestMessageWithCancellationToken!.RequestMessage, "Yay"); + await queue.ApplyResponse(response, requestMessageWithCancellationToken.RequestMessage.ActivityId); + + var responseMessage = await queueAndWaitAsync; + responseMessage.Result.Should().Be("Yay"); + + // Assert + bool heartBeatSent = false; + var cts = new CancellationTokenSource().CancelOnDispose(); + using var _ = redisTransport.SubscribeToNodeHeartBeatChannel(endpoint, request.ActivityId, HalibutQueueNodeSendingPulses.Receiver, async () => + { + await Task.CompletedTask; + heartBeatSent = true; + }, + cts.CancellationToken); + + await Task.Delay(5000); + heartBeatSent.Should().BeFalse(); + } [Test] public async Task FullSendAndReceiveWithDataStreamShouldWork() @@ -220,7 +337,7 @@ public async Task FullSendAndReceiveWithDataStreamShouldWork() // Or should it to try to encourage a new TCP connection which could go to a different node. [Test] - public async Task WhenTheReceiverConnectionToRedisIsInterruptedAndRestoredBeforeWorkIsPublished_TheRecieverShouldBeAbleToCollectThatWorkQuickly() + public async Task WhenTheReceiversConnectionToRedisIsInterruptedAndRestoredBeforeWorkIsPublished_TheRecieverShouldBeAbleToCollectThatWorkQuickly() { // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); @@ -408,7 +525,7 @@ public async Task WhenTheReceiverDequeuesWork_AndTheSenderDisconnects_AndNeverRe } [Test] - public async Task WhenTheSenderDisconnectsFromRedisRightWhenTheReceiverSendsTheResponseBack_TheSenderStillGetsTheResponse() + public async Task WhenTheSenderBrieflyDisconnectsFromRedisRightWhenTheReceiverSendsTheResponseBack_TheSenderStillGetsTheResponse() { // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); diff --git a/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs b/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs index 5ce7cce37..37f27b0fd 100644 --- a/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs +++ b/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs @@ -126,7 +126,7 @@ private static async Task WatchForPulsesFrom Guid requestActivityId, HalibutRedisTransport halibutRedisTransport, ILog log, - TimeSpan maxTimeBetweenHeartBeetsBeforeProcessingNodeIsAssumedToBeOffline, + TimeSpan maxTimeBetweenHeartBeetsBeforeNodeIsAssumedToBeOffline, HalibutQueueNodeSendingPulses watchingForPulsesFrom, CancellationToken watchCancellationToken) { @@ -151,7 +151,7 @@ private static async Task WatchForPulsesFrom // TODO: I am sure a fancy pants delay could be done here calculated from the now and last heart beat etc. await Try.IgnoringError(async () => await Task.Delay(TimeSpan.FromSeconds(10), watchCancellationToken)); var timeSinceLastHeartBeat = DateTimeOffset.Now - lastHeartBeat.Value; - if (timeSinceLastHeartBeat > maxTimeBetweenHeartBeetsBeforeProcessingNodeIsAssumedToBeOffline) + if (timeSinceLastHeartBeat > maxTimeBetweenHeartBeetsBeforeNodeIsAssumedToBeOffline) { log.Write(EventType.Diagnostic, "{0} node appears disconnected, request {1}, last heartbeat was {2} seconds ago", watchingForPulsesFrom, requestActivityId, timeSinceLastHeartBeat.TotalSeconds); return NodeProcessingRequestWatcherResult.NodeMayHaveDisconnected; diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index eb61e613d..0e15f6ad6 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -35,7 +35,7 @@ class RedisPendingRequestQueue : IPendingRequestQueue, IDisposable readonly AsyncManualResetEvent hasItemsForEndpoint = new(); readonly CancellationTokenSource queueCts = new (); - ConcurrentDictionary disposablesForInFlightRequests = new(); + internal ConcurrentDictionary disposablesForInFlightRequests = new(); readonly CancellationToken queueToken; Task PulseChannelSubDisposer { get; } @@ -74,7 +74,6 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can using var cts = CancellationTokenSource.CreateLinkedTokenSource(queueCts.Token, requestCancellationToken); var cancellationToken = cts.Token; // TODO: redis goes down - // TODO: Other node goes down. // TODO RedisConnectionException can be raised out of here, what should the queue do? using var pending = new PendingRequest(request, log); @@ -198,28 +197,26 @@ async Task SubscribeToResponse(Guid activityId, { try { + log.Write(EventType.Diagnostic, "Waiting for response for request {0}", activityId); var responseJson = await sub.ResultTask; + log.Write(EventType.Diagnostic, "Received response JSON for request {0}, deserializing", activityId); var response = await messageReaderWriter.ReadResponse(responseJson, cancellationToken); + log.Write(EventType.Diagnostic, "Successfully deserialized response for request {0}, invoking callback", activityId); onResponse(response); } catch (OperationCanceledException) { - // TODO ignore + log.Write(EventType.Diagnostic, "Response subscription cancelled for request {0}", activityId); } - catch (Exception) + catch (Exception ex) { - // TODO log + log.WriteException(EventType.Error, "Error while processing response for request {0}", ex, activityId); } }); return sub; - // return await halibutRedisTransport.SubScribeToResponses(endpoint, activityId, async (responseJson) => - // { - // var response = await messageReaderWriter.ReadResponse(responseJson, cancellationToken); - // onResponse(response); - // }, cancellationToken); } - public bool IsEmpty => throw new NotImplementedException(); + public bool IsEmpty => Count == 0; public int Count => throw new NotImplementedException(); // Setting this too high means things above the RPC might not have time to retry. @@ -265,15 +262,52 @@ async Task SubscribeToResponse(Guid activityId, public async Task ApplyResponse(ResponseMessage response, Guid requestActivityId) { - if (response == null) return; - - var cancellationToken = CancellationToken.None; - - // This node has now completed the RPC, and so the response must be sent - // back to the node which sent the response + log.Write(EventType.MessageExchange, "Applying response for request {0}", requestActivityId); - var payload = await messageReaderWriter.PrepareResponse(response, cancellationToken); - await PollAndSubscribeForSingleMessage.TrySendMessage(ResponseMessageSubscriptionName, halibutRedisTransport, endpoint, requestActivityId, payload, log); + try + { + if (response == null) + { + log.Write(EventType.Diagnostic, "Response is null for request {0}, skipping apply", requestActivityId); + return; + } + + log.Write(EventType.MessageExchange, "Preparing response payload for request {0}", requestActivityId); + var cancellationToken = CancellationToken.None; + + // This node has now completed the RPC, and so the response must be sent + // back to the node which sent the response + + var payload = await messageReaderWriter.PrepareResponse(response, cancellationToken); + log.Write(EventType.MessageExchange, "Sending response message for request {0}", requestActivityId); + await PollAndSubscribeForSingleMessage.TrySendMessage(ResponseMessageSubscriptionName, halibutRedisTransport, endpoint, requestActivityId, payload, log); + log.Write(EventType.MessageExchange, "Successfully applied response for request {0}", requestActivityId); + } + catch (Exception ex) + { + log.WriteException(EventType.Error, "Error applying response for request {0}", ex, requestActivityId); + throw; + } + finally + { + if (disposablesForInFlightRequests.TryRemove(requestActivityId, out var disposables)) + { + log.Write(EventType.Diagnostic, "Disposing in-flight request resources for request {0}", requestActivityId); + try + { + await disposables.DisposeAsync(); + log.Write(EventType.Diagnostic, "Successfully disposed in-flight request resources for request {0}", requestActivityId); + } + catch (Exception ex) + { + log.WriteException(EventType.Diagnostic, "Error disposing in-flight request resources for request {0}", ex, requestActivityId); + } + } + else + { + log.Write(EventType.Diagnostic, "No in-flight request resources found to dispose for request {0}", requestActivityId); + } + } } async Task DequeueNextAsync() From 8ae84b6d0b369ed7a764d53a695516f006c45216 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Thu, 7 Aug 2025 08:19:12 +1000 Subject: [PATCH 024/137] Now we can detect a redis that drops all it data like its hot --- .../RedisFacadeWhenRedisGoesDownAwayTests.cs | 28 ++-- .../Redis/RedisPendingRequestQueueFixture.cs | 20 +-- .../Redis/Utils/RedisContainerBuilder.cs | 113 ++++++++++++++++ .../Queue/Redis/HalibutRedisTransport.cs | 13 +- .../Redis/PollAndSubscribeForSingleMessage.cs | 3 +- source/Halibut/Queue/Redis/RedisFacade.cs | 21 ++- .../Queue/Redis/RedisPendingRequestQueue.cs | 28 ++-- .../Redis/WatchForRedisLosingAllItsData.cs | 128 ++++++++++++++++++ .../Redis/WatchForRequestCancellation.cs | 10 +- 9 files changed, 312 insertions(+), 52 deletions(-) create mode 100644 source/Halibut.Tests/Queue/Redis/Utils/RedisContainerBuilder.cs create mode 100644 source/Halibut/Queue/Redis/WatchForRedisLosingAllItsData.cs diff --git a/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs b/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs index 2196b68fa..70f9a1e2f 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs @@ -46,9 +46,9 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); - await redisFacade.SetString("foo", "bar"); + await redisFacade.SetString("foo", "bar", TimeSpan.FromMinutes(1), CancellationToken); - (await redisFacade.GetString("foo")).Should().Be("bar"); + (await redisFacade.GetString("foo", CancellationToken)).Should().Be("bar"); portForwarder.EnterKillNewAndExistingConnectionsMode(); portForwarder.ReturnToNormalMode(); @@ -56,7 +56,7 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe // After a short delay it does seem to work again. await Task.Delay(1000); - await redisFacade.GetString("foo"); + await redisFacade.GetString("foo", CancellationToken); } [Test] @@ -66,16 +66,16 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); - await redisFacade.SetString("foo", "bar"); + await redisFacade.SetString("foo", "bar", TimeSpan.FromMinutes(1), CancellationToken); - (await redisFacade.GetString("foo")).Should().Be("bar"); + (await redisFacade.GetString("foo", CancellationToken)).Should().Be("bar"); portForwarder.EnterKillNewAndExistingConnectionsMode(); portForwarder.ReturnToNormalMode(); // No delay here - await redisFacade.GetString("foo"); + await redisFacade.GetString("foo", CancellationToken); } [Test] @@ -86,7 +86,7 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); // Establish connection first - await redisFacade.SetString("connection", "established"); + await redisFacade.SetString("connection", "established", TimeSpan.FromMinutes(1), CancellationToken); portForwarder.EnterKillNewAndExistingConnectionsMode(); portForwarder.ReturnToNormalMode(); @@ -103,7 +103,7 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); // Establish connection first - await redisFacade.SetString("connection", "established"); + await redisFacade.SetString("connection", "established", TimeSpan.FromMinutes(1), CancellationToken); portForwarder.EnterKillNewAndExistingConnectionsMode(); portForwarder.ReturnToNormalMode(); @@ -138,7 +138,7 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); // Establish connection first - await redisFacade.SetString("connection", "established"); + await redisFacade.SetString("connection", "established", TimeSpan.FromMinutes(1), CancellationToken); portForwarder.EnterKillNewAndExistingConnectionsMode(); portForwarder.ReturnToNormalMode(); @@ -173,13 +173,13 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); // Establish connection first - await redisFacade.SetString("connection", "established"); + await redisFacade.SetString("connection", "established", TimeSpan.FromMinutes(1), CancellationToken); portForwarder.EnterKillNewAndExistingConnectionsMode(); portForwarder.ReturnToNormalMode(); // No delay here - should retry and succeed - await redisFacade.SetString("test-key", "test-value"); + await redisFacade.SetString("test-key", "test-value", TimeSpan.FromMinutes(1), CancellationToken); } [Test] @@ -190,13 +190,13 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); // Establish connection and set up test data - await redisFacade.SetString("test-key", "test-value"); + await redisFacade.SetString("test-key", "test-value", TimeSpan.FromMinutes(1), CancellationToken); portForwarder.EnterKillNewAndExistingConnectionsMode(); portForwarder.ReturnToNormalMode(); // No delay here - should retry and succeed - var result = await redisFacade.GetString("test-key"); + var result = await redisFacade.GetString("test-key", CancellationToken); result.Should().Be("test-value"); } @@ -227,7 +227,7 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndWeTryT await using var redisViaPortForwarder = new RedisFacade("localhost:" + portForwarder.ListeningPort, guid, redisLogCreator.CreateNewForPrefix("Unstable")); await using var redisStableConnection = new RedisFacade("localhost:" + redisPort, guid, redisLogCreator.CreateNewForPrefix("Stable")); - await redisViaPortForwarder.SetString("Establish connection", "before we subscribe"); + await redisViaPortForwarder.SetString("Establish connection", "before we subscribe", TimeSpan.FromMinutes(1), CancellationToken); portForwarder.EnterKillNewAndExistingConnectionsMode(); diff --git a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs index dc2b94e56..4a4f0420b 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs @@ -225,7 +225,7 @@ public async Task OnceARequestIsComplete_NoSenderHeartBeatsShouldBeSent() var request = new RequestMessageBuilder("poll://test-endpoint").Build(); var queue = new RedisPendingRequestQueue(endpoint, log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); - queue.DelayBetweenHeartBeatsForSender = TimeSpan.FromSeconds(1); + queue.DelayBetweenHeartBeatsForRequestSender = TimeSpan.FromSeconds(1); // Act var queueAndWaitAsync = queue.QueueAndWaitAsync(request, CancellationToken.None); @@ -268,7 +268,7 @@ public async Task OnceARequestIsComplete_NoReceiverHeartBeatsShouldBeSent() var request = new RequestMessageBuilder("poll://test-endpoint").Build(); var queue = new RedisPendingRequestQueue(endpoint, log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); - queue.DelayBetweenHeartBeatsForReceiver = TimeSpan.FromSeconds(1); + queue.DelayBetweenHeartBeatsForRequestProcessor = TimeSpan.FromSeconds(1); // Act var queueAndWaitAsync = queue.QueueAndWaitAsync(request, CancellationToken.None); @@ -453,10 +453,10 @@ public async Task WhenTheReceiverDequeuesWorkAndThenDisconnectsFromRedisForEver_ var node2Receiver = new RedisPendingRequestQueue(endpoint, log, new HalibutRedisTransport(unstableRedisConnection), messageReaderWriter, halibutTimeoutAndLimits); // Lower this to complete the test sooner. - node1Sender.DelayBetweenHeartBeatsForReceiver = TimeSpan.FromSeconds(1); - node2Receiver.DelayBetweenHeartBeatsForReceiver = TimeSpan.FromSeconds(1); - node1Sender.NodeOfflineTimeoutBetweenHeartBeatsFromReceiver = TimeSpan.FromSeconds(10); - node2Receiver.NodeOfflineTimeoutBetweenHeartBeatsFromReceiver = TimeSpan.FromSeconds(10); + node1Sender.DelayBetweenHeartBeatsForRequestProcessor = TimeSpan.FromSeconds(1); + node2Receiver.DelayBetweenHeartBeatsForRequestProcessor = TimeSpan.FromSeconds(1); + node1Sender.NodeIsOfflineHeartBeatTimeoutForRequestProcessor = TimeSpan.FromSeconds(10); + node2Receiver.NodeIsOfflineHeartBeatTimeoutForRequestProcessor = TimeSpan.FromSeconds(10); var request = new RequestMessageBuilder("poll://test-endpoint").Build(); @@ -503,10 +503,10 @@ public async Task WhenTheReceiverDequeuesWork_AndTheSenderDisconnects_AndNeverRe var node1Sender = new RedisPendingRequestQueue(endpoint, log, new HalibutRedisTransport(unstableRedisConnection), messageReaderWriter, halibutTimeoutAndLimits); var node2Receiver = new RedisPendingRequestQueue(endpoint, log, new HalibutRedisTransport(stableRedisConnection), messageReaderWriter, halibutTimeoutAndLimits); - node1Sender.DelayBetweenHeartBeatsForSender= TimeSpan.FromSeconds(1); - node2Receiver.DelayBetweenHeartBeatsForSender= TimeSpan.FromSeconds(1); - node1Sender.NodeOfflineTimeoutBetweenHeartBeatsFromSender = TimeSpan.FromSeconds(15); - node2Receiver.NodeOfflineTimeoutBetweenHeartBeatsFromSender = TimeSpan.FromSeconds(15); + node1Sender.DelayBetweenHeartBeatsForRequestSender= TimeSpan.FromSeconds(1); + node2Receiver.DelayBetweenHeartBeatsForRequestSender= TimeSpan.FromSeconds(1); + node1Sender.NodeIsOfflineHeartBeatTimeoutForRequestSender = TimeSpan.FromSeconds(15); + node2Receiver.NodeIsOfflineHeartBeatTimeoutForRequestSender = TimeSpan.FromSeconds(15); var request = new RequestMessageBuilder("poll://test-endpoint").Build(); diff --git a/source/Halibut.Tests/Queue/Redis/Utils/RedisContainerBuilder.cs b/source/Halibut.Tests/Queue/Redis/Utils/RedisContainerBuilder.cs new file mode 100644 index 000000000..36a9212b7 --- /dev/null +++ b/source/Halibut.Tests/Queue/Redis/Utils/RedisContainerBuilder.cs @@ -0,0 +1,113 @@ +using System; +using System.IO; +using System.Threading.Tasks; +using DotNet.Testcontainers.Builders; +using DotNet.Testcontainers.Containers; +using Halibut.Tests.Support; +using NUnit.Framework; + +namespace Halibut.Tests.Queue.Redis.Utils +{ + public class RedisContainerBuilder + { + private string _image = "redis:7-alpine"; + private string? _customConfigPath; + private int? _hostPort; + + /// + /// Sets the Redis Docker image to use. Defaults to "redis:7-alpine". + /// + /// The Redis Docker image tag + /// The builder instance for method chaining + public RedisContainerBuilder WithImage(string image) + { + _image = image; + return this; + } + + /// + /// Sets a custom Redis configuration path to mount into the container. + /// If not specified, uses the default redis-conf directory from the project root. + /// + /// The path to the Redis configuration directory + /// The builder instance for method chaining + public RedisContainerBuilder WithCustomConfigPath(string configPath) + { + _customConfigPath = configPath; + return this; + } + + /// + /// Sets a specific host port to bind to. If not specified, finds a free port automatically. + /// + /// The host port to bind to + /// The builder instance for method chaining + public RedisContainerBuilder WithHostPort(int hostPort) + { + _hostPort = hostPort; + return this; + } + + /// + /// Builds and returns a configured Redis container with the specified settings. + /// The container is not started - call StartAsync() on the returned container to start it. + /// + /// A configured Redis container ready to be started + public RedisContainer Build() + { + var hostPort = _hostPort ?? TcpPortHelper.FindFreeTcpPort(); + var redisConfigPath = _customConfigPath ?? + Path.GetFullPath(Path.Combine(TestContext.CurrentContext.TestDirectory, "../../../../../redis-conf")); + + var container = new ContainerBuilder() + .WithImage(_image) + .WithPortBinding(hostPort, 6379) + .WithBindMount(redisConfigPath, "/usr/local/etc/redis") + .WithCommand("redis-server", "/usr/local/etc/redis/redis.conf") + .WithWaitStrategy(DotNet.Testcontainers.Builders.Wait.ForUnixContainer() + .UntilPortIsAvailable(6379)) + .Build(); + + return new RedisContainer(container, hostPort); + } + } + + /// + /// Wrapper around the testcontainers IContainer that provides Redis-specific functionality + /// + public class RedisContainer : IAsyncDisposable + { + private readonly IContainer _container; + + public RedisContainer(IContainer container, int redisPort) + { + _container = container; + RedisPort = redisPort; + } + + /// + /// The host port that Redis is bound to + /// + public int RedisPort { get; } + + /// + /// The connection string to connect to this Redis instance + /// + public string ConnectionString => $"localhost:{RedisPort}"; + + /// + /// Starts the Redis container + /// + public Task StartAsync() => _container.StartAsync(); + + /// + /// Stops the Redis container + /// + public Task StopAsync() => _container.StopAsync(); + + /// + /// Disposes the Redis container + /// + public ValueTask DisposeAsync() => _container.DisposeAsync(); + } +} \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/HalibutRedisTransport.cs b/source/Halibut/Queue/Redis/HalibutRedisTransport.cs index aaf6b0fb3..cff74aa0b 100644 --- a/source/Halibut/Queue/Redis/HalibutRedisTransport.cs +++ b/source/Halibut/Queue/Redis/HalibutRedisTransport.cs @@ -149,16 +149,16 @@ public string RequestCancelledMarkerKey(Uri endpoint, Guid requestId) return $"{Namespace}::RequestCancelledMarkerKey::{endpoint}::{requestId}"; } - public async Task MarkRequestAsCancelled(Uri endpoint, Guid requestId, CancellationToken cancellationToken) + public async Task MarkRequestAsCancelled(Uri endpoint, Guid requestId, TimeSpan ttl, CancellationToken cancellationToken) { var key = RequestCancelledMarkerKey(endpoint, requestId); - await facade.SetString(key, "{}"); + await facade.SetString(key, "{}", ttl, cancellationToken); } public async Task IsRequestMarkedAsCancelled(Uri endpoint, Guid requestId, CancellationToken cancellationToken) { var key = RequestCancelledMarkerKey(endpoint, requestId); - return (await facade.GetString(key)) != null; + return (await facade.GetString(key, cancellationToken)) != null; } @@ -234,17 +234,16 @@ string GenericMarkerKey(string thingToWatchFor, Uri endpoint, Guid identifier) return $"{Namespace}::GenericMarker::{thingToWatchFor}::{endpoint}::{identifier}"; } - public async Task SendValue(string thingToWatchFor, Uri endpoint, Guid identifier, string value, CancellationToken cancellationToken) + public async Task SendValue(string thingToWatchFor, Uri endpoint, Guid identifier, string value, TimeSpan ttl, CancellationToken cancellationToken) { - // TODO: really must have a TTL var key = GenericMarkerKey(thingToWatchFor, endpoint, identifier); - await facade.SetString(key, value); + await facade.SetString(key, value, ttl, cancellationToken); } public async Task GetGenericMarker(string thingToWatchFor, Uri endpoint, Guid identifier, CancellationToken cancellationToken) { var key = GenericMarkerKey(thingToWatchFor, endpoint, identifier); - return await facade.GetString(key); + return await facade.GetString(key, cancellationToken); } public async Task DeleteGenericMarker(string thingToWatchFor, Uri endpoint, Guid identifier, CancellationToken cancellationToken) diff --git a/source/Halibut/Queue/Redis/PollAndSubscribeForSingleMessage.cs b/source/Halibut/Queue/Redis/PollAndSubscribeForSingleMessage.cs index ff8af3ef5..87626eca1 100644 --- a/source/Halibut/Queue/Redis/PollAndSubscribeForSingleMessage.cs +++ b/source/Halibut/Queue/Redis/PollAndSubscribeForSingleMessage.cs @@ -29,6 +29,7 @@ public static async Task TrySendMessage( Uri endpoint, Guid activityId, string value, + TimeSpan ttl, ILog log) { log.Write(EventType.Diagnostic, "Attempting to set {0} for - Endpoint: {1}, ActivityId: {2}", messageTypeName, endpoint, activityId); @@ -39,7 +40,7 @@ public static async Task TrySendMessage( try { log.Write(EventType.Diagnostic, "Marking {0} as set - Endpoint: {1}, ActivityId: {2}", messageTypeName, endpoint, activityId); - await halibutRedisTransport.SendValue(messageTypeName, endpoint, activityId, value, cts.Token); + await halibutRedisTransport.SendValue(messageTypeName, endpoint, activityId, value, ttl, cts.Token); log.Write(EventType.Diagnostic, "Publishing {0} notification - Endpoint: {1}, ActivityId: {2}", messageTypeName, endpoint, activityId); await halibutRedisTransport.PublishThatValueIsAvailable(messageTypeName, endpoint, activityId, value, cts.Token); diff --git a/source/Halibut/Queue/Redis/RedisFacade.cs b/source/Halibut/Queue/Redis/RedisFacade.cs index f7f532c5d..eb4bb2fe6 100644 --- a/source/Halibut/Queue/Redis/RedisFacade.cs +++ b/source/Halibut/Queue/Redis/RedisFacade.cs @@ -349,7 +349,7 @@ await ExecuteWithRetry(async () => }, CancellationToken.None); } - public async Task SetString(string key, string value) + public async Task SetString(string key, string value, TimeSpan ttl, CancellationToken cancellationToken) { // TODO TTL key = "string:" + keyPrefix + ":" + key; @@ -357,17 +357,28 @@ await ExecuteWithRetry(async () => { var database = Connection.GetDatabase(); await database.StringSetAsync(key, value); - }, CancellationToken.None); + }, cancellationToken); + + await SetTtlForKey(key, ttl, cancellationToken); } - - public async Task GetString(string key) + + public async Task SetTtlForKey(string key, TimeSpan ttl, CancellationToken cancellationToken) + { + await ExecuteWithRetry(async () => + { + var database = Connection.GetDatabase(); + await database.KeyExpireAsync(key, ttl); + }, cancellationToken); + } + + public async Task GetString(string key, CancellationToken cancellationToken) { key = "string:" + keyPrefix + ":" + key; return await ExecuteWithRetry(async () => { var database = Connection.GetDatabase(); return await database.StringGetAsync(key); - }, CancellationToken.None); + }, cancellationToken); } public async Task DeleteString(string key) diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index 0e15f6ad6..b2de5f5c9 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -14,7 +14,6 @@ using System; using System.Collections.Concurrent; -using System.Security.Cryptography; using System.Threading; using System.Threading.Tasks; using Halibut.Diagnostics; @@ -87,7 +86,7 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can var tryClearRequestFromQueueAtMostOnce = new AsyncLazy(async () => await TryClearRequestFromQueue(request, pending)); try { - await using var senderPulse = new NodeHeartBeatSender(endpoint, request.ActivityId, halibutRedisTransport, log, HalibutQueueNodeSendingPulses.Sender, DelayBetweenHeartBeatsForSender); + await using var senderPulse = new NodeHeartBeatSender(endpoint, request.ActivityId, halibutRedisTransport, log, HalibutQueueNodeSendingPulses.Sender, DelayBetweenHeartBeatsForRequestSender); // Make the request available before we tell people it is available. await halibutRedisTransport.PutRequest(endpoint, request.ActivityId, payload, cancellationToken); await halibutRedisTransport.PushRequestGuidOnToQueue(endpoint, request.ActivityId, cancellationToken); @@ -121,14 +120,14 @@ async Task SendCancellationIfRequestWasCancelled(RequestMessage request, Pending } } - void WatchProcessingNodeIsStillConnectedInBackground(RequestMessage request, PendingRequest pending, CancellationTokenSourceAsyncDisposable watcherCts) + void WatchProcessingNodeIsStillConnectedInBackground(RequestMessage request, PendingRequest pending, CancelOnDisposeCancellationTokenSource watcherCts) { Task.Run(async () => { var watcherCtsCancellationToken = watcherCts.CancellationToken; try { - var disconnected = await NodeHeartBeatSender.WatchThatNodeProcessingTheRequestIsStillAlive(endpoint, request, pending, halibutRedisTransport, log, NodeOfflineTimeoutBetweenHeartBeatsFromReceiver, watcherCtsCancellationToken); + var disconnected = await NodeHeartBeatSender.WatchThatNodeProcessingTheRequestIsStillAlive(endpoint, request, pending, halibutRedisTransport, log, NodeIsOfflineHeartBeatTimeoutForRequestProcessor, watcherCtsCancellationToken); if (!watcherCtsCancellationToken.IsCancellationRequested && disconnected == NodeHeartBeatSender.NodeProcessingRequestWatcherResult.NodeMayHaveDisconnected) { // TODO: if(responseWatcher.CheckForResponseNow() == ResponseNotFound) { @@ -218,18 +217,21 @@ async Task SubscribeToResponse(Guid activityId, public bool IsEmpty => Count == 0; public int Count => throw new NotImplementedException(); - - // Setting this too high means things above the RPC might not have time to retry. - public TimeSpan NodeOfflineTimeoutBetweenHeartBeatsFromReceiver { get; set; } = TimeSpan.FromSeconds(60); + // The timespan is more generous for the sender going offline, since if it does go offline, // since under some cases the request completing is advantageous. That node needs to // re-do the entire RPC for idempotent RPCs this might mean that the task required is already done. - internal TimeSpan NodeOfflineTimeoutBetweenHeartBeatsFromSender { get; set; } = TimeSpan.FromSeconds(90); + internal TimeSpan NodeIsOfflineHeartBeatTimeoutForRequestSender { get; set; } = TimeSpan.FromSeconds(90); + + internal TimeSpan DelayBetweenHeartBeatsForRequestSender { get; set; } = TimeSpan.FromSeconds(15); + + // Setting this too high means things above the RPC might not have time to retry. + public TimeSpan NodeIsOfflineHeartBeatTimeoutForRequestProcessor { get; set; } = TimeSpan.FromSeconds(60); - internal TimeSpan DelayBetweenHeartBeatsForSender { get; set; } = TimeSpan.FromSeconds(15); + internal TimeSpan DelayBetweenHeartBeatsForRequestProcessor { get; set; } = TimeSpan.FromSeconds(15); - internal TimeSpan DelayBetweenHeartBeatsForReceiver { get; set; } = TimeSpan.FromSeconds(15); + internal TimeSpan TTLOfResponseMessage { get; set; } = TimeSpan.FromMinutes(5); @@ -246,9 +248,9 @@ async Task SubscribeToResponse(Guid activityId, var disposables = new DisposableCollection(); try { - disposables.AddAsyncDisposable(new NodeHeartBeatSender(endpoint, pending.ActivityId, halibutRedisTransport, log, HalibutQueueNodeSendingPulses.Receiver, DelayBetweenHeartBeatsForReceiver)); + disposables.AddAsyncDisposable(new NodeHeartBeatSender(endpoint, pending.ActivityId, halibutRedisTransport, log, HalibutQueueNodeSendingPulses.Receiver, DelayBetweenHeartBeatsForRequestProcessor)); - var watcher = new WatchForRequestCancellationOrSenderDisconnect(endpoint, pending.ActivityId, halibutRedisTransport, NodeOfflineTimeoutBetweenHeartBeatsFromSender, log); + var watcher = new WatchForRequestCancellationOrSenderDisconnect(endpoint, pending.ActivityId, halibutRedisTransport, NodeIsOfflineHeartBeatTimeoutForRequestSender, log); var response = new RequestMessageWithCancellationToken(pending, watcher.RequestProcessingCancellationToken); disposablesForInFlightRequests[pending.ActivityId] = disposables; return response; @@ -280,7 +282,7 @@ public async Task ApplyResponse(ResponseMessage response, Guid requestActivityId var payload = await messageReaderWriter.PrepareResponse(response, cancellationToken); log.Write(EventType.MessageExchange, "Sending response message for request {0}", requestActivityId); - await PollAndSubscribeForSingleMessage.TrySendMessage(ResponseMessageSubscriptionName, halibutRedisTransport, endpoint, requestActivityId, payload, log); + await PollAndSubscribeForSingleMessage.TrySendMessage(ResponseMessageSubscriptionName, halibutRedisTransport, endpoint, requestActivityId, payload, TTLOfResponseMessage, log); log.Write(EventType.MessageExchange, "Successfully applied response for request {0}", requestActivityId); } catch (Exception ex) diff --git a/source/Halibut/Queue/Redis/WatchForRedisLosingAllItsData.cs b/source/Halibut/Queue/Redis/WatchForRedisLosingAllItsData.cs new file mode 100644 index 000000000..fa05b1332 --- /dev/null +++ b/source/Halibut/Queue/Redis/WatchForRedisLosingAllItsData.cs @@ -0,0 +1,128 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; +using System.Threading; +using System.Threading.Tasks; +using Halibut.Diagnostics; +using Halibut.Util; + +namespace Halibut.Queue.Redis +{ + public class WatchForRedisLosingAllItsData : IAsyncDisposable + { + RedisFacade redisFacade; + readonly ILog log; + + internal TimeSpan SetupDelay { get;} + internal TimeSpan WatchInterval { get; } + internal TimeSpan KeyTTL { get; } + CancelOnDisposeCancellationTokenSource cancelOnDisposeCancellationTokenSource = new CancellationTokenSource().CancelOnDispose(); + + public WatchForRedisLosingAllItsData(RedisFacade redisFacade, ILog log, TimeSpan? setupDelay = null, TimeSpan? watchInterval = null, TimeSpan? keyTTL = null) + { + this.redisFacade = redisFacade; + this.log = log; + this.SetupDelay = setupDelay ?? TimeSpan.FromSeconds(1); + this.WatchInterval = watchInterval ?? TimeSpan.FromSeconds(60); + this.KeyTTL = keyTTL ?? TimeSpan.FromMinutes(60); + var _ = Task.Run(async () => await KeepWatchingForDataLose(cancelOnDisposeCancellationTokenSource.CancellationToken)); + } + + private TaskCompletionSource taskCompletionSource = new TaskCompletionSource(); + + /// + /// Will cause the caller to wait until we are connected to redis and so can detect datalose. + /// + /// Time to wait for this to reach a state where it can detect datalose + /// + /// A cancellation token which is triggered when data lose occurs. + public async Task GetTokenForDataLoseDetection(TimeSpan timeToWait, CancellationToken cancellationToken) + { + if (taskCompletionSource.Task.IsCompleted) + { + return await taskCompletionSource.Task; + } + + await using var cts = CancellationTokenSource.CreateLinkedTokenSource(cancellationToken).CancelOnDispose(); + cts.CancellationTokenSource.CancelAfter(timeToWait); + return await taskCompletionSource.Task.WaitAsync(cts.CancellationToken); + } + + private async Task KeepWatchingForDataLose(CancellationToken cancellationToken) + { + while (!cancellationToken.IsCancellationRequested) + { + await Try.IgnoringError(async () => await WatchForDataLose(cancellationToken)); + } + } + + async Task WatchForDataLose(CancellationToken cancellationToken) + { + string guid = Guid.NewGuid().ToString(); + var key = "WatchForDataLose::" + guid; + var hasSetKey = false; + + log.Write(EventType.Diagnostic, "Starting Redis data loss monitoring with key {0}", key); + + await using var cts = new CancellationTokenSource().CancelOnDispose(); + while (!cancellationToken.IsCancellationRequested) + { + try + { + if (!hasSetKey) + { + log.Write(EventType.Diagnostic, "Setting initial data loss monitoring key {0} with TTL {1} minutes", key, KeyTTL.TotalMinutes); + await redisFacade.SetString(key, guid.ToString(), KeyTTL, cancellationToken); + taskCompletionSource.TrySetResult(cts.CancellationToken); + hasSetKey = true; + log.Write(EventType.Diagnostic, "Successfully set initial data loss monitoring key {0}, monitoring is now active", key); + } + else + { + var data = await redisFacade.GetString(key, cancellationToken); + if (data != guid.ToString()) + { + log.Write(EventType.Error, "Redis data loss detected! Expected value {0} for key {1}, but got {2}. This indicates Redis has lost data.", guid.ToString(), key, data ?? "null"); + // Anyone new will be given a new thing to wait on. + taskCompletionSource = new TaskCompletionSource(); + await Try.IgnoringError(async () => await cts.CancellationTokenSource.CancelAsync()); + return; + } + } + } + catch (Exception ex) + { + log.Write(EventType.Diagnostic, "Error occurred during Redis data loss monitoring for key {0}: {1}. Will retry after delay.", key, ex.Message); + } + + await Try.IgnoringError(async () => + { + if (!hasSetKey) await Task.Delay(SetupDelay, cancellationToken); + else await Task.Delay(WatchInterval, cancellationToken); + }); + + } + + log.Write(EventType.Diagnostic, "Redis data loss monitoring stopped for key {0}, cleaning up", key); + await Try.IgnoringError(async () => await redisFacade.DeleteString(key)); + } + + public async ValueTask DisposeAsync() + { + log.Write(EventType.Diagnostic, "Disposing WatchForRedisLosingAllItsData"); + await cancelOnDisposeCancellationTokenSource.DisposeAsync(); + } + } +} \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs b/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs index cfb8a5838..92c921760 100644 --- a/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs +++ b/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs @@ -23,7 +23,6 @@ namespace Halibut.Queue.Redis { public class WatchForRequestCancellation : IAsyncDisposable { - public static async Task TrySendCancellation( HalibutRedisTransport halibutRedisTransport, Uri endpoint, @@ -41,7 +40,7 @@ public static async Task TrySendCancellation( await halibutRedisTransport.PublishCancellation(endpoint, request.ActivityId, cts.Token); log.Write(EventType.Diagnostic, "Marking request as cancelled - Endpoint: {0}, ActivityId: {1}", endpoint, request.ActivityId); - await halibutRedisTransport.MarkRequestAsCancelled(endpoint, request.ActivityId, cts.Token); + await halibutRedisTransport.MarkRequestAsCancelled(endpoint, request.ActivityId, CancelRequestMarkerTTL, cts.Token); log.Write(EventType.Diagnostic, "Successfully sent cancellation for request - Endpoint: {0}, ActivityId: {1}", endpoint, request.ActivityId); } @@ -54,6 +53,12 @@ public static async Task TrySendCancellation( log.Write(EventType.Error, "Failed to send cancellation for request - Endpoint: {0}, ActivityId: {1}, Error: {2}", endpoint, request.ActivityId, ex.Message); } } + + // How long the CancelRequestMarker will sit in redis before it times out. + // If it does timeout it won't matter since the request-sender will stop sending heart beats + // causing the request-processor to cancel the request anyway. + static TimeSpan CancelRequestMarkerTTL = TimeSpan.FromMinutes(5); + readonly CancellationTokenSource requestCancelledCts = new(); public CancellationToken RequestCancelledCancellationToken => requestCancelledCts.Token; @@ -90,6 +95,7 @@ async Task WatchForExceptions(Uri endpoint, Guid requestActivityId, HalibutRedis // Also poll to see if the request is cancelled since we can miss // the publication. + // TODO: reconsider if we need this since the heart beats should take care of this. while (!token.IsCancellationRequested) { try From a5d8b93b511a0980b2660a1d32cfdc02995a1086 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Thu, 7 Aug 2025 11:10:48 +1000 Subject: [PATCH 025/137] Add support to detect redis loss --- source/Halibut.Tests/BaseTest.cs | 7 ++ .../Redis/RedisPendingRequestQueueFixture.cs | 105 +++++++++++++----- .../Queue/Redis/RedisPendingRequestQueue.cs | 33 +++++- .../Redis/RedisPendingRequestQueueFactory.cs | 8 ++ .../Redis/WatchForRedisLosingAllItsData.cs | 14 ++- .../ServiceModel/PendingRequestQueueAsync.cs | 2 +- .../Halibut/ServiceModel/PendingRequestV2.cs | 51 ++++++--- 7 files changed, 169 insertions(+), 51 deletions(-) diff --git a/source/Halibut.Tests/BaseTest.cs b/source/Halibut.Tests/BaseTest.cs index 2c2b0302d..1d9ac4c1a 100644 --- a/source/Halibut.Tests/BaseTest.cs +++ b/source/Halibut.Tests/BaseTest.cs @@ -1,9 +1,12 @@ using System; using System.Threading; using System.Threading.Tasks; +using Halibut.Logging; using Halibut.Tests.Support; +using Halibut.Tests.Support.Logging; using NUnit.Framework; using NUnit.Framework.Interfaces; +using ILog = Halibut.Diagnostics.ILog; using ILogger = Serilog.ILogger; namespace Halibut.Tests @@ -15,6 +18,8 @@ public class BaseTest public CancellationToken CancellationToken { get; private set; } public ILogger Logger { get; private set; } = null!; + + public ILog HalibutLog { get; private set; } = null!; [SetUp] public void SetUp() @@ -25,6 +30,8 @@ public void SetUp() .Build() .ForContext(GetType()); + HalibutLog = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); + Logger.Information("Trace log file {LogFile}", traceLogFileLogger.logFilePath); Logger.Information("Test started"); diff --git a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs index 4a4f0420b..55f9e9e5b 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs @@ -15,6 +15,7 @@ using Halibut.Queue.Redis; using Halibut.ServiceModel; using Halibut.Tests.Builders; +using Halibut.Tests.Queue.Redis.Utils; using Halibut.Tests.Support; using Halibut.Tests.Support.Logging; using Halibut.Tests.Support.TestAttributes; @@ -52,7 +53,7 @@ public async Task DequeueAsync_ShouldReturnRequestFromRedis() var request = new RequestMessageBuilder("poll://test-endpoint").Build(); - var sut = new RedisPendingRequestQueue(endpoint, log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + var sut = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); var task = sut.QueueAndWaitAsync(request, CancellationToken.None); @@ -81,7 +82,7 @@ public async Task WhenThePickupTimeoutExpires_AnErrorsReturned_AndTheRequestCanN .WithServiceEndpoint(b => b.WithPollingRequestQueueTimeout(TimeSpan.FromMilliseconds(100))) .Build(); - var sut = new RedisPendingRequestQueue(endpoint, log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + var sut = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); var response = await sut.QueueAndWaitAsync(request, CancellationToken.None); @@ -110,7 +111,7 @@ public async Task When100kTentaclesAreSubscribed_TheQueueStillWorks() await using var disposableCollection = new DisposableCollection(); for (int i = 0; i < 300000; i++) { - disposableCollection.Add(new RedisPendingRequestQueue(new Uri("poll://" + Guid.NewGuid()), log, redisTransport, messageReaderWriter, halibutTimeoutsAndLimits)); + disposableCollection.Add(new RedisPendingRequestQueue(new Uri("poll://" + Guid.NewGuid()), new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, halibutTimeoutsAndLimits)); if (i % 10000 == 0) { Logger.Information("Up to: {i}", i); @@ -125,7 +126,7 @@ public async Task When100kTentaclesAreSubscribed_TheQueueStillWorks() { var request = new RequestMessageBuilder(endpoint.ToString()).Build(); - await using var sut = new RedisPendingRequestQueue(endpoint, log, new HalibutRedisTransport(CreateRedisFacade()), messageReaderWriter, halibutTimeoutsAndLimits); + await using var sut = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(CreateRedisFacade()), messageReaderWriter, halibutTimeoutsAndLimits); var resultTask = sut.DequeueAsync(CancellationToken); @@ -160,12 +161,12 @@ public async Task FullSendAndReceiveShouldWork() var request = new RequestMessageBuilder("poll://test-endpoint").Build(); - var node1Sender = new RedisPendingRequestQueue(endpoint, log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); - var node2Reciever = new RedisPendingRequestQueue(endpoint, log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + var node2Receiver = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); var queueAndWaitAsync = node1Sender.QueueAndWaitAsync(request, CancellationToken.None); - var requestMessageWithCancellationToken = await node2Reciever.DequeueAsync(CancellationToken); + var requestMessageWithCancellationToken = await node2Receiver.DequeueAsync(CancellationToken); requestMessageWithCancellationToken.Should().NotBeNull(); requestMessageWithCancellationToken!.RequestMessage.Id.Should().Be(request.Id); @@ -173,13 +174,62 @@ public async Task FullSendAndReceiveShouldWork() requestMessageWithCancellationToken.RequestMessage.ServiceName.Should().Be(request.ServiceName); var response = ResponseMessage.FromResult(requestMessageWithCancellationToken.RequestMessage, "Yay"); - await node2Reciever.ApplyResponse(response, requestMessageWithCancellationToken.RequestMessage.ActivityId); + await node2Receiver.ApplyResponse(response, requestMessageWithCancellationToken.RequestMessage.ActivityId); var responseMessage = await queueAndWaitAsync; responseMessage.Result.Should().Be("Yay"); } + [Test] + public async Task WhenDataLostIsDetected_InFlightRequestShouldBeAbandoned() + { + // Arrange + var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); + var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); + var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); + var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); + var messageSerializer = new QueueMessageSerializerBuilder().Build(); + var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); + + var request = new RequestMessageBuilder("poll://test-endpoint").Build(); + + await using var dataLossWatcher = new CancellableDataLossWatchForRedisLosingAllItsData(); + + var node1Sender = new RedisPendingRequestQueue(endpoint, dataLossWatcher, log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + var node2Receiver = new RedisPendingRequestQueue(endpoint, dataLossWatcher, log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + + var queueAndWaitAsync = node1Sender.QueueAndWaitAsync(request, CancellationToken.None); + + var requestMessageWithCancellationToken = await node2Receiver.DequeueAsync(CancellationToken); + + requestMessageWithCancellationToken.Should().NotBeNull(); + + // Act + await dataLossWatcher.DataLossHasOccured(); + + // Assert + requestMessageWithCancellationToken!.CancellationToken.IsCancellationRequested.Should().BeTrue("The receiver of the data should just give up processing"); + + // We don't want to just await queueAndWaitAsync, since timeouts will kick in resulting in a response no matter what. + // We cant to see that it quickly returns. + await Task.WhenAny(Task.Delay(5000), queueAndWaitAsync); + + queueAndWaitAsync.IsCompleted.Should().BeTrue("As soon as data loss is detected the queueAndWait should return."); + + // Sigh it can go down either of these paths! + if (queueAndWaitAsync.IsCompletedSuccessfully) + { + var responseMessage = await queueAndWaitAsync; + responseMessage!.Error!.Message.Should().Contain("Cancelled because data loss on redis was detected."); + } + else + { + var e = await AssertException.Throws(queueAndWaitAsync); + e.And.InnerException!.Message.Should().Contain("Cancelled because data loss on redis was detected."); + } + } + [Test] public async Task OnceARequestIsComplete_NoInflightDisposableShouldExist() { @@ -193,7 +243,7 @@ public async Task OnceARequestIsComplete_NoInflightDisposableShouldExist() var request = new RequestMessageBuilder("poll://test-endpoint").Build(); - var queue = new RedisPendingRequestQueue(endpoint, log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + var queue = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); // Act var queueAndWaitAsync = queue.QueueAndWaitAsync(request, CancellationToken.None); @@ -224,7 +274,7 @@ public async Task OnceARequestIsComplete_NoSenderHeartBeatsShouldBeSent() var request = new RequestMessageBuilder("poll://test-endpoint").Build(); - var queue = new RedisPendingRequestQueue(endpoint, log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + var queue = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); queue.DelayBetweenHeartBeatsForRequestSender = TimeSpan.FromSeconds(1); // Act @@ -267,7 +317,7 @@ public async Task OnceARequestIsComplete_NoReceiverHeartBeatsShouldBeSent() var request = new RequestMessageBuilder("poll://test-endpoint").Build(); - var queue = new RedisPendingRequestQueue(endpoint, log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + var queue = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); queue.DelayBetweenHeartBeatsForRequestProcessor = TimeSpan.FromSeconds(1); // Act @@ -310,12 +360,12 @@ public async Task FullSendAndReceiveWithDataStreamShouldWork() var request = new RequestMessageBuilder("poll://test-endpoint").Build(); request.Params = new[] { new ComplexObjectMultipleDataStreams(DataStream.FromString("hello"), DataStream.FromString("world")) }; - var node1Sender = new RedisPendingRequestQueue(endpoint, log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); - var node2Reciever = new RedisPendingRequestQueue(endpoint, log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + var node2Receiver = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); var queueAndWaitAsync = node1Sender.QueueAndWaitAsync(request, CancellationToken.None); - var requestMessageWithCancellationToken = await node2Reciever.DequeueAsync(CancellationToken); + var requestMessageWithCancellationToken = await node2Receiver.DequeueAsync(CancellationToken); var objWithDataStreams = (ComplexObjectMultipleDataStreams)requestMessageWithCancellationToken!.RequestMessage.Params[0]; (await objWithDataStreams.Payload1!.ReadAsString(CancellationToken)).Should().Be("hello"); @@ -324,7 +374,7 @@ public async Task FullSendAndReceiveWithDataStreamShouldWork() var response = ResponseMessage.FromResult(requestMessageWithCancellationToken.RequestMessage, new ComplexObjectMultipleDataStreams(DataStream.FromString("good"), DataStream.FromString("bye"))); - await node2Reciever.ApplyResponse(response, requestMessageWithCancellationToken.RequestMessage.ActivityId); + await node2Receiver.ApplyResponse(response, requestMessageWithCancellationToken.RequestMessage.ActivityId); var responseMessage = await queueAndWaitAsync; @@ -337,7 +387,7 @@ public async Task FullSendAndReceiveWithDataStreamShouldWork() // Or should it to try to encourage a new TCP connection which could go to a different node. [Test] - public async Task WhenTheReceiversConnectionToRedisIsInterruptedAndRestoredBeforeWorkIsPublished_TheRecieverShouldBeAbleToCollectThatWorkQuickly() + public async Task WhenTheReceiversConnectionToRedisIsInterruptedAndRestoredBeforeWorkIsPublished_TheReceiverShouldBeAbleToCollectThatWorkQuickly() { // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); @@ -357,8 +407,8 @@ public async Task WhenTheReceiversConnectionToRedisIsInterruptedAndRestoredBefor var halibutTimeoutAndLimits = new HalibutTimeoutsAndLimits(); halibutTimeoutAndLimits.PollingQueueWaitTimeout = TimeSpan.FromDays(1); // We should not need to rely on the timeout working for very short disconnects. - var node1Sender = new RedisPendingRequestQueue(endpoint, log, new HalibutRedisTransport(redisFacadeSender), messageReaderWriter, halibutTimeoutAndLimits); - var node2Receiver = new RedisPendingRequestQueue(endpoint, log, new HalibutRedisTransport(redisFacadeReceiver), messageReaderWriter, halibutTimeoutAndLimits); + var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(redisFacadeSender), messageReaderWriter, halibutTimeoutAndLimits); + var node2Receiver = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(redisFacadeReceiver), messageReaderWriter, halibutTimeoutAndLimits); var dequeueTask = node2Receiver.DequeueAsync(CancellationToken); await Task.Delay(5000, CancellationToken); // Allow some time for the receiver to subscribe to work. @@ -407,8 +457,8 @@ public async Task WhenTheSendersConnectionToRedisIsBrieflyInterruptedWhileSendin halibutTimeoutAndLimits.PollingQueueWaitTimeout = TimeSpan.FromDays(1); // We should not need to rely on the timeout working for very short disconnects. - var node1Sender = new RedisPendingRequestQueue(endpoint, log, new HalibutRedisTransport(redisFacadeSender), messageReaderWriter, halibutTimeoutAndLimits); - var node2Receiver = new RedisPendingRequestQueue(endpoint, log, new HalibutRedisTransport(redisFacadeSender), messageReaderWriter, halibutTimeoutAndLimits); + var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(redisFacadeSender), messageReaderWriter, halibutTimeoutAndLimits); + var node2Receiver = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(redisFacadeSender), messageReaderWriter, halibutTimeoutAndLimits); portForwarder.EnterKillNewAndExistingConnectionsMode(); @@ -449,8 +499,8 @@ public async Task WhenTheReceiverDequeuesWorkAndThenDisconnectsFromRedisForEver_ using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); await using var unstableRedisConnection = CreateRedisFacade(portForwarder.ListeningPort, guid: guid); - var node1Sender = new RedisPendingRequestQueue(endpoint, log, new HalibutRedisTransport(stableRedisConnection), messageReaderWriter, halibutTimeoutAndLimits); - var node2Receiver = new RedisPendingRequestQueue(endpoint, log, new HalibutRedisTransport(unstableRedisConnection), messageReaderWriter, halibutTimeoutAndLimits); + var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(stableRedisConnection), messageReaderWriter, halibutTimeoutAndLimits); + var node2Receiver = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(unstableRedisConnection), messageReaderWriter, halibutTimeoutAndLimits); // Lower this to complete the test sooner. node1Sender.DelayBetweenHeartBeatsForRequestProcessor = TimeSpan.FromSeconds(1); @@ -500,8 +550,8 @@ public async Task WhenTheReceiverDequeuesWork_AndTheSenderDisconnects_AndNeverRe using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); await using var unstableRedisConnection = CreateRedisFacade(portForwarder.ListeningPort, guid: guid); - var node1Sender = new RedisPendingRequestQueue(endpoint, log, new HalibutRedisTransport(unstableRedisConnection), messageReaderWriter, halibutTimeoutAndLimits); - var node2Receiver = new RedisPendingRequestQueue(endpoint, log, new HalibutRedisTransport(stableRedisConnection), messageReaderWriter, halibutTimeoutAndLimits); + var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(unstableRedisConnection), messageReaderWriter, halibutTimeoutAndLimits); + var node2Receiver = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(stableRedisConnection), messageReaderWriter, halibutTimeoutAndLimits); node1Sender.DelayBetweenHeartBeatsForRequestSender= TimeSpan.FromSeconds(1); node2Receiver.DelayBetweenHeartBeatsForRequestSender= TimeSpan.FromSeconds(1); @@ -540,8 +590,8 @@ public async Task WhenTheSenderBrieflyDisconnectsFromRedisRightWhenTheReceiverSe using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); await using var unreliableConnection = CreateRedisFacade(portForwarder.ListeningPort, guid: guid); - var node1Sender = new RedisPendingRequestQueue(endpoint, log, new HalibutRedisTransport(unreliableConnection), messageReaderWriter, new HalibutTimeoutsAndLimits()); - var node2Receiver = new RedisPendingRequestQueue(endpoint, log, new HalibutRedisTransport(stableConnection), messageReaderWriter, new HalibutTimeoutsAndLimits()); + var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(unreliableConnection), messageReaderWriter, new HalibutTimeoutsAndLimits()); + var node2Receiver = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(stableConnection), messageReaderWriter, new HalibutTimeoutsAndLimits()); var request = new RequestMessageBuilder("poll://test-endpoint").Build(); // TODO: This only needs to be set because we do not detect the work was collected as soon as it has been collected. @@ -582,6 +632,7 @@ public async Task WhenUsingTheRedisQueue_ASimpleEchoServiceCanBeCalled(ClientAnd new RedisPendingRequestQueueFactory( queueMessageSerializer, dataStreamStore, + new NeverLosingDataWatchForRedisLosingAllItsData(), redisTransport, new HalibutTimeoutsAndLimits(), logFactory)) @@ -612,7 +663,7 @@ public async Task CancellingARequestShouldResultInTheDequeuedResponseTokenBeingC var request = new RequestMessageBuilder("poll://test-endpoint").Build(); request.Params = new[] { new ComplexObjectMultipleDataStreams(DataStream.FromString("hello"), DataStream.FromString("world")) }; - var node1Sender = new RedisPendingRequestQueue(endpoint, log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); using var cts = new CancellationTokenSource(); diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index b2de5f5c9..c6f404c9b 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -27,6 +27,7 @@ namespace Halibut.Queue.Redis class RedisPendingRequestQueue : IPendingRequestQueue, IDisposable { readonly Uri endpoint; + readonly IWatchForRedisLosingAllItsData watchForRedisLosingAllItsData; readonly ILog log; readonly HalibutRedisTransport halibutRedisTransport; readonly HalibutTimeoutsAndLimits halibutTimeoutsAndLimits; @@ -35,18 +36,22 @@ class RedisPendingRequestQueue : IPendingRequestQueue, IDisposable readonly CancellationTokenSource queueCts = new (); internal ConcurrentDictionary disposablesForInFlightRequests = new(); + + // TODO: this needs to be used in all public methods. readonly CancellationToken queueToken; Task PulseChannelSubDisposer { get; } public RedisPendingRequestQueue( Uri endpoint, + IWatchForRedisLosingAllItsData watchForRedisLosingAllItsData, ILog log, HalibutRedisTransport halibutRedisTransport, MessageReaderWriter messageReaderWriter, HalibutTimeoutsAndLimits halibutTimeoutsAndLimits) { this.endpoint = endpoint; + this.watchForRedisLosingAllItsData = watchForRedisLosingAllItsData; this.log = log; this.messageReaderWriter = messageReaderWriter; this.halibutRedisTransport = halibutRedisTransport; @@ -68,9 +73,19 @@ public async ValueTask DisposeAsync() await Try.IgnoringError(async () => await (await PulseChannelSubDisposer).DisposeAsync()); } + private async Task DataLossCancellationToken(CancellationToken? cancellationToken) + { + using var cts = CancellationTokenSource.CreateLinkedTokenSource(queueCts.Token, cancellationToken ?? CancellationToken.None); + return await watchForRedisLosingAllItsData.GetTokenForDataLoseDetection(TimeSpan.FromSeconds(30), cts.Token); + } + public async Task QueueAndWaitAsync(RequestMessage request, CancellationToken requestCancellationToken) { - using var cts = CancellationTokenSource.CreateLinkedTokenSource(queueCts.Token, requestCancellationToken); + + var dataLoseCt = await DataLossCancellationToken(requestCancellationToken); + + using var cts = CancellationTokenSource.CreateLinkedTokenSource(queueCts.Token, requestCancellationToken, dataLoseCt); + var cancellationToken = cts.Token; // TODO: redis goes down // TODO RedisConnectionException can be raised out of here, what should the queue do? @@ -95,7 +110,10 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can await using var watcherCts = CancellationTokenSource.CreateLinkedTokenSource(cts.Token).CancelOnDispose(); WatchProcessingNodeIsStillConnectedInBackground(request, pending, watcherCts); - await pending.WaitUntilComplete(async () => await tryClearRequestFromQueueAtMostOnce.Task, cancellationToken); + await pending.WaitUntilComplete( + async () => await tryClearRequestFromQueueAtMostOnce.Task, + () => dataLoseCt.IsCancellationRequested ? " Cancelled because data loss on redis was detected." : "", + cancellationToken); } finally { @@ -248,10 +266,17 @@ async Task SubscribeToResponse(Guid activityId, var disposables = new DisposableCollection(); try { + // There is a chance the data loss occured after we got the data but before here. + // In that case we will just time out because of the lack of heart beats. + var dataLossCT = await this.watchForRedisLosingAllItsData.GetTokenForDataLoseDetection(TimeSpan.FromSeconds(30), queueToken); + disposables.AddAsyncDisposable(new NodeHeartBeatSender(endpoint, pending.ActivityId, halibutRedisTransport, log, HalibutQueueNodeSendingPulses.Receiver, DelayBetweenHeartBeatsForRequestProcessor)); - var watcher = new WatchForRequestCancellationOrSenderDisconnect(endpoint, pending.ActivityId, halibutRedisTransport, NodeIsOfflineHeartBeatTimeoutForRequestSender, log); - var response = new RequestMessageWithCancellationToken(pending, watcher.RequestProcessingCancellationToken); + + var cts = CancellationTokenSource.CreateLinkedTokenSource(watcher.RequestProcessingCancellationToken, dataLossCT); + disposables.Add(cts); + + var response = new RequestMessageWithCancellationToken(pending, cts.Token); disposablesForInFlightRequests[pending.ActivityId] = disposables; return response; } diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueueFactory.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueueFactory.cs index 1d100120a..65fe38582 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueueFactory.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueueFactory.cs @@ -13,6 +13,7 @@ // limitations under the License. using System; +using System.Threading.Tasks; using Halibut.Diagnostics; using Halibut.Queue.QueuedDataStreams; using Halibut.ServiceModel; @@ -26,10 +27,12 @@ public class RedisPendingRequestQueueFactory : IPendingRequestQueueFactory readonly HalibutRedisTransport halibutRedisTransport; readonly ILogFactory logFactory; readonly HalibutTimeoutsAndLimits halibutTimeoutsAndLimits; + readonly IWatchForRedisLosingAllItsData watchForRedisLosingAllItsData; public RedisPendingRequestQueueFactory( QueueMessageSerializer queueMessageSerializer, IStoreDataStreamsForDistributedQueues dataStreamStorage, + IWatchForRedisLosingAllItsData watchForRedisLosingAllItsData, HalibutRedisTransport halibutRedisTransport, HalibutTimeoutsAndLimits halibutTimeoutsAndLimits, ILogFactory logFactory) @@ -39,15 +42,20 @@ public RedisPendingRequestQueueFactory( this.halibutRedisTransport = halibutRedisTransport; this.logFactory = logFactory; this.halibutTimeoutsAndLimits = halibutTimeoutsAndLimits; + this.watchForRedisLosingAllItsData = watchForRedisLosingAllItsData; } + + public IPendingRequestQueue CreateQueue(Uri endpoint) { return new RedisPendingRequestQueue(endpoint, + watchForRedisLosingAllItsData, logFactory.ForEndpoint(endpoint), halibutRedisTransport, new MessageReaderWriter(queueMessageSerializer, dataStreamStorage), halibutTimeoutsAndLimits); } + } } \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/WatchForRedisLosingAllItsData.cs b/source/Halibut/Queue/Redis/WatchForRedisLosingAllItsData.cs index fa05b1332..fe534311a 100644 --- a/source/Halibut/Queue/Redis/WatchForRedisLosingAllItsData.cs +++ b/source/Halibut/Queue/Redis/WatchForRedisLosingAllItsData.cs @@ -20,7 +20,18 @@ namespace Halibut.Queue.Redis { - public class WatchForRedisLosingAllItsData : IAsyncDisposable + public interface IWatchForRedisLosingAllItsData : IAsyncDisposable + { + /// + /// Will cause the caller to wait until we are connected to redis and so can detect datalose. + /// + /// Time to wait for this to reach a state where it can detect datalose + /// + /// A cancellation token which is triggered when data lose occurs. + Task GetTokenForDataLoseDetection(TimeSpan timeToWait, CancellationToken cancellationToken); + } + + public class WatchForRedisLosingAllItsData : IWatchForRedisLosingAllItsData { RedisFacade redisFacade; readonly ILog log; @@ -55,6 +66,7 @@ public async Task GetTokenForDataLoseDetection(TimeSpan timeT return await taskCompletionSource.Task; } + // TODO: Check if tentacle needs this to be classified as exception that can be retried. await using var cts = CancellationTokenSource.CreateLinkedTokenSource(cancellationToken).CancelOnDispose(); cts.CancellationTokenSource.CancelAfter(timeToWait); return await taskCompletionSource.Task.WaitAsync(cts.CancellationToken); diff --git a/source/Halibut/ServiceModel/PendingRequestQueueAsync.cs b/source/Halibut/ServiceModel/PendingRequestQueueAsync.cs index 9750699b2..d76fab3a2 100644 --- a/source/Halibut/ServiceModel/PendingRequestQueueAsync.cs +++ b/source/Halibut/ServiceModel/PendingRequestQueueAsync.cs @@ -58,7 +58,7 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can try { - await pending.WaitUntilComplete(() => Task.CompletedTask, cancellationToken); + await pending.WaitUntilComplete(() => Task.CompletedTask, () => null, cancellationToken); } finally { diff --git a/source/Halibut/ServiceModel/PendingRequestV2.cs b/source/Halibut/ServiceModel/PendingRequestV2.cs index a88ce3de4..01e15a3ec 100644 --- a/source/Halibut/ServiceModel/PendingRequestV2.cs +++ b/source/Halibut/ServiceModel/PendingRequestV2.cs @@ -58,39 +58,55 @@ public PendingRequest(RequestMessage request, ILog log) /// This gives the user an opportunity to remove the pending request from shared places and optionally /// call BeginTransfer /// + /// /// - public async Task WaitUntilComplete(Func checkIfPendingRequestWasCollectedOrRemoveIt, CancellationToken cancellationToken) + public async Task WaitUntilComplete(Func checkIfPendingRequestWasCollectedOrRemoveIt, + Func cancellationReason, + CancellationToken cancellationToken) { log.Write(EventType.MessageExchange, "Request {0} was queued", request); - await using var cts = CancellationTokenSource.CreateLinkedTokenSource(cancellationToken).CancelOnDispose(); - var pendingRequestPickupTimeout = Try.IgnoringError(async () => await Task.Delay(request.Destination.PollingRequestQueueTimeout, cts.CancellationToken)); - var responseWaiterTask = responseWaiter.WaitAsync(cts.CancellationToken); + var pendingRequestPickupTimeout = Try.IgnoringError(async () => await Task.Delay(request.Destination.PollingRequestQueueTimeout, cancellationToken)); + var responseWaiterTask = responseWaiter.WaitAsync(cancellationToken); await Task.WhenAny(pendingRequestPickupTimeout, responseWaiterTask); - using (await transferLock.LockAsync(CancellationToken.None)) + + // Response has been returned so just say we are done. + if (responseWaiter.IsSet) { - // Response has been returned so just say we are done. - if (responseWaiter.IsSet) - { - log.Write(EventType.MessageExchange, "Request {0} was collected by the polling endpoint", request); - return; - } + log.Write(EventType.MessageExchange, "Request {0} was collected by the polling endpoint", request); + return; } if (!requestCollected.IsSet) await checkIfPendingRequestWasCollectedOrRemoveIt(); using (await transferLock.LockAsync(CancellationToken.None)) { + if (responseWaiter.IsSet) + { + log.Write(EventType.MessageExchange, "Request {0} was collected by the polling endpoint", request); + return; + } + if (cancellationToken.IsCancellationRequested) { - if (!requestCollected.IsSet) log.Write(EventType.MessageExchange, "Request {0} was cancelled before it could be collected by the polling endpoint", request); - else log.Write(EventType.MessageExchange, "Request {0} was collected by the polling endpoint, will try to cancel the request", request); + // TODO: This seems sus, we throw here but we don't throw below. This should be straightened out. + OperationCanceledException operationCanceledException; + if (!requestCollected.IsSet) + { + log.Write(EventType.MessageExchange, "Request {0} was cancelled before it could be collected by the polling endpoint" + cancellationReason()??"", request); + operationCanceledException = new OperationCanceledException($"Request {request} was cancelled before it could be collected by the polling endpoint" + cancellationReason()??""); + } + else + { + log.Write(EventType.MessageExchange, "Request {0} was collected by the polling endpoint, will try to cancel the request" + cancellationReason()??"", request); + operationCanceledException = new OperationCanceledException($"Request {request} was collected by the polling endpoint, will try to cancel the request" + cancellationReason() ?? ""); + } await Try.IgnoringError(async () => await pendingRequestCancellationTokenSource.CancelAsync()); - cancellationToken.ThrowIfCancellationRequested(); + throw requestCollected.IsSet ? new TransferringRequestCancelledException(operationCanceledException) : new ConnectingRequestCancelledException(operationCanceledException); } if (!requestCollected.IsSet) @@ -116,20 +132,19 @@ public async Task WaitUntilComplete(Func checkIfPendingRequestWasCollected { await responseWaiterTask; } - catch (Exception) when (!cts.CancellationToken.IsCancellationRequested) + catch (Exception) when (cancellationToken.IsCancellationRequested) { using (await transferLock.LockAsync(CancellationToken.None)) { if (!responseWaiter.IsSet) { - log.Write(EventType.MessageExchange, "Request {0} was cancelled before a response was received", request); + log.Write(EventType.MessageExchange, "Request {0} was cancelled before a response was received" + cancellationReason()??"", request); SetResponseNoLock(ResponseMessage.FromException( request, - new TimeoutException($"A request was sent to a polling endpoint, the polling endpoint collected it but the request was cancelled before the polling endpoint responded."), + new TimeoutException($"A request was sent to a polling endpoint, the polling endpoint collected it but the request was cancelled before the polling endpoint responded." + cancellationReason()??""), ConnectionState.Connecting), false); await Try.IgnoringError(async () => await pendingRequestCancellationTokenSource.CancelAsync()); - cancellationToken.ThrowIfCancellationRequested(); } } } From a8aacd34a5645934737e0c29aee193a7d9732857 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Thu, 7 Aug 2025 13:13:48 +1000 Subject: [PATCH 026/137] . --- source/Halibut/Queue/Redis/NodeHeartBeatSender.cs | 1 + source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs | 5 +++-- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs b/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs index 37f27b0fd..0025b0c46 100644 --- a/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs +++ b/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs @@ -173,6 +173,7 @@ static async Task WaitForRequestToBeCollected(Uri endpoint, RequestMessage reque { log.Write(EventType.Diagnostic, "Waiting for request {0} to be collected from queue", request.ActivityId); + // TODO: um like listen to the heart beat here, because heart beat == collected (maybe) while (!cancellationToken.IsCancellationRequested) { try diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index c6f404c9b..332c9812c 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -112,8 +112,8 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can await pending.WaitUntilComplete( async () => await tryClearRequestFromQueueAtMostOnce.Task, - () => dataLoseCt.IsCancellationRequested ? " Cancelled because data loss on redis was detected." : "", - cancellationToken); + () => dataLoseCt.IsCancellationRequested ? " Cancelled because data loss on redis was detected." : "", + cancellationToken); } finally { @@ -339,6 +339,7 @@ public async Task ApplyResponse(ResponseMessage response, Guid requestActivityId async Task DequeueNextAsync() { + // TODO use queue token. var cancellationToken = CancellationToken.None; using var cancellationTokenSource = CancellationTokenSource.CreateLinkedTokenSource(cancellationToken); From ee7a794066a2c9e495a5c30a1398026743120e02 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Thu, 7 Aug 2025 13:15:49 +1000 Subject: [PATCH 027/137] . --- .../Queue/Redis/NodeHeartBeatSenderFixture.cs | 443 ++++++++++++++++++ .../Queue/Redis/RedisFacadeFixture.cs | 392 ++++++++++++++++ ...leDataLossWatchForRedisLosingAllItsData.cs | 54 +++ ...LosingDataWatchForRedisLosingAllItsData.cs | 38 ++ .../WatchForRedisLosingAllItsDataFixture.cs | 107 +++++ 5 files changed, 1034 insertions(+) create mode 100644 source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs create mode 100644 source/Halibut.Tests/Queue/Redis/RedisFacadeFixture.cs create mode 100644 source/Halibut.Tests/Queue/Redis/Utils/CancellableDataLossWatchForRedisLosingAllItsData.cs create mode 100644 source/Halibut.Tests/Queue/Redis/Utils/NeverLosingDataWatchForRedisLosingAllItsData.cs create mode 100644 source/Halibut.Tests/Queue/Redis/WatchForRedisLosingAllItsDataFixture.cs diff --git a/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs b/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs new file mode 100644 index 000000000..13969c515 --- /dev/null +++ b/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs @@ -0,0 +1,443 @@ +using System; +using System.Collections.Concurrent; +using System.Collections.Generic; +using System.Diagnostics.CodeAnalysis; +using System.Linq; +using System.Threading; +using System.Threading.Tasks; +using FluentAssertions; +using Halibut.Diagnostics; +using Halibut.Logging; +using Halibut.Queue.Redis; +using Halibut.ServiceModel; +using Halibut.Tests.Builders; +using Halibut.Tests.Support.Logging; +using Halibut.Transport.Protocol; +using Halibut.Util; +using Nito.AsyncEx; +using NUnit.Framework; +using Octopus.TestPortForwarder; + +namespace Halibut.Tests.Queue.Redis +{ + [SuppressMessage("Usage", "VSTHRD003:Avoid awaiting foreign Tasks")] + public class NodeHeartBeatSenderFixture : BaseTest + { + const int redisPort = 6379; + + private static RedisFacade CreateRedisFacade(int? port = redisPort, Guid? guid = null) => + new("localhost:" + port, (guid ?? Guid.NewGuid()).ToString(), + new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix("")); + + // TODO: ai tests need review + [Test] + public async Task WhenCreated_ShouldStartSendingHeartbeats() + { + // Arrange + var endpoint = new Uri("poll://" + Guid.NewGuid()); + var requestActivityId = Guid.NewGuid(); + var log = new TestContextLogCreator("NodeHeartBeat", LogLevel.Trace).CreateNewForPrefix(""); + var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); + + var anyHeartBeatReceived = new AsyncManualResetEvent(false); + + // Subscribe to heartbeats before creating the sender + await using var subscription = await redisTransport.SubscribeToNodeHeartBeatChannel( + endpoint, requestActivityId, HalibutQueueNodeSendingPulses.Receiver, async () => + { + await Task.CompletedTask; + anyHeartBeatReceived.Set(); + }, CancellationToken); + + // Act + await using var heartBeatSender = new NodeHeartBeatSender(endpoint, requestActivityId, redisTransport, log, HalibutQueueNodeSendingPulses.Receiver, defaultDelayBetweenPulses: TimeSpan.FromSeconds(1)); + + // Wait for a few heartbeats + await Task.WhenAny(Task.Delay(TimeSpan.FromSeconds(20), CancellationToken), anyHeartBeatReceived.WaitAsync()); + + // Assert + anyHeartBeatReceived.IsSet.Should().BeTrue("Should have received at least one heartbeat"); + } + + [Test] + public async Task WhenRedisConnectionIsInterrupted_ShouldSwitchToPanicMode() + { + // Arrange + var endpoint = new Uri("poll://" + Guid.NewGuid()); + var requestActivityId = Guid.NewGuid(); + var log = new TestContextLogCreator("NodeHeartBeat", LogLevel.Trace).CreateNewForPrefix(""); + var guid = Guid.NewGuid(); + + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); + await using var unstableRedisFacade = CreateRedisFacade(portForwarder.ListeningPort, guid); + await using var stableRedisFacade = CreateRedisFacade(redisPort, guid); + + var redisTransport = new HalibutRedisTransport(unstableRedisFacade); + + var heartbeatsReceived = new ConcurrentBag(); + + // Subscribe with stable connection to monitor heartbeats + await using var subscription = await new HalibutRedisTransport(stableRedisFacade) + .SubscribeToNodeHeartBeatChannel( + endpoint, requestActivityId, HalibutQueueNodeSendingPulses.Receiver, async () => + { + await Task.CompletedTask; + heartbeatsReceived.Add(DateTimeOffset.Now); + }, CancellationToken); + + // Act + await using var heartBeatSender = new NodeHeartBeatSender(endpoint, requestActivityId, redisTransport, log, HalibutQueueNodeSendingPulses.Receiver, TimeSpan.FromSeconds(1)); + + // Wait for initial heartbeat + await Task.Delay(TimeSpan.FromSeconds(2), CancellationToken); + var initialHeartbeatCount = heartbeatsReceived.Count; + + // Interrupt connection + portForwarder.EnterKillNewAndExistingConnectionsMode(); + + // Wait during the outage + await Task.Delay(TimeSpan.FromSeconds(10), CancellationToken); + var heartbeatsDuringOutage = heartbeatsReceived.Count - initialHeartbeatCount; + + // Restore connection + portForwarder.ReturnToNormalMode(); + + // Wait for recovery + await Task.Delay(TimeSpan.FromSeconds(15), CancellationToken); + var heartbeatsAfterRecovery = heartbeatsReceived.Count - initialHeartbeatCount - heartbeatsDuringOutage; + + // Assert + initialHeartbeatCount.Should().BeGreaterThan(0, "Should have received initial heartbeats"); + heartbeatsDuringOutage.Should().Be(0, "Should not receive heartbeats during network outage"); + heartbeatsAfterRecovery.Should().BeGreaterThan(0, "Should resume sending heartbeats after recovery"); + } + + [Test] + public async Task WhenDisposed_ShouldStopSendingHeartbeats() + { + // Arrange + var endpoint = new Uri("poll://" + Guid.NewGuid()); + var requestActivityId = Guid.NewGuid(); + var log = new TestContextLogCreator("NodeHeartBeat", LogLevel.Trace).CreateNewForPrefix(""); + var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); + + var heartbeatsReceived = new ConcurrentBag(); + var anyHeartBeatReceived = new AsyncManualResetEvent(false); + + await using var subscription = await redisTransport.SubscribeToNodeHeartBeatChannel( + endpoint, requestActivityId, HalibutQueueNodeSendingPulses.Receiver, async () => + { + await Task.CompletedTask; + anyHeartBeatReceived.Set(); + heartbeatsReceived.Add(DateTimeOffset.Now); + }, CancellationToken); + + // Act + var heartBeatSender = new NodeHeartBeatSender(endpoint, requestActivityId, redisTransport, log, HalibutQueueNodeSendingPulses.Receiver, defaultDelayBetweenPulses: TimeSpan.FromSeconds(1)); + + // Wait for some heartbeats + await anyHeartBeatReceived.WaitAsync(CancellationToken); + + // Dispose the sender + await heartBeatSender.DisposeAsync(); + + await heartBeatSender.TaskSendingPulses; + + anyHeartBeatReceived.Reset(); + + await Task.WhenAny(Task.Delay(TimeSpan.FromSeconds(5), CancellationToken), anyHeartBeatReceived.WaitAsync()); + + anyHeartBeatReceived.IsSet.Should().BeFalse(); + } + + [Test] + public async Task WaitUntilNodeProcessingRequestFlatLines_WhenHeartbeatsStop_ShouldReturnProcessingNodeIsLikelyDisconnected() + { + // Arrange + var endpoint = new Uri("poll://" + Guid.NewGuid()); + var requestActivityId = Guid.NewGuid(); + var log = new TestContextLogCreator("NodeHeartBeat", LogLevel.Trace).CreateNewForPrefix(""); + var guid = Guid.NewGuid(); + + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); + await using var unstableRedisFacade = CreateRedisFacade(portForwarder.ListeningPort, guid); + await using var stableRedisFacade = CreateRedisFacade(redisPort, guid); + + var unstableRedisTransport = new HalibutRedisTransport(unstableRedisFacade); + var stableRedisTransport = new HalibutRedisTransport(stableRedisFacade); + + var request = new RequestMessageBuilder(endpoint.ToString()) + .WithActivityId(requestActivityId) + .Build(); + var pendingRequest = new PendingRequest(request, log); + + // Start heartbeat sender + await using var heartBeatSender = new NodeHeartBeatSender(endpoint, requestActivityId, unstableRedisTransport, log, HalibutQueueNodeSendingPulses.Receiver, TimeSpan.FromSeconds(1)); + + // Mark request as collected so watcher proceeds to monitoring phase + await pendingRequest.RequestHasBeenCollectedAndWillBeTransferred(); + + // Start the watcher + var watcherTask = NodeHeartBeatSender.WatchThatNodeProcessingTheRequestIsStillAlive( + endpoint, + request, + pendingRequest, + stableRedisTransport, + log, + TimeSpan.FromSeconds(10), // Short timeout for test + CancellationToken); + + // Wait for initial heartbeats to establish baseline + await Task.Delay(TimeSpan.FromSeconds(3), CancellationToken); + + // Act - Kill the connection to stop heartbeats + portForwarder.EnterKillNewAndExistingConnectionsMode(); + + // Assert + var result = await watcherTask; + result.Should().Be(NodeHeartBeatSender.NodeProcessingRequestWatcherResult.NodeMayHaveDisconnected); + } + + [Test] + public async Task NoIssueReturnedWhenNodeProcessingRequestIsNotSeenToGoOffline() + { + // Arrange + var endpoint = new Uri("poll://" + Guid.NewGuid()); + var requestActivityId = Guid.NewGuid(); + var log = new TestContextLogCreator("NodeHeartBeat", LogLevel.Trace).CreateNewForPrefix(""); + var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); + + var request = new RequestMessageBuilder(endpoint.ToString()) + .WithActivityId(requestActivityId) + .Build(); + var pendingRequest = new PendingRequest(request, log); + await pendingRequest.RequestHasBeenCollectedAndWillBeTransferred(); + + // Start heartbeat sender + await using var heartBeatSender = new NodeHeartBeatSender(endpoint, requestActivityId, redisTransport, log, HalibutQueueNodeSendingPulses.Receiver, TimeSpan.FromSeconds(1)); + + // Start the watcher without marking request as collected + using var cts = new CancellationTokenSource(); + var watcherTask = NodeHeartBeatSender.WatchThatNodeProcessingTheRequestIsStillAlive( + endpoint, + request, + pendingRequest, + redisTransport, + log, + TimeSpan.FromMinutes(5), + cts.Token); + + await cts.CancelAsync(); + + // Assert + var result = await watcherTask; + result.Should().Be(NodeHeartBeatSender.NodeProcessingRequestWatcherResult.NoDisconnectSeen); + } + + [Test] + public async Task WaitUntilNodeProcessingRequestFlatLines_WhenConnectionInterruptedDuringMonitoring_ShouldStillDetectFlatline() + { + // Arrange + var endpoint = new Uri("poll://" + Guid.NewGuid()); + var requestActivityId = Guid.NewGuid(); + var log = new TestContextLogCreator("NodeHeartBeat", LogLevel.Trace).CreateNewForPrefix(""); + var guid = Guid.NewGuid(); + + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); + await using var unstableRedisFacade = CreateRedisFacade(portForwarder.ListeningPort, guid); + await using var stableRedisFacade = CreateRedisFacade(redisPort, guid); + + var unstableRedisTransport = new HalibutRedisTransport(unstableRedisFacade); + var stableRedisTransport = new HalibutRedisTransport(stableRedisFacade); + + var request = new RequestMessageBuilder(endpoint.ToString()) + .WithActivityId(requestActivityId) + .Build(); + var pendingRequest = new PendingRequest(request, log); + + // Start heartbeat sender with unstable connection + await using var heartBeatSender = new NodeHeartBeatSender(endpoint, requestActivityId, unstableRedisTransport, log, HalibutQueueNodeSendingPulses.Receiver, TimeSpan.FromSeconds(1)); + + // Mark request as collected + await pendingRequest.RequestHasBeenCollectedAndWillBeTransferred(); + + // Start watcher with stable connection + var watcherTask = NodeHeartBeatSender.WatchThatNodeProcessingTheRequestIsStillAlive( + endpoint, + request, + pendingRequest, + stableRedisTransport, + log, + TimeSpan.FromSeconds(15), // Short timeout for test + CancellationToken); + + // Wait for initial heartbeats + await Task.Delay(TimeSpan.FromSeconds(3), CancellationToken); + + // Act - Interrupt the heartbeat sender's connection + portForwarder.EnterKillNewAndExistingConnectionsMode(); + + // Assert - Watcher should detect flatline + var result = await watcherTask; + result.Should().Be(NodeHeartBeatSender.NodeProcessingRequestWatcherResult.NodeMayHaveDisconnected); + } + + [Test] + public async Task WhenMultipleHeartBeatSendersForSameRequest_OnlyOneSetOfHeartbeatsShouldBeReceived() + { + // Arrange + var endpoint = new Uri("poll://" + Guid.NewGuid()); + var requestActivityId = Guid.NewGuid(); + var log = new TestContextLogCreator("NodeHeartBeat", LogLevel.Trace).CreateNewForPrefix(""); + var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); + + var heartbeatsReceived = new ConcurrentBag(); + + await using var subscription = await redisTransport.SubscribeToNodeHeartBeatChannel( + endpoint, requestActivityId, HalibutQueueNodeSendingPulses.Receiver, async () => + { + await Task.CompletedTask; + heartbeatsReceived.Add(DateTimeOffset.Now); + }, CancellationToken); + + // Act - Create multiple senders for the same request + await using var heartBeatSender1 = new NodeHeartBeatSender(endpoint, requestActivityId, redisTransport, log, HalibutQueueNodeSendingPulses.Receiver, TimeSpan.FromSeconds(1)); + await using var heartBeatSender2 = new NodeHeartBeatSender(endpoint, requestActivityId, redisTransport, log, HalibutQueueNodeSendingPulses.Receiver, TimeSpan.FromSeconds(1)); + + // Wait for heartbeats + await Task.Delay(TimeSpan.FromSeconds(10), CancellationToken); + + // Assert + heartbeatsReceived.Should().NotBeEmpty("Should have received heartbeats"); + // Note: We can't easily assert the exact count since both senders are publishing, + // but we can verify the system handles multiple senders gracefully + } + + [Test] + public async Task WhenHeartBeatSenderConnectionRecovery_ShouldResumeNormalHeartbeatInterval() + { + // Arrange + var endpoint = new Uri("poll://" + Guid.NewGuid()); + var requestActivityId = Guid.NewGuid(); + var log = new TestContextLogCreator("NodeHeartBeat", LogLevel.Trace).CreateNewForPrefix(""); + var guid = Guid.NewGuid(); + + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); + await using var unstableRedisFacade = CreateRedisFacade(portForwarder.ListeningPort, guid); + await using var stableRedisFacade = CreateRedisFacade(redisPort, guid); + + var unstableRedisTransport = new HalibutRedisTransport(unstableRedisFacade); + var stableRedisTransport = new HalibutRedisTransport(stableRedisFacade); + + var heartbeatTimestamps = new ConcurrentBag(); + + await using var subscription = await stableRedisTransport.SubscribeToNodeHeartBeatChannel( + endpoint, requestActivityId, HalibutQueueNodeSendingPulses.Receiver, async () => + { + await Task.CompletedTask; + heartbeatTimestamps.Add(DateTimeOffset.Now); + }, CancellationToken); + + // Act + await using var heartBeatSender = new NodeHeartBeatSender(endpoint, requestActivityId, unstableRedisTransport, log, HalibutQueueNodeSendingPulses.Receiver, TimeSpan.FromSeconds(1)); + + // Wait for initial heartbeats (normal 15s interval) + await Task.Delay(TimeSpan.FromSeconds(5), CancellationToken); + + // Interrupt connection to trigger panic mode (7s interval) + portForwarder.EnterKillNewAndExistingConnectionsMode(); + await Task.Delay(TimeSpan.FromSeconds(2), CancellationToken); + + // Restore connection + portForwarder.ReturnToNormalMode(); + + // Wait for recovery and return to normal intervals + await Task.Delay(TimeSpan.FromSeconds(20), CancellationToken); + + // Assert + heartbeatTimestamps.Should().NotBeEmpty("Should have received heartbeats after recovery"); + + // Verify we have heartbeats spanning the recovery period + var timestamps = heartbeatTimestamps.ToArray(); + if (timestamps.Length > 1) + { + var timeSpan = timestamps.Max() - timestamps.Min(); + timeSpan.Should().BeGreaterThan(TimeSpan.FromSeconds(10), "Should have heartbeats over recovery period"); + } + } + + [Test] + public async Task SenderAndReceiverNodeTypes_ShouldUseDistinctChannels() + { + // Arrange + var endpoint = new Uri("poll://" + Guid.NewGuid()); + var requestActivityId = Guid.NewGuid(); + var log = new TestContextLogCreator("NodeHeartBeat", LogLevel.Trace).CreateNewForPrefix(""); + var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); + + var senderHeartbeatsReceived = new AsyncManualResetEvent(false); + var receiverHeartbeatsReceived = new AsyncManualResetEvent(false); + + // Subscribe to sender heartbeats + await using var senderSubscription = await redisTransport.SubscribeToNodeHeartBeatChannel( + endpoint, requestActivityId, HalibutQueueNodeSendingPulses.Sender, async () => + { + await Task.CompletedTask; + senderHeartbeatsReceived.Set(); + }, CancellationToken); + + // Subscribe to receiver heartbeats + await using var receiverSubscription = await redisTransport.SubscribeToNodeHeartBeatChannel( + endpoint, requestActivityId, HalibutQueueNodeSendingPulses.Receiver, async () => + { + await Task.CompletedTask; + receiverHeartbeatsReceived.Set(); + }, CancellationToken); + + // Act - Create sender node heartbeat sender + await using var senderHeartBeatSender = new NodeHeartBeatSender(endpoint, requestActivityId, redisTransport, log, HalibutQueueNodeSendingPulses.Sender, defaultDelayBetweenPulses: TimeSpan.FromSeconds(1)); + + // Wait for sender heartbeat + await Task.WhenAny(Task.Delay(TimeSpan.FromSeconds(5), CancellationToken), senderHeartbeatsReceived.WaitAsync()); + + // Create receiver node heartbeat sender + await using var receiverHeartBeatSender = new NodeHeartBeatSender(endpoint, requestActivityId, redisTransport, log, HalibutQueueNodeSendingPulses.Receiver, defaultDelayBetweenPulses: TimeSpan.FromSeconds(1)); + + // Wait for receiver heartbeat + await Task.WhenAny(Task.Delay(TimeSpan.FromSeconds(5), CancellationToken), receiverHeartbeatsReceived.WaitAsync()); + + // Assert + senderHeartbeatsReceived.IsSet.Should().BeTrue("Should have received sender heartbeat"); + receiverHeartbeatsReceived.IsSet.Should().BeTrue("Should have received receiver heartbeat"); + } + + [Test] + public async Task SenderNodeHeartbeats_ShouldNotBeReceivedByReceiverSubscription() + { + // Arrange + var endpoint = new Uri("poll://" + Guid.NewGuid()); + var requestActivityId = Guid.NewGuid(); + var log = new TestContextLogCreator("NodeHeartBeat", LogLevel.Trace).CreateNewForPrefix(""); + var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); + + var receiverHeartbeatsReceived = new AsyncManualResetEvent(false); + + // Subscribe only to receiver heartbeats + await using var receiverSubscription = await redisTransport.SubscribeToNodeHeartBeatChannel( + endpoint, requestActivityId, HalibutQueueNodeSendingPulses.Receiver, async () => + { + await Task.CompletedTask; + receiverHeartbeatsReceived.Set(); + }, CancellationToken); + + // Act - Create sender node heartbeat sender (should not trigger receiver subscription) + await using var senderHeartBeatSender = new NodeHeartBeatSender(endpoint, requestActivityId, redisTransport, log, HalibutQueueNodeSendingPulses.Sender, defaultDelayBetweenPulses: TimeSpan.FromSeconds(1)); + + // Wait to see if receiver subscription gets triggered (it shouldn't) + await Task.WhenAny(Task.Delay(TimeSpan.FromSeconds(3), CancellationToken), receiverHeartbeatsReceived.WaitAsync()); + + // Assert + receiverHeartbeatsReceived.IsSet.Should().BeFalse("Should not have received sender heartbeat on receiver subscription"); + } + } +} \ No newline at end of file diff --git a/source/Halibut.Tests/Queue/Redis/RedisFacadeFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisFacadeFixture.cs new file mode 100644 index 000000000..a1c58ccc1 --- /dev/null +++ b/source/Halibut.Tests/Queue/Redis/RedisFacadeFixture.cs @@ -0,0 +1,392 @@ +using System; +using System.Collections.Generic; +using System.Threading.Tasks; +using FluentAssertions; +using Halibut.Logging; +using Halibut.Queue.Redis; +using Halibut.Tests.Support.Logging; +using Halibut.Util.AsyncEx; +using NUnit.Framework; + +namespace Halibut.Tests.Queue.Redis +{ + public class RedisFacadeFixture : BaseTest + { + // AI generated :S + private static RedisFacade CreateRedisFacade() => new("localhost", Guid.NewGuid().ToString(), new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix("")); + + [Test] + public void Constructor_WithRedisHostAndKeyPrefix_ShouldCreateInstance() + { + // Arrange & Act + var redisFacade = new RedisFacade("localhost", "test-prefix", new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix("")); + + // Assert + redisFacade.Should().NotBeNull(); + } + + [Test] + public void Constructor_WithNullKeyPrefix_ShouldUseDefaultPrefix() + { + // Arrange & Act + var redisFacade = new RedisFacade("localhost", null, new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix("")); + + // Assert + redisFacade.Should().NotBeNull(); + } + + [Test] + public async Task SetString_AndGetString_ShouldStoreAndRetrieveValue() + { + // Arrange + await using var redisFacade = CreateRedisFacade(); + var key = Guid.NewGuid().ToString(); + var value = "test-value"; + + // Act + await redisFacade.SetString(key, value, TimeSpan.FromMinutes(1), CancellationToken); + var retrievedValue = await redisFacade.GetString(key, CancellationToken); + + // Assert + retrievedValue.Should().Be(value); + } + + [Test] + public async Task GetString_WithNonExistentKey_ShouldReturnNull() + { + // Arrange + await using var redisFacade = CreateRedisFacade(); + var nonExistentKey = Guid.NewGuid().ToString(); + + // Act + var retrievedValue = await redisFacade.GetString(nonExistentKey, CancellationToken); + + // Assert + retrievedValue.Should().BeNull(); + } + + [Test] + public async Task SetInHash_ShouldStoreValueInHash() + { + // Arrange + await using var redisFacade = CreateRedisFacade(); + var key = Guid.NewGuid().ToString(); + var field = "test-field"; + var payload = "test-payload"; + + // Act + await redisFacade.SetInHash(key, field, payload); + + // Assert - We'll verify by trying to get and delete it + var retrievedValue = await redisFacade.TryGetAndDeleteFromHash(key, field); + retrievedValue.Should().Be(payload); + } + + [Test] + public async Task TryGetAndDeleteFromHash_WithExistingValue_ShouldReturnValueAndDelete() + { + // Arrange + await using var redisFacade = CreateRedisFacade(); + var key = Guid.NewGuid().ToString(); + var field = "test-field"; + var payload = "test-payload"; + + await redisFacade.SetInHash(key, field, payload); + + // Act + var retrievedValue = await redisFacade.TryGetAndDeleteFromHash(key, field); + + // Assert + retrievedValue.Should().Be(payload); + } + + [Test] + public async Task HashContainsKey_WithExistingField_ShouldReturnTrue() + { + // Arrange + await using var redisFacade = CreateRedisFacade(); + var key = Guid.NewGuid().ToString(); + var field = "test-field"; + var payload = "test-payload"; + + await redisFacade.SetInHash(key, field, payload); + + // Act + var exists = await redisFacade.HashContainsKey(key, field); + + // Assert + exists.Should().BeTrue(); + } + + [Test] + public async Task HashContainsKey_WithNonExistentField_ShouldReturnFalse() + { + // Arrange + await using var redisFacade = CreateRedisFacade(); + var key = Guid.NewGuid().ToString(); + var nonExistentField = "non-existent-field"; + + // Act + var exists = await redisFacade.HashContainsKey(key, nonExistentField); + + // Assert + exists.Should().BeFalse(); + } + + [Test] + public async Task HashContainsKey_WithNonExistentKey_ShouldReturnFalse() + { + // Arrange + await using var redisFacade = CreateRedisFacade(); + var nonExistentKey = Guid.NewGuid().ToString(); + var field = "test-field"; + + // Act + var exists = await redisFacade.HashContainsKey(nonExistentKey, field); + + // Assert + exists.Should().BeFalse(); + } + + [Test] + public async Task TryGetAndDeleteFromHash_ShouldDeleteTheEntireKey() + { + // Arrange + await using var redisFacade = CreateRedisFacade(); + var key = Guid.NewGuid().ToString(); + var field = "test-field"; + var payload = "test-payload"; + + await redisFacade.SetInHash(key, field, payload); + + // Verify the hash field exists + var existsBefore = await redisFacade.HashContainsKey(key, field); + existsBefore.Should().BeTrue(); + + // Act + var retrievedValue = await redisFacade.TryGetAndDeleteFromHash(key, field); + + // Assert + retrievedValue.Should().Be(payload); + + // Verify the entire key was deleted (not just the field) + var existsAfter = await redisFacade.HashContainsKey(key, field); + existsAfter.Should().BeFalse(); + + // Verify trying to get it again returns null + var secondRetrieval = await redisFacade.TryGetAndDeleteFromHash(key, field); + secondRetrieval.Should().BeNull(); + } + + [Test] + public async Task ListRightPushAsync_AndListLeftPopAsync_ShouldWorkAsQueue() + { + // Arrange + await using var redisFacade = CreateRedisFacade(); + var key = Guid.NewGuid().ToString(); + var payload1 = "first-item"; + var payload2 = "second-item"; + + // Act - Push items to the right + await redisFacade.ListRightPushAsync(key, payload1); + await redisFacade.ListRightPushAsync(key, payload2); + + // Pop items from the left (FIFO) + var firstItem = await redisFacade.ListLeftPopAsync(key); + var secondItem = await redisFacade.ListLeftPopAsync(key); + + // Assert + firstItem.Should().Be(payload1); + secondItem.Should().Be(payload2); + } + + [Test] + public async Task ListLeftPopAsync_WithEmptyList_ShouldReturnNull() + { + // Arrange + await using var redisFacade = CreateRedisFacade(); + var emptyListKey = Guid.NewGuid().ToString(); + + // Act + var result = await redisFacade.ListLeftPopAsync(emptyListKey); + + // Assert + result.Should().BeNull(); + } + + [Test] + public async Task PublishToChannel_AndSubscribeToChannel_ShouldDeliverMessage() + { + // Arrange + await using var redisFacade = CreateRedisFacade(); + var channelName = Guid.NewGuid().ToString(); + var testMessage = "test-message"; + var receivedMessages = new List(); + var messageReceived = new TaskCompletionSource(); + + // Subscribe to the channel + await using var subscription = await redisFacade.SubscribeToChannel(channelName, async message => + { + await Task.Yield(); // Make it properly async + if (!message.Message.IsNull) + { + receivedMessages.Add(message.Message!); + messageReceived.SetResult(true); + } + }, CancellationToken); + + // Act - Publish a message + await redisFacade.PublishToChannel(channelName, testMessage); + + // Wait for the message to be received + await messageReceived.Task.TimeoutAfter(TimeSpan.FromSeconds(5), CancellationToken); + + // Assert + receivedMessages.Should().HaveCount(1); + receivedMessages[0].Should().Be(testMessage); + } + + [Test] + public async Task PublishToChannel_WithMultipleMessages_ShouldDeliverAllMessages() + { + // Arrange + await using var redisFacade = CreateRedisFacade(); + var channelName = Guid.NewGuid().ToString(); + var messages = new[] { "message1", "message2", "message3" }; + var receivedMessages = new List(); + var allMessagesReceived = new TaskCompletionSource(); + + // Subscribe to the channel + await using var subscription = await redisFacade.SubscribeToChannel(channelName, async message => + { + await Task.Yield(); + if (!message.Message.IsNull) + { + receivedMessages.Add(message.Message!); + if (receivedMessages.Count == messages.Length) + { + allMessagesReceived.SetResult(true); + } + } + }, CancellationToken); + + // Act - Publish multiple messages + foreach (var msg in messages) + { + await redisFacade.PublishToChannel(channelName, msg); + } + + // Wait for all messages to be received + await allMessagesReceived.Task.TimeoutAfter(TimeSpan.FromSeconds(5), CancellationToken); + + // Assert + receivedMessages.Should().HaveCount(3); + receivedMessages.Should().Contain(messages); + } + + [Test] + public async Task SubscribeToChannel_WhenDisposed_ShouldUnsubscribe() + { + // Arrange + await using var redisFacade = CreateRedisFacade(); + var channelName = Guid.NewGuid().ToString(); + var receivedMessages = new List(); + + var subscription = await redisFacade.SubscribeToChannel(channelName, async message => + { + await Task.Yield(); + if (!message.Message.IsNull) + { + receivedMessages.Add(message.Message!); + } + }, CancellationToken); + + // Act - Dispose the subscription + await subscription.DisposeAsync(); + + // Publish a message after unsubscribing + await redisFacade.PublishToChannel(channelName, "should-not-receive"); + + // Wait a bit to ensure no message is received + await Task.Delay(100); + + // Assert + receivedMessages.Should().BeEmpty(); + } + + [Test] + public async Task KeyPrefixing_ShouldIsolateDataBetweenDifferentPrefixes() + { + // Arrange + var prefix1 = Guid.NewGuid().ToString(); + var prefix2 = Guid.NewGuid().ToString(); + + await using var redisFacade1 = new RedisFacade("localhost", prefix1, new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix("")); + await using var redisFacade2 = new RedisFacade("localhost", prefix2, new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix("")); + + var key = "shared-key"; + var value1 = "value-from-facade1"; + var value2 = "value-from-facade2"; + + // Act - Set values with the same key but different prefixes + await redisFacade1.SetString(key, value1, TimeSpan.FromMinutes(1), CancellationToken); + await redisFacade2.SetString(key, value2, TimeSpan.FromMinutes(1), CancellationToken); + + // Get values using both facades + var retrievedValue1 = await redisFacade1.GetString(key, CancellationToken); + var retrievedValue2 = await redisFacade2.GetString(key, CancellationToken); + + // Assert - Each facade should retrieve its own value + retrievedValue1.Should().Be(value1); + retrievedValue2.Should().Be(value2); + } + + // [Test] + // public void Dispose_ShouldNotThrowException() + // { + // // Arrange + // var redisFacade = CreateRedisFacade(); + // + // // Act & Assert + // Action act = () => redisFacade.Dispose(); + // act.Should().NotThrow(); + // } + // + // [Test] + // public void Dispose_CalledMultipleTimes_ShouldNotThrowException() + // { + // // Arrange + // var redisFacade = CreateRedisFacade(); + // + // // Act & Assert + // Action act = () => + // { + // redisFacade.Dispose(); + // redisFacade.Dispose(); // Second call + // }; + // act.Should().NotThrow(); + // } + + [Test] + public async Task SetInHash_WithTTL_ShouldExpireAfterSpecifiedTime() + { + // Arrange + await using var redisFacade = CreateRedisFacade(); + var key = Guid.NewGuid().ToString(); + var field = "test-field"; + var payload = "test-payload"; + + // Act - Set a value in hash (it has a TTL of 9:9:9 according to the implementation) + await redisFacade.SetInHash(key, field, payload); + + // Immediately try to get the value - should exist + var immediateValue = await redisFacade.TryGetAndDeleteFromHash(key, field); + + // Assert + immediateValue.Should().Be(payload); + + // Note: We can't easily test the actual TTL expiration in a unit test + // as it would require waiting 9+ hours, but we've verified the value is set correctly + } + } +} \ No newline at end of file diff --git a/source/Halibut.Tests/Queue/Redis/Utils/CancellableDataLossWatchForRedisLosingAllItsData.cs b/source/Halibut.Tests/Queue/Redis/Utils/CancellableDataLossWatchForRedisLosingAllItsData.cs new file mode 100644 index 000000000..e7c7e6d23 --- /dev/null +++ b/source/Halibut.Tests/Queue/Redis/Utils/CancellableDataLossWatchForRedisLosingAllItsData.cs @@ -0,0 +1,54 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; +using System.Threading; +using System.Threading.Tasks; +using Halibut.Queue.Redis; +using Halibut.Util; +using Try = Halibut.Tests.Support.Try; + +namespace Halibut.Tests.Queue.Redis.Utils +{ + public class CancellableDataLossWatchForRedisLosingAllItsData : IWatchForRedisLosingAllItsData + { + CancelOnDisposeCancellationTokenSource cancellationTokenSource = new CancellationTokenSource().CancelOnDispose(); + + public TaskCompletionSource TaskCompletionSource = new(); + public CancellableDataLossWatchForRedisLosingAllItsData() + { + TaskCompletionSource.SetResult(cancellationTokenSource.CancellationToken); + } + + public async Task DataLossHasOccured() + { + await cancellationTokenSource.DisposeAsync(); + cancellationTokenSource = new CancellationTokenSource().CancelOnDispose(); + TaskCompletionSource = new TaskCompletionSource(); + TaskCompletionSource.SetResult(cancellationTokenSource.CancellationToken); + } + + public async ValueTask DisposeAsync() + { + await Try.CatchingError(async () => await cancellationTokenSource.DisposeAsync()); + } + + public async Task GetTokenForDataLoseDetection(TimeSpan timeToWait, CancellationToken cancellationToken) + { +#pragma warning disable VSTHRD003 + return await TaskCompletionSource.Task; +#pragma warning restore VSTHRD003 + } + } +} \ No newline at end of file diff --git a/source/Halibut.Tests/Queue/Redis/Utils/NeverLosingDataWatchForRedisLosingAllItsData.cs b/source/Halibut.Tests/Queue/Redis/Utils/NeverLosingDataWatchForRedisLosingAllItsData.cs new file mode 100644 index 000000000..12c366755 --- /dev/null +++ b/source/Halibut.Tests/Queue/Redis/Utils/NeverLosingDataWatchForRedisLosingAllItsData.cs @@ -0,0 +1,38 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; +using System.Threading; +using System.Threading.Tasks; +using Halibut.Queue.Redis; + +namespace Halibut.Tests.Queue.Redis.Utils +{ + /// + /// Test implementation of IWatchForRedisLosingAllItsData that returns CancellationToken.None + /// to indicate no data loss detection is active during testing. + /// + public class NeverLosingDataWatchForRedisLosingAllItsData : IWatchForRedisLosingAllItsData + { + public Task GetTokenForDataLoseDetection(TimeSpan timeToWait, CancellationToken cancellationToken) + { + return Task.FromResult(CancellationToken.None); + } + + public ValueTask DisposeAsync() + { + return ValueTask.CompletedTask; + } + } +} \ No newline at end of file diff --git a/source/Halibut.Tests/Queue/Redis/WatchForRedisLosingAllItsDataFixture.cs b/source/Halibut.Tests/Queue/Redis/WatchForRedisLosingAllItsDataFixture.cs new file mode 100644 index 000000000..bde58690d --- /dev/null +++ b/source/Halibut.Tests/Queue/Redis/WatchForRedisLosingAllItsDataFixture.cs @@ -0,0 +1,107 @@ +using System; +using System.Threading.Tasks; +using FluentAssertions; +using Halibut.Logging; +using Halibut.Queue.Redis; +using Halibut.Tests.Queue.Redis.Utils; +using Halibut.Tests.Support; +using Halibut.Tests.Support.Logging; +using NUnit.Framework; +using Octopus.TestPortForwarder; + +namespace Halibut.Tests.Queue.Redis +{ + public class WatchForRedisLosingAllItsDataFixture : BaseTest + { + const int redisPort = 6379; + private static RedisFacade CreateRedisFacade(int? port = redisPort, Guid? guid = null) => new("localhost:" + port, (guid ?? Guid.NewGuid()).ToString(), new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix("")); + + [Test] + public async Task WhenTheConnectionToRedisCanNotBeCreated_WhenAskingForALostDataCancellationToken_ATimeoutOccurs() + { + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); + portForwarder.EnterKillNewAndExistingConnectionsMode(); + await using var redisFacade = CreateRedisFacade(port: portForwarder.ListeningPort); + await using var watcher = new WatchForRedisLosingAllItsData(redisFacade, HalibutLog, watchInterval:TimeSpan.FromSeconds(1)); + + + await AssertException.Throws(watcher.GetTokenForDataLoseDetection(TimeSpan.FromSeconds(1), CancellationToken)); + } + + [Test] + public async Task WhenTheConnectionToRedisIsInitiallyDown_WhenAskingForALostDataCancellationToken_AndTheConnectionToRedisReturns_TheCTIsReturned() + { + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); + portForwarder.EnterKillNewAndExistingConnectionsMode(); + await using var redisFacade = CreateRedisFacade(port: portForwarder.ListeningPort); + await using var watcher = new WatchForRedisLosingAllItsData(redisFacade, HalibutLog, watchInterval:TimeSpan.FromSeconds(1)); + + var _ = Task.Run(async () => + { + await Task.Delay(2000); + portForwarder.ReturnToNormalMode(); + + }); + + await watcher.GetTokenForDataLoseDetection(TimeSpan.FromSeconds(20), CancellationToken); + } + + [Test] + public async Task WatchForARealRedisLosingAllOfItsData_TimesOutWhenWaitingForCTWhenNoConnectionToRedisCanBeEstablished() + { + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); + portForwarder.EnterKillNewAndExistingConnectionsMode(); + await using var redisFacade = CreateRedisFacade(port: portForwarder.ListeningPort); + await using var watcher = new WatchForRedisLosingAllItsData(redisFacade, HalibutLog, watchInterval:TimeSpan.FromSeconds(1)); + + + await AssertException.Throws(watcher.GetTokenForDataLoseDetection(TimeSpan.FromSeconds(1), CancellationToken)); + } + + [Test] + public async Task WatchForARealRedisLosingAllOfItsData_E2E_Test() + { + Logger.Information("Starting WatchForARealRedisLosingAllOfItsData_E2E_Test"); + + // Arrange - Create Redis container using the builder + Logger.Information("Creating Redis container"); + await using var container = new RedisContainerBuilder() + .Build(); + + Logger.Information("Starting Redis container"); + await container.StartAsync(); + Logger.Information("Redis container started successfully with connection string: {ConnectionString}", container.ConnectionString); + + // Create RedisFacade connected to the containerized Redis + await using var redisFacade = CreateRedisFacade(port: container.RedisPort); + + await using var watcher = new WatchForRedisLosingAllItsData(redisFacade, HalibutLog, watchInterval:TimeSpan.FromSeconds(1)); + + Logger.Information("Getting initial cancellation token for data loss detection (20 second timeout)"); + var watcherCT = await watcher.GetTokenForDataLoseDetection(TimeSpan.FromSeconds(20), CancellationToken); + Logger.Information("Initial cancellation token obtained, IsCancellationRequested: {IsCancellationRequested}", watcherCT.IsCancellationRequested); + + watcherCT.IsCancellationRequested.Should().BeFalse(); + + // Act + Logger.Information("Stopping Redis container to simulate data loss"); + await container.StopAsync(); + Logger.Information("Redis container stopped"); + + Logger.Information("Starting Redis container again (fresh instance, data lost)"); + await container.StartAsync(); + Logger.Information("Redis container restarted"); + + // Assert + Logger.Information("Waiting up to 10 seconds for data loss detection to trigger cancellation token"); + await Task.WhenAny(Task.Delay(TimeSpan.FromSeconds(10), watcherCT)); + + watcherCT.IsCancellationRequested.Should().BeTrue("Should have detected the data loss"); + + Logger.Information("Getting new cancellation token to verify recovery"); + var nextToken = await watcher.GetTokenForDataLoseDetection(TimeSpan.FromSeconds(20), CancellationToken); + + nextToken.IsCancellationRequested.Should().BeFalse("The new token should have no data loss"); + } + } +} \ No newline at end of file From c3172e4c98b57af0838164fcbeecaeda574ebeb0 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Thu, 7 Aug 2025 13:33:45 +1000 Subject: [PATCH 028/137] . --- ...CancellationTokenSourceExtensionMethods.cs | 55 +++++++++++++++++++ 1 file changed, 55 insertions(+) create mode 100644 source/Halibut/Util/CancellationTokenSourceExtensionMethods.cs diff --git a/source/Halibut/Util/CancellationTokenSourceExtensionMethods.cs b/source/Halibut/Util/CancellationTokenSourceExtensionMethods.cs new file mode 100644 index 000000000..1b862141b --- /dev/null +++ b/source/Halibut/Util/CancellationTokenSourceExtensionMethods.cs @@ -0,0 +1,55 @@ +#nullable enable +using System; +using System.Threading; +using System.Threading.Tasks; + +namespace Halibut.Util +{ + public static class CancellationTokenSourceExtensionMethods + { + /// + /// Creates an async disposable wrapper for the CancellationTokenSource that will safely cancel and dispose it. + /// + /// The CancellationTokenSource to wrap + /// An IAsyncDisposable that will cancel and dispose the CancellationTokenSource + public static CancelOnDisposeCancellationTokenSource CancelOnDispose(this CancellationTokenSource cancellationTokenSource) + { + return new CancelOnDisposeCancellationTokenSource(cancellationTokenSource); + } + } + + /// + /// An async disposable wrapper for CancellationTokenSource that safely cancels and disposes it. + /// + public sealed class CancelOnDisposeCancellationTokenSource : IAsyncDisposable + { + public readonly CancellationTokenSource CancellationTokenSource; + private bool disposed = false; + + internal CancelOnDisposeCancellationTokenSource(CancellationTokenSource cancellationTokenSource) + { + this.CancellationTokenSource = cancellationTokenSource ?? throw new ArgumentNullException(nameof(cancellationTokenSource)); + this.CancellationToken = cancellationTokenSource.Token; + } + + public CancellationToken CancellationToken { get; } + + public async ValueTask DisposeAsync() + { + if (disposed) + return; + + disposed = true; + + // First, attempt to cancel the cancellation token source +#if NET8_0_OR_GREATER + await Try.IgnoringError(async () => await CancellationTokenSource.CancelAsync()); +#else + Try.IgnoringError(() => CancellationTokenSource.Cancel()); +#endif + + // Then, dispose the cancellation token source + Try.IgnoringError(() => CancellationTokenSource.Dispose()); + } + } +} \ No newline at end of file From 9675eb1b9e34a16a1c690c1e885ae445afde73f7 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Thu, 7 Aug 2025 14:21:08 +1000 Subject: [PATCH 029/137] Fix tests under load --- .../Redis/RedisPendingRequestQueueFixture.cs | 19 ++++++-- .../TestRedisPendingRequestQueueFactory.cs | 47 +++++++++++++++++++ .../Queue/Redis/RedisPendingRequestQueue.cs | 4 +- 3 files changed, 65 insertions(+), 5 deletions(-) create mode 100644 source/Halibut.Tests/Queue/Redis/TestRedisPendingRequestQueueFactory.cs diff --git a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs index 55f9e9e5b..d701d33c9 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs @@ -83,6 +83,7 @@ public async Task WhenThePickupTimeoutExpires_AnErrorsReturned_AndTheRequestCanN .Build(); var sut = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + await sut.WaitUntilQueueIsSubscribedToReceiveMessages(); var response = await sut.QueueAndWaitAsync(request, CancellationToken.None); @@ -163,6 +164,7 @@ public async Task FullSendAndReceiveShouldWork() var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); var node2Receiver = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + await node2Receiver.WaitUntilQueueIsSubscribedToReceiveMessages(); var queueAndWaitAsync = node1Sender.QueueAndWaitAsync(request, CancellationToken.None); @@ -198,6 +200,7 @@ public async Task WhenDataLostIsDetected_InFlightRequestShouldBeAbandoned() var node1Sender = new RedisPendingRequestQueue(endpoint, dataLossWatcher, log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); var node2Receiver = new RedisPendingRequestQueue(endpoint, dataLossWatcher, log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + await node2Receiver.WaitUntilQueueIsSubscribedToReceiveMessages(); var queueAndWaitAsync = node1Sender.QueueAndWaitAsync(request, CancellationToken.None); @@ -351,7 +354,8 @@ public async Task FullSendAndReceiveWithDataStreamShouldWork() { // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); - var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); + var senderLog = new TestContextLogCreator("QueueSender", LogLevel.Trace).CreateNewForPrefix(""); + var receiverLog = new TestContextLogCreator("ReceiverLog", LogLevel.Trace).CreateNewForPrefix(""); var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); @@ -360,8 +364,9 @@ public async Task FullSendAndReceiveWithDataStreamShouldWork() var request = new RequestMessageBuilder("poll://test-endpoint").Build(); request.Params = new[] { new ComplexObjectMultipleDataStreams(DataStream.FromString("hello"), DataStream.FromString("world")) }; - var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); - var node2Receiver = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), senderLog, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + var node2Receiver = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), receiverLog, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + await node2Receiver.WaitUntilQueueIsSubscribedToReceiveMessages(); var queueAndWaitAsync = node1Sender.QueueAndWaitAsync(request, CancellationToken.None); @@ -409,6 +414,7 @@ public async Task WhenTheReceiversConnectionToRedisIsInterruptedAndRestoredBefor var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(redisFacadeSender), messageReaderWriter, halibutTimeoutAndLimits); var node2Receiver = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(redisFacadeReceiver), messageReaderWriter, halibutTimeoutAndLimits); + await node2Receiver.WaitUntilQueueIsSubscribedToReceiveMessages(); var dequeueTask = node2Receiver.DequeueAsync(CancellationToken); await Task.Delay(5000, CancellationToken); // Allow some time for the receiver to subscribe to work. @@ -459,6 +465,7 @@ public async Task WhenTheSendersConnectionToRedisIsBrieflyInterruptedWhileSendin var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(redisFacadeSender), messageReaderWriter, halibutTimeoutAndLimits); var node2Receiver = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(redisFacadeSender), messageReaderWriter, halibutTimeoutAndLimits); + await node2Receiver.WaitUntilQueueIsSubscribedToReceiveMessages(); portForwarder.EnterKillNewAndExistingConnectionsMode(); @@ -501,6 +508,7 @@ public async Task WhenTheReceiverDequeuesWorkAndThenDisconnectsFromRedisForEver_ var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(stableRedisConnection), messageReaderWriter, halibutTimeoutAndLimits); var node2Receiver = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(unstableRedisConnection), messageReaderWriter, halibutTimeoutAndLimits); + await node2Receiver.WaitUntilQueueIsSubscribedToReceiveMessages(); // Lower this to complete the test sooner. node1Sender.DelayBetweenHeartBeatsForRequestProcessor = TimeSpan.FromSeconds(1); @@ -552,6 +560,7 @@ public async Task WhenTheReceiverDequeuesWork_AndTheSenderDisconnects_AndNeverRe var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(unstableRedisConnection), messageReaderWriter, halibutTimeoutAndLimits); var node2Receiver = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(stableRedisConnection), messageReaderWriter, halibutTimeoutAndLimits); + await node2Receiver.WaitUntilQueueIsSubscribedToReceiveMessages(); node1Sender.DelayBetweenHeartBeatsForRequestSender= TimeSpan.FromSeconds(1); node2Receiver.DelayBetweenHeartBeatsForRequestSender= TimeSpan.FromSeconds(1); @@ -592,6 +601,7 @@ public async Task WhenTheSenderBrieflyDisconnectsFromRedisRightWhenTheReceiverSe var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(unreliableConnection), messageReaderWriter, new HalibutTimeoutsAndLimits()); var node2Receiver = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(stableConnection), messageReaderWriter, new HalibutTimeoutsAndLimits()); + await node2Receiver.WaitUntilQueueIsSubscribedToReceiveMessages(); var request = new RequestMessageBuilder("poll://test-endpoint").Build(); // TODO: This only needs to be set because we do not detect the work was collected as soon as it has been collected. @@ -635,7 +645,8 @@ public async Task WhenUsingTheRedisQueue_ASimpleEchoServiceCanBeCalled(ClientAnd new NeverLosingDataWatchForRedisLosingAllItsData(), redisTransport, new HalibutTimeoutsAndLimits(), - logFactory)) + logFactory) + .WithWaitForReceiverToBeReady()) .Build(CancellationToken)) { var echo = clientAndService.CreateAsyncClient(); diff --git a/source/Halibut.Tests/Queue/Redis/TestRedisPendingRequestQueueFactory.cs b/source/Halibut.Tests/Queue/Redis/TestRedisPendingRequestQueueFactory.cs new file mode 100644 index 000000000..cc846b25f --- /dev/null +++ b/source/Halibut.Tests/Queue/Redis/TestRedisPendingRequestQueueFactory.cs @@ -0,0 +1,47 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; +using Halibut.Queue.Redis; +using Halibut.ServiceModel; + +namespace Halibut.Tests.Queue.Redis +{ + public class TestRedisPendingRequestQueueFactory : IPendingRequestQueueFactory + { + RedisPendingRequestQueueFactory redisPendingRequestQueueFactory; + + public TestRedisPendingRequestQueueFactory(RedisPendingRequestQueueFactory redisPendingRequestQueueFactory) + { + this.redisPendingRequestQueueFactory = redisPendingRequestQueueFactory; + } + + public IPendingRequestQueue CreateQueue(Uri endpoint) + { + var queue = (RedisPendingRequestQueue) redisPendingRequestQueueFactory.CreateQueue(endpoint); +#pragma warning disable VSTHRD002 + queue.WaitUntilQueueIsSubscribedToReceiveMessages().GetAwaiter().GetResult(); +#pragma warning restore VSTHRD002 + return queue; + } + } + + public static class RedisPendingRequestQueueFactoryExtensionMethods + { + public static IPendingRequestQueueFactory WithWaitForReceiverToBeReady(this RedisPendingRequestQueueFactory redisPendingRequestQueueFactory) + { + return new TestRedisPendingRequestQueueFactory(redisPendingRequestQueueFactory); + } + } +} \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index 332c9812c..69d8422cd 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -63,8 +63,10 @@ public RedisPendingRequestQueue( // The advice is many channels with few subscribers is better than a single channel with many subscribers. // If we end up with too many channels, we could shared the channels based on modulo of the hash of the endpoint, // which means we might have only 1000 channels and num_tentacles/1000 subscribers to each channel. For 300K tentacles. - PulseChannelSubDisposer = Task.Run(() => this.halibutRedisTransport.SubscribeToRequestMessagePulseChannel(endpoint, _ => hasItemsForEndpoint.Set(), queueToken)); + PulseChannelSubDisposer = Task.Run(async () => await this.halibutRedisTransport.SubscribeToRequestMessagePulseChannel(endpoint, _ => hasItemsForEndpoint.Set(), queueToken)); } + + internal async Task WaitUntilQueueIsSubscribedToReceiveMessages() => await PulseChannelSubDisposer; public async ValueTask DisposeAsync() { From 3ed16cad193d8f8c862037c086cfdfd324c4246b Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Thu, 7 Aug 2025 16:09:25 +1000 Subject: [PATCH 030/137] Improve test --- .../Queue/Redis/NodeHeartBeatSenderFixture.cs | 8 ++- .../Redis/RedisPendingRequestQueueFixture.cs | 7 +++ .../Queue/Redis/NodeHeartBeatSender.cs | 49 ++++++++++++++++--- .../Queue/Redis/RedisPendingRequestQueue.cs | 12 ++++- 4 files changed, 65 insertions(+), 11 deletions(-) diff --git a/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs b/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs index 13969c515..d784a6637 100644 --- a/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs @@ -59,7 +59,8 @@ public async Task WhenCreated_ShouldStartSendingHeartbeats() anyHeartBeatReceived.IsSet.Should().BeTrue("Should have received at least one heartbeat"); } - [Test] + // Not sure this is a good test + //[Test] public async Task WhenRedisConnectionIsInterrupted_ShouldSwitchToPanicMode() { // Arrange @@ -183,6 +184,7 @@ public async Task WaitUntilNodeProcessingRequestFlatLines_WhenHeartbeatsStop_Sho request, pendingRequest, stableRedisTransport, + TimeSpan.FromSeconds(1), log, TimeSpan.FromSeconds(10), // Short timeout for test CancellationToken); @@ -223,6 +225,7 @@ public async Task NoIssueReturnedWhenNodeProcessingRequestIsNotSeenToGoOffline() request, pendingRequest, redisTransport, + TimeSpan.FromSeconds(1), log, TimeSpan.FromMinutes(5), cts.Token); @@ -266,7 +269,8 @@ public async Task WaitUntilNodeProcessingRequestFlatLines_WhenConnectionInterrup endpoint, request, pendingRequest, - stableRedisTransport, + stableRedisTransport, + TimeSpan.FromSeconds(1), log, TimeSpan.FromSeconds(15), // Short timeout for test CancellationToken); diff --git a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs index d701d33c9..68ae6106f 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs @@ -54,6 +54,7 @@ public async Task DequeueAsync_ShouldReturnRequestFromRedis() var request = new RequestMessageBuilder("poll://test-endpoint").Build(); var sut = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + await sut.WaitUntilQueueIsSubscribedToReceiveMessages(); var task = sut.QueueAndWaitAsync(request, CancellationToken.None); @@ -247,6 +248,7 @@ public async Task OnceARequestIsComplete_NoInflightDisposableShouldExist() var request = new RequestMessageBuilder("poll://test-endpoint").Build(); var queue = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + await queue.WaitUntilQueueIsSubscribedToReceiveMessages(); // Act var queueAndWaitAsync = queue.QueueAndWaitAsync(request, CancellationToken.None); @@ -278,6 +280,7 @@ public async Task OnceARequestIsComplete_NoSenderHeartBeatsShouldBeSent() var request = new RequestMessageBuilder("poll://test-endpoint").Build(); var queue = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + await queue.WaitUntilQueueIsSubscribedToReceiveMessages(); queue.DelayBetweenHeartBeatsForRequestSender = TimeSpan.FromSeconds(1); // Act @@ -321,6 +324,7 @@ public async Task OnceARequestIsComplete_NoReceiverHeartBeatsShouldBeSent() var request = new RequestMessageBuilder("poll://test-endpoint").Build(); var queue = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + await queue.WaitUntilQueueIsSubscribedToReceiveMessages(); queue.DelayBetweenHeartBeatsForRequestProcessor = TimeSpan.FromSeconds(1); // Act @@ -566,6 +570,8 @@ public async Task WhenTheReceiverDequeuesWork_AndTheSenderDisconnects_AndNeverRe node2Receiver.DelayBetweenHeartBeatsForRequestSender= TimeSpan.FromSeconds(1); node1Sender.NodeIsOfflineHeartBeatTimeoutForRequestSender = TimeSpan.FromSeconds(15); node2Receiver.NodeIsOfflineHeartBeatTimeoutForRequestSender = TimeSpan.FromSeconds(15); + node1Sender.TimeBetweenCheckingIfRequestWasCollected = TimeSpan.FromSeconds(1); + node2Receiver.TimeBetweenCheckingIfRequestWasCollected = TimeSpan.FromSeconds(1); var request = new RequestMessageBuilder("poll://test-endpoint").Build(); @@ -675,6 +681,7 @@ public async Task CancellingARequestShouldResultInTheDequeuedResponseTokenBeingC request.Params = new[] { new ComplexObjectMultipleDataStreams(DataStream.FromString("hello"), DataStream.FromString("world")) }; var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + await node1Sender.WaitUntilQueueIsSubscribedToReceiveMessages(); using var cts = new CancellationTokenSource(); diff --git a/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs b/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs index 0025b0c46..8f692845f 100644 --- a/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs +++ b/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs @@ -20,6 +20,7 @@ using Halibut.ServiceModel; using Halibut.Transport.Protocol; using Halibut.Util; +using Nito.AsyncEx; namespace Halibut.Queue.Redis { @@ -98,16 +99,29 @@ public static async Task WatchThatNodeProces RequestMessage request, PendingRequest pending, HalibutRedisTransport halibutRedisTransport, + TimeSpan timeBetweenCheckingIfRequestWasCollected, ILog log, TimeSpan maxTimeBetweenHeartBeetsBeforeProcessingNodeIsAssumedToBeOffline, CancellationToken watchCancellationToken) { // Once the pending's CT has been cancelled we no longer care to keep observing using var cts = CancellationTokenSource.CreateLinkedTokenSource(watchCancellationToken, pending.PendingRequestCancellationToken); + var cancellationToken = cts.Token; // TODO: test this is indeed called first. - await WaitForRequestToBeCollected(endpoint, request, pending, halibutRedisTransport, log, cts.Token); - - return await WatchForPulsesFromNode(endpoint, request.ActivityId, halibutRedisTransport, log, maxTimeBetweenHeartBeetsBeforeProcessingNodeIsAssumedToBeOffline, HalibutQueueNodeSendingPulses.Receiver, cts.Token); + try + { + await WaitForRequestToBeCollected(endpoint, request, pending, halibutRedisTransport, timeBetweenCheckingIfRequestWasCollected, log, cts.Token); + + return await WatchForPulsesFromNode(endpoint, request.ActivityId, halibutRedisTransport, log, maxTimeBetweenHeartBeetsBeforeProcessingNodeIsAssumedToBeOffline, HalibutQueueNodeSendingPulses.Receiver, cts.Token); + } + catch (Exception) when (!cts.Token.IsCancellationRequested) + { + throw; + } + catch (Exception) + { + return NodeProcessingRequestWatcherResult.NoDisconnectSeen; + } } public static async Task WatchThatNodeWhichSentTheRequestIsStillAlive( @@ -130,7 +144,7 @@ private static async Task WatchForPulsesFrom HalibutQueueNodeSendingPulses watchingForPulsesFrom, CancellationToken watchCancellationToken) { - log.Write(EventType.Diagnostic, "Starting to watch for {0} node flatline, request {1}, endpoint {2}", watchingForPulsesFrom, requestActivityId, endpoint); + log.Write(EventType.Diagnostic, "Starting to watch for pulses from {0} node, request {1}, endpoint {2}", watchingForPulsesFrom, requestActivityId, endpoint); DateTimeOffset? lastHeartBeat = DateTimeOffset.Now; @@ -168,16 +182,29 @@ private static async Task WatchForPulsesFrom } } - - static async Task WaitForRequestToBeCollected(Uri endpoint, RequestMessage request, PendingRequest pending, HalibutRedisTransport halibutRedisTransport, ILog log, CancellationToken cancellationToken) + static async Task WaitForRequestToBeCollected(Uri endpoint, RequestMessage request, PendingRequest pending, HalibutRedisTransport halibutRedisTransport, + TimeSpan timeBetweenCheckingIfRequestWasCollected, + ILog log, CancellationToken cancellationToken) { log.Write(EventType.Diagnostic, "Waiting for request {0} to be collected from queue", request.ActivityId); + + // Is this worthwhile? + var asyncManualResetEvent = new AsyncManualResetEvent(false); + // await using var subscription = await halibutRedisTransport.SubscribeToNodeHeartBeatChannel( + // endpoint, + // request.ActivityId, + // HalibutQueueNodeSendingPulses.Receiver, async () => + // { + // await Task.CompletedTask; + // asyncManualResetEvent.Set(); + // log.Write(EventType.Diagnostic, "While waiting for request to be collected received heartbeat from {0} node, request {1}", HalibutQueueNodeSendingPulses.Receiver, request.ActivityId); + // }, cancellationToken); - // TODO: um like listen to the heart beat here, because heart beat == collected (maybe) while (!cancellationToken.IsCancellationRequested) { try { + asyncManualResetEvent.Reset(); // Has something else determined the request was collected? // TODO should we bail out of here if the PendingRequest is complete? if(pending.HasRequestBeenMarkedAsCollected) @@ -201,7 +228,13 @@ static async Task WaitForRequestToBeCollected(Uri endpoint, RequestMessage reque log.WriteException(EventType.Diagnostic, "Error checking if request {0} is still on queue", ex, request.ActivityId); } - await Try.IgnoringError(async () => await Task.WhenAny(Task.Delay(TimeSpan.FromSeconds(30), cancellationToken), pending.WaitForRequestToBeMarkedAsCollected(cancellationToken))); + await Try.IgnoringError(async () => + { + await Task.WhenAny( + Task.Delay(timeBetweenCheckingIfRequestWasCollected, cancellationToken), + asyncManualResetEvent.WaitAsync(cancellationToken), + pending.WaitForRequestToBeMarkedAsCollected(cancellationToken)); + }); } log.Write(EventType.Diagnostic, "Stopped waiting for request {0} to be collected (cancelled)", request.ActivityId); diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index 69d8422cd..929758b0c 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -147,7 +147,15 @@ void WatchProcessingNodeIsStillConnectedInBackground(RequestMessage request, Pen var watcherCtsCancellationToken = watcherCts.CancellationToken; try { - var disconnected = await NodeHeartBeatSender.WatchThatNodeProcessingTheRequestIsStillAlive(endpoint, request, pending, halibutRedisTransport, log, NodeIsOfflineHeartBeatTimeoutForRequestProcessor, watcherCtsCancellationToken); + var disconnected = await NodeHeartBeatSender.WatchThatNodeProcessingTheRequestIsStillAlive( + endpoint, + request, + pending, + halibutRedisTransport, + TimeBetweenCheckingIfRequestWasCollected, + log, + NodeIsOfflineHeartBeatTimeoutForRequestProcessor, + watcherCtsCancellationToken); if (!watcherCtsCancellationToken.IsCancellationRequested && disconnected == NodeHeartBeatSender.NodeProcessingRequestWatcherResult.NodeMayHaveDisconnected) { // TODO: if(responseWatcher.CheckForResponseNow() == ResponseNotFound) { @@ -253,6 +261,8 @@ async Task SubscribeToResponse(Guid activityId, internal TimeSpan TTLOfResponseMessage { get; set; } = TimeSpan.FromMinutes(5); + internal TimeSpan TimeBetweenCheckingIfRequestWasCollected { get; set; } = TimeSpan.FromSeconds(30); + public async Task DequeueAsync(CancellationToken cancellationToken) From 1ea826bb04a15eb50994ffdd3b9bf3029608e47c Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Thu, 7 Aug 2025 16:19:32 +1000 Subject: [PATCH 031/137] More reliable timeout --- source/Halibut/Queue/Redis/NodeHeartBeatSender.cs | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs b/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs index 8f692845f..ce2222d1d 100644 --- a/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs +++ b/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs @@ -163,13 +163,19 @@ private static async Task WatchForPulsesFrom while (!watchCancellationToken.IsCancellationRequested) { // TODO: I am sure a fancy pants delay could be done here calculated from the now and last heart beat etc. - await Try.IgnoringError(async () => await Task.Delay(TimeSpan.FromSeconds(10), watchCancellationToken)); + var timeSinceLastHeartBeat = DateTimeOffset.Now - lastHeartBeat.Value; if (timeSinceLastHeartBeat > maxTimeBetweenHeartBeetsBeforeNodeIsAssumedToBeOffline) { log.Write(EventType.Diagnostic, "{0} node appears disconnected, request {1}, last heartbeat was {2} seconds ago", watchingForPulsesFrom, requestActivityId, timeSinceLastHeartBeat.TotalSeconds); return NodeProcessingRequestWatcherResult.NodeMayHaveDisconnected; } + + var timeToWait = TimeSpan.FromSeconds(30); + var timeBeforeTimeoutPlusOneSecond = maxTimeBetweenHeartBeetsBeforeNodeIsAssumedToBeOffline - timeSinceLastHeartBeat + TimeSpan.FromSeconds(1); + if (timeBeforeTimeoutPlusOneSecond < timeToWait) timeToWait = timeBeforeTimeoutPlusOneSecond; + + await Try.IgnoringError(async () => await Task.Delay(timeToWait, watchCancellationToken)); } log.Write(EventType.Diagnostic, "{0} node watcher cancelled, request {1}", watchingForPulsesFrom, requestActivityId); From b31a64d3cd1b02bba7d06bbfaab79a31c6583f2b Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Thu, 7 Aug 2025 16:54:48 +1000 Subject: [PATCH 032/137] Start testing both queues at the same time --- .../Builders/PendingRequestQueueBuilder.cs | 108 ++++++++- .../Builders/ResponseMessageBuilder.cs | 2 +- ...gTentacleDequeuesRequestsInOrderFixture.cs | 4 +- .../Queue/QueueMessageSerializerBuilder.cs | 56 +++++ .../Queue/QueueMessageSerializerFixture.cs | 48 +--- .../PendingRequestQueueFixture.cs | 224 +++++++++--------- .../AllQueuesTestCasesAttribute.cs | 69 ++++++ .../Timeouts/PollingQueueTimeouts.cs | 4 +- .../Timeouts/TimeoutsApplyDuringHandShake.cs | 2 +- 9 files changed, 344 insertions(+), 173 deletions(-) create mode 100644 source/Halibut.Tests/Queue/QueueMessageSerializerBuilder.cs create mode 100644 source/Halibut.Tests/Support/TestAttributes/AllQueuesTestCasesAttribute.cs diff --git a/source/Halibut.Tests/Builders/PendingRequestQueueBuilder.cs b/source/Halibut.Tests/Builders/PendingRequestQueueBuilder.cs index a833615f2..78a3dd699 100644 --- a/source/Halibut.Tests/Builders/PendingRequestQueueBuilder.cs +++ b/source/Halibut.Tests/Builders/PendingRequestQueueBuilder.cs @@ -1,41 +1,113 @@ using System; +using System.Threading.Tasks; using Halibut.Diagnostics; +using Halibut.Logging; +using Halibut.Queue.QueuedDataStreams; +using Halibut.Queue.Redis; using Halibut.ServiceModel; +using Halibut.Tests.Queue; +using Halibut.Tests.Queue.Redis.Utils; +using Halibut.Tests.Support; +using Halibut.Tests.Support.Logging; +using DisposableCollection = Halibut.Util.DisposableCollection; +using ILog = Halibut.Diagnostics.ILog; namespace Halibut.Tests.Builders { - public class PendingRequestQueueBuilder + + public interface IPendingRequestQueueBuilder + { + public IPendingRequestQueueBuilder WithEndpoint(string endpoint); + public IPendingRequestQueueBuilder WithLog(ILog log); + public IPendingRequestQueueBuilder WithPollingQueueWaitTimeout(TimeSpan? pollingQueueWaitTimeout); + public QueueHolder Build(); + } + + public class RedisPendingRequestQueueBuilder : IPendingRequestQueueBuilder { + + const int redisPort = 6379; + ILog? log; string? endpoint; TimeSpan? pollingQueueWaitTimeout; - bool? relyOnConnectionTimeoutsInsteadOfPollingRequestMaximumMessageProcessingTimeout; - public PendingRequestQueueBuilder WithEndpoint(string endpoint) + public IPendingRequestQueueBuilder WithEndpoint(string endpoint) { this.endpoint = endpoint; return this; } - public PendingRequestQueueBuilder WithLog(ILog log) + public IPendingRequestQueueBuilder WithLog(ILog log) { this.log = log; return this; } - public PendingRequestQueueBuilder WithPollingQueueWaitTimeout(TimeSpan? pollingQueueWaitTimeout) + public IPendingRequestQueueBuilder WithPollingQueueWaitTimeout(TimeSpan? pollingQueueWaitTimeout) { this.pollingQueueWaitTimeout = pollingQueueWaitTimeout; return this; } - public PendingRequestQueueBuilder WithRelyOnConnectionTimeoutsInsteadOfPollingRequestMaximumMessageProcessingTimeout(bool relyOnConnectionTimeoutsInsteadOfPollingRequestMaximumMessageProcessingTimeout) + public QueueHolder Build() { - this.relyOnConnectionTimeoutsInsteadOfPollingRequestMaximumMessageProcessingTimeout = relyOnConnectionTimeoutsInsteadOfPollingRequestMaximumMessageProcessingTimeout; + var endpoint = new Uri(this.endpoint ?? "poll://endpoint001"); + var halibutTimeoutsAndLimits = new HalibutTimeoutsAndLimitsForTestsBuilder().Build(); + var log = this.log ?? new TestContextLogCreator("Queue", LogLevel.Trace).CreateNewForPrefix(""); + + if (this.pollingQueueWaitTimeout != null) + { + halibutTimeoutsAndLimits.PollingQueueWaitTimeout = pollingQueueWaitTimeout.Value; + } + + var disposableCollection = new DisposableCollection(); + + var redisFacade = new RedisFacade("localhost:" + redisPort, (Guid.NewGuid()).ToString(), log); + disposableCollection.AddAsyncDisposable(redisFacade); + + var redisTransport = new HalibutRedisTransport(redisFacade); + var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); + var messageSerializer = new QueueMessageSerializerBuilder().Build(); + var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); + + var request = new RequestMessageBuilder("poll://test-endpoint").Build(); + + var queue = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, halibutTimeoutsAndLimits); + +#pragma warning disable VSTHRD002 + queue.WaitUntilQueueIsSubscribedToReceiveMessages().GetAwaiter().GetResult(); +#pragma warning restore VSTHRD002 + + return new QueueHolder(queue, disposableCollection); + } + } + + public class PendingRequestQueueBuilder : IPendingRequestQueueBuilder + { + ILog? log; + string? endpoint; + TimeSpan? pollingQueueWaitTimeout; + + public IPendingRequestQueueBuilder WithEndpoint(string endpoint) + { + this.endpoint = endpoint; return this; } - public IPendingRequestQueue Build() + public IPendingRequestQueueBuilder WithLog(ILog log) + { + this.log = log; + return this; + } + + public IPendingRequestQueueBuilder WithPollingQueueWaitTimeout(TimeSpan? pollingQueueWaitTimeout) + { + this.pollingQueueWaitTimeout = pollingQueueWaitTimeout; + return this; + } + + public QueueHolder Build() { var endpoint = this.endpoint ?? "poll://endpoint001"; var halibutTimeoutsAndLimits = new HalibutTimeoutsAndLimitsForTestsBuilder().Build(); @@ -43,7 +115,25 @@ public IPendingRequestQueue Build() var pollingQueueWaitTimeout = this.pollingQueueWaitTimeout ?? halibutTimeoutsAndLimits.PollingQueueWaitTimeout; - return new PendingRequestQueueAsync(log, pollingQueueWaitTimeout); + return new QueueHolder(new PendingRequestQueueAsync(log, pollingQueueWaitTimeout), new DisposableCollection()); + } + } + + public class QueueHolder : IAsyncDisposable + { + public IPendingRequestQueue PendingRequestQueue { get; } + public DisposableCollection DisposableCollection { get; } + + public QueueHolder(IPendingRequestQueue pendingRequestQueue, Halibut.Util.DisposableCollection disposableCollection) + { + this.PendingRequestQueue = pendingRequestQueue; + this.DisposableCollection = disposableCollection; + } + + public async ValueTask DisposeAsync() + { + this.DisposableCollection.AddAsyncDisposable(PendingRequestQueue); + await DisposableCollection.DisposeAsync(); } } } \ No newline at end of file diff --git a/source/Halibut.Tests/Builders/ResponseMessageBuilder.cs b/source/Halibut.Tests/Builders/ResponseMessageBuilder.cs index 8ecd61b8e..acb9ea9e9 100644 --- a/source/Halibut.Tests/Builders/ResponseMessageBuilder.cs +++ b/source/Halibut.Tests/Builders/ResponseMessageBuilder.cs @@ -22,7 +22,7 @@ public ResponseMessage Build() var response = new ResponseMessage { Id = id, - Result = new object() + Result = "Hello World" }; return response; } diff --git a/source/Halibut.Tests/PollingTentacleDequeuesRequestsInOrderFixture.cs b/source/Halibut.Tests/PollingTentacleDequeuesRequestsInOrderFixture.cs index a12cfbeb5..ee7686dc0 100644 --- a/source/Halibut.Tests/PollingTentacleDequeuesRequestsInOrderFixture.cs +++ b/source/Halibut.Tests/PollingTentacleDequeuesRequestsInOrderFixture.cs @@ -30,10 +30,12 @@ public async Task QueuedUpRequestsShouldBeDequeuedInOrder(ClientAndServiceTestCa { return new FuncPendingRequestQueueFactory(uri => { + // TODO: Test the new queue here. pendingRequestQueue = new PendingRequestQueueBuilder() .WithLog(logFactory.ForEndpoint(uri)) .WithPollingQueueWaitTimeout(TimeSpan.FromSeconds(1)) - .Build(); + .Build() + .PendingRequestQueue; return pendingRequestQueue; }); }) diff --git a/source/Halibut.Tests/Queue/QueueMessageSerializerBuilder.cs b/source/Halibut.Tests/Queue/QueueMessageSerializerBuilder.cs new file mode 100644 index 000000000..5a0194b3f --- /dev/null +++ b/source/Halibut.Tests/Queue/QueueMessageSerializerBuilder.cs @@ -0,0 +1,56 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; +using Halibut.Diagnostics; +using Halibut.Queue; +using Halibut.Transport.Protocol; +using Newtonsoft.Json; + +namespace Halibut.Tests.Queue +{ + public class QueueMessageSerializerBuilder + { + ITypeRegistry? typeRegistry; + Action? configureSerializer; + + public QueueMessageSerializerBuilder WithTypeRegistry(ITypeRegistry typeRegistry) + { + this.typeRegistry = typeRegistry; + return this; + } + + public QueueMessageSerializerBuilder WithSerializerSettings(Action configure) + { + configureSerializer = configure; + return this; + } + + public QueueMessageSerializer Build() + { + var typeRegistry = this.typeRegistry ?? new TypeRegistry(); + + StreamCapturingJsonSerializer StreamCapturingSerializer() + { + var settings = MessageSerializerBuilder.CreateSerializer(); + var binder = new RegisteredSerializationBinder(typeRegistry); + settings.SerializationBinder = binder; + configureSerializer?.Invoke(settings); + return new StreamCapturingJsonSerializer(settings); + } + + return new QueueMessageSerializer(StreamCapturingSerializer); + } + } +} \ No newline at end of file diff --git a/source/Halibut.Tests/Queue/QueueMessageSerializerFixture.cs b/source/Halibut.Tests/Queue/QueueMessageSerializerFixture.cs index 880dcfa4e..4838512e5 100644 --- a/source/Halibut.Tests/Queue/QueueMessageSerializerFixture.cs +++ b/source/Halibut.Tests/Queue/QueueMessageSerializerFixture.cs @@ -5,10 +5,8 @@ using System.Threading.Tasks; using FluentAssertions; using Halibut.Diagnostics; -using Halibut.Queue; using Halibut.Tests.Support; using Halibut.Transport.Protocol; -using Newtonsoft.Json; using NUnit.Framework; namespace Halibut.Tests.Queue @@ -19,7 +17,7 @@ public class QueueMessageSerializerFixture : BaseTest public void SerializeAndDeserializeMessage_ShouldRoundTrip() { // Arrange - var sut = new QueueMessageSerializerBuilder(new LogFactory()) + var sut = new QueueMessageSerializerBuilder() .Build(); const string testMessage = "Hello, Queue!"; @@ -38,7 +36,7 @@ public void SerializeAndDeserializeMessage_ShouldRoundTrip() public void SerializeAndDeserializeMessage_ShouldRoundTrip_RequestMessage() { // Arrange - var sut = new QueueMessageSerializerBuilder(new LogFactory()) + var sut = new QueueMessageSerializerBuilder() .Build(); var request = new RequestMessage() @@ -67,7 +65,7 @@ public void SerializeAndDeserializeMessageWithDataStream_ShouldRoundTrip_Request var typeRegistry = new TypeRegistry(); typeRegistry.Register(typeof(IHaveTypeWithDataStreamsService)); // Arrange - var sut = new QueueMessageSerializerBuilder(new LogFactory()) + var sut = new QueueMessageSerializerBuilder() .WithTypeRegistry(typeRegistry) .Build(); @@ -138,45 +136,5 @@ static Func WriteRepeatedStringsAsync(string to }); } } - - public class QueueMessageSerializerBuilder - { - readonly ILogFactory logFactory; - ITypeRegistry? typeRegistry; - Action? configureSerializer; - - public QueueMessageSerializerBuilder(ILogFactory logFactory) - { - this.logFactory = logFactory; - } - - public QueueMessageSerializerBuilder WithTypeRegistry(ITypeRegistry typeRegistry) - { - this.typeRegistry = typeRegistry; - return this; - } - - public QueueMessageSerializerBuilder WithSerializerSettings(Action configure) - { - configureSerializer = configure; - return this; - } - - public QueueMessageSerializer Build() - { - var typeRegistry = this.typeRegistry ?? new TypeRegistry(); - - StreamCapturingJsonSerializer StreamCapturingSerializer() - { - var settings = MessageSerializerBuilder.CreateSerializer(); - var binder = new RegisteredSerializationBinder(typeRegistry); - settings.SerializationBinder = binder; - configureSerializer?.Invoke(settings); - return new StreamCapturingJsonSerializer(settings); - } - - return new QueueMessageSerializer(StreamCapturingSerializer); - } - } } } \ No newline at end of file diff --git a/source/Halibut.Tests/ServiceModel/PendingRequestQueueFixture.cs b/source/Halibut.Tests/ServiceModel/PendingRequestQueueFixture.cs index 86936736b..9be202741 100644 --- a/source/Halibut.Tests/ServiceModel/PendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/ServiceModel/PendingRequestQueueFixture.cs @@ -9,24 +9,28 @@ using Halibut.ServiceModel; using Halibut.Tests.Builders; using Halibut.Tests.Support; +using Halibut.Tests.Support.TestAttributes; using Halibut.Transport.Protocol; using NUnit.Framework; namespace Halibut.Tests.ServiceModel { + public class PendingRequestQueueFixture : BaseTest { [Test] - public async Task QueueAndWait_WillContinueWaitingUntilResponseIsApplied() + [AllQueuesTestCases] + public async Task QueueAndWait_WillContinueWaitingUntilResponseIsApplied(PendingRequestQueueTestCase queueTestCase) { // Arrange const string endpoint = "poll://endpoint001"; - await using var sut = new PendingRequestQueueBuilder().WithEndpoint(endpoint).Build(); + await using var queueHolder = queueTestCase.Builder.WithEndpoint(endpoint).Build(); + var sut = queueHolder.PendingRequestQueue; var request = new RequestMessageBuilder(endpoint).Build(); var expectedResponse = ResponseMessageBuilder.FromRequest(request).Build(); - var queueAndWaitTask = await StartQueueAndWaitAndWaitForRequestToBeQueued(sut, request, CancellationToken); + var queueAndWaitTask = sut.QueueAndWaitAsync(request, CancellationToken); await sut.DequeueAsync(CancellationToken); // Act @@ -37,21 +41,23 @@ public async Task QueueAndWait_WillContinueWaitingUntilResponseIsApplied() // Assert var response = await queueAndWaitTask; - response.Should().Be(expectedResponse); + response.Should().BeEquivalentTo(expectedResponse); } [Test] - public async Task QueueAndWait_WillIgnoreUnrelatedApplyResponses_AndShouldContinueWaiting() + [AllQueuesTestCases] + public async Task QueueAndWait_WillIgnoreUnrelatedApplyResponses_AndShouldContinueWaiting(PendingRequestQueueTestCase queueTestCase) { // Arrange const string endpoint = "poll://endpoint001"; - await using var sut = new PendingRequestQueueBuilder().WithEndpoint(endpoint).Build(); + await using var queueHolder = queueTestCase.Builder.WithEndpoint(endpoint).Build(); + var sut = queueHolder.PendingRequestQueue; var request = new RequestMessageBuilder(endpoint).Build(); var expectedResponse = ResponseMessageBuilder.FromRequest(request).Build(); var unexpectedResponse = new ResponseMessageBuilder(Guid.NewGuid().ToString()).Build(); - var queueAndWaitTask = await StartQueueAndWaitAndWaitForRequestToBeQueued(sut, request, CancellationToken); + var queueAndWaitTask = sut.QueueAndWaitAsync(request, CancellationToken); await sut.DequeueAsync(CancellationToken); @@ -60,8 +66,8 @@ public async Task QueueAndWait_WillIgnoreUnrelatedApplyResponses_AndShouldContin queueAndWaitTask.IsCompleted.Should().BeFalse(); // Apply unrelated responses - await sut.ApplyResponse(null!, request.ActivityId); - await sut.ApplyResponse(unexpectedResponse, request.ActivityId); + await sut.ApplyResponse(null!, Guid.NewGuid()); + await sut.ApplyResponse(unexpectedResponse, Guid.NewGuid()); await Task.Delay(1000, CancellationToken); queueAndWaitTask.IsCompleted.Should().BeFalse(); @@ -71,23 +77,25 @@ public async Task QueueAndWait_WillIgnoreUnrelatedApplyResponses_AndShouldContin // Assert var response = await queueAndWaitTask; - response.Should().Be(expectedResponse); + response.Should().BeEquivalentTo(expectedResponse); } [Test] - public async Task QueueAndWait_WhenPollingRequestQueueTimeoutIsReached_WillStopWaitingAndClearRequest() + [AllQueuesTestCases] + public async Task QueueAndWait_WhenPollingRequestQueueTimeoutIsReached_WillStopWaitingAndClearRequest(PendingRequestQueueTestCase queueTestCase) { // Arrange const string endpoint = "poll://endpoint001"; - await using var sut = new PendingRequestQueueBuilder().WithEndpoint(endpoint).Build(); + await using var queueHolder = queueTestCase.Builder.WithEndpoint(endpoint).Build(); + var sut = queueHolder.PendingRequestQueue; var request = new RequestMessageBuilder(endpoint) .WithServiceEndpoint(seb => seb.WithPollingRequestQueueTimeout(TimeSpan.FromMilliseconds(1000))) .Build(); // Act var stopwatch = Stopwatch.StartNew(); - var queueAndWaitTask = await StartQueueAndWaitAndWaitForRequestToBeQueued(sut, request, CancellationToken); + var queueAndWaitTask = sut.QueueAndWaitAsync(request, CancellationToken); var response = await queueAndWaitTask; // Assert @@ -101,52 +109,26 @@ public async Task QueueAndWait_WhenPollingRequestQueueTimeoutIsReached_WillStopW } [Test] - public async Task QueueAndWait_WithRelyOnConnectionTimeoutsInsteadOfPollingRequestMaximumMessageProcessingTimeout_WhenPollingRequestQueueTimeoutIsReached_WillStopWaitingAndClearRequest() + [AllQueuesTestCases] + public async Task QueueAndWait_WhenRequestIsDequeued_WithRelyOnConnectionTimeoutsInsteadOfPollingRequestMaximumMessageProcessingTimeout_WillWaitForeverUntilResponseIsSet(PendingRequestQueueTestCase queueTestCase) { // Arrange const string endpoint = "poll://endpoint001"; - await using var sut = new PendingRequestQueueBuilder() - .WithRelyOnConnectionTimeoutsInsteadOfPollingRequestMaximumMessageProcessingTimeout(true) - .WithEndpoint(endpoint) - .Build(); - var request = new RequestMessageBuilder(endpoint) - .WithServiceEndpoint(seb => seb.WithPollingRequestQueueTimeout(TimeSpan.FromMilliseconds(1000))) - .Build(); - - // Act - var stopwatch = Stopwatch.StartNew(); - var queueAndWaitTask = await StartQueueAndWaitAndWaitForRequestToBeQueued(sut, request, CancellationToken); - var response = await queueAndWaitTask; - - // Assert - // Although we sleep for 1 second, sometimes it can be just under. So be generous with the buffer. - stopwatch.Elapsed.Should().BeGreaterThan(TimeSpan.FromMilliseconds(800)); - response.Id.Should().Be(request.Id); - response.Error!.Message.Should().Be("A request was sent to a polling endpoint, but the polling endpoint did not collect the request within the allowed time (00:00:01), so the request timed out."); - - var next = await sut.DequeueAsync(CancellationToken); - next.Should().BeNull(); - } - - [Test] - public async Task QueueAndWait_WhenRequestIsDequeued_WithRelyOnConnectionTimeoutsInsteadOfPollingRequestMaximumMessageProcessingTimeout_WillWaitForeverUntilResponseIsSet() - { - // Arrange - const string endpoint = "poll://endpoint001"; - - await using var sut = new PendingRequestQueueBuilder() + await using var queueHolder = queueTestCase.Builder .WithEndpoint(endpoint) .WithPollingQueueWaitTimeout(TimeSpan.Zero) // Remove delay, otherwise we wait the full 20 seconds for DequeueAsync at the end of the test - .WithRelyOnConnectionTimeoutsInsteadOfPollingRequestMaximumMessageProcessingTimeout(true) .Build(); - var request = new RequestMessageBuilder(endpoint) - .WithServiceEndpoint(seb => seb.WithPollingRequestQueueTimeout(TimeSpan.FromMilliseconds(1000))) - .Build(); - var expectedResponse = ResponseMessageBuilder.FromRequest(request).Build(); - + var sut = queueHolder.PendingRequestQueue; + // Act - var (queueAndWaitTask, dequeued) = await QueueAndDequeueRequest_ForTimeoutTestingOnly_ToCopeWithRaceCondition(sut, request, CancellationToken); + var lowTimeoutToShowWeAreWaitingForTheResponseToComeBack = TimeSpan.FromMilliseconds(1000); + var (request, queueAndWaitTask, dequeued) = await QueueAndDequeueRequest_ForVeryLowPickupTimeouts( + sut, + endpoint, + builder => builder.WithServiceEndpoint(seb => seb.WithPollingRequestQueueTimeout(lowTimeoutToShowWeAreWaitingForTheResponseToComeBack)), + CancellationToken); + var expectedResponse = ResponseMessageBuilder.FromRequest(request).Build(); var delayTask = Task.Delay(TimeSpan.FromSeconds(10)); var finishedTask = await Task.WhenAny(queueAndWaitTask, delayTask); @@ -158,47 +140,49 @@ public async Task QueueAndWait_WhenRequestIsDequeued_WithRelyOnConnectionTimeout var response = await queueAndWaitTask; // Assert - dequeued.RequestMessage.Should().NotBeNull("We should have removed the item from the queue before it timed out.").And.Be(request); - response.Should().Be(expectedResponse); + dequeued.RequestMessage.Should().NotBeNull("We should have removed the item from the queue before it timed out.") + .And.BeEquivalentTo(request); + response.Should().BeEquivalentTo(expectedResponse); var next = await sut.DequeueAsync(CancellationToken); next.Should().BeNull(); } - [Test] - public async Task QueueAndWait_WhenRequestIsDequeued_ButPollingRequestQueueTimeoutIsReached_ShouldWaitTillRequestRespondsAndClearRequest() - { - // Arrange - const string endpoint = "poll://endpoint001"; - - await using var sut = new PendingRequestQueueBuilder() - .WithEndpoint(endpoint) - .WithPollingQueueWaitTimeout(TimeSpan.Zero) // Remove delay, otherwise we wait the full 20 seconds for DequeueAsync at the end of the test - .Build(); - - var request = new RequestMessageBuilder(endpoint) - .WithServiceEndpoint(seb => seb.WithPollingRequestQueueTimeout(TimeSpan.FromMilliseconds(1000))) - .Build(); - - var expectedResponse = ResponseMessageBuilder.FromRequest(request).Build(); - - // Act - var (queueAndWaitTask, dequeued) = await QueueAndDequeueRequest_ForTimeoutTestingOnly_ToCopeWithRaceCondition(sut, request, CancellationToken); - - await Task.Delay(2000, CancellationToken); - dequeued.CancellationToken.IsCancellationRequested.Should().BeFalse("Should not have cancelled the request after PollingRequestQueueTimeout is reached"); - - await sut.ApplyResponse(expectedResponse, request.ActivityId); - - var response = await queueAndWaitTask; - - // Assert - dequeued.RequestMessage.Should().NotBeNull("We should have removed the item from the queue before it timed out.").And.Be(request); - response.Should().Be(expectedResponse); - - var next = await sut.DequeueAsync(CancellationToken); - next.Should().BeNull(); - } + // [Test] + // public async Task QueueAndWait_WhenRequestIsDequeued_ButPollingRequestQueueTimeoutIsReached_ShouldWaitTillRequestRespondsAndClearRequest() + // { + // // Arrange + // const string endpoint = "poll://endpoint001"; + // + // await using var queueHolder = new PendingRequestQueueBuilder() + // .WithEndpoint(endpoint) + // .WithPollingQueueWaitTimeout(TimeSpan.Zero) // Remove delay, otherwise we wait the full 20 seconds for DequeueAsync at the end of the test + // .Build(); + // var sut = queueHolder.PendingRequestQueue; + // + // var request = new RequestMessageBuilder(endpoint) + // .WithServiceEndpoint(seb => seb.WithPollingRequestQueueTimeout(TimeSpan.FromMilliseconds(1000))) + // .Build(); + // + // var expectedResponse = ResponseMessageBuilder.FromRequest(request).Build(); + // + // // Act + // var (queueAndWaitTask, dequeued) = await QueueAndDequeueRequest_ForTimeoutTestingOnly_ToCopeWithRaceCondition(sut, request, CancellationToken); + // + // await Task.Delay(2000, CancellationToken); + // dequeued.CancellationToken.IsCancellationRequested.Should().BeFalse("Should not have cancelled the request after PollingRequestQueueTimeout is reached"); + // + // await sut.ApplyResponse(expectedResponse, request.ActivityId); + // + // var response = await queueAndWaitTask; + // + // // Assert + // dequeued.RequestMessage.Should().NotBeNull("We should have removed the item from the queue before it timed out.").And.Be(request); + // response.Should().Be(expectedResponse); + // + // var next = await sut.DequeueAsync(CancellationToken); + // next.Should().BeNull(); + // } [Test] public async Task QueueAndWait_AddingMultipleItemsToQueueInOrder_ShouldDequeueInOrder() @@ -206,7 +190,8 @@ public async Task QueueAndWait_AddingMultipleItemsToQueueInOrder_ShouldDequeueIn // Arrange const string endpoint = "poll://endpoint001"; - await using var sut = new PendingRequestQueueBuilder().WithEndpoint(endpoint).Build(); + await using var queueHolder = new PendingRequestQueueBuilder().WithEndpoint(endpoint).Build(); + var sut = queueHolder.PendingRequestQueue; var requestsInOrder = Enumerable.Range(0, 3) .Select(_ => new RequestMessageBuilder(endpoint).Build()) @@ -236,7 +221,8 @@ public async Task QueueAndWait_AddingMultipleItemsToQueueConcurrently_AllRequest // Arrange const string endpoint = "poll://endpoint001"; - await using var sut = new PendingRequestQueueBuilder().WithEndpoint(endpoint).Build(); + await using var queueHolder = new PendingRequestQueueBuilder().WithEndpoint(endpoint).Build(); + var sut = queueHolder.PendingRequestQueue; var requestsInOrder = Enumerable.Range(0, 30) .Select(_ => new RequestMessageBuilder(endpoint).Build()) @@ -268,7 +254,8 @@ public async Task QueueAndWait_Can_Queue_Dequeue_Apply_VeryLargeNumberOfRequests // Arrange const string endpoint = "poll://endpoint001"; - await using var sut = new PendingRequestQueueBuilder().WithEndpoint(endpoint).Build(); + await using var queueHolder = new PendingRequestQueueBuilder().WithEndpoint(endpoint).Build(); + var sut = queueHolder.PendingRequestQueue; // Choose a number large enough that it will fail when doing the 'synchronous' version. var requestsInOrder = Enumerable.Range(0, 5000) @@ -301,7 +288,8 @@ public async Task QueueAndWait_Can_Queue_Dequeue_WhenRequestsAreBeingCancelled() const int totalRequest = 500; const int minimumCancelledRequest = 100; - await using var sut = new PendingRequestQueueBuilder().WithEndpoint(endpoint).Build(); + await using var queueHolder = new PendingRequestQueueBuilder().WithEndpoint(endpoint).Build(); + var sut = queueHolder.PendingRequestQueue; var requestsInOrder = Enumerable.Range(0, totalRequest) .Select(_ => new RequestMessageBuilder(endpoint).Build()) @@ -375,7 +363,8 @@ public async Task QueueAndWait_CancellingAPendingRequestBeforeItIsDequeued_Shoul // Arrange const string endpoint = "poll://endpoint001"; - await using var sut = new PendingRequestQueueBuilder().WithEndpoint(endpoint).Build(); + await using var queueHolder = new PendingRequestQueueBuilder().WithEndpoint(endpoint).Build(); + var sut = queueHolder.PendingRequestQueue; var request = new RequestMessageBuilder(endpoint).Build(); var cancellationTokenSource = new CancellationTokenSource(); @@ -402,10 +391,11 @@ public async Task QueueAndWait_CancellingAPendingRequestAfterItIsDequeued_Should // Arrange const string endpoint = "poll://endpoint001"; - await using var sut = new PendingRequestQueueBuilder() + await using var queueHolder = new PendingRequestQueueBuilder() .WithEndpoint(endpoint) .WithPollingQueueWaitTimeout(TimeSpan.Zero) // Remove delay, otherwise we wait the full 20 seconds for DequeueAsync at the end of the test .Build(); + var sut = queueHolder.PendingRequestQueue; var request = new RequestMessageBuilder(endpoint).Build(); @@ -442,10 +432,11 @@ public async Task DequeueAsync_WithNothingQueued_WillWaitPollingQueueWaitTimeout // Arrange const string endpoint = "poll://endpoint001"; - await using var sut = new PendingRequestQueueBuilder() + await using var queueHolder = new PendingRequestQueueBuilder() .WithEndpoint(endpoint) .WithPollingQueueWaitTimeout(TimeSpan.FromSeconds(1)) .Build(); + var sut = queueHolder.PendingRequestQueue; // Act var stopwatch = Stopwatch.StartNew(); @@ -463,10 +454,11 @@ public async Task DequeueAsync_WithNothingQueued_WillWaitPollingQueueWaitTimeout // Arrange const string endpoint = "poll://endpoint001"; - await using var sut = new PendingRequestQueueBuilder() + await using var queueHolder = new PendingRequestQueueBuilder() .WithEndpoint(endpoint) .WithPollingQueueWaitTimeout(TimeSpan.FromSeconds(1)) .Build(); + var sut = queueHolder.PendingRequestQueue; // Queue, Dequeue, and respond to a previous request var previousRequest = new RequestMessageBuilder(endpoint).Build(); @@ -493,7 +485,8 @@ public async Task DequeueAsync_WillContinueWaitingUntilItemIsQueued() // Arrange const string endpoint = "poll://endpoint001"; - await using var sut = new PendingRequestQueueBuilder().WithEndpoint(endpoint).Build(); + await using var queueHolder = new PendingRequestQueueBuilder().WithEndpoint(endpoint).Build(); + var sut = queueHolder.PendingRequestQueue; var request = new RequestMessageBuilder(endpoint).Build(); var expectedResponse = ResponseMessageBuilder.FromRequest(request).Build(); @@ -525,7 +518,8 @@ public async Task DequeueAsync_WithMultipleDequeueCallsWaiting_WhenSingleRequest // Arrange const string endpoint = "poll://endpoint001"; - await using var sut = new PendingRequestQueueBuilder().WithEndpoint(endpoint).Build(); + await using var queueHolder = new PendingRequestQueueBuilder().WithEndpoint(endpoint).Build(); + var sut = queueHolder.PendingRequestQueue; var request = new RequestMessageBuilder(endpoint).Build(); var expectedResponse = ResponseMessageBuilder.FromRequest(request).Build(); @@ -554,10 +548,11 @@ public async Task ApplyResponse_AfterRequestHasBeenCollected_AndWaitingHasBeenCa // Arrange const string endpoint = "poll://endpoint001"; - await using var sut = new PendingRequestQueueBuilder() + await using var queueHolder = new PendingRequestQueueBuilder() .WithEndpoint(endpoint) .WithPollingQueueWaitTimeout(TimeSpan.Zero) // Remove delay, otherwise we wait the full 20 seconds for DequeueAsync at the end of the test .Build(); + var sut = queueHolder.PendingRequestQueue; var request = new RequestMessageBuilder(endpoint).Build(); var expectedResponse = ResponseMessageBuilder.FromRequest(request).Build(); @@ -620,30 +615,31 @@ Task StartQueueAndWait( return task; } - async Task<(Task queueAndWaitTask, RequestMessageWithCancellationToken dequeued)> QueueAndDequeueRequest_ForTimeoutTestingOnly_ToCopeWithRaceCondition( + async Task<(RequestMessage, Task queueAndWaitTask, RequestMessageWithCancellationToken dequeued)> QueueAndDequeueRequest_ForVeryLowPickupTimeouts( IPendingRequestQueue sut, - RequestMessage request, + string endpoint, + Func requestBuilderFactory, CancellationToken cancellationToken) { //For most tests, this is not a good method to use. It is a fix for some specific tests to cope with a race condition when Team City runs out of resources (and causes tests to become flaky) while (true) { - var queueAndWaitTask = await StartQueueAndWaitAndWaitForRequestToBeQueued(sut, request, cancellationToken); - sut.Count.Should().Be(1, "Item should be queued"); - - var dequeued = await sut.DequeueAsync(cancellationToken); - sut.Count.Should().Be(0, "Item should be dequeued"); - - // There is a race condition where the task/thread that queues the request can actually progress far enough that it times out before DequeueAsync can take the request. - // This tends to happen in tests where PollingRequestQueueTimeout has been reduced. - // If this happens, then the item is 'completed', and DequeueAsync returns null (not the state we wish to be in) - // So if dequeued is null, then try again. - if (dequeued is not null) + var request = requestBuilderFactory(new RequestMessageBuilder(endpoint)).Build(); + + // Since the pickup timeout is low, there is a good chance that queueAndWaitTask will finish before + // we can dequeue the work. In that case we will try again. + var queueAndWaitTask = Task.Run(() => sut.QueueAndWaitAsync(request, cancellationToken)); + + // While the queueAndWaitTask is not complete, keep trying to dequeue the work. + while (!queueAndWaitTask.IsCompleted) { - return (queueAndWaitTask, dequeued); + var dequeued = await sut.DequeueAsync(cancellationToken); + if (dequeued is not null) + { + return (request, queueAndWaitTask, dequeued); + } + cancellationToken.ThrowIfCancellationRequested(); } - - cancellationToken.ThrowIfCancellationRequested(); } } diff --git a/source/Halibut.Tests/Support/TestAttributes/AllQueuesTestCasesAttribute.cs b/source/Halibut.Tests/Support/TestAttributes/AllQueuesTestCasesAttribute.cs new file mode 100644 index 000000000..34d7f91e0 --- /dev/null +++ b/source/Halibut.Tests/Support/TestAttributes/AllQueuesTestCasesAttribute.cs @@ -0,0 +1,69 @@ +using System; +using System.Collections; +using System.Collections.Generic; +using System.Linq; +using Halibut.ServiceModel; +using Halibut.Tests.Builders; +using Halibut.Tests.Support.BackwardsCompatibility; +using Halibut.Tests.Support.TestCases; +using NUnit.Framework; + +namespace Halibut.Tests.Support.TestAttributes +{ + [AttributeUsage(AttributeTargets.Method, AllowMultiple = true, Inherited = false)] + public class AllQueuesTestCasesAttribute : HalibutTestCaseSourceAttribute + { + public AllQueuesTestCasesAttribute() : + base( + typeof(PendingRequestQueueFactories), + nameof(PendingRequestQueueFactories.GetEnumerator), null) + { + } + + static class PendingRequestQueueFactories + { + public static IEnumerable GetEnumerator() + { + var factories = new List(); + factories.Add(new PendingRequestQueueTestCase("Redis", () => new RedisPendingRequestQueueBuilder())); + factories.Add(new PendingRequestQueueTestCase("InMemory", () => new PendingRequestQueueBuilder())); + + return factories; + } + } + } + + public class PendingRequestQueueTestCase + { + public readonly string Name; + private Func BuilderBuilder { get; } + + public IPendingRequestQueueBuilder Builder => BuilderBuilder(); + + public PendingRequestQueueTestCase(string name, Func builder) + { + Name = name; + BuilderBuilder = builder; + } + + public override string ToString() => Name; + + protected bool Equals(PendingRequestQueueTestCase other) + { + return Name == other.Name; + } + + public override bool Equals(object? obj) + { + if (obj is null) return false; + if (ReferenceEquals(this, obj)) return true; + if (obj.GetType() != GetType()) return false; + return Equals((PendingRequestQueueTestCase)obj); + } + + public override int GetHashCode() + { + return Name.GetHashCode(); + } + } +} diff --git a/source/Halibut.Tests/Timeouts/PollingQueueTimeouts.cs b/source/Halibut.Tests/Timeouts/PollingQueueTimeouts.cs index 0020ab006..34e852425 100644 --- a/source/Halibut.Tests/Timeouts/PollingQueueTimeouts.cs +++ b/source/Halibut.Tests/Timeouts/PollingQueueTimeouts.cs @@ -43,7 +43,7 @@ public async Task WhenNoMessagesAreSentToAPollingTentacle_ThePollingRequestQueue .WithPendingRequestQueueFactory(logFactory => new FuncPendingRequestQueueFactory(uri => new PendingRequestQueueBuilder() .WithLog(logFactory.ForEndpoint(uri)) .WithPollingQueueWaitTimeout(TimeSpan.FromSeconds(1)) - .Build())) + .Build().PendingRequestQueue)) // TODO: Test redis queue here as well. .WithEchoService() .Build(CancellationToken)) { @@ -76,7 +76,7 @@ public async Task WhenThePollingQueueHasNoMessagesAndDoesNotReturnNullResponsesP .WithPendingRequestQueueFactory(logFactory => new FuncPendingRequestQueueFactory(uri => new PendingRequestQueueBuilder() .WithLog(logFactory.ForEndpoint(uri)) .WithPollingQueueWaitTimeout(TimeSpan.FromSeconds(100)) // Increase the time between sending null requests back to trigger the timeout. - .Build())) + .Build().PendingRequestQueue)) .WithEchoService() .WithHalibutTimeoutsAndLimits(new HalibutTimeoutsAndLimitsForTestsBuilder() .Build() diff --git a/source/Halibut.Tests/Timeouts/TimeoutsApplyDuringHandShake.cs b/source/Halibut.Tests/Timeouts/TimeoutsApplyDuringHandShake.cs index 83c3babe9..989b2ddbd 100644 --- a/source/Halibut.Tests/Timeouts/TimeoutsApplyDuringHandShake.cs +++ b/source/Halibut.Tests/Timeouts/TimeoutsApplyDuringHandShake.cs @@ -56,7 +56,7 @@ int writeNumberToPauseOn // Ie pause on the first or second write .WithPendingRequestQueueFactory(logFactory => new FuncPendingRequestQueueFactory(uri => new PendingRequestQueueBuilder() .WithLog(logFactory.ForEndpoint(uri)) .WithPollingQueueWaitTimeout(TimeSpan.FromSeconds(1)) - .Build())) + .Build().PendingRequestQueue)) .WithPollingReconnectRetryPolicy(() => new RetryPolicy(1, TimeSpan.Zero, TimeSpan.Zero)) .WithEchoService() .WithHalibutTimeoutsAndLimits(halibutTimeoutsAndLimits) From c2093553f69c0b2bf09dd04892239af3e02de1e7 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Thu, 7 Aug 2025 16:58:11 +1000 Subject: [PATCH 033/137] . --- .../Queue/Redis/Utils/RedisContainerBuilder.cs | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/source/Halibut.Tests/Queue/Redis/Utils/RedisContainerBuilder.cs b/source/Halibut.Tests/Queue/Redis/Utils/RedisContainerBuilder.cs index 36a9212b7..65392f431 100644 --- a/source/Halibut.Tests/Queue/Redis/Utils/RedisContainerBuilder.cs +++ b/source/Halibut.Tests/Queue/Redis/Utils/RedisContainerBuilder.cs @@ -5,6 +5,7 @@ using DotNet.Testcontainers.Containers; using Halibut.Tests.Support; using NUnit.Framework; +using Try = Halibut.Util.Try; namespace Halibut.Tests.Queue.Redis.Utils { @@ -98,7 +99,15 @@ public RedisContainer(IContainer container, int redisPort) /// /// Starts the Redis container /// - public Task StartAsync() => _container.StartAsync(); + public async Task StartAsync() + { + // Since I have seen errors here. + for (int i = 0; i < 5; i++) + { + await Try.IgnoringError(async () => await _container.StartAsync()); + } + await _container.StartAsync(); + } /// /// Stops the Redis container From ee0fe397c18028b364d5aaa1043c7ba9e4b53d6d Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Mon, 11 Aug 2025 10:33:39 +1000 Subject: [PATCH 034/137] Queue tests are now shared between in mem and redis queue --- .../PendingRequestQueueFixture.cs | 156 ++++++++++-------- .../Queue/Redis/RedisPendingRequestQueue.cs | 35 ++-- .../ServiceModel/IPendingRequestQueue.cs | 6 + 3 files changed, 119 insertions(+), 78 deletions(-) diff --git a/source/Halibut.Tests/ServiceModel/PendingRequestQueueFixture.cs b/source/Halibut.Tests/ServiceModel/PendingRequestQueueFixture.cs index 9be202741..448d51aa3 100644 --- a/source/Halibut.Tests/ServiceModel/PendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/ServiceModel/PendingRequestQueueFixture.cs @@ -110,7 +110,8 @@ public async Task QueueAndWait_WhenPollingRequestQueueTimeoutIsReached_WillStopW [Test] [AllQueuesTestCases] - public async Task QueueAndWait_WhenRequestIsDequeued_WithRelyOnConnectionTimeoutsInsteadOfPollingRequestMaximumMessageProcessingTimeout_WillWaitForeverUntilResponseIsSet(PendingRequestQueueTestCase queueTestCase) + public async Task QueueAndWait_WhenRequestIsDequeued_WithRelyOnConnectionTimeoutsInsteadOfPollingRequestMaximumMessageProcessingTimeout_WillWaitForeverUntilResponseIsSet( + PendingRequestQueueTestCase queueTestCase) { // Arrange const string endpoint = "poll://endpoint001"; @@ -148,49 +149,53 @@ public async Task QueueAndWait_WhenRequestIsDequeued_WithRelyOnConnectionTimeout next.Should().BeNull(); } - // [Test] - // public async Task QueueAndWait_WhenRequestIsDequeued_ButPollingRequestQueueTimeoutIsReached_ShouldWaitTillRequestRespondsAndClearRequest() - // { - // // Arrange - // const string endpoint = "poll://endpoint001"; - // - // await using var queueHolder = new PendingRequestQueueBuilder() - // .WithEndpoint(endpoint) - // .WithPollingQueueWaitTimeout(TimeSpan.Zero) // Remove delay, otherwise we wait the full 20 seconds for DequeueAsync at the end of the test - // .Build(); - // var sut = queueHolder.PendingRequestQueue; - // - // var request = new RequestMessageBuilder(endpoint) - // .WithServiceEndpoint(seb => seb.WithPollingRequestQueueTimeout(TimeSpan.FromMilliseconds(1000))) - // .Build(); - // - // var expectedResponse = ResponseMessageBuilder.FromRequest(request).Build(); - // - // // Act - // var (queueAndWaitTask, dequeued) = await QueueAndDequeueRequest_ForTimeoutTestingOnly_ToCopeWithRaceCondition(sut, request, CancellationToken); - // - // await Task.Delay(2000, CancellationToken); - // dequeued.CancellationToken.IsCancellationRequested.Should().BeFalse("Should not have cancelled the request after PollingRequestQueueTimeout is reached"); - // - // await sut.ApplyResponse(expectedResponse, request.ActivityId); - // - // var response = await queueAndWaitTask; - // - // // Assert - // dequeued.RequestMessage.Should().NotBeNull("We should have removed the item from the queue before it timed out.").And.Be(request); - // response.Should().Be(expectedResponse); - // - // var next = await sut.DequeueAsync(CancellationToken); - // next.Should().BeNull(); - // } + [Test] + [AllQueuesTestCases] + public async Task QueueAndWait_WhenRequestIsDequeued_ButPollingRequestQueueTimeoutIsReached_ShouldWaitTillRequestRespondsAndClearRequest( + PendingRequestQueueTestCase queueTestCase) + { + // Arrange + const string endpoint = "poll://endpoint001"; + + await using var queueHolder = queueTestCase.Builder + .WithEndpoint(endpoint) + .WithPollingQueueWaitTimeout(TimeSpan.Zero) // Remove delay, otherwise we wait the full 20 seconds for DequeueAsync at the end of the test + .Build(); + var sut = queueHolder.PendingRequestQueue; + + // Act + var lowTimeoutToShowWeAreWaitingForTheResponseToComeBack = TimeSpan.FromMilliseconds(1000); + var (request, queueAndWaitTask, dequeued) = await QueueAndDequeueRequest_ForVeryLowPickupTimeouts( + sut, + endpoint, + builder => builder.WithServiceEndpoint(seb => seb.WithPollingRequestQueueTimeout(lowTimeoutToShowWeAreWaitingForTheResponseToComeBack)), + CancellationToken); + var expectedResponse = ResponseMessageBuilder.FromRequest(request).Build(); + + await Task.Delay(2000, CancellationToken); + dequeued.CancellationToken.IsCancellationRequested.Should().BeFalse("Should not have cancelled the request after PollingRequestQueueTimeout is reached"); + + await sut.ApplyResponse(expectedResponse, request.ActivityId); + + var response = await queueAndWaitTask; + + // Assert + dequeued.RequestMessage.Should().NotBeNull("We should have removed the item from the queue before it timed out.") + .And.BeEquivalentTo(request); + response.Should().BeEquivalentTo(expectedResponse); + + var next = await sut.DequeueAsync(CancellationToken); + next.Should().BeNull(); + } [Test] - public async Task QueueAndWait_AddingMultipleItemsToQueueInOrder_ShouldDequeueInOrder() + [AllQueuesTestCases] + public async Task QueueAndWait_AddingMultipleItemsToQueueInOrder_ShouldDequeueInOrder(PendingRequestQueueTestCase queueTestCase) { // Arrange const string endpoint = "poll://endpoint001"; - await using var queueHolder = new PendingRequestQueueBuilder().WithEndpoint(endpoint).Build(); + await using var queueHolder = queueTestCase.Builder.WithEndpoint(endpoint).Build(); var sut = queueHolder.PendingRequestQueue; var requestsInOrder = Enumerable.Range(0, 3) @@ -209,19 +214,20 @@ public async Task QueueAndWait_AddingMultipleItemsToQueueInOrder_ShouldDequeueIn foreach (var expectedRequest in requestsInOrder) { var request = await sut.DequeueAsync(CancellationToken); - request!.RequestMessage.Should().Be(expectedRequest); + request!.RequestMessage.Should().BeEquivalentTo(expectedRequest); } await ApplyResponsesConcurrentlyAndEnsureAllQueueResponsesMatch(sut, requestsInOrder, queueAndWaitTasksInOrder); } [Test] - public async Task QueueAndWait_AddingMultipleItemsToQueueConcurrently_AllRequestsShouldBeSuccessfullyQueued() + [AllQueuesTestCases] + public async Task QueueAndWait_AddingMultipleItemsToQueueConcurrently_AllRequestsShouldBeSuccessfullyQueued(PendingRequestQueueTestCase queueTestCase) { // Arrange const string endpoint = "poll://endpoint001"; - await using var queueHolder = new PendingRequestQueueBuilder().WithEndpoint(endpoint).Build(); + await using var queueHolder = queueTestCase.Builder.WithEndpoint(endpoint).Build(); var sut = queueHolder.PendingRequestQueue; var requestsInOrder = Enumerable.Range(0, 30) @@ -249,12 +255,13 @@ public async Task QueueAndWait_AddingMultipleItemsToQueueConcurrently_AllRequest [Test] [NonParallelizable] - public async Task QueueAndWait_Can_Queue_Dequeue_Apply_VeryLargeNumberOfRequests() + [AllQueuesTestCases] + public async Task QueueAndWait_Can_Queue_Dequeue_Apply_VeryLargeNumberOfRequests(PendingRequestQueueTestCase queueTestCase) { // Arrange const string endpoint = "poll://endpoint001"; - await using var queueHolder = new PendingRequestQueueBuilder().WithEndpoint(endpoint).Build(); + await using var queueHolder = queueTestCase.Builder.WithEndpoint(endpoint).Build(); var sut = queueHolder.PendingRequestQueue; // Choose a number large enough that it will fail when doing the 'synchronous' version. @@ -281,14 +288,15 @@ public async Task QueueAndWait_Can_Queue_Dequeue_Apply_VeryLargeNumberOfRequests [Test] [NonParallelizable] - public async Task QueueAndWait_Can_Queue_Dequeue_WhenRequestsAreBeingCancelled() + [AllQueuesTestCases] + public async Task QueueAndWait_Can_Queue_Dequeue_WhenRequestsAreBeingCancelled(PendingRequestQueueTestCase queueTestCase) { // Arrange const string endpoint = "poll://endpoint001"; const int totalRequest = 500; const int minimumCancelledRequest = 100; - await using var queueHolder = new PendingRequestQueueBuilder().WithEndpoint(endpoint).Build(); + await using var queueHolder = queueTestCase.Builder.WithEndpoint(endpoint).Build(); var sut = queueHolder.PendingRequestQueue; var requestsInOrder = Enumerable.Range(0, totalRequest) @@ -358,12 +366,14 @@ public async Task QueueAndWait_Can_Queue_Dequeue_WhenRequestsAreBeingCancelled() } [Test] - public async Task QueueAndWait_CancellingAPendingRequestBeforeItIsDequeued_ShouldThrowExceptionAndClearRequest() + [AllQueuesTestCases] + public async Task QueueAndWait_CancellingAPendingRequestBeforeItIsDequeued_ShouldThrowExceptionAndClearRequest( + PendingRequestQueueTestCase queueTestCase) { // Arrange const string endpoint = "poll://endpoint001"; - await using var queueHolder = new PendingRequestQueueBuilder().WithEndpoint(endpoint).Build(); + await using var queueHolder = queueTestCase.Builder.WithEndpoint(endpoint).Build(); var sut = queueHolder.PendingRequestQueue; var request = new RequestMessageBuilder(endpoint).Build(); @@ -386,12 +396,14 @@ public async Task QueueAndWait_CancellingAPendingRequestBeforeItIsDequeued_Shoul } [Test] - public async Task QueueAndWait_CancellingAPendingRequestAfterItIsDequeued_ShouldThrowExceptionAndClearRequest() + [AllQueuesTestCases] + public async Task QueueAndWait_CancellingAPendingRequestAfterItIsDequeued_ShouldThrowExceptionAndClearRequest( + PendingRequestQueueTestCase queueTestCase) { // Arrange const string endpoint = "poll://endpoint001"; - await using var queueHolder = new PendingRequestQueueBuilder() + await using var queueHolder = queueTestCase.Builder .WithEndpoint(endpoint) .WithPollingQueueWaitTimeout(TimeSpan.Zero) // Remove delay, otherwise we wait the full 20 seconds for DequeueAsync at the end of the test .Build(); @@ -420,19 +432,22 @@ public async Task QueueAndWait_CancellingAPendingRequestAfterItIsDequeued_Should await AssertException.Throws(queueAndWaitTask); // Assert - dequeued?.RequestMessage.Should().NotBeNull().And.Be(request); + dequeued?.RequestMessage.Should().NotBeNull() + .And.BeEquivalentTo(request); var next = await sut.DequeueAsync(CancellationToken); next.Should().BeNull(); } [Test] - public async Task DequeueAsync_WithNothingQueued_WillWaitPollingQueueWaitTimeout_ShouldReturnNull() + [AllQueuesTestCases] + public async Task DequeueAsync_WithNothingQueued_WillWaitPollingQueueWaitTimeout_ShouldReturnNull( + PendingRequestQueueTestCase queueTestCase) { // Arrange const string endpoint = "poll://endpoint001"; - await using var queueHolder = new PendingRequestQueueBuilder() + await using var queueHolder = queueTestCase.Builder .WithEndpoint(endpoint) .WithPollingQueueWaitTimeout(TimeSpan.FromSeconds(1)) .Build(); @@ -449,12 +464,14 @@ public async Task DequeueAsync_WithNothingQueued_WillWaitPollingQueueWaitTimeout } [Test] - public async Task DequeueAsync_WithNothingQueued_WillWaitPollingQueueWaitTimeout_AfterPreviousRequestWasQueuedAndDequeued() + [AllQueuesTestCases] + public async Task DequeueAsync_WithNothingQueued_WillWaitPollingQueueWaitTimeout_AfterPreviousRequestWasQueuedAndDequeued( + PendingRequestQueueTestCase queueTestCase) { // Arrange const string endpoint = "poll://endpoint001"; - await using var queueHolder = new PendingRequestQueueBuilder() + await using var queueHolder = queueTestCase.Builder .WithEndpoint(endpoint) .WithPollingQueueWaitTimeout(TimeSpan.FromSeconds(1)) .Build(); @@ -465,7 +482,8 @@ public async Task DequeueAsync_WithNothingQueued_WillWaitPollingQueueWaitTimeout var expectedPreviousResponse = ResponseMessageBuilder.FromRequest(previousRequest).Build(); var queueAndWaitTask = await StartQueueAndWaitAndWaitForRequestToBeQueued(sut, previousRequest, CancellationToken); - await sut.DequeueAsync(CancellationToken); + var dequeued = await sut.DequeueAsync(CancellationToken); + dequeued.Should().NotBeNull(); await sut.ApplyResponse(expectedPreviousResponse, previousRequest.ActivityId); await queueAndWaitTask; @@ -475,17 +493,19 @@ public async Task DequeueAsync_WithNothingQueued_WillWaitPollingQueueWaitTimeout // Assert // Although we sleep for 1 second, sometimes it can be just under. So be generous with the buffer. - stopwatch.Elapsed.Should().BeGreaterThan(TimeSpan.FromMilliseconds(800)); dequeuedRequest.Should().BeNull(); + stopwatch.Elapsed.Should().BeGreaterThan(TimeSpan.FromMilliseconds(800)); + } [Test] - public async Task DequeueAsync_WillContinueWaitingUntilItemIsQueued() + [AllQueuesTestCases] + public async Task DequeueAsync_WillContinueWaitingUntilItemIsQueued(PendingRequestQueueTestCase queueTestCase) { // Arrange const string endpoint = "poll://endpoint001"; - await using var queueHolder = new PendingRequestQueueBuilder().WithEndpoint(endpoint).Build(); + await using var queueHolder = queueTestCase.Builder.WithEndpoint(endpoint).Build(); var sut = queueHolder.PendingRequestQueue; var request = new RequestMessageBuilder(endpoint).Build(); var expectedResponse = ResponseMessageBuilder.FromRequest(request).Build(); @@ -504,21 +524,23 @@ public async Task DequeueAsync_WillContinueWaitingUntilItemIsQueued() // Although we sleep for 1 second, sometimes it can be just under. So be generous with the buffer. stopwatch.Elapsed.Should().BeGreaterThan(TimeSpan.FromMilliseconds(800)); - dequeuedRequest!.RequestMessage.Should().Be(request); + dequeuedRequest!.RequestMessage.Should().BeEquivalentTo(request); // Apply a response so we can prove this counts as taking a message. await sut.ApplyResponse(expectedResponse, request.ActivityId); var response = await queueAndWaitTask; - response.Should().Be(expectedResponse); + response.Should().BeEquivalentTo(expectedResponse); } [Test] - public async Task DequeueAsync_WithMultipleDequeueCallsWaiting_WhenSingleRequestIsQueued_ThenOnlyOneCallersReceivesRequest() + [AllQueuesTestCases] + public async Task DequeueAsync_WithMultipleDequeueCallsWaiting_WhenSingleRequestIsQueued_ThenOnlyOneCallersReceivesRequest( + PendingRequestQueueTestCase queueTestCase) { // Arrange const string endpoint = "poll://endpoint001"; - await using var queueHolder = new PendingRequestQueueBuilder().WithEndpoint(endpoint).Build(); + await using var queueHolder = queueTestCase.Builder.WithEndpoint(endpoint).Build(); var sut = queueHolder.PendingRequestQueue; var request = new RequestMessageBuilder(endpoint).Build(); var expectedResponse = ResponseMessageBuilder.FromRequest(request).Build(); @@ -539,16 +561,18 @@ public async Task DequeueAsync_WithMultipleDequeueCallsWaiting_WhenSingleRequest await queueAndWaitTask; var singleDequeuedRequest = await dequeueTasks.Should().ContainSingle(t => t.Result != null).Subject; - singleDequeuedRequest!.RequestMessage.Should().Be(request); + singleDequeuedRequest!.RequestMessage.Should().BeEquivalentTo(request); } [Test] - public async Task ApplyResponse_AfterRequestHasBeenCollected_AndWaitingHasBeenCancelled_ResponseIsIgnored() + [AllQueuesTestCases] + public async Task ApplyResponse_AfterRequestHasBeenCollected_AndWaitingHasBeenCancelled_ResponseIsIgnored( + PendingRequestQueueTestCase queueTestCase) { // Arrange const string endpoint = "poll://endpoint001"; - await using var queueHolder = new PendingRequestQueueBuilder() + await using var queueHolder = queueTestCase.Builder .WithEndpoint(endpoint) .WithPollingQueueWaitTimeout(TimeSpan.Zero) // Remove delay, otherwise we wait the full 20 seconds for DequeueAsync at the end of the test .Build(); @@ -665,7 +689,7 @@ static async Task ApplyResponsesConcurrentlyAndEnsureAllQueueResponsesMatch( var response = await queueAndWaitTask; var expectedResponse = expectedResponsesInOrder[index++]; - response.Should().Be(expectedResponse); + response.Should().BeEquivalentTo(expectedResponse); } } } diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index 929758b0c..ae85959e0 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -40,6 +40,8 @@ class RedisPendingRequestQueue : IPendingRequestQueue, IDisposable // TODO: this needs to be used in all public methods. readonly CancellationToken queueToken; + int numberOfInFlightRequestsThatHaveReachedTheStageOfBeingReadyForCollection = 0; + Task PulseChannelSubDisposer { get; } public RedisPendingRequestQueue( @@ -108,14 +110,22 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can await halibutRedisTransport.PutRequest(endpoint, request.ActivityId, payload, cancellationToken); await halibutRedisTransport.PushRequestGuidOnToQueue(endpoint, request.ActivityId, cancellationToken); await halibutRedisTransport.PulseRequestPushedToEndpoint(endpoint, cancellationToken); + Interlocked.Increment(ref numberOfInFlightRequestsThatHaveReachedTheStageOfBeingReadyForCollection); + try + { - await using var watcherCts = CancellationTokenSource.CreateLinkedTokenSource(cts.Token).CancelOnDispose(); - WatchProcessingNodeIsStillConnectedInBackground(request, pending, watcherCts); + await using var watcherCts = CancellationTokenSource.CreateLinkedTokenSource(cts.Token).CancelOnDispose(); + WatchProcessingNodeIsStillConnectedInBackground(request, pending, watcherCts); - await pending.WaitUntilComplete( - async () => await tryClearRequestFromQueueAtMostOnce.Task, - () => dataLoseCt.IsCancellationRequested ? " Cancelled because data loss on redis was detected." : "", - cancellationToken); + await pending.WaitUntilComplete( + async () => await tryClearRequestFromQueueAtMostOnce.Task, + () => dataLoseCt.IsCancellationRequested ? " Cancelled because data loss on redis was detected." : "", + cancellationToken); + } + finally + { + Interlocked.Decrement(ref numberOfInFlightRequestsThatHaveReachedTheStageOfBeingReadyForCollection); + } } finally { @@ -244,8 +254,7 @@ async Task SubscribeToResponse(Guid activityId, } public bool IsEmpty => Count == 0; - public int Count => throw new NotImplementedException(); - + public int Count => numberOfInFlightRequestsThatHaveReachedTheStageOfBeingReadyForCollection; // The timespan is more generous for the sender going offline, since if it does go offline, // since under some cases the request completing is advantageous. That node needs to @@ -358,8 +367,13 @@ public async Task ApplyResponse(ResponseMessage response, Guid requestActivityId try { // TODO can we avoid going to redis here? + // TODO: Does this work well for multiple clients? We might go round before we collect work. + // TODO: test this. + hasItemsForEndpoint.Reset(); + var first = await TryRemoveNextItemFromQueue(cancellationToken); if (first != null) return first; + await Task.WhenAny( hasItemsForEndpoint.WaitAsync(cancellationTokenSource.Token), @@ -371,10 +385,7 @@ await Task.WhenAny( // to keep the connection healthy. return null; } - - // TODO: Does this work well for multiple clients? We might go round before we collect work. - // TODO: test this. - hasItemsForEndpoint.Reset(); + return await TryRemoveNextItemFromQueue(cancellationToken); } finally diff --git a/source/Halibut/ServiceModel/IPendingRequestQueue.cs b/source/Halibut/ServiceModel/IPendingRequestQueue.cs index 5d2ed930f..c1796f671 100644 --- a/source/Halibut/ServiceModel/IPendingRequestQueue.cs +++ b/source/Halibut/ServiceModel/IPendingRequestQueue.cs @@ -8,6 +8,12 @@ namespace Halibut.ServiceModel public interface IPendingRequestQueue : IAsyncDisposable { bool IsEmpty { get; } + + /// + /// For testing only, use only as an indicator that the call to QueueAndWaitAsync has made the request ready for collection. + /// The number of request that are ready for collection, + /// OR the number of requests that are still in flight but passed the point of ready for collection. + /// int Count { get; } Task ApplyResponse(ResponseMessage response, Guid requestActivityId); Task DequeueAsync(CancellationToken cancellationToken); From 7b46dc3ded893899d4d50d281fad6fdaa8bd274c Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Tue, 12 Aug 2025 13:39:22 +1000 Subject: [PATCH 035/137] Fixes a bug where subscriptions would not be unsubscribed from --- .../Halibut.Tests/ManyPollingTentacleTests.cs | 179 ++++++++++++++++++ .../Queue/Redis/HalibutRedisTransport.cs | 8 +- .../Queue/Redis/NodeHeartBeatSender.cs | 13 +- source/Halibut/Queue/Redis/RedisFacade.cs | 36 ++-- .../Queue/Redis/RedisPendingRequestQueue.cs | 1 + .../Redis/WatchForRequestCancellation.cs | 4 +- ...orRequestCancellationOrSenderDisconnect.cs | 8 +- 7 files changed, 221 insertions(+), 28 deletions(-) create mode 100644 source/Halibut.Tests/ManyPollingTentacleTests.cs diff --git a/source/Halibut.Tests/ManyPollingTentacleTests.cs b/source/Halibut.Tests/ManyPollingTentacleTests.cs new file mode 100644 index 000000000..d04ab0344 --- /dev/null +++ b/source/Halibut.Tests/ManyPollingTentacleTests.cs @@ -0,0 +1,179 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; +using System.Collections.Generic; +using System.Diagnostics; +using System.Linq; +using System.Threading; +using System.Threading.Tasks; +using Halibut.Diagnostics; +using Halibut.Logging; +using Halibut.Queue.QueuedDataStreams; +using Halibut.Queue.Redis; +using Halibut.ServiceModel; +using Halibut.Tests.Support; +using Halibut.Tests.Support.Logging; +using Halibut.Tests.Support.TestAttributes; +using Halibut.Tests.TestServices; +using Halibut.Tests.TestServices.Async; +using Halibut.TestUtils.Contracts; +using NUnit.Framework; +using DisposableCollection = Halibut.Util.DisposableCollection; + +namespace Halibut.Tests +{ + public class ManyPollingTentacleTests : BaseTest + { + [Test] + [AllQueuesTestCases] + [NonParallelizable] + public async Task ManyPollingTentacles(PendingRequestQueueTestCase queueTestCase) + { + var services = GetDelegateServiceFactory(); + await using var disposables = new DisposableCollection(); + var log = new TestContextLogCreator("Redis", LogLevel.Fatal); + await using var redisFacade = new RedisFacade("localhost:6379", Guid.NewGuid().ToString(), log.CreateNewForPrefix("")); + await using (var octopus = new HalibutRuntimeBuilder() + .WithServerCertificate(Certificates.Octopus) + .WithPendingRequestQueueFactory(msgSer => + { + if (queueTestCase.ToString().ToLower().Contains("redis")) + { + var watchForRedisLosingAllItsData = new WatchForRedisLosingAllItsData(redisFacade, log.CreateNewForPrefix("watcher")); + disposables.AddAsyncDisposable(watchForRedisLosingAllItsData); + + return new RedisPendingRequestQueueFactory(msgSer, + new InMemoryStoreDataStreamsForDistributedQueues(), + watchForRedisLosingAllItsData, + new HalibutRedisTransport(redisFacade), + new HalibutTimeoutsAndLimitsForTestsBuilder().Build(), + new LogFactory()); + } + + return new PendingRequestQueueFactoryAsync(new HalibutTimeoutsAndLimitsForTestsBuilder().Build(), + new LogFactory()); + }) + .WithHalibutTimeoutsAndLimits(new HalibutTimeoutsAndLimitsForTestsBuilder().Build()) + .Build()) + { + var listenPort = octopus.Listen(); + octopus.Trust(Certificates.TentacleListening.Thumbprint); + + var _ = Task.Run(async () => + { + while (!CancellationToken.IsCancellationRequested) + { + GC.Collect(); + Logger.Information("Total subscribers: {TotalSubs}", redisFacade.TotalSubscribers); + await Task.Delay(10000); + } + }); + + var serviceEndpoint = new ServiceEndPoint(new Uri("https://localhost:" + listenPort), Certificates.Octopus.Thumbprint, new HalibutTimeoutsAndLimitsForTestsBuilder().Build()); + + var pollEndpoints = Enumerable.Range(0, 10000).Select(i => new Uri("poll://" + i + "Bob")).ToArray(); + + foreach (var pollEndpoint in pollEndpoints) + { + var tentacleListening = new HalibutRuntimeBuilder() + .WithServerCertificate(Certificates.TentacleListening) + .WithServiceFactory(services) + .WithHalibutTimeoutsAndLimits(new HalibutTimeoutsAndLimitsForTestsBuilder().Build()) + .Build(); + tentacleListening.Poll(pollEndpoint, serviceEndpoint, CancellationToken); + } + + var clients = pollEndpoints.Select(pollEndpoint => + octopus.CreateAsyncClient(new ServiceEndPoint(pollEndpoint, Certificates.Octopus.Thumbprint, new HalibutTimeoutsAndLimitsForTestsBuilder().Build()))) + .ToList(); + + var tasks = new List(); + + int concurrency = 0; + int limit = 50; + int total = concurrency * Math.Min(clients.Count, limit); + int callsMade = 0; + + var totalSw = Stopwatch.StartNew(); + for (int i = 0; i < concurrency; i++) + { + tasks.Add(Task.Run(async () => + { + var shuffle = clients.ToArray(); + Random.Shared.Shuffle(shuffle); + shuffle = shuffle.Take(limit).ToArray(); + foreach (var client in shuffle) + { + await client.SayHelloAsync("World"); + var v = Interlocked.Increment(ref callsMade); + if (v % 5000 == 0) + { + var timePerCall = totalSw.ElapsedMilliseconds / v; + Logger.Information("Done: {CallsMade} / {Total} avg: {A}", v, total, timePerCall); + } + + } + })); + } + + + + await Task.WhenAll(tasks); + + totalSw.Stop(); + + Logger.Information("Time was {T}", totalSw.ElapsedMilliseconds); + + await Task.Delay(10000000); + } + } + + static DelegateServiceFactory GetDelegateServiceFactory() + { + var services = new DelegateServiceFactory(); + services.Register(() => new AsyncEchoServiceWithDelay()); + return services; + } + } + + public class AsyncEchoServiceWithDelay : IAsyncEchoService + { + + public async Task LongRunningOperationAsync(CancellationToken cancellationToken) + { + await Task.Delay(10000, cancellationToken); + return 16; + } + + public async Task SayHelloAsync(string name, CancellationToken cancellationToken) + { + + await Task.Delay(0); + return name + "..."; + } + + public async Task CrashAsync(CancellationToken cancellationToken) + { + await Task.CompletedTask; + throw new DivideByZeroException(); + } + + public async Task CountBytesAsync(DataStream dataStream, CancellationToken cancellationToken) + { + await Task.CompletedTask; + throw new Exception(); + } + } +} \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/HalibutRedisTransport.cs b/source/Halibut/Queue/Redis/HalibutRedisTransport.cs index cff74aa0b..57e776556 100644 --- a/source/Halibut/Queue/Redis/HalibutRedisTransport.cs +++ b/source/Halibut/Queue/Redis/HalibutRedisTransport.cs @@ -163,9 +163,9 @@ public async Task IsRequestMarkedAsCancelled(Uri endpoint, Guid requestId, // Node Processing the request heart beat channel - static string NodeProcessingTheRequestHeartBeatChannel(Uri endpoint, Guid requestId, HalibutQueueNodeSendingPulses nodeSendingPulsesType) + static string NodeHeartBeatChannel(Uri endpoint, Guid requestId, HalibutQueueNodeSendingPulses nodeSendingPulsesType) { - return $"{Namespace}::NodeProcessingTheRequestHeartBeatChannel::{endpoint}::{requestId}::{nodeSendingPulsesType}"; + return $"{Namespace}::NodeHeartBeatChannel::{endpoint}::{requestId}::{nodeSendingPulsesType}"; } public async Task SubscribeToNodeHeartBeatChannel( @@ -175,7 +175,7 @@ public async Task SubscribeToNodeHeartBeatChannel( Func onHeartBeat, CancellationToken cancellationToken) { - var channelName = NodeProcessingTheRequestHeartBeatChannel(endpoint, request, nodeSendingPulsesType); + var channelName = NodeHeartBeatChannel(endpoint, request, nodeSendingPulsesType); return await facade.SubscribeToChannel(channelName, async foo => { string? response = foo.Message; @@ -185,7 +185,7 @@ public async Task SubscribeToNodeHeartBeatChannel( public async Task SendHeartBeatFromNodeProcessingTheRequest(Uri endpoint, Guid requestId, HalibutQueueNodeSendingPulses nodeSendingPulsesType, CancellationToken cancellationToken) { - var channelName = NodeProcessingTheRequestHeartBeatChannel(endpoint, requestId, nodeSendingPulsesType); + var channelName = NodeHeartBeatChannel(endpoint, requestId, nodeSendingPulsesType); await facade.PublishToChannel(channelName, "{}"); } diff --git a/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs b/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs index ce2222d1d..85c1df4c6 100644 --- a/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs +++ b/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs @@ -40,7 +40,7 @@ public enum NodeProcessingRequestWatcherResult private readonly Uri endpoint; private readonly Guid requestActivityId; private readonly HalibutRedisTransport halibutRedisTransport; - private readonly CancellationTokenSource cancellationTokenSource; + private readonly CancelOnDisposeCancellationTokenSource cancellationTokenSource; private readonly ILog log; private readonly HalibutQueueNodeSendingPulses nodeSendingPulsesType; @@ -57,10 +57,10 @@ public NodeHeartBeatSender( this.requestActivityId = requestActivityId; this.halibutRedisTransport = halibutRedisTransport; this.nodeSendingPulsesType = nodeSendingPulsesType; - cancellationTokenSource = new CancellationTokenSource(); + cancellationTokenSource = new CancellationTokenSource().CancelOnDispose(); this.log = log; log.Write(EventType.Diagnostic, "Starting NodeHeartBeatSender for {0} node, request {1}, endpoint {2}", nodeSendingPulsesType, requestActivityId, endpoint); - TaskSendingPulses = Task.Run(() => SendPulsesWhileProcessingRequest(defaultDelayBetweenPulses, cancellationTokenSource.Token)); + TaskSendingPulses = Task.Run(() => SendPulsesWhileProcessingRequest(defaultDelayBetweenPulses, cancellationTokenSource.CancellationToken)); } private async Task SendPulsesWhileProcessingRequest(TimeSpan defaultDelayBetweenPulses, CancellationToken cancellationToken) @@ -106,7 +106,6 @@ public static async Task WatchThatNodeProces { // Once the pending's CT has been cancelled we no longer care to keep observing using var cts = CancellationTokenSource.CreateLinkedTokenSource(watchCancellationToken, pending.PendingRequestCancellationToken); - var cancellationToken = cts.Token; // TODO: test this is indeed called first. try { @@ -153,7 +152,8 @@ private static async Task WatchForPulsesFrom await using var subscription = await halibutRedisTransport.SubscribeToNodeHeartBeatChannel( endpoint, requestActivityId, - watchingForPulsesFrom, async () => + watchingForPulsesFrom, + async () => { await Task.CompletedTask; lastHeartBeat = DateTimeOffset.Now; @@ -250,8 +250,7 @@ public async ValueTask DisposeAsync() { log.Write(EventType.Diagnostic, "Disposing NodeHeartBeatSender for {0} node, request {1}", nodeSendingPulsesType, requestActivityId); - await Try.IgnoringError(async () => await cancellationTokenSource.CancelAsync()); - Try.IgnoringError(() => cancellationTokenSource.Dispose()); + await Try.IgnoringError(async () => await cancellationTokenSource.DisposeAsync()); log.Write(EventType.Diagnostic, "NodeHeartBeatSender disposed for {0} node, request {1}", nodeSendingPulsesType, requestActivityId); } diff --git a/source/Halibut/Queue/Redis/RedisFacade.cs b/source/Halibut/Queue/Redis/RedisFacade.cs index eb4bb2fe6..edaa66221 100644 --- a/source/Halibut/Queue/Redis/RedisFacade.cs +++ b/source/Halibut/Queue/Redis/RedisFacade.cs @@ -18,10 +18,7 @@ using System.Threading.Tasks; using Halibut.Util; using Halibut.Diagnostics; // Add logging support -using Newtonsoft.Json; -using Nito.AsyncEx; using StackExchange.Redis; -using StackExchange.Redis.KeyspaceIsolation; namespace Halibut.Queue.Redis { @@ -224,8 +221,9 @@ public async ValueTask DisposeAsync() await Try.IgnoringError(async () => await conn.DisposeAsync()); } } - - + + + internal int TotalSubscribers = 0; public async Task SubscribeToChannel(string channelName, Func onMessage, CancellationToken cancellationToken) { @@ -239,12 +237,28 @@ public async Task SubscribeToChannel(string channelName, Func< // This can throw if we are unable to connect to redis. var channel = await Connection.GetSubscriber() .SubscribeAsync(new RedisChannel(channelName, RedisChannel.PatternMode.Literal)); - - // Once we are connected to redis, it seems even if the connection to redis dies. - // The client will take care of re-connecting to redis. - channel.OnMessage(onMessage); - - return new FuncAsyncDisposable(async () => await channel.UnsubscribeAsync()); + + var disposable = new FuncAsyncDisposable(async () => + { + Interlocked.Decrement(ref TotalSubscribers); + await channel.UnsubscribeAsync(); + }); + + Interlocked.Increment(ref TotalSubscribers); + try + { + // Once we are connected to redis, it seems even if the connection to redis dies. + // The client will take care of re-connecting to redis. + channel.OnMessage(onMessage); + } + catch (Exception) + { + await disposable.DisposeAsync(); + throw; + } + + return disposable; + } catch { diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index ae85959e0..6055a76f8 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -293,6 +293,7 @@ async Task SubscribeToResponse(Guid activityId, disposables.AddAsyncDisposable(new NodeHeartBeatSender(endpoint, pending.ActivityId, halibutRedisTransport, log, HalibutQueueNodeSendingPulses.Receiver, DelayBetweenHeartBeatsForRequestProcessor)); var watcher = new WatchForRequestCancellationOrSenderDisconnect(endpoint, pending.ActivityId, halibutRedisTransport, NodeIsOfflineHeartBeatTimeoutForRequestSender, log); + disposables.AddAsyncDisposable(watcher); var cts = CancellationTokenSource.CreateLinkedTokenSource(watcher.RequestProcessingCancellationToken, dataLossCT); disposables.Add(cts); diff --git a/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs b/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs index 92c921760..c692e15dd 100644 --- a/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs +++ b/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs @@ -72,10 +72,10 @@ public WatchForRequestCancellation(Uri endpoint, Guid requestActivityId, Halibut log.Write(EventType.Diagnostic, "Starting to watch for request cancellation - Endpoint: {0}, ActivityId: {1}", endpoint, requestActivityId); var token = watchForCancellationTokenSource.Token; - var _ = Task.Run(async () => await WatchForExceptions(endpoint, requestActivityId, halibutRedisTransport, token)); + var _ = Task.Run(async () => await WatchForCancellation(endpoint, requestActivityId, halibutRedisTransport, token)); } - async Task WatchForExceptions(Uri endpoint, Guid requestActivityId, HalibutRedisTransport halibutRedisTransport, CancellationToken token) + async Task WatchForCancellation(Uri endpoint, Guid requestActivityId, HalibutRedisTransport halibutRedisTransport, CancellationToken token) { try { diff --git a/source/Halibut/Queue/Redis/WatchForRequestCancellationOrSenderDisconnect.cs b/source/Halibut/Queue/Redis/WatchForRequestCancellationOrSenderDisconnect.cs index 6d9a3820a..d348dee68 100644 --- a/source/Halibut/Queue/Redis/WatchForRequestCancellationOrSenderDisconnect.cs +++ b/source/Halibut/Queue/Redis/WatchForRequestCancellationOrSenderDisconnect.cs @@ -28,7 +28,7 @@ public class WatchForRequestCancellationOrSenderDisconnect : IAsyncDisposable public CancellationToken RequestProcessingCancellationToken { get; } readonly WatchForRequestCancellation watchForRequestCancellation; - readonly CancellationTokenSource KeepWatchingCancellationTokenSource; + readonly CancelOnDisposeCancellationTokenSource KeepWatchingCancellationTokenSource; DisposableCollection disposableCollection = new DisposableCollection(); @@ -48,8 +48,8 @@ public WatchForRequestCancellationOrSenderDisconnect( disposableCollection.Add(RequestCancellationTokenSource); RequestProcessingCancellationToken = RequestCancellationTokenSource.Token; - KeepWatchingCancellationTokenSource = new CancellationTokenSource(); - disposableCollection.Add(KeepWatchingCancellationTokenSource); + KeepWatchingCancellationTokenSource = new CancellationTokenSource().CancelOnDispose(); + disposableCollection.AddAsyncDisposable(KeepWatchingCancellationTokenSource); Task.Run(() => WatchThatNodeWhichSentTheRequestIsStillAlive(endpoint, requestActivityId, halibutRedisTransport, nodeOfflineTimeoutBetweenHeartBeatsFromSender, log)); } @@ -63,7 +63,7 @@ public WatchForRequestCancellationOrSenderDisconnect( async Task WatchThatNodeWhichSentTheRequestIsStillAlive(Uri endpoint, Guid requestActivityId, HalibutRedisTransport halibutRedisTransport, TimeSpan nodeOfflineTimeoutBetweenHeartBeatsFromSender, ILog log) { - var watchCancellationToken = KeepWatchingCancellationTokenSource.Token; + var watchCancellationToken = KeepWatchingCancellationTokenSource.CancellationToken; try { var res = await NodeHeartBeatSender From b73eead2942f1e28059fed6f91e00483ac5bd39e Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Tue, 12 Aug 2025 14:09:37 +1000 Subject: [PATCH 036/137] ignore some exceptions --- .../Queue/Redis/PollAndSubscribeForSingleMessage.cs | 7 +++++-- source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs | 3 +++ source/Halibut/Queue/Redis/WatchForRequestCancellation.cs | 7 +++++-- 3 files changed, 13 insertions(+), 4 deletions(-) diff --git a/source/Halibut/Queue/Redis/PollAndSubscribeForSingleMessage.cs b/source/Halibut/Queue/Redis/PollAndSubscribeForSingleMessage.cs index 87626eca1..b2b97c885 100644 --- a/source/Halibut/Queue/Redis/PollAndSubscribeForSingleMessage.cs +++ b/source/Halibut/Queue/Redis/PollAndSubscribeForSingleMessage.cs @@ -160,14 +160,17 @@ async Task WatchAndWaitForMessage(CancellationToken token) pollBackoffStrategy.Try(); var delay = pollBackoffStrategy.GetSleepPeriod(); log.Write(EventType.Diagnostic, "Waiting {0} seconds before next poll for {1} - Endpoint: {2}, ActivityId: {3}", delay.TotalSeconds, messageTypeName, endpoint, activityId); - await Task.Delay(delay, token); + await Try.IgnoringError(async () => await Task.Delay(delay, token)); } log.Write(EventType.Diagnostic, "Exiting watch loop for {0} - Endpoint: {1}, ActivityId: {2}", messageTypeName, endpoint, activityId); } catch (Exception ex) { - log.Write(EventType.Error, "Unexpected error in {0} watcher - Endpoint: {1}, ActivityId: {2}, Error: {3}", messageTypeName, endpoint, activityId, ex.Message); + if (!token.IsCancellationRequested) + { + log.Write(EventType.Error, "Unexpected error in {0} watcher - Endpoint: {1}, ActivityId: {2}, Error: {3}", messageTypeName, endpoint, activityId, ex.Message); + } } } diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index 6055a76f8..04e7391be 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -117,6 +117,9 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can await using var watcherCts = CancellationTokenSource.CreateLinkedTokenSource(cts.Token).CancelOnDispose(); WatchProcessingNodeIsStillConnectedInBackground(request, pending, watcherCts); + // TODO: We need to ensure that no matter what exceptions are thrown we eventually exit. + // For example can the subscription to the response, fail and never come back? + // Can the WatchProcessProcessingNodeIsStillConnected fail and never come back? await pending.WaitUntilComplete( async () => await tryClearRequestFromQueueAtMostOnce.Task, () => dataLoseCt.IsCancellationRequested ? " Cancelled because data loss on redis was detected." : "", diff --git a/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs b/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs index c692e15dd..262146b22 100644 --- a/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs +++ b/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs @@ -112,14 +112,17 @@ async Task WatchForCancellation(Uri endpoint, Guid requestActivityId, HalibutRed { log.Write(EventType.Diagnostic, "Error while polling for request cancellation - Endpoint: {0}, ActivityId: {1}, Error: {2}", endpoint, requestActivityId, ex.Message); } - await Task.Delay(TimeSpan.FromSeconds(60), token); + await Try.IgnoringError(async () => await Task.Delay(TimeSpan.FromSeconds(60), token)); } log.Write(EventType.Diagnostic, "Exiting watch loop for request cancellation - Endpoint: {0}, ActivityId: {1}", endpoint, requestActivityId); } catch (Exception ex) { - log.Write(EventType.Error, "Unexpected error in request cancellation watcher - Endpoint: {0}, ActivityId: {1}, Error: {2}", endpoint, requestActivityId, ex.Message); + if (!token.IsCancellationRequested) + { + log.Write(EventType.Error, "Unexpected error in request cancellation watcher - Endpoint: {0}, ActivityId: {1}, Error: {2}", endpoint, requestActivityId, ex.Message); + } } } From 7ec738fb9694ca49077fab23fd82929486f65b12 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Wed, 13 Aug 2025 07:04:25 +1000 Subject: [PATCH 037/137] Fix big in detecting if redis lost its data --- .../Halibut.Tests/ManyPollingTentacleTests.cs | 30 ++++++++++++++----- .../WatchForRedisLosingAllItsDataFixture.cs | 13 +++++++- source/Halibut/Queue/Redis/RedisFacade.cs | 20 ++++++++++--- .../Redis/WatchForRedisLosingAllItsData.cs | 8 +++-- 4 files changed, 56 insertions(+), 15 deletions(-) diff --git a/source/Halibut.Tests/ManyPollingTentacleTests.cs b/source/Halibut.Tests/ManyPollingTentacleTests.cs index d04ab0344..1d476eabd 100644 --- a/source/Halibut.Tests/ManyPollingTentacleTests.cs +++ b/source/Halibut.Tests/ManyPollingTentacleTests.cs @@ -18,6 +18,8 @@ using System.Linq; using System.Threading; using System.Threading.Tasks; +using Docker.DotNet.Models; +using FluentAssertions; using Halibut.Diagnostics; using Halibut.Logging; using Halibut.Queue.QueuedDataStreams; @@ -34,12 +36,13 @@ namespace Halibut.Tests { + [NonParallelizable] public class ManyPollingTentacleTests : BaseTest { [Test] [AllQueuesTestCases] [NonParallelizable] - public async Task ManyPollingTentacles(PendingRequestQueueTestCase queueTestCase) + public async Task ManyRequestToPollingTentacles_Works_AndDoesNotUseTooManyResources(PendingRequestQueueTestCase queueTestCase) { var services = GetDelegateServiceFactory(); await using var disposables = new DisposableCollection(); @@ -83,7 +86,8 @@ public async Task ManyPollingTentacles(PendingRequestQueueTestCase queueTestCase var serviceEndpoint = new ServiceEndPoint(new Uri("https://localhost:" + listenPort), Certificates.Octopus.Thumbprint, new HalibutTimeoutsAndLimitsForTestsBuilder().Build()); - var pollEndpoints = Enumerable.Range(0, 10000).Select(i => new Uri("poll://" + i + "Bob")).ToArray(); + + var pollEndpoints = Enumerable.Range(0, 100).Select(i => new Uri("poll://" + i + "Bob")).ToArray(); foreach (var pollEndpoint in pollEndpoints) { @@ -101,8 +105,8 @@ public async Task ManyPollingTentacles(PendingRequestQueueTestCase queueTestCase var tasks = new List(); - int concurrency = 0; - int limit = 50; + int concurrency = 20; + int limit = 20; int total = concurrency * Math.Min(clients.Count, limit); int callsMade = 0; @@ -128,15 +132,27 @@ public async Task ManyPollingTentacles(PendingRequestQueueTestCase queueTestCase })); } - - await Task.WhenAll(tasks); totalSw.Stop(); Logger.Information("Time was {T}", totalSw.ElapsedMilliseconds); + + callsMade.Should().Be(total); - await Task.Delay(10000000); + + for (int i = 0; i < 200; i++) + { + if (redisFacade.TotalSubscribers == pollEndpoints.Length) break; + await Task.Delay(100); + } + + redisFacade.TotalSubscribers.Should().Be(pollEndpoints.Length); + + foreach (var task in tasks) + { + await task; + } } } diff --git a/source/Halibut.Tests/Queue/Redis/WatchForRedisLosingAllItsDataFixture.cs b/source/Halibut.Tests/Queue/Redis/WatchForRedisLosingAllItsDataFixture.cs index bde58690d..ba0af11b6 100644 --- a/source/Halibut.Tests/Queue/Redis/WatchForRedisLosingAllItsDataFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/WatchForRedisLosingAllItsDataFixture.cs @@ -59,7 +59,18 @@ public async Task WatchForARealRedisLosingAllOfItsData_TimesOutWhenWaitingForCTW } [Test] - public async Task WatchForARealRedisLosingAllOfItsData_E2E_Test() + public async Task WhenRedisRunsForLongerThanTheKeyTTL_NoDataLoseShouldBeDetected() + { + await using var redisFacade = CreateRedisFacade(); + await using var watcher = new WatchForRedisLosingAllItsData(redisFacade, HalibutLog, watchInterval:TimeSpan.FromMilliseconds(100), keyTTL: TimeSpan.FromSeconds(2)); + var watcherCt = await watcher.GetTokenForDataLoseDetection(TimeSpan.FromSeconds(20), CancellationToken); + + await Task.Delay(TimeSpan.FromSeconds(4)); + watcherCt.IsCancellationRequested.Should().BeFalse(); + } + + [Test] + public async Task WhenRedisLosesAllOfIts_TheWatcherShouldDetectTheDataLose() { Logger.Information("Starting WatchForARealRedisLosingAllOfItsData_E2E_Test"); diff --git a/source/Halibut/Queue/Redis/RedisFacade.cs b/source/Halibut/Queue/Redis/RedisFacade.cs index edaa66221..43caa8bed 100644 --- a/source/Halibut/Queue/Redis/RedisFacade.cs +++ b/source/Halibut/Queue/Redis/RedisFacade.cs @@ -366,17 +366,29 @@ await ExecuteWithRetry(async () => public async Task SetString(string key, string value, TimeSpan ttl, CancellationToken cancellationToken) { // TODO TTL - key = "string:" + keyPrefix + ":" + key; + key = ToStringKey(key); await ExecuteWithRetry(async () => { var database = Connection.GetDatabase(); await database.StringSetAsync(key, value); }, cancellationToken); - await SetTtlForKey(key, ttl, cancellationToken); + // TODO unit test. + await SetTtlForKeyRaw(key, ttl, cancellationToken); } - public async Task SetTtlForKey(string key, TimeSpan ttl, CancellationToken cancellationToken) + string ToStringKey(string key) + { + return "string:" + keyPrefix + ":" + key; + } + + public async Task SetTtlForString(string key, TimeSpan ttl, CancellationToken cancellationToken) + { + await SetTtlForKeyRaw(ToStringKey(key), ttl, cancellationToken); + + } + + async Task SetTtlForKeyRaw(string key, TimeSpan ttl, CancellationToken cancellationToken) { await ExecuteWithRetry(async () => { @@ -387,7 +399,7 @@ await ExecuteWithRetry(async () => public async Task GetString(string key, CancellationToken cancellationToken) { - key = "string:" + keyPrefix + ":" + key; + key = ToStringKey(key); return await ExecuteWithRetry(async () => { var database = Connection.GetDatabase(); diff --git a/source/Halibut/Queue/Redis/WatchForRedisLosingAllItsData.cs b/source/Halibut/Queue/Redis/WatchForRedisLosingAllItsData.cs index fe534311a..2a7bdc9f6 100644 --- a/source/Halibut/Queue/Redis/WatchForRedisLosingAllItsData.cs +++ b/source/Halibut/Queue/Redis/WatchForRedisLosingAllItsData.cs @@ -96,7 +96,7 @@ async Task WatchForDataLose(CancellationToken cancellationToken) if (!hasSetKey) { log.Write(EventType.Diagnostic, "Setting initial data loss monitoring key {0} with TTL {1} minutes", key, KeyTTL.TotalMinutes); - await redisFacade.SetString(key, guid.ToString(), KeyTTL, cancellationToken); + await redisFacade.SetString(key, guid, KeyTTL, cancellationToken); taskCompletionSource.TrySetResult(cts.CancellationToken); hasSetKey = true; log.Write(EventType.Diagnostic, "Successfully set initial data loss monitoring key {0}, monitoring is now active", key); @@ -104,14 +104,16 @@ async Task WatchForDataLose(CancellationToken cancellationToken) else { var data = await redisFacade.GetString(key, cancellationToken); - if (data != guid.ToString()) + if (data != guid) { - log.Write(EventType.Error, "Redis data loss detected! Expected value {0} for key {1}, but got {2}. This indicates Redis has lost data.", guid.ToString(), key, data ?? "null"); + log.Write(EventType.Error, "Redis data loss detected! Expected value {0} for key {1}, but got {2}. This indicates Redis has lost data.", guid, key, data ?? "null"); // Anyone new will be given a new thing to wait on. taskCompletionSource = new TaskCompletionSource(); await Try.IgnoringError(async () => await cts.CancellationTokenSource.CancelAsync()); return; } + + await redisFacade.SetTtlForString(key, KeyTTL, cancellationToken); } } catch (Exception ex) From 2de5e7334c4c9e2c85bf3f8f3515a8d43914d287 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Wed, 13 Aug 2025 11:47:37 +1000 Subject: [PATCH 038/137] Better logging, dispose extra queues created --- .../Support/TestContextConnectionLog.cs | 5 +++++ .../Halibut.Tests/ManyPollingTentacleTests.cs | 18 ++++++++++++--- .../Support/Logging/InMemoryLogWriter.cs | 5 +++++ .../Logging/TestContextConnectionLog.cs | 5 +++++ source/Halibut/Diagnostics/ILog.cs | 2 ++ .../Diagnostics/InMemoryConnectionLog.cs | 2 ++ .../AggregateLogWriterLogCreator.cs | 4 ++-- .../LogWriters/AggregateLogWriter.cs | 10 +++++++-- source/Halibut/HalibutRuntime.cs | 9 +++++++- .../Queue/Redis/NodeHeartBeatSender.cs | 7 ++++-- ...ssage.cs => PollAndSubscribeToResponse.cs} | 22 ++++++++++++------- source/Halibut/Queue/Redis/RedisFacade.cs | 7 ++++++ .../Queue/Redis/RedisPendingRequestQueue.cs | 9 ++++---- .../Halibut/ServiceModel/PendingRequestV2.cs | 2 +- 14 files changed, 84 insertions(+), 23 deletions(-) rename source/Halibut/Queue/Redis/{PollAndSubscribeForSingleMessage.cs => PollAndSubscribeToResponse.cs} (85%) diff --git a/source/Halibut.Tests.DotMemory/Support/TestContextConnectionLog.cs b/source/Halibut.Tests.DotMemory/Support/TestContextConnectionLog.cs index 8db4d103e..8304a1d18 100644 --- a/source/Halibut.Tests.DotMemory/Support/TestContextConnectionLog.cs +++ b/source/Halibut.Tests.DotMemory/Support/TestContextConnectionLog.cs @@ -36,6 +36,11 @@ public IList GetLogs() throw new NotImplementedException(); } + public ILog ForContext() + { + return this; + } + void WriteInternal(LogEvent logEvent) { var logEventLogLevel = GetLogLevel(logEvent); diff --git a/source/Halibut.Tests/ManyPollingTentacleTests.cs b/source/Halibut.Tests/ManyPollingTentacleTests.cs index 1d476eabd..f69edc9fc 100644 --- a/source/Halibut.Tests/ManyPollingTentacleTests.cs +++ b/source/Halibut.Tests/ManyPollingTentacleTests.cs @@ -44,6 +44,7 @@ public class ManyPollingTentacleTests : BaseTest [NonParallelizable] public async Task ManyRequestToPollingTentacles_Works_AndDoesNotUseTooManyResources(PendingRequestQueueTestCase queueTestCase) { + var logFactory = new CachingLogFactory(new TestContextLogCreator("", LogLevel.Trace)); var services = GetDelegateServiceFactory(); await using var disposables = new DisposableCollection(); var log = new TestContextLogCreator("Redis", LogLevel.Fatal); @@ -62,11 +63,11 @@ public async Task ManyRequestToPollingTentacles_Works_AndDoesNotUseTooManyResour watchForRedisLosingAllItsData, new HalibutRedisTransport(redisFacade), new HalibutTimeoutsAndLimitsForTestsBuilder().Build(), - new LogFactory()); + logFactory); } return new PendingRequestQueueFactoryAsync(new HalibutTimeoutsAndLimitsForTestsBuilder().Build(), - new LogFactory()); + logFactory); }) .WithHalibutTimeoutsAndLimits(new HalibutTimeoutsAndLimitsForTestsBuilder().Build()) .Build()) @@ -141,11 +142,22 @@ public async Task ManyRequestToPollingTentacles_Works_AndDoesNotUseTooManyResour callsMade.Should().Be(total); - for (int i = 0; i < 200; i++) + for (int i = 0; i < 20; i++) { if (redisFacade.TotalSubscribers == pollEndpoints.Length) break; await Task.Delay(100); } + + var s = ""; + foreach (var subsKey in redisFacade.subs.Keys) + { + if(subsKey.Contains("RequestMessagesPulseChannelName")) continue; + s += "\n" + subsKey; + } + + Logger.Information("F {F}", s); + + Logger.Information("Created queues: {Count}", RedisPendingRequestQueue.CountCreated); redisFacade.TotalSubscribers.Should().Be(pollEndpoints.Length); diff --git a/source/Halibut.Tests/Support/Logging/InMemoryLogWriter.cs b/source/Halibut.Tests/Support/Logging/InMemoryLogWriter.cs index b37fe2f0c..dd0af97a8 100644 --- a/source/Halibut.Tests/Support/Logging/InMemoryLogWriter.cs +++ b/source/Halibut.Tests/Support/Logging/InMemoryLogWriter.cs @@ -30,5 +30,10 @@ public IList GetLogs() { return events.ToArray(); } + + public ILog ForContext() + { + return this; + } } } \ No newline at end of file diff --git a/source/Halibut.Tests/Support/Logging/TestContextConnectionLog.cs b/source/Halibut.Tests/Support/Logging/TestContextConnectionLog.cs index 2f2b5233e..1d5b58d1a 100644 --- a/source/Halibut.Tests/Support/Logging/TestContextConnectionLog.cs +++ b/source/Halibut.Tests/Support/Logging/TestContextConnectionLog.cs @@ -38,6 +38,11 @@ public IList GetLogs() throw new NotImplementedException(); } + public ILog ForContext() + { + return new TestContextConnectionLog(endpoint, typeof(T).Name, logLevel); + } + void WriteInternal(LogEvent logEvent) { var logEventLogLevel = GetLogLevel(logEvent); diff --git a/source/Halibut/Diagnostics/ILog.cs b/source/Halibut/Diagnostics/ILog.cs index 2c907aaf3..29c31ea1a 100644 --- a/source/Halibut/Diagnostics/ILog.cs +++ b/source/Halibut/Diagnostics/ILog.cs @@ -7,5 +7,7 @@ namespace Halibut.Diagnostics public interface ILog : ILogWriter { IList GetLogs(); + + ILog ForContext(); } } \ No newline at end of file diff --git a/source/Halibut/Diagnostics/InMemoryConnectionLog.cs b/source/Halibut/Diagnostics/InMemoryConnectionLog.cs index af1ca6bd5..94e18246e 100644 --- a/source/Halibut/Diagnostics/InMemoryConnectionLog.cs +++ b/source/Halibut/Diagnostics/InMemoryConnectionLog.cs @@ -39,6 +39,8 @@ public IList GetLogs() return events.ToArray(); } + public ILog ForContext() => this; + void WriteInternal(LogEvent logEvent) { var logLevel = GetLogLevel(logEvent); diff --git a/source/Halibut/Diagnostics/LogCreators/AggregateLogWriterLogCreator.cs b/source/Halibut/Diagnostics/LogCreators/AggregateLogWriterLogCreator.cs index 4cdb24575..a3effecab 100644 --- a/source/Halibut/Diagnostics/LogCreators/AggregateLogWriterLogCreator.cs +++ b/source/Halibut/Diagnostics/LogCreators/AggregateLogWriterLogCreator.cs @@ -6,9 +6,9 @@ namespace Halibut.Diagnostics.LogCreators public class AggregateLogWriterLogCreator : ICreateNewILog { readonly ICreateNewILog logCreator; - readonly Func logWriterFactoryForPrefix; + readonly Func logWriterFactoryForPrefix; - public AggregateLogWriterLogCreator(ICreateNewILog logCreator, Func logWriterFactoryForPrefix) + public AggregateLogWriterLogCreator(ICreateNewILog logCreator, Func logWriterFactoryForPrefix) { this.logCreator = logCreator; this.logWriterFactoryForPrefix = logWriterFactoryForPrefix; diff --git a/source/Halibut/Diagnostics/LogWriters/AggregateLogWriter.cs b/source/Halibut/Diagnostics/LogWriters/AggregateLogWriter.cs index 38da98d4b..4d7484b0a 100644 --- a/source/Halibut/Diagnostics/LogWriters/AggregateLogWriter.cs +++ b/source/Halibut/Diagnostics/LogWriters/AggregateLogWriter.cs @@ -1,5 +1,6 @@ using System; using System.Collections.Generic; +using System.Linq; namespace Halibut.Diagnostics.LogWriters { @@ -9,9 +10,9 @@ namespace Halibut.Diagnostics.LogWriters public class AggregateLogWriter : ILog { readonly ILog log; - readonly ILogWriter[] logWriter; + readonly ILog[] logWriter; - public AggregateLogWriter(ILog log, ILogWriter[] logWriter) + public AggregateLogWriter(ILog log, ILog[] logWriter) { this.log = log; this.logWriter = logWriter; @@ -39,5 +40,10 @@ public IList GetLogs() { return log.GetLogs(); } + + public ILog ForContext() + { + return new AggregateLogWriter(log.ForContext(), logWriter.Select(lw => lw.ForContext()).ToArray()); + } } } \ No newline at end of file diff --git a/source/Halibut/HalibutRuntime.cs b/source/Halibut/HalibutRuntime.cs index effb8abc9..12b98eadb 100644 --- a/source/Halibut/HalibutRuntime.cs +++ b/source/Halibut/HalibutRuntime.cs @@ -87,7 +87,14 @@ internal HalibutRuntime( IPendingRequestQueue GetQueue(Uri target) { - return queues.GetOrAdd(target, u => queueFactory.CreateQueue(target)); + IPendingRequestQueue? createdQueue = null; + var queue = queues.GetOrAdd(target, u => createdQueue = queueFactory.CreateQueue(target)); + if (createdQueue != null && !ReferenceEquals(createdQueue, queue)) + { + createdQueue.DisposeAsync(); + } + + return queue; } public int Listen() diff --git a/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs b/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs index 85c1df4c6..7407a4924 100644 --- a/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs +++ b/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs @@ -58,8 +58,8 @@ public NodeHeartBeatSender( this.halibutRedisTransport = halibutRedisTransport; this.nodeSendingPulsesType = nodeSendingPulsesType; cancellationTokenSource = new CancellationTokenSource().CancelOnDispose(); - this.log = log; - log.Write(EventType.Diagnostic, "Starting NodeHeartBeatSender for {0} node, request {1}, endpoint {2}", nodeSendingPulsesType, requestActivityId, endpoint); + this.log = log.ForContext(); + this.log.Write(EventType.Diagnostic, "Starting NodeHeartBeatSender for {0} node, request {1}, endpoint {2}", nodeSendingPulsesType, requestActivityId, endpoint); TaskSendingPulses = Task.Run(() => SendPulsesWhileProcessingRequest(defaultDelayBetweenPulses, cancellationTokenSource.CancellationToken)); } @@ -104,6 +104,7 @@ public static async Task WatchThatNodeProces TimeSpan maxTimeBetweenHeartBeetsBeforeProcessingNodeIsAssumedToBeOffline, CancellationToken watchCancellationToken) { + log = log.ForContext(); // Once the pending's CT has been cancelled we no longer care to keep observing using var cts = CancellationTokenSource.CreateLinkedTokenSource(watchCancellationToken, pending.PendingRequestCancellationToken); // TODO: test this is indeed called first. @@ -143,6 +144,7 @@ private static async Task WatchForPulsesFrom HalibutQueueNodeSendingPulses watchingForPulsesFrom, CancellationToken watchCancellationToken) { + log.ForContext(); log.Write(EventType.Diagnostic, "Starting to watch for pulses from {0} node, request {1}, endpoint {2}", watchingForPulsesFrom, requestActivityId, endpoint); DateTimeOffset? lastHeartBeat = DateTimeOffset.Now; @@ -192,6 +194,7 @@ static async Task WaitForRequestToBeCollected(Uri endpoint, RequestMessage reque TimeSpan timeBetweenCheckingIfRequestWasCollected, ILog log, CancellationToken cancellationToken) { + log = log.ForContext(); log.Write(EventType.Diagnostic, "Waiting for request {0} to be collected from queue", request.ActivityId); // Is this worthwhile? diff --git a/source/Halibut/Queue/Redis/PollAndSubscribeForSingleMessage.cs b/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs similarity index 85% rename from source/Halibut/Queue/Redis/PollAndSubscribeForSingleMessage.cs rename to source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs index b2b97c885..b79e89a1d 100644 --- a/source/Halibut/Queue/Redis/PollAndSubscribeForSingleMessage.cs +++ b/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs @@ -21,10 +21,10 @@ namespace Halibut.Queue.Redis { - public class PollAndSubscribeForSingleMessage : IAsyncDisposable + public class PollAndSubscribeToResponse : IAsyncDisposable { public static async Task TrySendMessage( - string messageTypeName, + string messageTypeName, // TODO drop this, it is always response. HalibutRedisTransport halibutRedisTransport, Uri endpoint, Guid activityId, @@ -70,9 +70,9 @@ public static async Task TrySendMessage( public Task ResultTask => message.Task; - public PollAndSubscribeForSingleMessage(string messageTypeName, Uri endpoint, Guid activityId, HalibutRedisTransport halibutRedisTransport, ILog log) + public PollAndSubscribeToResponse(string messageTypeName, Uri endpoint, Guid activityId, HalibutRedisTransport halibutRedisTransport, ILog log) { - this.log = log; + this.log = log.ForContext(); this.messageTypeName = messageTypeName; this.endpoint = endpoint; this.activityId = activityId; @@ -82,7 +82,7 @@ public PollAndSubscribeForSingleMessage(string messageTypeName, Uri endpoint, Gu TimeSpan.FromSeconds(15), // Increment: 15s TimeSpan.FromMinutes(2) // Maximum delay: 2 minutes ); - log.Write(EventType.Diagnostic, "Starting to watch for {0} - Endpoint: {1}, ActivityId: {2}", messageTypeName, endpoint, activityId); + this.log.Write(EventType.Diagnostic, "Starting to watch for {0} - Endpoint: {1}, ActivityId: {2}", messageTypeName, endpoint, activityId); var token = watcherTokenSource.Token; var _ = Task.Run(async () => await WatchAndWaitForMessage(token)); @@ -130,7 +130,12 @@ async Task WatchAndWaitForMessage(CancellationToken token) await TrySetResultAndRemoveValueFromRedis(value, token); } - await watcherTokenSource.CancelAsync(); + log.Write(EventType.Diagnostic, "Cancelling polling loop for {0} - Endpoint: {1}, ActivityId: {2}", messageTypeName, endpoint, activityId); + + await Try.IgnoringError(async () => await watcherTokenSource.CancelAsync()); + + log.Write(EventType.Diagnostic, "Is token cancelled: {0} - Endpoint: {1}, ActivityId: {2} {3}", messageTypeName, endpoint, activityId, token.IsCancellationRequested); + }, token); @@ -161,6 +166,7 @@ async Task WatchAndWaitForMessage(CancellationToken token) var delay = pollBackoffStrategy.GetSleepPeriod(); log.Write(EventType.Diagnostic, "Waiting {0} seconds before next poll for {1} - Endpoint: {2}, ActivityId: {3}", delay.TotalSeconds, messageTypeName, endpoint, activityId); await Try.IgnoringError(async () => await Task.Delay(delay, token)); + log.Write(EventType.Diagnostic, "Done waiting going around the loop {0} - Endpoint: {1}, ActivityId: {2}", messageTypeName, endpoint, activityId); } log.Write(EventType.Diagnostic, "Exiting watch loop for {0} - Endpoint: {1}, ActivityId: {2}", messageTypeName, endpoint, activityId); @@ -176,7 +182,7 @@ async Task WatchAndWaitForMessage(CancellationToken token) public async ValueTask DisposeAsync() { - log.Write(EventType.Diagnostic, "Disposing GenericWatcher for {0}", messageTypeName); + log.Write(EventType.Diagnostic, "Disposing GenericWatcher for {0} - Endpoint: {1}, ActivityId: {2}", messageTypeName, endpoint, activityId); await Try.IgnoringError(async () => await watcherTokenSource.CancelAsync()); Try.IgnoringError(() => watcherTokenSource.Dispose()); @@ -184,7 +190,7 @@ public async ValueTask DisposeAsync() // If the message task is not yet complete, then complete if now with null since we have nothing for it. Try.IgnoringError(() => message.TrySetCanceled()); - log.Write(EventType.Diagnostic, "GenericWatcher for {0} disposed", messageTypeName); + log.Write(EventType.Diagnostic, "Disposed GenericWatcher for {0} - Endpoint: {1}, ActivityId: {2}", messageTypeName, endpoint, activityId); } } } \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/RedisFacade.cs b/source/Halibut/Queue/Redis/RedisFacade.cs index 43caa8bed..356b372fb 100644 --- a/source/Halibut/Queue/Redis/RedisFacade.cs +++ b/source/Halibut/Queue/Redis/RedisFacade.cs @@ -13,6 +13,7 @@ // limitations under the License. using System; +using System.Collections.Concurrent; using System.Diagnostics; using System.Threading; using System.Threading.Tasks; @@ -225,6 +226,8 @@ public async ValueTask DisposeAsync() internal int TotalSubscribers = 0; + internal ConcurrentDictionary subs = new ConcurrentDictionary(); + public async Task SubscribeToChannel(string channelName, Func onMessage, CancellationToken cancellationToken) { @@ -238,9 +241,13 @@ public async Task SubscribeToChannel(string channelName, Func< var channel = await Connection.GetSubscriber() .SubscribeAsync(new RedisChannel(channelName, RedisChannel.PatternMode.Literal)); + var someGuid = channelName + "__" + Guid.NewGuid().ToString(); + subs.TryAdd(someGuid, false); + var disposable = new FuncAsyncDisposable(async () => { Interlocked.Decrement(ref TotalSubscribers); + subs.TryRemove(someGuid, out var _); await channel.UnsubscribeAsync(); }); diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index 04e7391be..e267ffb06 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -54,7 +54,7 @@ public RedisPendingRequestQueue( { this.endpoint = endpoint; this.watchForRedisLosingAllItsData = watchForRedisLosingAllItsData; - this.log = log; + this.log = log.ForContext(); this.messageReaderWriter = messageReaderWriter; this.halibutRedisTransport = halibutRedisTransport; this.halibutTimeoutsAndLimits = halibutTimeoutsAndLimits; @@ -79,6 +79,7 @@ public async ValueTask DisposeAsync() private async Task DataLossCancellationToken(CancellationToken? cancellationToken) { + // TODO must throw something that can be retried. using var cts = CancellationTokenSource.CreateLinkedTokenSource(queueCts.Token, cancellationToken ?? CancellationToken.None); return await watchForRedisLosingAllItsData.GetTokenForDataLoseDetection(TimeSpan.FromSeconds(30), cts.Token); } @@ -91,8 +92,8 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can using var cts = CancellationTokenSource.CreateLinkedTokenSource(queueCts.Token, requestCancellationToken, dataLoseCt); var cancellationToken = cts.Token; - // TODO: redis goes down // TODO RedisConnectionException can be raised out of here, what should the queue do? + // TODO it must raise an exception that supports being retried. using var pending = new PendingRequest(request, log); // TODO: What if this payload was gigantic @@ -232,7 +233,7 @@ async Task SubscribeToResponse(Guid activityId, CancellationToken cancellationToken) { await Task.CompletedTask; - var sub = new PollAndSubscribeForSingleMessage(ResponseMessageSubscriptionName, endpoint, activityId, halibutRedisTransport, log); + var sub = new PollAndSubscribeToResponse(ResponseMessageSubscriptionName, endpoint, activityId, halibutRedisTransport, log); var _ = Task.Run(async () => { try @@ -332,7 +333,7 @@ public async Task ApplyResponse(ResponseMessage response, Guid requestActivityId var payload = await messageReaderWriter.PrepareResponse(response, cancellationToken); log.Write(EventType.MessageExchange, "Sending response message for request {0}", requestActivityId); - await PollAndSubscribeForSingleMessage.TrySendMessage(ResponseMessageSubscriptionName, halibutRedisTransport, endpoint, requestActivityId, payload, TTLOfResponseMessage, log); + await PollAndSubscribeToResponse.TrySendMessage(ResponseMessageSubscriptionName, halibutRedisTransport, endpoint, requestActivityId, payload, TTLOfResponseMessage, log); log.Write(EventType.MessageExchange, "Successfully applied response for request {0}", requestActivityId); } catch (Exception ex) diff --git a/source/Halibut/ServiceModel/PendingRequestV2.cs b/source/Halibut/ServiceModel/PendingRequestV2.cs index 01e15a3ec..239175124 100644 --- a/source/Halibut/ServiceModel/PendingRequestV2.cs +++ b/source/Halibut/ServiceModel/PendingRequestV2.cs @@ -38,7 +38,7 @@ public class PendingRequest : IDisposable public PendingRequest(RequestMessage request, ILog log) { this.request = request; - this.log = log; + this.log = log.ForContext(); pendingRequestCancellationTokenSource = new CancellationTokenSource(); PendingRequestCancellationToken = pendingRequestCancellationTokenSource.Token; From a85a5df1b0e3e30735fb726e6b0421e41dd828d9 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Wed, 13 Aug 2025 13:30:47 +1000 Subject: [PATCH 039/137] Never dispose CTS --- .../Halibut.Tests/ManyPollingTentacleTests.cs | 18 ++++-- .../Redis/RedisPendingRequestQueueFixture.cs | 8 +-- ...leDataLossWatchForRedisLosingAllItsData.cs | 12 ++-- .../Queue/Redis/NodeHeartBeatSender.cs | 12 ++-- .../Queue/Redis/PollAndSubscribeToResponse.cs | 19 +++--- source/Halibut/Queue/Redis/RedisFacade.cs | 11 ++-- .../Queue/Redis/RedisPendingRequestQueue.cs | 37 ++++++----- .../Redis/WatchForRedisLosingAllItsData.cs | 18 +++--- .../Redis/WatchForRequestCancellation.cs | 12 ++-- ...orRequestCancellationOrSenderDisconnect.cs | 24 ++++---- .../Util/CancelOnDisposeCancellationToken.cs | 61 +++++++++++++++++++ ...CancellationTokenSourceExtensionMethods.cs | 55 ----------------- 12 files changed, 147 insertions(+), 140 deletions(-) create mode 100644 source/Halibut/Util/CancelOnDisposeCancellationToken.cs delete mode 100644 source/Halibut/Util/CancellationTokenSourceExtensionMethods.cs diff --git a/source/Halibut.Tests/ManyPollingTentacleTests.cs b/source/Halibut.Tests/ManyPollingTentacleTests.cs index f69edc9fc..e87af566f 100644 --- a/source/Halibut.Tests/ManyPollingTentacleTests.cs +++ b/source/Halibut.Tests/ManyPollingTentacleTests.cs @@ -47,13 +47,14 @@ public async Task ManyRequestToPollingTentacles_Works_AndDoesNotUseTooManyResour var logFactory = new CachingLogFactory(new TestContextLogCreator("", LogLevel.Trace)); var services = GetDelegateServiceFactory(); await using var disposables = new DisposableCollection(); + var isRedis = queueTestCase.ToString().ToLower().Contains("redis"); var log = new TestContextLogCreator("Redis", LogLevel.Fatal); await using var redisFacade = new RedisFacade("localhost:6379", Guid.NewGuid().ToString(), log.CreateNewForPrefix("")); await using (var octopus = new HalibutRuntimeBuilder() .WithServerCertificate(Certificates.Octopus) .WithPendingRequestQueueFactory(msgSer => { - if (queueTestCase.ToString().ToLower().Contains("redis")) + if (isRedis) { var watchForRedisLosingAllItsData = new WatchForRedisLosingAllItsData(redisFacade, log.CreateNewForPrefix("watcher")); disposables.AddAsyncDisposable(watchForRedisLosingAllItsData); @@ -156,10 +157,17 @@ public async Task ManyRequestToPollingTentacles_Works_AndDoesNotUseTooManyResour } Logger.Information("F {F}", s); - - Logger.Information("Created queues: {Count}", RedisPendingRequestQueue.CountCreated); - - redisFacade.TotalSubscribers.Should().Be(pollEndpoints.Length); + + if(isRedis) + { + if (redisFacade.TotalSubscribers != pollEndpoints.Length) + { + Logger.Information("OH NO!"); + + await Task.Delay(TimeSpan.FromMinutes(1)); + } + redisFacade.TotalSubscribers.Should().Be(pollEndpoints.Length); + } foreach (var task in tasks) { diff --git a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs index 68ae6106f..ce380c64f 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs @@ -298,13 +298,13 @@ public async Task OnceARequestIsComplete_NoSenderHeartBeatsShouldBeSent() // Assert bool heartBeatSent = false; - var cts = new CancellationTokenSource().CancelOnDispose(); + var cts = new CancelOnDisposeCancellationToken(); using var _ = redisTransport.SubscribeToNodeHeartBeatChannel(endpoint, request.ActivityId, HalibutQueueNodeSendingPulses.Sender, async () => { await Task.CompletedTask; heartBeatSent = true; }, - cts.CancellationToken); + cts.Token); await Task.Delay(5000); heartBeatSent.Should().BeFalse(); @@ -341,13 +341,13 @@ public async Task OnceARequestIsComplete_NoReceiverHeartBeatsShouldBeSent() // Assert bool heartBeatSent = false; - var cts = new CancellationTokenSource().CancelOnDispose(); + var cts = new CancelOnDisposeCancellationToken(); using var _ = redisTransport.SubscribeToNodeHeartBeatChannel(endpoint, request.ActivityId, HalibutQueueNodeSendingPulses.Receiver, async () => { await Task.CompletedTask; heartBeatSent = true; }, - cts.CancellationToken); + cts.Token); await Task.Delay(5000); heartBeatSent.Should().BeFalse(); diff --git a/source/Halibut.Tests/Queue/Redis/Utils/CancellableDataLossWatchForRedisLosingAllItsData.cs b/source/Halibut.Tests/Queue/Redis/Utils/CancellableDataLossWatchForRedisLosingAllItsData.cs index e7c7e6d23..45abbd372 100644 --- a/source/Halibut.Tests/Queue/Redis/Utils/CancellableDataLossWatchForRedisLosingAllItsData.cs +++ b/source/Halibut.Tests/Queue/Redis/Utils/CancellableDataLossWatchForRedisLosingAllItsData.cs @@ -23,25 +23,25 @@ namespace Halibut.Tests.Queue.Redis.Utils { public class CancellableDataLossWatchForRedisLosingAllItsData : IWatchForRedisLosingAllItsData { - CancelOnDisposeCancellationTokenSource cancellationTokenSource = new CancellationTokenSource().CancelOnDispose(); + CancelOnDisposeCancellationToken cancellationToken = new(); public TaskCompletionSource TaskCompletionSource = new(); public CancellableDataLossWatchForRedisLosingAllItsData() { - TaskCompletionSource.SetResult(cancellationTokenSource.CancellationToken); + TaskCompletionSource.SetResult(cancellationToken.Token); } public async Task DataLossHasOccured() { - await cancellationTokenSource.DisposeAsync(); - cancellationTokenSource = new CancellationTokenSource().CancelOnDispose(); + await cancellationToken.DisposeAsync(); + cancellationToken = new CancelOnDisposeCancellationToken(); TaskCompletionSource = new TaskCompletionSource(); - TaskCompletionSource.SetResult(cancellationTokenSource.CancellationToken); + TaskCompletionSource.SetResult(cancellationToken.Token); } public async ValueTask DisposeAsync() { - await Try.CatchingError(async () => await cancellationTokenSource.DisposeAsync()); + await Try.CatchingError(async () => await cancellationToken.DisposeAsync()); } public async Task GetTokenForDataLoseDetection(TimeSpan timeToWait, CancellationToken cancellationToken) diff --git a/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs b/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs index 7407a4924..8d951ba8e 100644 --- a/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs +++ b/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs @@ -40,7 +40,7 @@ public enum NodeProcessingRequestWatcherResult private readonly Uri endpoint; private readonly Guid requestActivityId; private readonly HalibutRedisTransport halibutRedisTransport; - private readonly CancelOnDisposeCancellationTokenSource cancellationTokenSource; + private readonly CancelOnDisposeCancellationToken cancellationToken; private readonly ILog log; private readonly HalibutQueueNodeSendingPulses nodeSendingPulsesType; @@ -57,10 +57,10 @@ public NodeHeartBeatSender( this.requestActivityId = requestActivityId; this.halibutRedisTransport = halibutRedisTransport; this.nodeSendingPulsesType = nodeSendingPulsesType; - cancellationTokenSource = new CancellationTokenSource().CancelOnDispose(); + cancellationToken = new CancelOnDisposeCancellationToken(); this.log = log.ForContext(); this.log.Write(EventType.Diagnostic, "Starting NodeHeartBeatSender for {0} node, request {1}, endpoint {2}", nodeSendingPulsesType, requestActivityId, endpoint); - TaskSendingPulses = Task.Run(() => SendPulsesWhileProcessingRequest(defaultDelayBetweenPulses, cancellationTokenSource.CancellationToken)); + TaskSendingPulses = Task.Run(() => SendPulsesWhileProcessingRequest(defaultDelayBetweenPulses, cancellationToken.Token)); } private async Task SendPulsesWhileProcessingRequest(TimeSpan defaultDelayBetweenPulses, CancellationToken cancellationToken) @@ -106,10 +106,10 @@ public static async Task WatchThatNodeProces { log = log.ForContext(); // Once the pending's CT has been cancelled we no longer care to keep observing - using var cts = CancellationTokenSource.CreateLinkedTokenSource(watchCancellationToken, pending.PendingRequestCancellationToken); - // TODO: test this is indeed called first. + await using var cts = new CancelOnDisposeCancellationToken(watchCancellationToken, pending.PendingRequestCancellationToken); try { + // TODO: test this is indeed called first. await WaitForRequestToBeCollected(endpoint, request, pending, halibutRedisTransport, timeBetweenCheckingIfRequestWasCollected, log, cts.Token); return await WatchForPulsesFromNode(endpoint, request.ActivityId, halibutRedisTransport, log, maxTimeBetweenHeartBeetsBeforeProcessingNodeIsAssumedToBeOffline, HalibutQueueNodeSendingPulses.Receiver, cts.Token); @@ -253,7 +253,7 @@ public async ValueTask DisposeAsync() { log.Write(EventType.Diagnostic, "Disposing NodeHeartBeatSender for {0} node, request {1}", nodeSendingPulsesType, requestActivityId); - await Try.IgnoringError(async () => await cancellationTokenSource.DisposeAsync()); + await Try.IgnoringError(async () => await cancellationToken.DisposeAsync()); log.Write(EventType.Diagnostic, "NodeHeartBeatSender disposed for {0} node, request {1}", nodeSendingPulsesType, requestActivityId); } diff --git a/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs b/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs index b79e89a1d..1b0304656 100644 --- a/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs +++ b/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs @@ -34,7 +34,7 @@ public static async Task TrySendMessage( { log.Write(EventType.Diagnostic, "Attempting to set {0} for - Endpoint: {1}, ActivityId: {2}", messageTypeName, endpoint, activityId); - using var cts = new CancellationTokenSource(); + await using var cts = new CancelOnDisposeCancellationToken(); cts.CancelAfter(TimeSpan.FromMinutes(2)); // Best efforts. try @@ -57,7 +57,7 @@ public static async Task TrySendMessage( } } - readonly CancellationTokenSource watcherTokenSource = new(); + readonly CancelOnDisposeCancellationToken watcherToken; readonly ILog log; readonly string messageTypeName; @@ -83,8 +83,9 @@ public PollAndSubscribeToResponse(string messageTypeName, Uri endpoint, Guid act TimeSpan.FromMinutes(2) // Maximum delay: 2 minutes ); this.log.Write(EventType.Diagnostic, "Starting to watch for {0} - Endpoint: {1}, ActivityId: {2}", messageTypeName, endpoint, activityId); - - var token = watcherTokenSource.Token; + + watcherToken = new CancelOnDisposeCancellationToken(); + var token = watcherToken.Token; var _ = Task.Run(async () => await WatchAndWaitForMessage(token)); } @@ -132,7 +133,7 @@ async Task WatchAndWaitForMessage(CancellationToken token) log.Write(EventType.Diagnostic, "Cancelling polling loop for {0} - Endpoint: {1}, ActivityId: {2}", messageTypeName, endpoint, activityId); - await Try.IgnoringError(async () => await watcherTokenSource.CancelAsync()); + await Try.IgnoringError(async () => await watcherToken.CancelAsync()); log.Write(EventType.Diagnostic, "Is token cancelled: {0} - Endpoint: {1}, ActivityId: {2} {3}", messageTypeName, endpoint, activityId, token.IsCancellationRequested); @@ -153,7 +154,7 @@ async Task WatchAndWaitForMessage(CancellationToken token) log.Write(EventType.Diagnostic, "{0} detected via polling - Endpoint: {1}, ActivityId: {2}", messageTypeName, endpoint, activityId); pollBackoffStrategy.Success(); // Reset backoff strategy on successful retrieval await TrySetResultAndRemoveValueFromRedis(value, token); - await watcherTokenSource.CancelAsync(); + await watcherToken.CancelAsync(); break; } } @@ -184,11 +185,11 @@ public async ValueTask DisposeAsync() { log.Write(EventType.Diagnostic, "Disposing GenericWatcher for {0} - Endpoint: {1}, ActivityId: {2}", messageTypeName, endpoint, activityId); - await Try.IgnoringError(async () => await watcherTokenSource.CancelAsync()); - Try.IgnoringError(() => watcherTokenSource.Dispose()); + await Try.IgnoringError(async () => await watcherToken.CancelAsync()); + //Try.IgnoringError(() => watcherTokenSource.Dispose()); // If the message task is not yet complete, then complete if now with null since we have nothing for it. - Try.IgnoringError(() => message.TrySetCanceled()); + Try.IgnoringError(() => message.TrySetCanceled()); // TODO: try set cancelled? you sure? log.Write(EventType.Diagnostic, "Disposed GenericWatcher for {0} - Endpoint: {1}, ActivityId: {2}", messageTypeName, endpoint, activityId); } diff --git a/source/Halibut/Queue/Redis/RedisFacade.cs b/source/Halibut/Queue/Redis/RedisFacade.cs index 356b372fb..cd40837ab 100644 --- a/source/Halibut/Queue/Redis/RedisFacade.cs +++ b/source/Halibut/Queue/Redis/RedisFacade.cs @@ -64,7 +64,7 @@ public class RedisFacade : IAsyncDisposable string keyPrefix; - CancellationTokenSource cts; + CancelOnDisposeCancellationToken cts; CancellationToken facadeCancellationToken; public RedisFacade(string configuration, string? keyPrefix, ILog log) : this(ConfigurationOptions.Parse(configuration), keyPrefix, log) @@ -75,7 +75,7 @@ public RedisFacade(ConfigurationOptions redisOptions, string? keyPrefix, ILog lo { this.keyPrefix = keyPrefix ?? "halibut"; this.log = log; - this.cts = new CancellationTokenSource(); + this.cts = new CancelOnDisposeCancellationToken(); this.facadeCancellationToken = cts.Token; // aka have more goes at connecting. @@ -150,7 +150,7 @@ private void OnConnectionRestored(object? sender, ConnectionFailedEventArgs e) /// private async Task ExecuteWithRetry(Func> operation, CancellationToken cancellationToken) { - using var linkedTokenSource = CancellationTokenSource.CreateLinkedTokenSource(cancellationToken, facadeCancellationToken); + await using var linkedTokenSource = new CancelOnDisposeCancellationToken(cancellationToken, facadeCancellationToken); var combinedToken = linkedTokenSource.Token; var retryDelay = TimeSpan.FromSeconds(1); @@ -177,7 +177,7 @@ private async Task ExecuteWithRetry(Func> operation, CancellationT /// private async Task ExecuteWithRetry(Func operation, CancellationToken cancellationToken) { - using var linkedTokenSource = CancellationTokenSource.CreateLinkedTokenSource(cancellationToken, facadeCancellationToken); + await using var linkedTokenSource = new CancelOnDisposeCancellationToken(cancellationToken, facadeCancellationToken); var combinedToken = linkedTokenSource.Token; var retryDelay = TimeSpan.FromSeconds(1); @@ -204,8 +204,7 @@ private async Task ExecuteWithRetry(Func operation, CancellationToken canc public async ValueTask DisposeAsync() { - await Try.IgnoringError(async () => await cts.CancelAsync()); - Try.IgnoringError(() => cts.Dispose()); + await Try.IgnoringError(async () => await cts.DisposeAsync()); if (connection.IsValueCreated) { diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index e267ffb06..1654f369f 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -34,7 +34,7 @@ class RedisPendingRequestQueue : IPendingRequestQueue, IDisposable readonly MessageReaderWriter messageReaderWriter; readonly AsyncManualResetEvent hasItemsForEndpoint = new(); - readonly CancellationTokenSource queueCts = new (); + readonly CancelOnDisposeCancellationToken queueCts = new (); internal ConcurrentDictionary disposablesForInFlightRequests = new(); // TODO: this needs to be used in all public methods. @@ -72,15 +72,14 @@ public RedisPendingRequestQueue( public async ValueTask DisposeAsync() { - await Try.IgnoringError(async () => await queueCts.CancelAsync()); - Try.IgnoringError(() => queueCts.Dispose()); + await Try.IgnoringError(async () => await queueCts.DisposeAsync()); await Try.IgnoringError(async () => await (await PulseChannelSubDisposer).DisposeAsync()); } private async Task DataLossCancellationToken(CancellationToken? cancellationToken) { - // TODO must throw something that can be retried. - using var cts = CancellationTokenSource.CreateLinkedTokenSource(queueCts.Token, cancellationToken ?? CancellationToken.None); + // TODO this must throw something that can be retried. + await using var cts = new CancelOnDisposeCancellationToken(queueCts.Token, cancellationToken ?? CancellationToken.None); return await watchForRedisLosingAllItsData.GetTokenForDataLoseDetection(TimeSpan.FromSeconds(30), cts.Token); } @@ -89,7 +88,7 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can var dataLoseCt = await DataLossCancellationToken(requestCancellationToken); - using var cts = CancellationTokenSource.CreateLinkedTokenSource(queueCts.Token, requestCancellationToken, dataLoseCt); + await using var cts = new CancelOnDisposeCancellationToken(queueCts.Token, requestCancellationToken, dataLoseCt); var cancellationToken = cts.Token; // TODO RedisConnectionException can be raised out of here, what should the queue do? @@ -115,7 +114,7 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can try { - await using var watcherCts = CancellationTokenSource.CreateLinkedTokenSource(cts.Token).CancelOnDispose(); + await using var watcherCts = new CancelOnDisposeCancellationToken(cts.Token); WatchProcessingNodeIsStillConnectedInBackground(request, pending, watcherCts); // TODO: We need to ensure that no matter what exceptions are thrown we eventually exit. @@ -154,11 +153,11 @@ async Task SendCancellationIfRequestWasCancelled(RequestMessage request, Pending } } - void WatchProcessingNodeIsStillConnectedInBackground(RequestMessage request, PendingRequest pending, CancelOnDisposeCancellationTokenSource watcherCts) + void WatchProcessingNodeIsStillConnectedInBackground(RequestMessage request, PendingRequest pending, CancelOnDisposeCancellationToken watcherCts) { Task.Run(async () => { - var watcherCtsCancellationToken = watcherCts.CancellationToken; + var watcherCtsCancellationToken = watcherCts.Token; try { var disconnected = await NodeHeartBeatSender.WatchThatNodeProcessingTheRequestIsStillAlive( @@ -203,7 +202,7 @@ async Task TryClearRequestFromQueue(RequestMessage request, PendingRequest // TODO: log return false; } - using var cts = new CancellationTokenSource(); + await using var cts = new CancelOnDisposeCancellationToken(); cts.CancelAfter(TimeSpan.FromMinutes(2)); // Best efforts. var requestJson = await halibutRedisTransport.TryGetAndRemoveRequest(endpoint, request.ActivityId, cts.Token); if (requestJson != null) @@ -299,8 +298,8 @@ async Task SubscribeToResponse(Guid activityId, var watcher = new WatchForRequestCancellationOrSenderDisconnect(endpoint, pending.ActivityId, halibutRedisTransport, NodeIsOfflineHeartBeatTimeoutForRequestSender, log); disposables.AddAsyncDisposable(watcher); - var cts = CancellationTokenSource.CreateLinkedTokenSource(watcher.RequestProcessingCancellationToken, dataLossCT); - disposables.Add(cts); + var cts = new CancelOnDisposeCancellationToken(watcher.RequestProcessingCancellationToken, dataLossCT); + disposables.AddAsyncDisposable(cts); var response = new RequestMessageWithCancellationToken(pending, cts.Token); disposablesForInFlightRequests[pending.ActivityId] = disposables; @@ -365,10 +364,8 @@ public async Task ApplyResponse(ResponseMessage response, Guid requestActivityId async Task DequeueNextAsync() { - // TODO use queue token. - var cancellationToken = CancellationToken.None; - using var cancellationTokenSource = CancellationTokenSource.CreateLinkedTokenSource(cancellationToken); + await using var cts = new CancelOnDisposeCancellationToken(queueToken); try { // TODO can we avoid going to redis here? @@ -376,13 +373,13 @@ public async Task ApplyResponse(ResponseMessage response, Guid requestActivityId // TODO: test this. hasItemsForEndpoint.Reset(); - var first = await TryRemoveNextItemFromQueue(cancellationToken); + var first = await TryRemoveNextItemFromQueue(cts.Token); if (first != null) return first; await Task.WhenAny( - hasItemsForEndpoint.WaitAsync(cancellationTokenSource.Token), - Task.Delay(halibutTimeoutsAndLimits.PollingQueueWaitTimeout, cancellationTokenSource.Token)); + hasItemsForEndpoint.WaitAsync(cts.Token), + Task.Delay(halibutTimeoutsAndLimits.PollingQueueWaitTimeout, cts.Token)); if (!hasItemsForEndpoint.IsSet) { @@ -391,11 +388,11 @@ await Task.WhenAny( return null; } - return await TryRemoveNextItemFromQueue(cancellationToken); + return await TryRemoveNextItemFromQueue(cts.Token); } finally { - await cancellationTokenSource.CancelAsync(); + await cts.CancelAsync(); } } diff --git a/source/Halibut/Queue/Redis/WatchForRedisLosingAllItsData.cs b/source/Halibut/Queue/Redis/WatchForRedisLosingAllItsData.cs index 2a7bdc9f6..5c48e4398 100644 --- a/source/Halibut/Queue/Redis/WatchForRedisLosingAllItsData.cs +++ b/source/Halibut/Queue/Redis/WatchForRedisLosingAllItsData.cs @@ -39,7 +39,7 @@ public class WatchForRedisLosingAllItsData : IWatchForRedisLosingAllItsData internal TimeSpan SetupDelay { get;} internal TimeSpan WatchInterval { get; } internal TimeSpan KeyTTL { get; } - CancelOnDisposeCancellationTokenSource cancelOnDisposeCancellationTokenSource = new CancellationTokenSource().CancelOnDispose(); + CancelOnDisposeCancellationToken cts = new(); public WatchForRedisLosingAllItsData(RedisFacade redisFacade, ILog log, TimeSpan? setupDelay = null, TimeSpan? watchInterval = null, TimeSpan? keyTTL = null) { @@ -48,7 +48,7 @@ public WatchForRedisLosingAllItsData(RedisFacade redisFacade, ILog log, TimeSpan this.SetupDelay = setupDelay ?? TimeSpan.FromSeconds(1); this.WatchInterval = watchInterval ?? TimeSpan.FromSeconds(60); this.KeyTTL = keyTTL ?? TimeSpan.FromMinutes(60); - var _ = Task.Run(async () => await KeepWatchingForDataLose(cancelOnDisposeCancellationTokenSource.CancellationToken)); + var _ = Task.Run(async () => await KeepWatchingForDataLose(cts.Token)); } private TaskCompletionSource taskCompletionSource = new TaskCompletionSource(); @@ -67,9 +67,9 @@ public async Task GetTokenForDataLoseDetection(TimeSpan timeT } // TODO: Check if tentacle needs this to be classified as exception that can be retried. - await using var cts = CancellationTokenSource.CreateLinkedTokenSource(cancellationToken).CancelOnDispose(); - cts.CancellationTokenSource.CancelAfter(timeToWait); - return await taskCompletionSource.Task.WaitAsync(cts.CancellationToken); + await using var cts = new CancelOnDisposeCancellationToken(cancellationToken); + cts.CancelAfter(timeToWait); + return await taskCompletionSource.Task.WaitAsync(cts.Token); } private async Task KeepWatchingForDataLose(CancellationToken cancellationToken) @@ -88,7 +88,7 @@ async Task WatchForDataLose(CancellationToken cancellationToken) log.Write(EventType.Diagnostic, "Starting Redis data loss monitoring with key {0}", key); - await using var cts = new CancellationTokenSource().CancelOnDispose(); + await using var cts = new CancelOnDisposeCancellationToken(); while (!cancellationToken.IsCancellationRequested) { try @@ -97,7 +97,7 @@ async Task WatchForDataLose(CancellationToken cancellationToken) { log.Write(EventType.Diagnostic, "Setting initial data loss monitoring key {0} with TTL {1} minutes", key, KeyTTL.TotalMinutes); await redisFacade.SetString(key, guid, KeyTTL, cancellationToken); - taskCompletionSource.TrySetResult(cts.CancellationToken); + taskCompletionSource.TrySetResult(cts.Token); hasSetKey = true; log.Write(EventType.Diagnostic, "Successfully set initial data loss monitoring key {0}, monitoring is now active", key); } @@ -109,7 +109,7 @@ async Task WatchForDataLose(CancellationToken cancellationToken) log.Write(EventType.Error, "Redis data loss detected! Expected value {0} for key {1}, but got {2}. This indicates Redis has lost data.", guid, key, data ?? "null"); // Anyone new will be given a new thing to wait on. taskCompletionSource = new TaskCompletionSource(); - await Try.IgnoringError(async () => await cts.CancellationTokenSource.CancelAsync()); + await Try.IgnoringError(async () => await cts.CancelAsync()); return; } @@ -136,7 +136,7 @@ await Try.IgnoringError(async () => public async ValueTask DisposeAsync() { log.Write(EventType.Diagnostic, "Disposing WatchForRedisLosingAllItsData"); - await cancelOnDisposeCancellationTokenSource.DisposeAsync(); + await cts.DisposeAsync(); } } } \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs b/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs index 262146b22..8c634a475 100644 --- a/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs +++ b/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs @@ -31,7 +31,7 @@ public static async Task TrySendCancellation( { log.Write(EventType.Diagnostic, "Attempting to send cancellation for request - Endpoint: {0}, ActivityId: {1}", endpoint, request.ActivityId); - using var cts = new CancellationTokenSource(); + await using var cts = new CancelOnDisposeCancellationToken(); cts.CancelAfter(TimeSpan.FromMinutes(2)); // Best efforts. try @@ -59,10 +59,10 @@ public static async Task TrySendCancellation( // causing the request-processor to cancel the request anyway. static TimeSpan CancelRequestMarkerTTL = TimeSpan.FromMinutes(5); - readonly CancellationTokenSource requestCancelledCts = new(); + readonly CancelOnDisposeCancellationToken requestCancelledCts = new(); public CancellationToken RequestCancelledCancellationToken => requestCancelledCts.Token; - readonly CancellationTokenSource watchForCancellationTokenSource = new(); + readonly CancelOnDisposeCancellationToken watchForCancellationTokenSource = new(); readonly ILog log; @@ -130,10 +130,8 @@ public async ValueTask DisposeAsync() { log.Write(EventType.Diagnostic, "Disposing WatchForRequestCancellation"); - await Try.IgnoringError(async () => await watchForCancellationTokenSource.CancelAsync()); - Try.IgnoringError(() => watchForCancellationTokenSource.Dispose()); - await Try.IgnoringError(async () => await requestCancelledCts.CancelAsync()); - Try.IgnoringError(() => requestCancelledCts.Dispose()); + await Try.IgnoringError(async () => await watchForCancellationTokenSource.DisposeAsync()); + await Try.IgnoringError(async () => await requestCancelledCts.DisposeAsync()); log.Write(EventType.Diagnostic, "WatchForRequestCancellation disposed"); } diff --git a/source/Halibut/Queue/Redis/WatchForRequestCancellationOrSenderDisconnect.cs b/source/Halibut/Queue/Redis/WatchForRequestCancellationOrSenderDisconnect.cs index d348dee68..9126cf919 100644 --- a/source/Halibut/Queue/Redis/WatchForRequestCancellationOrSenderDisconnect.cs +++ b/source/Halibut/Queue/Redis/WatchForRequestCancellationOrSenderDisconnect.cs @@ -24,13 +24,12 @@ namespace Halibut.Queue.Redis public class WatchForRequestCancellationOrSenderDisconnect : IAsyncDisposable { - readonly CancellationTokenSource RequestCancellationTokenSource; + readonly CancelOnDisposeCancellationToken requestCancellationTokenSource; public CancellationToken RequestProcessingCancellationToken { get; } - readonly WatchForRequestCancellation watchForRequestCancellation; - readonly CancelOnDisposeCancellationTokenSource KeepWatchingCancellationTokenSource; + readonly CancelOnDisposeCancellationToken keepWatchingCancellationToken; - DisposableCollection disposableCollection = new DisposableCollection(); + DisposableCollection disposableCollection = new(); public WatchForRequestCancellationOrSenderDisconnect( Uri endpoint, @@ -41,15 +40,15 @@ public WatchForRequestCancellationOrSenderDisconnect( { try { - watchForRequestCancellation = new WatchForRequestCancellation(endpoint, requestActivityId, halibutRedisTransport, log); + var watchForRequestCancellation = new WatchForRequestCancellation(endpoint, requestActivityId, halibutRedisTransport, log); disposableCollection.AddAsyncDisposable(watchForRequestCancellation); - RequestCancellationTokenSource = CancellationTokenSource.CreateLinkedTokenSource(watchForRequestCancellation.RequestCancelledCancellationToken); - disposableCollection.Add(RequestCancellationTokenSource); - RequestProcessingCancellationToken = RequestCancellationTokenSource.Token; + requestCancellationTokenSource = new CancelOnDisposeCancellationToken(watchForRequestCancellation.RequestCancelledCancellationToken); + disposableCollection.AddAsyncDisposable(requestCancellationTokenSource); + RequestProcessingCancellationToken = requestCancellationTokenSource.Token; - KeepWatchingCancellationTokenSource = new CancellationTokenSource().CancelOnDispose(); - disposableCollection.AddAsyncDisposable(KeepWatchingCancellationTokenSource); + keepWatchingCancellationToken = new CancelOnDisposeCancellationToken(); + disposableCollection.AddAsyncDisposable(keepWatchingCancellationToken); Task.Run(() => WatchThatNodeWhichSentTheRequestIsStillAlive(endpoint, requestActivityId, halibutRedisTransport, nodeOfflineTimeoutBetweenHeartBeatsFromSender, log)); } @@ -58,19 +57,18 @@ public WatchForRequestCancellationOrSenderDisconnect( Try.IgnoringError(async () => await disposableCollection.DisposeAsync()).GetAwaiter().GetResult(); throw; } - } async Task WatchThatNodeWhichSentTheRequestIsStillAlive(Uri endpoint, Guid requestActivityId, HalibutRedisTransport halibutRedisTransport, TimeSpan nodeOfflineTimeoutBetweenHeartBeatsFromSender, ILog log) { - var watchCancellationToken = KeepWatchingCancellationTokenSource.CancellationToken; + var watchCancellationToken = keepWatchingCancellationToken.Token; try { var res = await NodeHeartBeatSender .WatchThatNodeWhichSentTheRequestIsStillAlive(endpoint, requestActivityId, halibutRedisTransport, log, nodeOfflineTimeoutBetweenHeartBeatsFromSender, watchCancellationToken); if (res == NodeHeartBeatSender.NodeProcessingRequestWatcherResult.NodeMayHaveDisconnected) { - await RequestCancellationTokenSource.CancelAsync(); + await requestCancellationTokenSource.CancelAsync(); } } catch (Exception) when (watchCancellationToken.IsCancellationRequested) diff --git a/source/Halibut/Util/CancelOnDisposeCancellationToken.cs b/source/Halibut/Util/CancelOnDisposeCancellationToken.cs new file mode 100644 index 000000000..d41730768 --- /dev/null +++ b/source/Halibut/Util/CancelOnDisposeCancellationToken.cs @@ -0,0 +1,61 @@ +#nullable enable +using System; +using System.Threading; +using System.Threading.Tasks; + +namespace Halibut.Util +{ + + /// + /// An async disposable wrapper for CancellationTokenSource that safely cancels and DOES NOT dispose it. + /// + public sealed class CancelOnDisposeCancellationToken : IAsyncDisposable + { + readonly CancellationTokenSource cancellationTokenSource; + bool disposed; + + public CancelOnDisposeCancellationToken(params CancellationToken[] token) + : this(CancellationTokenSource.CreateLinkedTokenSource(token)) + { + } + public CancelOnDisposeCancellationToken() : this(new CancellationTokenSource()) + { + } + + private CancelOnDisposeCancellationToken(CancellationTokenSource cancellationTokenSource) + { + this.cancellationTokenSource = cancellationTokenSource; + Token = cancellationTokenSource.Token; + } + + public CancellationToken Token { get; } + + public async ValueTask DisposeAsync() + { + if (disposed) + return; + + disposed = true; + + await Try.IgnoringError(async () => await CancelAsync()); + + // And then don't dispose the CancellationTokenSource. + // Since doing so WILL result in race conditions where + // callbacks will be silently not executed. + } + + public async Task CancelAsync() + { +#if NET8_0_OR_GREATER + await cancellationTokenSource.CancelAsync(); +#else + CancellationTokenSource.Cancel(); +#endif + } + + public void CancelAfter(TimeSpan timeSpan) + { + cancellationTokenSource.CancelAfter(timeSpan); + } + } +} \ No newline at end of file diff --git a/source/Halibut/Util/CancellationTokenSourceExtensionMethods.cs b/source/Halibut/Util/CancellationTokenSourceExtensionMethods.cs deleted file mode 100644 index 1b862141b..000000000 --- a/source/Halibut/Util/CancellationTokenSourceExtensionMethods.cs +++ /dev/null @@ -1,55 +0,0 @@ -#nullable enable -using System; -using System.Threading; -using System.Threading.Tasks; - -namespace Halibut.Util -{ - public static class CancellationTokenSourceExtensionMethods - { - /// - /// Creates an async disposable wrapper for the CancellationTokenSource that will safely cancel and dispose it. - /// - /// The CancellationTokenSource to wrap - /// An IAsyncDisposable that will cancel and dispose the CancellationTokenSource - public static CancelOnDisposeCancellationTokenSource CancelOnDispose(this CancellationTokenSource cancellationTokenSource) - { - return new CancelOnDisposeCancellationTokenSource(cancellationTokenSource); - } - } - - /// - /// An async disposable wrapper for CancellationTokenSource that safely cancels and disposes it. - /// - public sealed class CancelOnDisposeCancellationTokenSource : IAsyncDisposable - { - public readonly CancellationTokenSource CancellationTokenSource; - private bool disposed = false; - - internal CancelOnDisposeCancellationTokenSource(CancellationTokenSource cancellationTokenSource) - { - this.CancellationTokenSource = cancellationTokenSource ?? throw new ArgumentNullException(nameof(cancellationTokenSource)); - this.CancellationToken = cancellationTokenSource.Token; - } - - public CancellationToken CancellationToken { get; } - - public async ValueTask DisposeAsync() - { - if (disposed) - return; - - disposed = true; - - // First, attempt to cancel the cancellation token source -#if NET8_0_OR_GREATER - await Try.IgnoringError(async () => await CancellationTokenSource.CancelAsync()); -#else - Try.IgnoringError(() => CancellationTokenSource.Cancel()); -#endif - - // Then, dispose the cancellation token source - Try.IgnoringError(() => CancellationTokenSource.Dispose()); - } - } -} \ No newline at end of file From 90157f15ce2737030e549670ae5b827c67df486b Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Wed, 13 Aug 2025 14:54:40 +1000 Subject: [PATCH 040/137] Allow retrying when redis loses all of its data --- .../Redis/RedisPendingRequestQueueFixture.cs | 15 +++----- source/Halibut.Tests/Support/Try.cs | 13 +++++++ ...onReturnedByHalibutProxyExtensionMethod.cs | 30 +++++++++++++++ .../RedisDataLoseHalibutClientException.cs | 25 +++++++++++++ .../Queue/Redis/RedisPendingRequestQueue.cs | 4 +- .../Halibut/ServiceModel/PendingRequestV2.cs | 37 ++++++++++++++----- 6 files changed, 103 insertions(+), 21 deletions(-) create mode 100644 source/Halibut/Queue/Redis/RedisDataLoseHalibutClientException.cs diff --git a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs index ce380c64f..67b4ab700 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs @@ -32,6 +32,7 @@ using Serilog; using DisposableCollection = Halibut.Util.DisposableCollection; using ILog = Halibut.Diagnostics.ILog; +using Try = Halibut.Tests.Support.Try; namespace Halibut.Tests.Queue.Redis { @@ -222,16 +223,10 @@ public async Task WhenDataLostIsDetected_InFlightRequestShouldBeAbandoned() queueAndWaitAsync.IsCompleted.Should().BeTrue("As soon as data loss is detected the queueAndWait should return."); // Sigh it can go down either of these paths! - if (queueAndWaitAsync.IsCompletedSuccessfully) - { - var responseMessage = await queueAndWaitAsync; - responseMessage!.Error!.Message.Should().Contain("Cancelled because data loss on redis was detected."); - } - else - { - var e = await AssertException.Throws(queueAndWaitAsync); - e.And.InnerException!.Message.Should().Contain("Cancelled because data loss on redis was detected."); - } + var e = await AssertException.Throws(queueAndWaitAsync); + e.And.IsRetryableError().Should().Be(HalibutRetryableErrorType.IsRetryable); + e.And.Should().BeOfType(); + } [Test] diff --git a/source/Halibut.Tests/Support/Try.cs b/source/Halibut.Tests/Support/Try.cs index 09c35ceb0..dfc9295d2 100644 --- a/source/Halibut.Tests/Support/Try.cs +++ b/source/Halibut.Tests/Support/Try.cs @@ -6,6 +6,19 @@ namespace Halibut.Tests.Support { public static class Try { + public static Exception? CatchingError(Action tryThisAction) + { + try + { + tryThisAction(); + } + catch (Exception e) + { + return e; + } + + return null; + } public static void CatchingError(Action tryThisAction, Action onFailure) { try diff --git a/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs b/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs index aa0bafb2d..32b063144 100644 --- a/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs +++ b/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs @@ -2,14 +2,38 @@ using System.IO; using System.Net.Sockets; using Halibut.Exceptions; +using Halibut.Queue.Redis; using Halibut.Transport; using Halibut.Transport.Protocol; using Halibut.Transport.Proxy.Exceptions; namespace Halibut.Diagnostics { + public enum HalibutRetryableErrorType + { + IsRetryable, + UnknownError, + NotRetryable + } + public static class ExceptionReturnedByHalibutProxyExtensionMethod { + public static HalibutRetryableErrorType IsRetryableError(this Exception exception) + { + var halibutNetworkExceptionType = IsNetworkError(exception); + switch (halibutNetworkExceptionType) + { + case HalibutNetworkExceptionType.IsNetworkError: + return HalibutRetryableErrorType.IsRetryable; + case HalibutNetworkExceptionType.UnknownError: + return HalibutRetryableErrorType.UnknownError; + case HalibutNetworkExceptionType.NotANetworkError: + return HalibutRetryableErrorType.NotRetryable; + default: + throw new ArgumentOutOfRangeException(); + } + } + /// /// Classifies the exception thrown from a halibut proxy as a network error or not. /// In some cases it is not possible to tell if the exception is a network error. @@ -19,6 +43,12 @@ public static class ExceptionReturnedByHalibutProxyExtensionMethod /// public static HalibutNetworkExceptionType IsNetworkError(this Exception exception) { + // TODO: This should be in is retryable but for now it needs to be here to work with tentacle client. + if (exception is RedisDataLoseHalibutClientException) + { + return HalibutNetworkExceptionType.IsNetworkError; + } + if (exception is NoMatchingServiceOrMethodHalibutClientException) { return HalibutNetworkExceptionType.NotANetworkError; diff --git a/source/Halibut/Queue/Redis/RedisDataLoseHalibutClientException.cs b/source/Halibut/Queue/Redis/RedisDataLoseHalibutClientException.cs new file mode 100644 index 000000000..662d00ef9 --- /dev/null +++ b/source/Halibut/Queue/Redis/RedisDataLoseHalibutClientException.cs @@ -0,0 +1,25 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; + +namespace Halibut.Queue.Redis +{ + public class RedisDataLoseHalibutClientException : HalibutClientException + { + public RedisDataLoseHalibutClientException(string message) : base(message) + { + } + } +} \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index 1654f369f..1448abc85 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -122,7 +122,9 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can // Can the WatchProcessProcessingNodeIsStillConnected fail and never come back? await pending.WaitUntilComplete( async () => await tryClearRequestFromQueueAtMostOnce.Task, - () => dataLoseCt.IsCancellationRequested ? " Cancelled because data loss on redis was detected." : "", + () => dataLoseCt.IsCancellationRequested ? + new RedisDataLoseHalibutClientException($"Request {request.ActivityId} was cancelled because we detected that redis lost all of its data.") + : null, cancellationToken); } finally diff --git a/source/Halibut/ServiceModel/PendingRequestV2.cs b/source/Halibut/ServiceModel/PendingRequestV2.cs index 239175124..70df71b1f 100644 --- a/source/Halibut/ServiceModel/PendingRequestV2.cs +++ b/source/Halibut/ServiceModel/PendingRequestV2.cs @@ -58,10 +58,10 @@ public PendingRequest(RequestMessage request, ILog log) /// This gives the user an opportunity to remove the pending request from shared places and optionally /// call BeginTransfer /// - /// + /// /// public async Task WaitUntilComplete(Func checkIfPendingRequestWasCollectedOrRemoveIt, - Func cancellationReason, + Func overrideCancellationReason, CancellationToken cancellationToken) { log.Write(EventType.MessageExchange, "Request {0} was queued", request); @@ -93,20 +93,29 @@ public async Task WaitUntilComplete(Func checkIfPendingRequestWasCollected if (cancellationToken.IsCancellationRequested) { // TODO: This seems sus, we throw here but we don't throw below. This should be straightened out. + await Try.IgnoringError(async () => await pendingRequestCancellationTokenSource.CancelAsync()); + + var cancellationException = overrideCancellationReason(); + if (cancellationException != null) + { + log.Write(EventType.MessageExchange, "Request {0} did not complete because: " + cancellationException.Message, request); + throw cancellationException; + } + OperationCanceledException operationCanceledException; if (!requestCollected.IsSet) { - log.Write(EventType.MessageExchange, "Request {0} was cancelled before it could be collected by the polling endpoint" + cancellationReason()??"", request); - operationCanceledException = new OperationCanceledException($"Request {request} was cancelled before it could be collected by the polling endpoint" + cancellationReason()??""); + log.Write(EventType.MessageExchange, "Request {0} was cancelled before it could be collected by the polling endpoint", request); + operationCanceledException = new OperationCanceledException($"Request {request} was cancelled before it could be collected by the polling endpoint"); } else { - log.Write(EventType.MessageExchange, "Request {0} was collected by the polling endpoint, will try to cancel the request" + cancellationReason()??"", request); - operationCanceledException = new OperationCanceledException($"Request {request} was collected by the polling endpoint, will try to cancel the request" + cancellationReason() ?? ""); + log.Write(EventType.MessageExchange, "Request {0} was collected by the polling endpoint, will try to cancel the request", request); + operationCanceledException = new OperationCanceledException($"Request {request} was collected by the polling endpoint, will try to cancel the request"); } - - await Try.IgnoringError(async () => await pendingRequestCancellationTokenSource.CancelAsync()); + throw requestCollected.IsSet ? new TransferringRequestCancelledException(operationCanceledException) : new ConnectingRequestCancelledException(operationCanceledException); + } if (!requestCollected.IsSet) @@ -138,10 +147,18 @@ public async Task WaitUntilComplete(Func checkIfPendingRequestWasCollected { if (!responseWaiter.IsSet) { - log.Write(EventType.MessageExchange, "Request {0} was cancelled before a response was received" + cancellationReason()??"", request); + var cancellationException = overrideCancellationReason(); + if (cancellationException != null) + { + await Try.IgnoringError(async () => await pendingRequestCancellationTokenSource.CancelAsync()); + log.Write(EventType.MessageExchange, "Request {0} did not complete because: " + cancellationException.Message, request); + throw cancellationException; + } + + log.Write(EventType.MessageExchange, "Request {0} was cancelled before a response was received" + overrideCancellationReason()??"", request); SetResponseNoLock(ResponseMessage.FromException( request, - new TimeoutException($"A request was sent to a polling endpoint, the polling endpoint collected it but the request was cancelled before the polling endpoint responded." + cancellationReason()??""), + new TimeoutException($"A request was sent to a polling endpoint, the polling endpoint collected it but the request was cancelled before the polling endpoint responded." + overrideCancellationReason()??""), ConnectionState.Connecting), false); await Try.IgnoringError(async () => await pendingRequestCancellationTokenSource.CancelAsync()); From a03aed0738c6d377f2ec6fc374f69a88b611e77f Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Wed, 13 Aug 2025 15:21:09 +1000 Subject: [PATCH 041/137] Account for requests being abandoned --- ...onReturnedByHalibutProxyExtensionMethod.cs | 7 +++ .../Queue/Redis/RedisPendingRequestQueue.cs | 56 +++++++++++++------ .../Redis/WatchForRequestCancellation.cs | 14 +++-- ...orRequestCancellationOrSenderDisconnect.cs | 5 +- .../Transport/Protocol/ResponseMessage.cs | 5 ++ 5 files changed, 66 insertions(+), 21 deletions(-) diff --git a/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs b/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs index 32b063144..91bdf5702 100644 --- a/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs +++ b/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs @@ -48,7 +48,14 @@ public static HalibutNetworkExceptionType IsNetworkError(this Exception exceptio { return HalibutNetworkExceptionType.IsNetworkError; } + + if (exception is HalibutClientException) + { + if (exception.Message.Contains(RedisPendingRequestQueue.RequestAbandonedMessage)) return HalibutNetworkExceptionType.IsNetworkError; + } + // TODO end + if (exception is NoMatchingServiceOrMethodHalibutClientException) { return HalibutNetworkExceptionType.NotANetworkError; diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index 1448abc85..7f88e1beb 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -35,7 +35,7 @@ class RedisPendingRequestQueue : IPendingRequestQueue, IDisposable readonly AsyncManualResetEvent hasItemsForEndpoint = new(); readonly CancelOnDisposeCancellationToken queueCts = new (); - internal ConcurrentDictionary disposablesForInFlightRequests = new(); + internal ConcurrentDictionary disposablesForInFlightRequests = new(); // TODO: this needs to be used in all public methods. readonly CancellationToken queueToken; @@ -295,6 +295,8 @@ async Task SubscribeToResponse(Guid activityId, // There is a chance the data loss occured after we got the data but before here. // In that case we will just time out because of the lack of heart beats. var dataLossCT = await this.watchForRedisLosingAllItsData.GetTokenForDataLoseDetection(TimeSpan.FromSeconds(30), queueToken); + // TODO: We should only mark the request as cancelled in the event we are told it was cancelled. Otherwise + // we need to correctly say that we decided to abandon the request such that it can be retried. disposables.AddAsyncDisposable(new NodeHeartBeatSender(endpoint, pending.ActivityId, halibutRedisTransport, log, HalibutQueueNodeSendingPulses.Receiver, DelayBetweenHeartBeatsForRequestProcessor)); var watcher = new WatchForRequestCancellationOrSenderDisconnect(endpoint, pending.ActivityId, halibutRedisTransport, NodeIsOfflineHeartBeatTimeoutForRequestSender, log); @@ -304,7 +306,7 @@ async Task SubscribeToResponse(Guid activityId, disposables.AddAsyncDisposable(cts); var response = new RequestMessageWithCancellationToken(pending, cts.Token); - disposablesForInFlightRequests[pending.ActivityId] = disposables; + disposablesForInFlightRequests[pending.ActivityId] = new WatcherAndDisposables(disposables, cts.Token, watcher); return response; } catch (Exception) @@ -314,9 +316,34 @@ async Task SubscribeToResponse(Guid activityId, } } + public class WatcherAndDisposables : IAsyncDisposable + { + readonly DisposableCollection disposableCollection; + public CancellationToken RequestCancelledForAnyReasonCancellationToken { get; } + public WatchForRequestCancellationOrSenderDisconnect watcher { get; } + + public WatcherAndDisposables(DisposableCollection disposableCollection, CancellationToken requestCancelledForAnyReasonCancellationToken, WatchForRequestCancellationOrSenderDisconnect watcher) + { + this.disposableCollection = disposableCollection; + this.RequestCancelledForAnyReasonCancellationToken = requestCancelledForAnyReasonCancellationToken; + this.watcher = watcher; + } + + public async ValueTask DisposeAsync() + { + await Try.IgnoringError(async () => await disposableCollection.DisposeAsync()); + } + } + + public const string RequestAbandonedMessage = "The request was abandoned, possibly because the node processing the request shutdown or redis lost all of its data."; public async Task ApplyResponse(ResponseMessage response, Guid requestActivityId) { log.Write(EventType.MessageExchange, "Applying response for request {0}", requestActivityId); + WatcherAndDisposables? watcherAndDisposables = null; + if (!disposablesForInFlightRequests.TryRemove(requestActivityId, out watcherAndDisposables)) + { + log.Write(EventType.Diagnostic, "No in-flight request resources found to dispose for request {0}", requestActivityId); + } try { @@ -332,6 +359,14 @@ public async Task ApplyResponse(ResponseMessage response, Guid requestActivityId // This node has now completed the RPC, and so the response must be sent // back to the node which sent the response + if (watcherAndDisposables != null && watcherAndDisposables.RequestCancelledForAnyReasonCancellationToken.IsCancellationRequested) + { + if (!watcherAndDisposables.watcher.SenderCancelledTheRequest) + { + log.Write(EventType.Diagnostic, "Response for request {0}, has been overridden with an abandon message as the request was abandoned", requestActivityId); + response = ResponseMessage.FromException(response, new HalibutClientException(RequestAbandonedMessage)); + } + } var payload = await messageReaderWriter.PrepareResponse(response, cancellationToken); log.Write(EventType.MessageExchange, "Sending response message for request {0}", requestActivityId); await PollAndSubscribeToResponse.TrySendMessage(ResponseMessageSubscriptionName, halibutRedisTransport, endpoint, requestActivityId, payload, TTLOfResponseMessage, log); @@ -344,24 +379,13 @@ public async Task ApplyResponse(ResponseMessage response, Guid requestActivityId } finally { - if (disposablesForInFlightRequests.TryRemove(requestActivityId, out var disposables)) - { log.Write(EventType.Diagnostic, "Disposing in-flight request resources for request {0}", requestActivityId); - try - { - await disposables.DisposeAsync(); - log.Write(EventType.Diagnostic, "Successfully disposed in-flight request resources for request {0}", requestActivityId); - } - catch (Exception ex) + if (watcherAndDisposables != null) { - log.WriteException(EventType.Diagnostic, "Error disposing in-flight request resources for request {0}", ex, requestActivityId); + await watcherAndDisposables.DisposeAsync(); } - } - else - { - log.Write(EventType.Diagnostic, "No in-flight request resources found to dispose for request {0}", requestActivityId); - } } + } async Task DequeueNextAsync() diff --git a/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs b/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs index 8c634a475..b8dd13e4a 100644 --- a/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs +++ b/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs @@ -61,6 +61,7 @@ public static async Task TrySendCancellation( readonly CancelOnDisposeCancellationToken requestCancelledCts = new(); public CancellationToken RequestCancelledCancellationToken => requestCancelledCts.Token; + public bool SenderCancelledTheRequest { get; private set; } readonly CancelOnDisposeCancellationToken watchForCancellationTokenSource = new(); @@ -86,8 +87,7 @@ async Task WatchForCancellation(Uri endpoint, Guid requestActivityId, HalibutRed { await Task.CompletedTask; log.Write(EventType.Diagnostic, "Received cancellation notification via subscription - Endpoint: {0}, ActivityId: {1}", endpoint, requestActivityId); - await requestCancelledCts.CancelAsync(); - await watchForCancellationTokenSource.CancelAsync(); + await RequestHasBeenCancelled(); }, token); @@ -103,8 +103,7 @@ async Task WatchForCancellation(Uri endpoint, Guid requestActivityId, HalibutRed if (await halibutRedisTransport.IsRequestMarkedAsCancelled(endpoint, requestActivityId, token)) { log.Write(EventType.Diagnostic, "Request cancellation detected via polling - Endpoint: {0}, ActivityId: {1}", endpoint, requestActivityId); - await requestCancelledCts.CancelAsync(); - await watchForCancellationTokenSource.CancelAsync(); + await RequestHasBeenCancelled(); break; } } @@ -126,6 +125,13 @@ async Task WatchForCancellation(Uri endpoint, Guid requestActivityId, HalibutRed } } + async Task RequestHasBeenCancelled() + { + SenderCancelledTheRequest = true; + await requestCancelledCts.CancelAsync(); + await watchForCancellationTokenSource.CancelAsync(); + } + public async ValueTask DisposeAsync() { log.Write(EventType.Diagnostic, "Disposing WatchForRequestCancellation"); diff --git a/source/Halibut/Queue/Redis/WatchForRequestCancellationOrSenderDisconnect.cs b/source/Halibut/Queue/Redis/WatchForRequestCancellationOrSenderDisconnect.cs index 9126cf919..f9e6b3d1e 100644 --- a/source/Halibut/Queue/Redis/WatchForRequestCancellationOrSenderDisconnect.cs +++ b/source/Halibut/Queue/Redis/WatchForRequestCancellationOrSenderDisconnect.cs @@ -31,6 +31,9 @@ public class WatchForRequestCancellationOrSenderDisconnect : IAsyncDisposable DisposableCollection disposableCollection = new(); + WatchForRequestCancellation watchForRequestCancellation; + public bool SenderCancelledTheRequest => watchForRequestCancellation.SenderCancelledTheRequest; + public WatchForRequestCancellationOrSenderDisconnect( Uri endpoint, Guid requestActivityId, @@ -40,7 +43,7 @@ public WatchForRequestCancellationOrSenderDisconnect( { try { - var watchForRequestCancellation = new WatchForRequestCancellation(endpoint, requestActivityId, halibutRedisTransport, log); + watchForRequestCancellation = new WatchForRequestCancellation(endpoint, requestActivityId, halibutRedisTransport, log); disposableCollection.AddAsyncDisposable(watchForRequestCancellation); requestCancellationTokenSource = new CancelOnDisposeCancellationToken(watchForRequestCancellation.RequestCancelledCancellationToken); diff --git a/source/Halibut/Transport/Protocol/ResponseMessage.cs b/source/Halibut/Transport/Protocol/ResponseMessage.cs index 6e6a1c3a1..f73dc6ca8 100644 --- a/source/Halibut/Transport/Protocol/ResponseMessage.cs +++ b/source/Halibut/Transport/Protocol/ResponseMessage.cs @@ -32,6 +32,11 @@ public static ResponseMessage FromException(RequestMessage request, Exception ex { return new ResponseMessage { Id = request.Id, Error = ServerErrorFromException(ex, connectionState) }; } + + public static ResponseMessage FromException(ResponseMessage response, Exception ex, ConnectionState connectionState = ConnectionState.Unknown) + { + return new ResponseMessage { Id = response.Id, Error = ServerErrorFromException(ex, connectionState) }; + } internal static ServerError ServerErrorFromException(Exception ex, ConnectionState connectionState = ConnectionState.Unknown) { From 824e79345abff99740e2887b75ab22072a0b1e79 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Wed, 13 Aug 2025 17:05:12 +1000 Subject: [PATCH 042/137] Update some TODOs --- source/Halibut/Queue/Redis/HalibutRedisTransport.cs | 6 +----- source/Halibut/Queue/Redis/NodeHeartBeatSender.cs | 4 +--- source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs | 2 +- source/Halibut/Queue/Redis/RedisFacade.cs | 1 + source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs | 5 ++--- 5 files changed, 6 insertions(+), 12 deletions(-) diff --git a/source/Halibut/Queue/Redis/HalibutRedisTransport.cs b/source/Halibut/Queue/Redis/HalibutRedisTransport.cs index 57e776556..1c86f5321 100644 --- a/source/Halibut/Queue/Redis/HalibutRedisTransport.cs +++ b/source/Halibut/Queue/Redis/HalibutRedisTransport.cs @@ -139,11 +139,7 @@ public async Task PublishCancellation(Uri endpoint, Guid requestId, Cancellation var channelName = RequestCancelledChannel(endpoint, requestId); await facade.PublishToChannel(channelName, "{}"); } - - // Cancellation Notification - // TODO: I think this wants to become some sort of: is the sender still interested in the results - // e.g. if the sender of the request is gone we should treat that as a cancellation. - + public string RequestCancelledMarkerKey(Uri endpoint, Guid requestId) { return $"{Namespace}::RequestCancelledMarkerKey::{endpoint}::{requestId}"; diff --git a/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs b/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs index 8d951ba8e..99b1cdc07 100644 --- a/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs +++ b/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs @@ -164,7 +164,6 @@ private static async Task WatchForPulsesFrom while (!watchCancellationToken.IsCancellationRequested) { - // TODO: I am sure a fancy pants delay could be done here calculated from the now and last heart beat etc. var timeSinceLastHeartBeat = DateTimeOffset.Now - lastHeartBeat.Value; if (timeSinceLastHeartBeat > maxTimeBetweenHeartBeetsBeforeNodeIsAssumedToBeOffline) @@ -197,7 +196,7 @@ static async Task WaitForRequestToBeCollected(Uri endpoint, RequestMessage reque log = log.ForContext(); log.Write(EventType.Diagnostic, "Waiting for request {0} to be collected from queue", request.ActivityId); - // Is this worthwhile? + // TODO: Is this worthwhile? var asyncManualResetEvent = new AsyncManualResetEvent(false); // await using var subscription = await halibutRedisTransport.SubscribeToNodeHeartBeatChannel( // endpoint, @@ -215,7 +214,6 @@ static async Task WaitForRequestToBeCollected(Uri endpoint, RequestMessage reque { asyncManualResetEvent.Reset(); // Has something else determined the request was collected? - // TODO should we bail out of here if the PendingRequest is complete? if(pending.HasRequestBeenMarkedAsCollected) { log.Write(EventType.Diagnostic, "Request {0} has been marked as collected", request.ActivityId); diff --git a/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs b/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs index 1b0304656..d7d00b979 100644 --- a/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs +++ b/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs @@ -189,7 +189,7 @@ public async ValueTask DisposeAsync() //Try.IgnoringError(() => watcherTokenSource.Dispose()); // If the message task is not yet complete, then complete if now with null since we have nothing for it. - Try.IgnoringError(() => message.TrySetCanceled()); // TODO: try set cancelled? you sure? + Try.IgnoringError(() => message.TrySetCanceled()); log.Write(EventType.Diagnostic, "Disposed GenericWatcher for {0} - Endpoint: {1}, ActivityId: {2}", messageTypeName, endpoint, activityId); } diff --git a/source/Halibut/Queue/Redis/RedisFacade.cs b/source/Halibut/Queue/Redis/RedisFacade.cs index cd40837ab..6294a296f 100644 --- a/source/Halibut/Queue/Redis/RedisFacade.cs +++ b/source/Halibut/Queue/Redis/RedisFacade.cs @@ -328,6 +328,7 @@ public async Task HashContainsKey(string key, string field) }, CancellationToken.None); // TODO: If we retry this is not idempotent. + // TODO: Test var res = await ExecuteWithRetry(async () => { var database = Connection.GetDatabase(); diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index 7f88e1beb..123dbcf5a 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -281,7 +281,7 @@ async Task SubscribeToResponse(Guid activityId, public async Task DequeueAsync(CancellationToken cancellationToken) { - // TODO: is it god or bad that redis exceptions will bubble out of here. + // TODO: is it good or bad that redis exceptions will bubble out of here. // I think it will kill the TCP connection, which will force re-connect (in perhaps a backoff function) // This could result in connecting to a node that is actually connected to redis. It could also // cause a cascade of failure from high load. @@ -295,8 +295,6 @@ async Task SubscribeToResponse(Guid activityId, // There is a chance the data loss occured after we got the data but before here. // In that case we will just time out because of the lack of heart beats. var dataLossCT = await this.watchForRedisLosingAllItsData.GetTokenForDataLoseDetection(TimeSpan.FromSeconds(30), queueToken); - // TODO: We should only mark the request as cancelled in the event we are told it was cancelled. Otherwise - // we need to correctly say that we decided to abandon the request such that it can be retried. disposables.AddAsyncDisposable(new NodeHeartBeatSender(endpoint, pending.ActivityId, halibutRedisTransport, log, HalibutQueueNodeSendingPulses.Receiver, DelayBetweenHeartBeatsForRequestProcessor)); var watcher = new WatchForRequestCancellationOrSenderDisconnect(endpoint, pending.ActivityId, halibutRedisTransport, NodeIsOfflineHeartBeatTimeoutForRequestSender, log); @@ -361,6 +359,7 @@ public async Task ApplyResponse(ResponseMessage response, Guid requestActivityId if (watcherAndDisposables != null && watcherAndDisposables.RequestCancelledForAnyReasonCancellationToken.IsCancellationRequested) { + // TODO: test if (!watcherAndDisposables.watcher.SenderCancelledTheRequest) { log.Write(EventType.Diagnostic, "Response for request {0}, has been overridden with an abandon message as the request was abandoned", requestActivityId); From f0b3d5b6405a58aaaf0b8c329ea4dbed1c940894 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Thu, 14 Aug 2025 07:59:49 +1000 Subject: [PATCH 043/137] When the receiever of the requests detects data lose it returns a retryable error --- .../Redis/RedisPendingRequestQueueFixture.cs | 58 ++++++++++++++++++- 1 file changed, 57 insertions(+), 1 deletion(-) diff --git a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs index 67b4ab700..079407785 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs @@ -627,8 +627,62 @@ public async Task WhenTheSenderBrieflyDisconnectsFromRedisRightWhenTheReceiverSe response.Result.Should().Be("Yay"); } - + [Test] + public async Task WhenTheRequestReceiverDetectsRedisDataLose_AndTheRequestSenderDoesNot_TheSenderReceivesARetryableResponse() + { + // Arrange + var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); + var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); + var guid = Guid.NewGuid(); + + var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); + var messageSerializer = new QueueMessageSerializerBuilder().Build(); + var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); + + await using var stableConnection = CreateRedisFacade(guid: guid); + + var redisDataLoseDetectorOnReceiver = new CancellableDataLossWatchForRedisLosingAllItsData(); + var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(stableConnection), messageReaderWriter, new HalibutTimeoutsAndLimits()); + var node2Receiver = new RedisPendingRequestQueue(endpoint, redisDataLoseDetectorOnReceiver, log, new HalibutRedisTransport(stableConnection), messageReaderWriter, new HalibutTimeoutsAndLimits()); + await node2Receiver.WaitUntilQueueIsSubscribedToReceiveMessages(); + + var request = new RequestMessageBuilder("poll://test-endpoint").Build(); + var queueAndWaitTask = node1Sender.QueueAndWaitAsync(request, CancellationToken.None); + + var dequeuedRequest = await node2Receiver.DequeueAsync(CancellationToken); + + // Act + await redisDataLoseDetectorOnReceiver.DataLossHasOccured(); + + var responseThatWouldNotBeRetried = ResponseMessage.FromException(dequeuedRequest!.RequestMessage, new NoMatchingServiceOrMethodHalibutClientException("")); + CreateExceptionFromResponse(responseThatWouldNotBeRetried, log) + .IsRetryableError().Should().Be(HalibutRetryableErrorType.NotRetryable); + + await node2Receiver.ApplyResponse(ResponseMessage.FromResult(dequeuedRequest!.RequestMessage, "Yay"), dequeuedRequest!.RequestMessage.ActivityId); + + var response = await queueAndWaitTask; + response.Error.Should().NotBeNull(); + + // Assert + CreateExceptionFromResponse(response, log) + .IsRetryableError().Should().Be(HalibutRetryableErrorType.IsRetryable); + } + + static Exception CreateExceptionFromResponse(ResponseMessage responseThatWouldNotBeRetried, ILog log) + { + try + { + HalibutProxyWithAsync.ThrowExceptionFromReceivedError(responseThatWouldNotBeRetried.Error!, log); + } + catch (Exception e) + { + return e; + } + Assert.Fail("Excpected an exception in the response message"); + throw new Exception("it failed"); + } + [Test] [LatestClientAndLatestServiceTestCases(testNetworkConditions: false, testListening: false, testWebSocket: false)] public async Task WhenUsingTheRedisQueue_ASimpleEchoServiceCanBeCalled(ClientAndServiceTestCase clientAndServiceTestCase) @@ -659,6 +713,8 @@ public async Task WhenUsingTheRedisQueue_ASimpleEchoServiceCanBeCalled(ClientAnd } } } + + [Test] From c8375430be68b211564bfc2265092ab3d11099e9 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Thu, 14 Aug 2025 09:26:47 +1000 Subject: [PATCH 044/137] Set TTL, Use token, Add logging, Remove generic from PollAndSub class --- .../Halibut.Tests/ManyPollingTentacleTests.cs | 22 ----- .../Queue/Redis/NodeHeartBeatSenderFixture.cs | 6 +- .../Queue/Redis/RedisFacadeFixture.cs | 46 +++++----- .../RedisFacadeWhenRedisGoesDownAwayTests.cs | 34 ++++---- .../Queue/Redis/HalibutRedisTransport.cs | 56 ++++++------ .../Queue/Redis/NodeHeartBeatSender.cs | 14 +-- .../Queue/Redis/PollAndSubscribeToResponse.cs | 65 +++++++------- source/Halibut/Queue/Redis/RedisFacade.cs | 86 ++++++------------- .../Queue/Redis/RedisPendingRequestQueue.cs | 37 ++++---- .../Redis/WatchForRedisLosingAllItsData.cs | 2 +- .../ServiceModel/PendingRequestQueueAsync.cs | 10 +-- .../Halibut/ServiceModel/PendingRequestV2.cs | 9 +- 12 files changed, 163 insertions(+), 224 deletions(-) diff --git a/source/Halibut.Tests/ManyPollingTentacleTests.cs b/source/Halibut.Tests/ManyPollingTentacleTests.cs index e87af566f..e92898e83 100644 --- a/source/Halibut.Tests/ManyPollingTentacleTests.cs +++ b/source/Halibut.Tests/ManyPollingTentacleTests.cs @@ -142,30 +142,8 @@ public async Task ManyRequestToPollingTentacles_Works_AndDoesNotUseTooManyResour callsMade.Should().Be(total); - - for (int i = 0; i < 20; i++) - { - if (redisFacade.TotalSubscribers == pollEndpoints.Length) break; - await Task.Delay(100); - } - - var s = ""; - foreach (var subsKey in redisFacade.subs.Keys) - { - if(subsKey.Contains("RequestMessagesPulseChannelName")) continue; - s += "\n" + subsKey; - } - - Logger.Information("F {F}", s); - if(isRedis) { - if (redisFacade.TotalSubscribers != pollEndpoints.Length) - { - Logger.Information("OH NO!"); - - await Task.Delay(TimeSpan.FromMinutes(1)); - } redisFacade.TotalSubscribers.Should().Be(pollEndpoints.Length); } diff --git a/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs b/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs index d784a6637..5dbf959e8 100644 --- a/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs @@ -170,7 +170,7 @@ public async Task WaitUntilNodeProcessingRequestFlatLines_WhenHeartbeatsStop_Sho var request = new RequestMessageBuilder(endpoint.ToString()) .WithActivityId(requestActivityId) .Build(); - var pendingRequest = new PendingRequest(request, log); + var pendingRequest = new RedisPendingRequest(request, log); // Start heartbeat sender await using var heartBeatSender = new NodeHeartBeatSender(endpoint, requestActivityId, unstableRedisTransport, log, HalibutQueueNodeSendingPulses.Receiver, TimeSpan.FromSeconds(1)); @@ -212,7 +212,7 @@ public async Task NoIssueReturnedWhenNodeProcessingRequestIsNotSeenToGoOffline() var request = new RequestMessageBuilder(endpoint.ToString()) .WithActivityId(requestActivityId) .Build(); - var pendingRequest = new PendingRequest(request, log); + var pendingRequest = new RedisPendingRequest(request, log); await pendingRequest.RequestHasBeenCollectedAndWillBeTransferred(); // Start heartbeat sender @@ -256,7 +256,7 @@ public async Task WaitUntilNodeProcessingRequestFlatLines_WhenConnectionInterrup var request = new RequestMessageBuilder(endpoint.ToString()) .WithActivityId(requestActivityId) .Build(); - var pendingRequest = new PendingRequest(request, log); + var pendingRequest = new RedisPendingRequest(request, log); // Start heartbeat sender with unstable connection await using var heartBeatSender = new NodeHeartBeatSender(endpoint, requestActivityId, unstableRedisTransport, log, HalibutQueueNodeSendingPulses.Receiver, TimeSpan.FromSeconds(1)); diff --git a/source/Halibut.Tests/Queue/Redis/RedisFacadeFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisFacadeFixture.cs index a1c58ccc1..9f9e44e07 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisFacadeFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisFacadeFixture.cs @@ -75,10 +75,10 @@ public async Task SetInHash_ShouldStoreValueInHash() var payload = "test-payload"; // Act - await redisFacade.SetInHash(key, field, payload); + await redisFacade.SetInHash(key, field, payload, TimeSpan.FromMinutes(1), CancellationToken); // Assert - We'll verify by trying to get and delete it - var retrievedValue = await redisFacade.TryGetAndDeleteFromHash(key, field); + var retrievedValue = await redisFacade.TryGetAndDeleteFromHash(key, field, CancellationToken); retrievedValue.Should().Be(payload); } @@ -91,10 +91,10 @@ public async Task TryGetAndDeleteFromHash_WithExistingValue_ShouldReturnValueAnd var field = "test-field"; var payload = "test-payload"; - await redisFacade.SetInHash(key, field, payload); + await redisFacade.SetInHash(key, field, payload, TimeSpan.FromMinutes(1), CancellationToken); // Act - var retrievedValue = await redisFacade.TryGetAndDeleteFromHash(key, field); + var retrievedValue = await redisFacade.TryGetAndDeleteFromHash(key, field, CancellationToken); // Assert retrievedValue.Should().Be(payload); @@ -109,10 +109,10 @@ public async Task HashContainsKey_WithExistingField_ShouldReturnTrue() var field = "test-field"; var payload = "test-payload"; - await redisFacade.SetInHash(key, field, payload); + await redisFacade.SetInHash(key, field, payload, TimeSpan.FromMinutes(1), CancellationToken); // Act - var exists = await redisFacade.HashContainsKey(key, field); + var exists = await redisFacade.HashContainsKey(key, field, CancellationToken); // Assert exists.Should().BeTrue(); @@ -127,7 +127,7 @@ public async Task HashContainsKey_WithNonExistentField_ShouldReturnFalse() var nonExistentField = "non-existent-field"; // Act - var exists = await redisFacade.HashContainsKey(key, nonExistentField); + var exists = await redisFacade.HashContainsKey(key, nonExistentField, CancellationToken); // Assert exists.Should().BeFalse(); @@ -142,7 +142,7 @@ public async Task HashContainsKey_WithNonExistentKey_ShouldReturnFalse() var field = "test-field"; // Act - var exists = await redisFacade.HashContainsKey(nonExistentKey, field); + var exists = await redisFacade.HashContainsKey(nonExistentKey, field, CancellationToken); // Assert exists.Should().BeFalse(); @@ -157,24 +157,24 @@ public async Task TryGetAndDeleteFromHash_ShouldDeleteTheEntireKey() var field = "test-field"; var payload = "test-payload"; - await redisFacade.SetInHash(key, field, payload); + await redisFacade.SetInHash(key, field, payload, TimeSpan.FromMinutes(1), CancellationToken); // Verify the hash field exists - var existsBefore = await redisFacade.HashContainsKey(key, field); + var existsBefore = await redisFacade.HashContainsKey(key, field, CancellationToken); existsBefore.Should().BeTrue(); // Act - var retrievedValue = await redisFacade.TryGetAndDeleteFromHash(key, field); + var retrievedValue = await redisFacade.TryGetAndDeleteFromHash(key, field, CancellationToken); // Assert retrievedValue.Should().Be(payload); // Verify the entire key was deleted (not just the field) - var existsAfter = await redisFacade.HashContainsKey(key, field); + var existsAfter = await redisFacade.HashContainsKey(key, field, CancellationToken); existsAfter.Should().BeFalse(); // Verify trying to get it again returns null - var secondRetrieval = await redisFacade.TryGetAndDeleteFromHash(key, field); + var secondRetrieval = await redisFacade.TryGetAndDeleteFromHash(key, field, CancellationToken); secondRetrieval.Should().BeNull(); } @@ -188,12 +188,12 @@ public async Task ListRightPushAsync_AndListLeftPopAsync_ShouldWorkAsQueue() var payload2 = "second-item"; // Act - Push items to the right - await redisFacade.ListRightPushAsync(key, payload1); - await redisFacade.ListRightPushAsync(key, payload2); + await redisFacade.ListRightPushAsync(key, payload1, TimeSpan.FromMinutes(1), CancellationToken); + await redisFacade.ListRightPushAsync(key, payload2, TimeSpan.FromMinutes(1), CancellationToken); // Pop items from the left (FIFO) - var firstItem = await redisFacade.ListLeftPopAsync(key); - var secondItem = await redisFacade.ListLeftPopAsync(key); + var firstItem = await redisFacade.ListLeftPopAsync(key, CancellationToken); + var secondItem = await redisFacade.ListLeftPopAsync(key, CancellationToken); // Assert firstItem.Should().Be(payload1); @@ -208,7 +208,7 @@ public async Task ListLeftPopAsync_WithEmptyList_ShouldReturnNull() var emptyListKey = Guid.NewGuid().ToString(); // Act - var result = await redisFacade.ListLeftPopAsync(emptyListKey); + var result = await redisFacade.ListLeftPopAsync(emptyListKey, CancellationToken); // Assert result.Should().BeNull(); @@ -236,7 +236,7 @@ public async Task PublishToChannel_AndSubscribeToChannel_ShouldDeliverMessage() }, CancellationToken); // Act - Publish a message - await redisFacade.PublishToChannel(channelName, testMessage); + await redisFacade.PublishToChannel(channelName, testMessage, CancellationToken); // Wait for the message to be received await messageReceived.Task.TimeoutAfter(TimeSpan.FromSeconds(5), CancellationToken); @@ -273,7 +273,7 @@ public async Task PublishToChannel_WithMultipleMessages_ShouldDeliverAllMessages // Act - Publish multiple messages foreach (var msg in messages) { - await redisFacade.PublishToChannel(channelName, msg); + await redisFacade.PublishToChannel(channelName, msg, CancellationToken); } // Wait for all messages to be received @@ -305,7 +305,7 @@ public async Task SubscribeToChannel_WhenDisposed_ShouldUnsubscribe() await subscription.DisposeAsync(); // Publish a message after unsubscribing - await redisFacade.PublishToChannel(channelName, "should-not-receive"); + await redisFacade.PublishToChannel(channelName, "should-not-receive", CancellationToken); // Wait a bit to ensure no message is received await Task.Delay(100); @@ -377,10 +377,10 @@ public async Task SetInHash_WithTTL_ShouldExpireAfterSpecifiedTime() var payload = "test-payload"; // Act - Set a value in hash (it has a TTL of 9:9:9 according to the implementation) - await redisFacade.SetInHash(key, field, payload); + await redisFacade.SetInHash(key, field, payload, TimeSpan.FromMinutes(1), CancellationToken); // Immediately try to get the value - should exist - var immediateValue = await redisFacade.TryGetAndDeleteFromHash(key, field); + var immediateValue = await redisFacade.TryGetAndDeleteFromHash(key, field, CancellationToken); // Assert immediateValue.Should().Be(payload); diff --git a/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs b/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs index 70f9a1e2f..735703303 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs @@ -92,7 +92,7 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe portForwarder.ReturnToNormalMode(); // No delay here - should retry and succeed - await redisFacade.PublishToChannel("test-channel", "test-message"); + await redisFacade.PublishToChannel("test-channel", "test-message", CancellationToken); } [Test] @@ -109,7 +109,7 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe portForwarder.ReturnToNormalMode(); // No delay here - should retry and succeed - await redisFacade.SetInHash("test-hash", "test-field", "test-value"); + await redisFacade.SetInHash("test-hash", "test-field", "test-value", TimeSpan.FromMinutes(1), CancellationToken); } [Test] @@ -120,13 +120,13 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); // Establish connection and set up test data - await redisFacade.SetInHash("test-hash", "test-field", "test-value"); + await redisFacade.SetInHash("test-hash", "test-field", "test-value", TimeSpan.FromMinutes(1), CancellationToken); portForwarder.EnterKillNewAndExistingConnectionsMode(); portForwarder.ReturnToNormalMode(); // No delay here - should retry and succeed - var result = await redisFacade.TryGetAndDeleteFromHash("test-hash", "test-field"); + var result = await redisFacade.TryGetAndDeleteFromHash("test-hash", "test-field", CancellationToken); result.Should().Be("test-value"); } @@ -144,7 +144,7 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe portForwarder.ReturnToNormalMode(); // No delay here - should retry and succeed - await redisFacade.ListRightPushAsync("test-list", "test-item"); + await redisFacade.ListRightPushAsync("test-list", "test-item", TimeSpan.FromMinutes(1), CancellationToken ); } [Test] @@ -155,13 +155,13 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); // Establish connection and set up test data - await redisFacade.ListRightPushAsync("test-list", "test-item"); + await redisFacade.ListRightPushAsync("test-list", "test-item", TimeSpan.FromMinutes(1), CancellationToken); portForwarder.EnterKillNewAndExistingConnectionsMode(); portForwarder.ReturnToNormalMode(); // No delay here - should retry and succeed - var result = await redisFacade.ListLeftPopAsync("test-list"); + var result = await redisFacade.ListLeftPopAsync("test-list", CancellationToken); result.Should().Be("test-item"); } @@ -208,13 +208,13 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); // Establish connection and set up test data - await redisFacade.SetInHash("test-hash", "test-field", "test-value"); + await redisFacade.SetInHash("test-hash", "test-field", "test-value", TimeSpan.FromMinutes(1), CancellationToken); portForwarder.EnterKillNewAndExistingConnectionsMode(); portForwarder.ReturnToNormalMode(); // No delay here - should retry and succeed - var exists = await redisFacade.HashContainsKey("test-hash", "test-field"); + var exists = await redisFacade.HashContainsKey("test-hash", "test-field", CancellationToken); exists.Should().BeTrue(); } @@ -241,7 +241,7 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndWeTryT // Give everything enough time to have a crack at trying to subscribe to messages. await Task.Delay(2000); - await redisStableConnection.PublishToChannel("bob", "MISSED"); + await redisStableConnection.PublishToChannel("bob", "MISSED", CancellationToken); // Just in case the subscriber reconnects faster than the publish call. await Task.Delay(2000); @@ -252,7 +252,7 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndWeTryT while (msgs.Count == 0) { Logger.Information("Trying again"); - await redisStableConnection.PublishToChannel("bob", "RECONNECT"); + await redisStableConnection.PublishToChannel("bob", "RECONNECT", CancellationToken); await Task.Delay(1000); } @@ -283,7 +283,7 @@ public async Task WhenTheConnectionIsNeverEstablished_AndWeTryToSubscribe_WhenTh // Give everything enough time to have a crack at trying to subscribe to messages. await Task.Delay(2000); - await redisStableConnection.PublishToChannel("bob", "MISSED"); + await redisStableConnection.PublishToChannel("bob", "MISSED", CancellationToken); // Just in case the subscriber reconnects faster than the publish call. await Task.Delay(2000); @@ -294,7 +294,7 @@ public async Task WhenTheConnectionIsNeverEstablished_AndWeTryToSubscribe_WhenTh while (msgs.Count == 0) { Logger.Information("Trying again"); - await redisStableConnection.PublishToChannel("bob", "RECONNECT"); + await redisStableConnection.PublishToChannel("bob", "RECONNECT", CancellationToken); await Task.Delay(1000); } @@ -323,22 +323,22 @@ public async Task WhenSubscribedAndTheConnectionGoesDown_WhenTheConnectionIsRest }, CancellationToken); // Check both sides can publish. - await redisViaPortForwarder.PublishToChannel("bob", "hello unstable"); - await redisStableConnection.PublishToChannel("bob", "hello stable"); + await redisViaPortForwarder.PublishToChannel("bob", "hello unstable", CancellationToken); + await redisStableConnection.PublishToChannel("bob", "hello stable", CancellationToken); await Task.Delay(1000); // TODO better msgs.Should().BeEquivalentTo("hello unstable", "hello stable"); portForwarder.EnterKillNewAndExistingConnectionsMode(); // The stable connection should still be able to publish to redis. // But the subscriber on the unstable connection will not got the message. - await redisStableConnection.PublishToChannel("bob", "MISSED"); + await redisStableConnection.PublishToChannel("bob", "MISSED", CancellationToken); await Task.Delay(1111); portForwarder.ReturnToNormalMode(); while (msgs.Count <= 2) { Logger.Information("Trying again"); - await redisStableConnection.PublishToChannel("bob", "RECONNECT"); + await redisStableConnection.PublishToChannel("bob", "RECONNECT", CancellationToken); await Task.Delay(1000); } diff --git a/source/Halibut/Queue/Redis/HalibutRedisTransport.cs b/source/Halibut/Queue/Redis/HalibutRedisTransport.cs index 1c86f5321..067434d5e 100644 --- a/source/Halibut/Queue/Redis/HalibutRedisTransport.cs +++ b/source/Halibut/Queue/Redis/HalibutRedisTransport.cs @@ -55,7 +55,7 @@ public async Task PulseRequestPushedToEndpoint(Uri endpoint, CancellationToken c { var channelName = RequestMessagesPulseChannelName(endpoint); string emptyJson = "{}"; // Maybe we will actually want to share data in the future, empty json means we can add stuff later. - await facade.PublishToChannel(channelName, emptyJson); + await facade.PublishToChannel(channelName, emptyJson, cancellationToken); } // Request IDs list @@ -67,14 +67,14 @@ static string KeyForNextRequestGuidInListForEndpoint(Uri endpoint) public async Task PushRequestGuidOnToQueue(Uri endpoint, Guid guid, CancellationToken cancellationToken) { - // TODO should we overcomplicate this with json? - // TODO TTL - await facade.ListRightPushAsync(KeyForNextRequestGuidInListForEndpoint(endpoint), guid.ToString()); + // TTL is high since it applies to all GUIDs in the queue. + var ttlForAllRequestsGuidsInList = TimeSpan.FromDays(1); + await facade.ListRightPushAsync(KeyForNextRequestGuidInListForEndpoint(endpoint), guid.ToString(), ttlForAllRequestsGuidsInList, cancellationToken); } public async Task TryPopNextRequestGuid(Uri endpoint, CancellationToken cancellationToken) { - var result = await facade.ListLeftPopAsync(KeyForNextRequestGuidInListForEndpoint(endpoint)); + var result = await facade.ListLeftPopAsync(KeyForNextRequestGuidInListForEndpoint(endpoint), cancellationToken); return result.ToGuid(); } @@ -87,21 +87,23 @@ static string RequestMessageKey(Uri endpoint, Guid requestId) static string RequestField = "RequestField"; - public async Task PutRequest(Uri endpoint, Guid requestId, string payload, CancellationToken cancellationToken) + public async Task PutRequest(Uri endpoint, Guid requestId, string payload, TimeSpan requestPickupTimeout, CancellationToken cancellationToken) { var redisQueueItem = new RedisHalibutQueueItem2(requestId, payload); var serialisedQueueItem = JsonConvert.SerializeObject(redisQueueItem); var requestKey = RequestMessageKey(endpoint, requestId); + + var ttl = requestPickupTimeout + TimeSpan.FromMinutes(2); - await facade.SetInHash(requestKey, RequestField, serialisedQueueItem); + await facade.SetInHash(requestKey, RequestField, serialisedQueueItem, ttl, cancellationToken); } public async Task TryGetAndRemoveRequest(Uri endpoint, Guid requestId, CancellationToken cancellationToken) { var requestKey = RequestMessageKey(endpoint, requestId); - var requestMessage = await facade.TryGetAndDeleteFromHash(requestKey, RequestField); + var requestMessage = await facade.TryGetAndDeleteFromHash(requestKey, RequestField, cancellationToken); if (requestMessage == null) return null; var redisQueueItem = JsonConvert.DeserializeObject(requestMessage); @@ -113,7 +115,7 @@ public async Task PutRequest(Uri endpoint, Guid requestId, string payload, Cance public async Task IsRequestStillOnQueue(Uri endpoint, Guid requestId, CancellationToken cancellationToken) { var requestKey = RequestMessageKey(endpoint, requestId); - return await facade.HashContainsKey(requestKey, RequestField); + return await facade.HashContainsKey(requestKey, RequestField, cancellationToken); } // Cancellation channel @@ -137,7 +139,7 @@ public async Task SubscribeToRequestCancellation(Uri endpoint, public async Task PublishCancellation(Uri endpoint, Guid requestId, CancellationToken cancellationToken) { var channelName = RequestCancelledChannel(endpoint, requestId); - await facade.PublishToChannel(channelName, "{}"); + await facade.PublishToChannel(channelName, "{}", cancellationToken); } public string RequestCancelledMarkerKey(Uri endpoint, Guid requestId) @@ -182,7 +184,7 @@ public async Task SubscribeToNodeHeartBeatChannel( public async Task SendHeartBeatFromNodeProcessingTheRequest(Uri endpoint, Guid requestId, HalibutQueueNodeSendingPulses nodeSendingPulsesType, CancellationToken cancellationToken) { var channelName = NodeHeartBeatChannel(endpoint, requestId, nodeSendingPulsesType); - await facade.PublishToChannel(channelName, "{}"); + await facade.PublishToChannel(channelName, "{}", cancellationToken); } // Backward compatibility methods (defaulting to Receiver for existing code) @@ -202,16 +204,16 @@ public async Task SendHeartBeatFromNodeProcessingTheRequest(Uri endpoint, Guid r // Generic methods for watching for any string value being set - string GenericChannelName(string thingToWatchFor, Uri endpoint, Guid identifier) + string ResponseAvailableChannel(Uri endpoint, Guid identifier) { - return $"{Namespace}::GenericChannel::{thingToWatchFor}::{endpoint}::{identifier}"; + return $"{Namespace}::ResponseAvailableChannel::{endpoint}::{identifier}"; } - public async Task SubscribeToGenericNotification(string thingToWatchFor, Uri endpoint, Guid identifier, + public async Task SubscribeToResponseChannel(Uri endpoint, Guid identifier, Func onValueReceived, CancellationToken cancellationToken) { - var channelName = GenericChannelName(thingToWatchFor, endpoint, identifier); + var channelName = ResponseAvailableChannel(endpoint, identifier); return await facade.SubscribeToChannel(channelName, async foo => { string? value = foo.Message; @@ -219,33 +221,33 @@ public async Task SubscribeToGenericNotification(string thingT }, cancellationToken); } - public async Task PublishThatValueIsAvailable(string thingToWatchFor, Uri endpoint, Guid identifier, string value, CancellationToken cancellationToken) + public async Task PublishThatResponseIsAvailable(Uri endpoint, Guid identifier, string value, CancellationToken cancellationToken) { - var channelName = GenericChannelName(thingToWatchFor, endpoint, identifier); - await facade.PublishToChannel(channelName, value); + var channelName = ResponseAvailableChannel(endpoint, identifier); + await facade.PublishToChannel(channelName, value, cancellationToken); } - string GenericMarkerKey(string thingToWatchFor, Uri endpoint, Guid identifier) + string ResponseMarkerKey(Uri endpoint, Guid identifier) { - return $"{Namespace}::GenericMarker::{thingToWatchFor}::{endpoint}::{identifier}"; + return $"{Namespace}::ResponseMarkerKey::{endpoint}::{identifier}"; } - public async Task SendValue(string thingToWatchFor, Uri endpoint, Guid identifier, string value, TimeSpan ttl, CancellationToken cancellationToken) + public async Task MarkThatResponseIsSet(Uri endpoint, Guid identifier, string value, TimeSpan ttl, CancellationToken cancellationToken) { - var key = GenericMarkerKey(thingToWatchFor, endpoint, identifier); + var key = ResponseMarkerKey(endpoint, identifier); await facade.SetString(key, value, ttl, cancellationToken); } - public async Task GetGenericMarker(string thingToWatchFor, Uri endpoint, Guid identifier, CancellationToken cancellationToken) + public async Task GetGenericMarker(Uri endpoint, Guid identifier, CancellationToken cancellationToken) { - var key = GenericMarkerKey(thingToWatchFor, endpoint, identifier); + var key = ResponseMarkerKey(endpoint, identifier); return await facade.GetString(key, cancellationToken); } - public async Task DeleteGenericMarker(string thingToWatchFor, Uri endpoint, Guid identifier, CancellationToken cancellationToken) + public async Task DeleteResponseMarker(Uri endpoint, Guid identifier, CancellationToken cancellationToken) { - var key = GenericMarkerKey(thingToWatchFor, endpoint, identifier); - return await facade.DeleteString(key); + var key = ResponseMarkerKey(endpoint, identifier); + return await facade.DeleteString(key, cancellationToken); } } diff --git a/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs b/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs index 99b1cdc07..8f6edc9ea 100644 --- a/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs +++ b/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs @@ -97,7 +97,7 @@ private async Task SendPulsesWhileProcessingRequest(TimeSpan defaultDelayBetween public static async Task WatchThatNodeProcessingTheRequestIsStillAlive( Uri endpoint, RequestMessage request, - PendingRequest pending, + RedisPendingRequest redisPending, HalibutRedisTransport halibutRedisTransport, TimeSpan timeBetweenCheckingIfRequestWasCollected, ILog log, @@ -106,11 +106,11 @@ public static async Task WatchThatNodeProces { log = log.ForContext(); // Once the pending's CT has been cancelled we no longer care to keep observing - await using var cts = new CancelOnDisposeCancellationToken(watchCancellationToken, pending.PendingRequestCancellationToken); + await using var cts = new CancelOnDisposeCancellationToken(watchCancellationToken, redisPending.PendingRequestCancellationToken); try { // TODO: test this is indeed called first. - await WaitForRequestToBeCollected(endpoint, request, pending, halibutRedisTransport, timeBetweenCheckingIfRequestWasCollected, log, cts.Token); + await WaitForRequestToBeCollected(endpoint, request, redisPending, halibutRedisTransport, timeBetweenCheckingIfRequestWasCollected, log, cts.Token); return await WatchForPulsesFromNode(endpoint, request.ActivityId, halibutRedisTransport, log, maxTimeBetweenHeartBeetsBeforeProcessingNodeIsAssumedToBeOffline, HalibutQueueNodeSendingPulses.Receiver, cts.Token); } @@ -189,7 +189,7 @@ private static async Task WatchForPulsesFrom } } - static async Task WaitForRequestToBeCollected(Uri endpoint, RequestMessage request, PendingRequest pending, HalibutRedisTransport halibutRedisTransport, + static async Task WaitForRequestToBeCollected(Uri endpoint, RequestMessage request, RedisPendingRequest redisPending, HalibutRedisTransport halibutRedisTransport, TimeSpan timeBetweenCheckingIfRequestWasCollected, ILog log, CancellationToken cancellationToken) { @@ -214,7 +214,7 @@ static async Task WaitForRequestToBeCollected(Uri endpoint, RequestMessage reque { asyncManualResetEvent.Reset(); // Has something else determined the request was collected? - if(pending.HasRequestBeenMarkedAsCollected) + if(redisPending.HasRequestBeenMarkedAsCollected) { log.Write(EventType.Diagnostic, "Request {0} has been marked as collected", request.ActivityId); return; @@ -226,7 +226,7 @@ static async Task WaitForRequestToBeCollected(Uri endpoint, RequestMessage reque if(!requestIsStillOnQueue) { log.Write(EventType.Diagnostic, "Request {0} is no longer on queue", request.ActivityId); - await pending.RequestHasBeenCollectedAndWillBeTransferred(); + await redisPending.RequestHasBeenCollectedAndWillBeTransferred(); return; } } @@ -240,7 +240,7 @@ await Try.IgnoringError(async () => await Task.WhenAny( Task.Delay(timeBetweenCheckingIfRequestWasCollected, cancellationToken), asyncManualResetEvent.WaitAsync(cancellationToken), - pending.WaitForRequestToBeMarkedAsCollected(cancellationToken)); + redisPending.WaitForRequestToBeMarkedAsCollected(cancellationToken)); }); } diff --git a/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs b/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs index d7d00b979..fcd535a39 100644 --- a/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs +++ b/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs @@ -24,7 +24,6 @@ namespace Halibut.Queue.Redis public class PollAndSubscribeToResponse : IAsyncDisposable { public static async Task TrySendMessage( - string messageTypeName, // TODO drop this, it is always response. HalibutRedisTransport halibutRedisTransport, Uri endpoint, Guid activityId, @@ -32,35 +31,35 @@ public static async Task TrySendMessage( TimeSpan ttl, ILog log) { - log.Write(EventType.Diagnostic, "Attempting to set {0} for - Endpoint: {1}, ActivityId: {2}", messageTypeName, endpoint, activityId); + log.Write(EventType.Diagnostic, "Attempting to set response for - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); await using var cts = new CancelOnDisposeCancellationToken(); cts.CancelAfter(TimeSpan.FromMinutes(2)); // Best efforts. try { - log.Write(EventType.Diagnostic, "Marking {0} as set - Endpoint: {1}, ActivityId: {2}", messageTypeName, endpoint, activityId); - await halibutRedisTransport.SendValue(messageTypeName, endpoint, activityId, value, ttl, cts.Token); + log.Write(EventType.Diagnostic, "Marking response as set - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); + await halibutRedisTransport.MarkThatResponseIsSet(endpoint, activityId, value, ttl, cts.Token); - log.Write(EventType.Diagnostic, "Publishing {0} notification - Endpoint: {1}, ActivityId: {2}", messageTypeName, endpoint, activityId); - await halibutRedisTransport.PublishThatValueIsAvailable(messageTypeName, endpoint, activityId, value, cts.Token); + log.Write(EventType.Diagnostic, "Publishing response notification - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); + await halibutRedisTransport.PublishThatResponseIsAvailable(endpoint, activityId, value, cts.Token); - log.Write(EventType.Diagnostic, "Successfully set {0} - Endpoint: {1}, ActivityId: {2}", messageTypeName, endpoint, activityId); + log.Write(EventType.Diagnostic, "Successfully set response - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); } catch (OperationCanceledException ex) { - log.Write(EventType.Error, "Set {0} operation timed out after 2 minutes - Endpoint: {1}, ActivityId: {2}, Error: {3}", messageTypeName, endpoint, activityId, ex.Message); + log.Write(EventType.Error, "Set response operation timed out after 2 minutes - Endpoint: {0}, ActivityId: {1}, Error: {2}", endpoint, activityId, ex.Message); } catch (Exception ex) { - log.Write(EventType.Error, "Failed to set {0} - Endpoint: {1}, ActivityId: {2}, Error: {3}", messageTypeName, endpoint, activityId, ex.Message); + log.Write(EventType.Error, "Failed to set response - Endpoint: {0}, ActivityId: {1}, Error: {2}", endpoint, activityId, ex.Message); } } readonly CancelOnDisposeCancellationToken watcherToken; readonly ILog log; - readonly string messageTypeName; + readonly HalibutRedisTransport halibutRedisTransport; readonly Uri endpoint; readonly Guid activityId; @@ -70,10 +69,10 @@ public static async Task TrySendMessage( public Task ResultTask => message.Task; - public PollAndSubscribeToResponse(string messageTypeName, Uri endpoint, Guid activityId, HalibutRedisTransport halibutRedisTransport, ILog log) + public PollAndSubscribeToResponse(Uri endpoint, Guid activityId, HalibutRedisTransport halibutRedisTransport, ILog log) { this.log = log.ForContext(); - this.messageTypeName = messageTypeName; + this.endpoint = endpoint; this.activityId = activityId; this.halibutRedisTransport = halibutRedisTransport; @@ -82,11 +81,11 @@ public PollAndSubscribeToResponse(string messageTypeName, Uri endpoint, Guid act TimeSpan.FromSeconds(15), // Increment: 15s TimeSpan.FromMinutes(2) // Maximum delay: 2 minutes ); - this.log.Write(EventType.Diagnostic, "Starting to watch for {0} - Endpoint: {1}, ActivityId: {2}", messageTypeName, endpoint, activityId); + this.log.Write(EventType.Diagnostic, "Starting to watch for response - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); watcherToken = new CancelOnDisposeCancellationToken(); var token = watcherToken.Token; - var _ = Task.Run(async () => await WatchAndWaitForMessage(token)); + var _ = Task.Run(async () => await WaitForResponse(token)); } readonly SemaphoreSlim trySetResultSemaphore = new SemaphoreSlim(1, 1); @@ -105,7 +104,7 @@ public async Task TrySetResultAndRemoveValueFromRedis(string value, Cancellation try { - await halibutRedisTransport.DeleteGenericMarker(messageTypeName, endpoint, activityId, cancellationToken); + await halibutRedisTransport.DeleteResponseMarker(endpoint, activityId, cancellationToken); } catch (Exception) { @@ -113,34 +112,31 @@ public async Task TrySetResultAndRemoveValueFromRedis(string value, Cancellation } } - async Task WatchAndWaitForMessage(CancellationToken token) + async Task WaitForResponse(CancellationToken token) { try { - log.Write(EventType.Diagnostic, "Subscribing to {0} notifications - Endpoint: {1}, ActivityId: {2}", messageTypeName, endpoint, activityId); + log.Write(EventType.Diagnostic, "Subscribing to response notifications - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); - await using var _ = await halibutRedisTransport.SubscribeToGenericNotification(messageTypeName, endpoint, activityId, + await using var _ = await halibutRedisTransport.SubscribeToResponseChannel(endpoint, activityId, async _ => { - log.Write(EventType.Diagnostic, "Received {0} notification via subscription - Endpoint: {1}, ActivityId: {2}", messageTypeName, endpoint, activityId); + log.Write(EventType.Diagnostic, "Received response notification via subscription - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); - var value = await halibutRedisTransport.GetGenericMarker(messageTypeName, endpoint, activityId, token); + var value = await halibutRedisTransport.GetGenericMarker(endpoint, activityId, token); if (value != null) { await TrySetResultAndRemoveValueFromRedis(value, token); } - log.Write(EventType.Diagnostic, "Cancelling polling loop for {0} - Endpoint: {1}, ActivityId: {2}", messageTypeName, endpoint, activityId); + log.Write(EventType.Diagnostic, "Cancelling polling loop for response - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); await Try.IgnoringError(async () => await watcherToken.CancelAsync()); - - log.Write(EventType.Diagnostic, "Is token cancelled: {0} - Endpoint: {1}, ActivityId: {2} {3}", messageTypeName, endpoint, activityId, token.IsCancellationRequested); - }, token); - log.Write(EventType.Diagnostic, "Starting polling loop for {0} - Endpoint: {1}, ActivityId: {2}", messageTypeName, endpoint, activityId); + log.Write(EventType.Diagnostic, "Starting polling loop for response - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); // Also poll to see if the value is set since we can miss // the publication. @@ -148,11 +144,10 @@ async Task WatchAndWaitForMessage(CancellationToken token) { try { - var value = await halibutRedisTransport.GetGenericMarker(messageTypeName, endpoint, activityId, token); + var value = await halibutRedisTransport.GetGenericMarker(endpoint, activityId, token); if (value != null) { - log.Write(EventType.Diagnostic, "{0} detected via polling - Endpoint: {1}, ActivityId: {2}", messageTypeName, endpoint, activityId); - pollBackoffStrategy.Success(); // Reset backoff strategy on successful retrieval + log.Write(EventType.Diagnostic, "Response detected via polling - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); await TrySetResultAndRemoveValueFromRedis(value, token); await watcherToken.CancelAsync(); break; @@ -160,30 +155,30 @@ async Task WatchAndWaitForMessage(CancellationToken token) } catch (Exception ex) { - log.Write(EventType.Diagnostic, "Error while polling for {0} - Endpoint: {1}, ActivityId: {2}, Error: {3}", messageTypeName, endpoint, activityId, ex.Message); + log.Write(EventType.Diagnostic, "Error while polling for response - Endpoint: {0}, ActivityId: {1}, Error: {2}", endpoint, activityId, ex.Message); } pollBackoffStrategy.Try(); var delay = pollBackoffStrategy.GetSleepPeriod(); - log.Write(EventType.Diagnostic, "Waiting {0} seconds before next poll for {1} - Endpoint: {2}, ActivityId: {3}", delay.TotalSeconds, messageTypeName, endpoint, activityId); + log.Write(EventType.Diagnostic, "Waiting {0} seconds before next poll for response - Endpoint: {1}, ActivityId: {2}", delay.TotalSeconds, endpoint, activityId); await Try.IgnoringError(async () => await Task.Delay(delay, token)); - log.Write(EventType.Diagnostic, "Done waiting going around the loop {0} - Endpoint: {1}, ActivityId: {2}", messageTypeName, endpoint, activityId); + log.Write(EventType.Diagnostic, "Done waiting going around the loop response - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); } - log.Write(EventType.Diagnostic, "Exiting watch loop for {0} - Endpoint: {1}, ActivityId: {2}", messageTypeName, endpoint, activityId); + log.Write(EventType.Diagnostic, "Exiting watch loop for response - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); } catch (Exception ex) { if (!token.IsCancellationRequested) { - log.Write(EventType.Error, "Unexpected error in {0} watcher - Endpoint: {1}, ActivityId: {2}, Error: {3}", messageTypeName, endpoint, activityId, ex.Message); + log.Write(EventType.Error, "Unexpected error in response watcher - Endpoint: {0}, ActivityId: {1}, Error: {2}", endpoint, activityId, ex.Message); } } } public async ValueTask DisposeAsync() { - log.Write(EventType.Diagnostic, "Disposing GenericWatcher for {0} - Endpoint: {1}, ActivityId: {2}", messageTypeName, endpoint, activityId); + log.Write(EventType.Diagnostic, "Disposing GenericWatcher for response - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); await Try.IgnoringError(async () => await watcherToken.CancelAsync()); //Try.IgnoringError(() => watcherTokenSource.Dispose()); @@ -191,7 +186,7 @@ public async ValueTask DisposeAsync() // If the message task is not yet complete, then complete if now with null since we have nothing for it. Try.IgnoringError(() => message.TrySetCanceled()); - log.Write(EventType.Diagnostic, "Disposed GenericWatcher for {0} - Endpoint: {1}, ActivityId: {2}", messageTypeName, endpoint, activityId); + log.Write(EventType.Diagnostic, "Disposed GenericWatcher for response - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); } } } \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/RedisFacade.cs b/source/Halibut/Queue/Redis/RedisFacade.cs index 6294a296f..24c5657cf 100644 --- a/source/Halibut/Queue/Redis/RedisFacade.cs +++ b/source/Halibut/Queue/Redis/RedisFacade.cs @@ -13,7 +13,6 @@ // limitations under the License. using System; -using System.Collections.Concurrent; using System.Diagnostics; using System.Threading; using System.Threading.Tasks; @@ -106,26 +105,8 @@ public class ShouldAbandonAndReconnectHelper public Task WaitUntilShouldReSubscribeTask => connectionInError.Task; } - - private ShouldAbandonAndReconnectHelper ShouldAbandonAndReconnect = new ShouldAbandonAndReconnectHelper(); - private readonly object errorOccuredLock = new object(); - - private ShouldAbandonAndReconnectHelper ConnectionInErrorHelperProvider() => ShouldAbandonAndReconnect; - - - private void AdviseThatClientsShouldStartReconnecting() - { - lock (errorOccuredLock) - { - var shouldAbandonAndReconnect = this.ShouldAbandonAndReconnect; - this.ShouldAbandonAndReconnect = new (); - shouldAbandonAndReconnect.SetReconnectionIsAdvised(); - } - } private void OnConnectionFailed(object? sender, ConnectionFailedEventArgs e) { - AdviseThatClientsShouldStartReconnecting(); - var message = $"Redis connection failed - EndPoint: {e.EndPoint}, Failure: {e.FailureType}, Exception: {e.Exception?.Message}"; log?.Write(EventType.Error, message); } @@ -224,12 +205,9 @@ public async ValueTask DisposeAsync() internal int TotalSubscribers = 0; - - internal ConcurrentDictionary subs = new ConcurrentDictionary(); - + public async Task SubscribeToChannel(string channelName, Func onMessage, CancellationToken cancellationToken) { - channelName = "channel:" + keyPrefix + ":" + channelName; while (true) { @@ -240,13 +218,9 @@ public async Task SubscribeToChannel(string channelName, Func< var channel = await Connection.GetSubscriber() .SubscribeAsync(new RedisChannel(channelName, RedisChannel.PatternMode.Literal)); - var someGuid = channelName + "__" + Guid.NewGuid().ToString(); - subs.TryAdd(someGuid, false); - var disposable = new FuncAsyncDisposable(async () => { Interlocked.Decrement(ref TotalSubscribers); - subs.TryRemove(someGuid, out var _); await channel.UnsubscribeAsync(); }); @@ -264,59 +238,50 @@ public async Task SubscribeToChannel(string channelName, Func< } return disposable; - } - catch + catch (Exception ex) { - // TODO: Get AI to log. + log?.WriteException(EventType.Diagnostic, "Failed to subscribe to Redis channel {0}, retrying in 2 seconds", ex, channelName); await Try.IgnoringError(async () => await Task.Delay(2000, cancellationToken)); } } } - public async Task PublishToChannel(string channelName, string payload) + public async Task PublishToChannel(string channelName, string payload, CancellationToken cancellationToken) { channelName = "channel:" + keyPrefix + ":" + channelName; await ExecuteWithRetry(async () => { var subscriber = Connection.GetSubscriber(); await subscriber.PublishAsync(new RedisChannel(channelName, RedisChannel.PatternMode.Literal), payload); - }, CancellationToken.None); + }, cancellationToken); } - public async Task SetInHash(string key, string field, string payload) + public async Task SetInHash(string key, string field, string payload, TimeSpan ttl, CancellationToken cancellationToken) { key = "hash:" + keyPrefix + ":" + key; - // TODO: TTL - // TODO ever call needs to respect the cancellation token - var ttl = new TimeSpan(9, 9, 9); - // Retry each operation independently await ExecuteWithRetry(async () => { var database = Connection.GetDatabase(); await database.HashSetAsync(key, new RedisValue(field), new RedisValue(payload)); - }, CancellationToken.None); - - await ExecuteWithRetry(async () => - { - var database = Connection.GetDatabase(); - await database.KeyExpireAsync(key, ttl); - }, CancellationToken.None); + }, cancellationToken); + + await SetTtlForKeyRaw(key, ttl, cancellationToken); } - public async Task HashContainsKey(string key, string field) + public async Task HashContainsKey(string key, string field, CancellationToken cancellationToken) { key = "hash:" + keyPrefix + ":" + key; return await ExecuteWithRetry(async () => { var database = Connection.GetDatabase(); return await database.HashExistsAsync(key, new RedisValue(field)); - }, CancellationToken.None); + }, cancellationToken); } - public async Task TryGetAndDeleteFromHash(string key, string field) + public async Task TryGetAndDeleteFromHash(string key, string field, CancellationToken cancellationToken) { key = "hash:" + keyPrefix + ":" + key; @@ -325,15 +290,15 @@ public async Task HashContainsKey(string key, string field) { var database = Connection.GetDatabase(); return await database.HashGetAsync(key, new RedisValue(field)); - }, CancellationToken.None); + }, cancellationToken); // TODO: If we retry this is not idempotent. - // TODO: Test + // TODO: This needs to be tested in RedisPendingRequestsQueueFixture var res = await ExecuteWithRetry(async () => { var database = Connection.GetDatabase(); return await database.KeyDeleteAsync(key); - }, CancellationToken.None); + }, cancellationToken); if (!res) { @@ -343,18 +308,18 @@ public async Task HashContainsKey(string key, string field) return (string?)value; } - public async Task ListRightPushAsync(string key, string payload) + public async Task ListRightPushAsync(string key, string payload, TimeSpan ttlForAllInList, CancellationToken cancellationToken) { key = "list:" + keyPrefix + ":" + key; await ExecuteWithRetry(async () => { var database = Connection.GetDatabase(); - // TODO can we set TTL on this? await database.ListRightPushAsync(key, payload); - }, CancellationToken.None); - } + }, cancellationToken); + + await SetTtlForKeyRaw(key, ttlForAllInList, cancellationToken); } - public async Task ListLeftPopAsync(string key) + public async Task ListLeftPopAsync(string key, CancellationToken cancellationToken) { key = "list:" + keyPrefix + ":" + key; return await ExecuteWithRetry(async () => @@ -367,12 +332,11 @@ await ExecuteWithRetry(async () => } return (string?)value; - }, CancellationToken.None); + }, cancellationToken); } public async Task SetString(string key, string value, TimeSpan ttl, CancellationToken cancellationToken) { - // TODO TTL key = ToStringKey(key); await ExecuteWithRetry(async () => { @@ -380,7 +344,6 @@ await ExecuteWithRetry(async () => await database.StringSetAsync(key, value); }, cancellationToken); - // TODO unit test. await SetTtlForKeyRaw(key, ttl, cancellationToken); } @@ -414,15 +377,14 @@ await ExecuteWithRetry(async () => }, cancellationToken); } - public async Task DeleteString(string key) + public async Task DeleteString(string key, CancellationToken cancellationToken) { - key = "string:" + keyPrefix + ":" + key; + key = ToStringKey(key); return await ExecuteWithRetry(async () => { var database = Connection.GetDatabase(); return await database.KeyDeleteAsync(key); - }, CancellationToken.None); + }, cancellationToken); } } - } \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index 123dbcf5a..963d2c733 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -93,7 +93,7 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can var cancellationToken = cts.Token; // TODO RedisConnectionException can be raised out of here, what should the queue do? // TODO it must raise an exception that supports being retried. - using var pending = new PendingRequest(request, log); + using var pending = new RedisPendingRequest(request, log); // TODO: What if this payload was gigantic // TODO: Do we need to encrypt this? @@ -107,7 +107,7 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can { await using var senderPulse = new NodeHeartBeatSender(endpoint, request.ActivityId, halibutRedisTransport, log, HalibutQueueNodeSendingPulses.Sender, DelayBetweenHeartBeatsForRequestSender); // Make the request available before we tell people it is available. - await halibutRedisTransport.PutRequest(endpoint, request.ActivityId, payload, cancellationToken); + await halibutRedisTransport.PutRequest(endpoint, request.ActivityId, payload, request.Destination.PollingRequestQueueTimeout, cancellationToken); await halibutRedisTransport.PushRequestGuidOnToQueue(endpoint, request.ActivityId, cancellationToken); await halibutRedisTransport.PulseRequestPushedToEndpoint(endpoint, cancellationToken); Interlocked.Increment(ref numberOfInFlightRequestsThatHaveReachedTheStageOfBeingReadyForCollection); @@ -136,26 +136,29 @@ await pending.WaitUntilComplete( { // Make an attempt to ensure the request is removed from redis. var background = Task.Run(async () => await Try.IgnoringError(async () => await tryClearRequestFromQueueAtMostOnce.Task)); - var backgroundCancellation = Task.Run(async () => await SendCancellationIfRequestWasCancelled(request, pending)); + var backgroundCancellation = Task.Run(async () => + { + if(requestCancellationToken.IsCancellationRequested) await SendCancellationIfRequestWasCancelled(request, pending); + }); } return pending.Response!; } - async Task SendCancellationIfRequestWasCancelled(RequestMessage request, PendingRequest pending) + async Task SendCancellationIfRequestWasCancelled(RequestMessage request, RedisPendingRequest redisPending) { - if (pending.PendingRequestCancellationToken.IsCancellationRequested) + if (redisPending.PendingRequestCancellationToken.IsCancellationRequested) { - // TODO log + log.Write(EventType.Diagnostic, "Request {0} was cancelled, sending cancellation to endpoint {1}", request.ActivityId, endpoint); await WatchForRequestCancellation.TrySendCancellation(halibutRedisTransport, endpoint, request, log); } else { - // TODO log + log.Write(EventType.Diagnostic, "Request {0} was not cancelled, no cancellation needed for endpoint {1}", request.ActivityId, endpoint); } } - void WatchProcessingNodeIsStillConnectedInBackground(RequestMessage request, PendingRequest pending, CancelOnDisposeCancellationToken watcherCts) + void WatchProcessingNodeIsStillConnectedInBackground(RequestMessage request, RedisPendingRequest redisPending, CancelOnDisposeCancellationToken watcherCts) { Task.Run(async () => { @@ -165,7 +168,7 @@ void WatchProcessingNodeIsStillConnectedInBackground(RequestMessage request, Pen var disconnected = await NodeHeartBeatSender.WatchThatNodeProcessingTheRequestIsStillAlive( endpoint, request, - pending, + redisPending, halibutRedisTransport, TimeBetweenCheckingIfRequestWasCollected, log, @@ -174,7 +177,7 @@ void WatchProcessingNodeIsStillConnectedInBackground(RequestMessage request, Pen if (!watcherCtsCancellationToken.IsCancellationRequested && disconnected == NodeHeartBeatSender.NodeProcessingRequestWatcherResult.NodeMayHaveDisconnected) { // TODO: if(responseWatcher.CheckForResponseNow() == ResponseNotFound) { - pending.SetResponse(ResponseMessage.FromError(request, "The node processing the request did not send a heartbeat for long enough, and so the node is now assumed to be offline.")); + redisPending.SetResponse(ResponseMessage.FromError(request, "The node processing the request did not send a heartbeat for long enough, and so the node is now assumed to be offline.")); //} } } @@ -189,7 +192,7 @@ void WatchProcessingNodeIsStillConnectedInBackground(RequestMessage request, Pen }); } - async Task TryClearRequestFromQueue(RequestMessage request, PendingRequest pending) + async Task TryClearRequestFromQueue(RequestMessage request, RedisPendingRequest redisPending) { log.Write(EventType.Diagnostic, "Attempting to clear request {0} from queue for endpoint {1}", request.ActivityId, endpoint); @@ -199,9 +202,9 @@ async Task TryClearRequestFromQueue(RequestMessage request, PendingRequest // - We could not pop it, which means it was collected. try { - if (pending.HasRequestBeenMarkedAsCollected) + if (redisPending.HasRequestBeenMarkedAsCollected) { - // TODO: log + log.Write(EventType.Diagnostic, "Request {0} has already been marked as collected, skipping queue removal for endpoint {1}", request.ActivityId, endpoint); return false; } await using var cts = new CancelOnDisposeCancellationToken(); @@ -214,7 +217,7 @@ async Task TryClearRequestFromQueue(RequestMessage request, PendingRequest } else { - await pending.RequestHasBeenCollectedAndWillBeTransferred(); + await redisPending.RequestHasBeenCollectedAndWillBeTransferred(); log.Write(EventType.Diagnostic, "Request {0} was not found in queue - it was already collected by a processing node", request.ActivityId); } } @@ -234,7 +237,7 @@ async Task SubscribeToResponse(Guid activityId, CancellationToken cancellationToken) { await Task.CompletedTask; - var sub = new PollAndSubscribeToResponse(ResponseMessageSubscriptionName, endpoint, activityId, halibutRedisTransport, log); + var sub = new PollAndSubscribeToResponse(endpoint, activityId, halibutRedisTransport, log); var _ = Task.Run(async () => { try @@ -273,7 +276,7 @@ async Task SubscribeToResponse(Guid activityId, internal TimeSpan DelayBetweenHeartBeatsForRequestProcessor { get; set; } = TimeSpan.FromSeconds(15); - internal TimeSpan TTLOfResponseMessage { get; set; } = TimeSpan.FromMinutes(5); + internal TimeSpan TTLOfResponseMessage { get; set; } = TimeSpan.FromMinutes(20); internal TimeSpan TimeBetweenCheckingIfRequestWasCollected { get; set; } = TimeSpan.FromSeconds(30); @@ -368,7 +371,7 @@ public async Task ApplyResponse(ResponseMessage response, Guid requestActivityId } var payload = await messageReaderWriter.PrepareResponse(response, cancellationToken); log.Write(EventType.MessageExchange, "Sending response message for request {0}", requestActivityId); - await PollAndSubscribeToResponse.TrySendMessage(ResponseMessageSubscriptionName, halibutRedisTransport, endpoint, requestActivityId, payload, TTLOfResponseMessage, log); + await PollAndSubscribeToResponse.TrySendMessage(halibutRedisTransport, endpoint, requestActivityId, payload, TTLOfResponseMessage, log); log.Write(EventType.MessageExchange, "Successfully applied response for request {0}", requestActivityId); } catch (Exception ex) diff --git a/source/Halibut/Queue/Redis/WatchForRedisLosingAllItsData.cs b/source/Halibut/Queue/Redis/WatchForRedisLosingAllItsData.cs index 5c48e4398..16fc90794 100644 --- a/source/Halibut/Queue/Redis/WatchForRedisLosingAllItsData.cs +++ b/source/Halibut/Queue/Redis/WatchForRedisLosingAllItsData.cs @@ -130,7 +130,7 @@ await Try.IgnoringError(async () => } log.Write(EventType.Diagnostic, "Redis data loss monitoring stopped for key {0}, cleaning up", key); - await Try.IgnoringError(async () => await redisFacade.DeleteString(key)); + await Try.IgnoringError(async () => await redisFacade.DeleteString(key, cancellationToken)); } public async ValueTask DisposeAsync() diff --git a/source/Halibut/ServiceModel/PendingRequestQueueAsync.cs b/source/Halibut/ServiceModel/PendingRequestQueueAsync.cs index d76fab3a2..ba2b190db 100644 --- a/source/Halibut/ServiceModel/PendingRequestQueueAsync.cs +++ b/source/Halibut/ServiceModel/PendingRequestQueueAsync.cs @@ -12,8 +12,8 @@ namespace Halibut.ServiceModel { public class PendingRequestQueueAsync : IPendingRequestQueue, IAsyncDisposable { - readonly List queue = new(); - readonly Dictionary inProgress = new(); + readonly List queue = new(); + readonly Dictionary inProgress = new(); readonly SemaphoreSlim queueLock = new(1, 1); readonly AsyncManualResetEvent itemAddedToQueue = new(false); readonly ILog log; @@ -40,7 +40,7 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can using var cancellationTokenSource = CancellationTokenSource.CreateLinkedTokenSource(cancellationToken, this.entireQueueCancellationTokenSource.Token); cancellationToken = cancellationTokenSource.Token; - using var pending = new PendingRequest(request, log); + using var pending = new RedisPendingRequest(request, log); try { @@ -116,7 +116,7 @@ public int Count } } - async Task DequeueNextAsync(TimeSpan timeout, CancellationToken cancellationToken) + async Task DequeueNextAsync(TimeSpan timeout, CancellationToken cancellationToken) { var first = await TakeFirst(cancellationToken); if (first != null || timeout <= TimeSpan.Zero) @@ -139,7 +139,7 @@ await Task.WhenAny( return await TakeFirst(cancellationToken); } - async Task TakeFirst(CancellationToken cancellationToken) + async Task TakeFirst(CancellationToken cancellationToken) { using (await queueLock.LockAsync(cancellationToken)) { diff --git a/source/Halibut/ServiceModel/PendingRequestV2.cs b/source/Halibut/ServiceModel/PendingRequestV2.cs index 70df71b1f..f4dcc261f 100644 --- a/source/Halibut/ServiceModel/PendingRequestV2.cs +++ b/source/Halibut/ServiceModel/PendingRequestV2.cs @@ -24,21 +24,20 @@ namespace Halibut.ServiceModel { - public class PendingRequest : IDisposable + public class RedisPendingRequest : IDisposable { readonly RequestMessage request; readonly ILog log; readonly AsyncManualResetEvent responseWaiter = new(false); readonly SemaphoreSlim transferLock = new(1, 1); - //bool transferBegun; - AsyncManualResetEvent requestCollected = new(false); + readonly AsyncManualResetEvent requestCollected = new(false); readonly CancellationTokenSource pendingRequestCancellationTokenSource; ResponseMessage? response; - public PendingRequest(RequestMessage request, ILog log) + public RedisPendingRequest(RequestMessage request, ILog log) { this.request = request; - this.log = log.ForContext(); + this.log = log.ForContext(); pendingRequestCancellationTokenSource = new CancellationTokenSource(); PendingRequestCancellationToken = pendingRequestCancellationTokenSource.Token; From d2e2e9e41d9ce66335ac3d8a89c15a00f1b60ec0 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Thu, 14 Aug 2025 09:55:30 +1000 Subject: [PATCH 045/137] Redis Facade unit tests --- .../Queue/Redis/RedisFacadeFixture.cs | 272 +++++++++++++++--- source/Halibut/Queue/Redis/RedisFacade.cs | 2 +- 2 files changed, 241 insertions(+), 33 deletions(-) diff --git a/source/Halibut.Tests/Queue/Redis/RedisFacadeFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisFacadeFixture.cs index 9f9e44e07..f340c1a10 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisFacadeFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisFacadeFixture.cs @@ -4,6 +4,7 @@ using FluentAssertions; using Halibut.Logging; using Halibut.Queue.Redis; +using Halibut.Tests.Support; using Halibut.Tests.Support.Logging; using Halibut.Util.AsyncEx; using NUnit.Framework; @@ -12,7 +13,6 @@ namespace Halibut.Tests.Queue.Redis { public class RedisFacadeFixture : BaseTest { - // AI generated :S private static RedisFacade CreateRedisFacade() => new("localhost", Guid.NewGuid().ToString(), new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix("")); [Test] @@ -341,32 +341,6 @@ public async Task KeyPrefixing_ShouldIsolateDataBetweenDifferentPrefixes() retrievedValue2.Should().Be(value2); } - // [Test] - // public void Dispose_ShouldNotThrowException() - // { - // // Arrange - // var redisFacade = CreateRedisFacade(); - // - // // Act & Assert - // Action act = () => redisFacade.Dispose(); - // act.Should().NotThrow(); - // } - // - // [Test] - // public void Dispose_CalledMultipleTimes_ShouldNotThrowException() - // { - // // Arrange - // var redisFacade = CreateRedisFacade(); - // - // // Act & Assert - // Action act = () => - // { - // redisFacade.Dispose(); - // redisFacade.Dispose(); // Second call - // }; - // act.Should().NotThrow(); - // } - [Test] public async Task SetInHash_WithTTL_ShouldExpireAfterSpecifiedTime() { @@ -376,17 +350,251 @@ public async Task SetInHash_WithTTL_ShouldExpireAfterSpecifiedTime() var field = "test-field"; var payload = "test-payload"; - // Act - Set a value in hash (it has a TTL of 9:9:9 according to the implementation) - await redisFacade.SetInHash(key, field, payload, TimeSpan.FromMinutes(1), CancellationToken); + // Act - Set a value in hash with short TTL that we can actually test + await redisFacade.SetInHash(key, field, payload, TimeSpan.FromMinutes(3), CancellationToken); - // Immediately try to get the value - should exist + // Immediately verify it exists + var immediateExists = await redisFacade.HashContainsKey(key, field, CancellationToken); + immediateExists.Should().BeTrue(); + + // Also verify we can retrieve the value immediately var immediateValue = await redisFacade.TryGetAndDeleteFromHash(key, field, CancellationToken); + immediateValue.Should().Be(payload); + + // Set the value again to test expiration (since TryGetAndDeleteFromHash removes it) + await redisFacade.SetInHash(key, field, payload, TimeSpan.FromMilliseconds(3), CancellationToken); + + // Assert - Should eventually expire + await ShouldEventually.Eventually(async () => + { + var exists = await redisFacade.HashContainsKey(key, field, CancellationToken); + exists.Should().BeFalse("the hash key should expire after TTL"); + }, TimeSpan.FromSeconds(5), CancellationToken); + + // Verify TryGetAndDeleteFromHash also returns null for expired key + var expiredValue = await redisFacade.TryGetAndDeleteFromHash(key, field, CancellationToken); + expiredValue.Should().BeNull(); + } + + [Test] + public async Task DeleteString_WithExistingKey_ShouldReturnTrueAndDeleteValue() + { + // Arrange + await using var redisFacade = CreateRedisFacade(); + var key = Guid.NewGuid().ToString(); + var value = "test-value"; + + // Set a value first + await redisFacade.SetString(key, value, TimeSpan.FromMinutes(1), CancellationToken); + + // Verify it exists + var existingValue = await redisFacade.GetString(key, CancellationToken); + existingValue.Should().Be(value); + + // Act + var deleteResult = await redisFacade.DeleteString(key, CancellationToken); // Assert - immediateValue.Should().Be(payload); + deleteResult.Should().BeTrue(); + // Verify the value is gone + var deletedValue = await redisFacade.GetString(key, CancellationToken); + deletedValue.Should().BeNull(); + } + + [Test] + public async Task DeleteString_WithNonExistentKey_ShouldReturnFalse() + { + // Arrange + await using var redisFacade = CreateRedisFacade(); + var nonExistentKey = Guid.NewGuid().ToString(); + + // Act + var deleteResult = await redisFacade.DeleteString(nonExistentKey, CancellationToken); + + // Assert + deleteResult.Should().BeFalse(); + } + + [Test] + public async Task SetTtlForString_WithExistingKey_ShouldUpdateTTL() + { + // Arrange + await using var redisFacade = CreateRedisFacade(); + var key = Guid.NewGuid().ToString(); + var value = "test-value"; + + // Set a value first with a long TTL + await redisFacade.SetString(key, value, TimeSpan.FromHours(1), CancellationToken); + + // Verify it exists + var existingValue = await redisFacade.GetString(key, CancellationToken); + existingValue.Should().Be(value); + + // Act - Update TTL to a shorter time + await redisFacade.SetTtlForString(key, TimeSpan.FromMinutes(1), CancellationToken); + + // Assert - Value should still exist immediately after TTL update + var valueAfterTtlUpdate = await redisFacade.GetString(key, CancellationToken); + valueAfterTtlUpdate.Should().Be(value); + // Note: We can't easily test the actual TTL expiration in a unit test - // as it would require waiting 9+ hours, but we've verified the value is set correctly + // without waiting, but we've verified the operation completes successfully + } + + [Test] + public async Task SetString_WithShortTTL_ShouldExpire() + { + // Arrange + await using var redisFacade = CreateRedisFacade(); + var key = Guid.NewGuid().ToString(); + var value = "test-value"; + + // Act - Set with very short TTL + await redisFacade.SetString(key, value, TimeSpan.FromMilliseconds(3), CancellationToken); + + // Immediately verify it exists + var immediateValue = await redisFacade.GetString(key, CancellationToken); + immediateValue.Should().Be(value); + + // Assert - Should eventually expire + await ShouldEventually.Eventually(async () => + { + var expiredValue = await redisFacade.GetString(key, CancellationToken); + expiredValue.Should().BeNull("the string should expire after TTL"); + }, TimeSpan.FromSeconds(5), CancellationToken); + } + + [Test] + public async Task ListRightPushAsync_WithShortTTL_ShouldExpire() + { + // Arrange + await using var redisFacade = CreateRedisFacade(); + var key = Guid.NewGuid().ToString(); + var payload = "test-payload"; + + // Act - Push with very short TTL + await redisFacade.ListRightPushAsync(key, payload, TimeSpan.FromMilliseconds(3), CancellationToken); + + // Immediately verify it exists + var immediateValue = await redisFacade.ListLeftPopAsync(key, CancellationToken); + immediateValue.Should().Be(payload); + + // Push another item and test expiration + await redisFacade.ListRightPushAsync(key, payload, TimeSpan.FromMilliseconds(3), CancellationToken); + + // Assert - Should eventually expire + await ShouldEventually.Eventually(async () => + { + var listValue = await redisFacade.ListLeftPopAsync(key, CancellationToken); + listValue.Should().BeNull("the list should expire after TTL"); + }, TimeSpan.FromSeconds(5), CancellationToken); + } + + [Test] + public void IsConnected_WhenNotInitialized_ShouldReturnFalse() + { + // Arrange + var redisFacade = new RedisFacade("localhost", "test-prefix", new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix("")); + + // Act & Assert + redisFacade.IsConnected.Should().BeFalse(); } + + [Test] + public async Task IsConnected_AfterSuccessfulOperation_ShouldReturnTrue() + { + // Arrange + await using var redisFacade = CreateRedisFacade(); + + // Act - Perform an operation to initialize connection + await redisFacade.SetString(Guid.NewGuid().ToString(), "test", TimeSpan.FromMinutes(1), CancellationToken); + + // Assert + redisFacade.IsConnected.Should().BeTrue(); + } + + [Test] + public async Task TotalSubscribers_ShouldTrackActiveSubscriptions() + { + // Arrange + await using var redisFacade = CreateRedisFacade(); + var channelName = Guid.NewGuid().ToString(); + + // Act & Assert - Initially no subscribers + redisFacade.TotalSubscribers.Should().Be(0); + + // Subscribe to channels + await using var subscription1 = await redisFacade.SubscribeToChannel(channelName + "1", _ => Task.CompletedTask, CancellationToken); + redisFacade.TotalSubscribers.Should().Be(1); + + await using var subscription2 = await redisFacade.SubscribeToChannel(channelName + "2", _ => Task.CompletedTask, CancellationToken); + redisFacade.TotalSubscribers.Should().Be(2); + + // Dispose one subscription + await subscription1.DisposeAsync(); + redisFacade.TotalSubscribers.Should().Be(1); + + // Dispose second subscription + await subscription2.DisposeAsync(); + redisFacade.TotalSubscribers.Should().Be(0); + } + + [Test] + public async Task MultipleSetString_WithDifferentTTLs_ShouldRespectIndividualTTLs() + { + // Arrange + await using var redisFacade = CreateRedisFacade(); + var key1 = Guid.NewGuid().ToString(); + var key2 = Guid.NewGuid().ToString(); + var value1 = "value1"; + var value2 = "value2"; + + // Act - Set with different TTLs + await redisFacade.SetString(key1, value1, TimeSpan.FromMilliseconds(3), CancellationToken); // Short TTL + await redisFacade.SetString(key2, value2, TimeSpan.FromMinutes(1), CancellationToken); // Long TTL + + // Assert - First should eventually expire, second should still exist + await ShouldEventually.Eventually(async () => + { + var expiredValue1 = await redisFacade.GetString(key1, CancellationToken); + expiredValue1.Should().BeNull("the first string should expire after short TTL"); + }, TimeSpan.FromSeconds(5), CancellationToken); + + // Verify the second key still exists after the first expires + var stillExists2 = await redisFacade.GetString(key2, CancellationToken); + stillExists2.Should().Be(value2); + } + + [Test] + public async Task DisposeAsync_ShouldCleanupResourcesAndNotThrow() + { + // Arrange + var redisFacade = CreateRedisFacade(); + + // Perform some operations to initialize resources + await redisFacade.SetString(Guid.NewGuid().ToString(), "test", TimeSpan.FromMinutes(1), CancellationToken); + await using var subscription = await redisFacade.SubscribeToChannel(Guid.NewGuid().ToString(), _ => Task.CompletedTask, CancellationToken); + + // Act & Assert - Dispose should not throw + Func disposeAction = async () => await redisFacade.DisposeAsync(); + await disposeAction.Should().NotThrowAsync(); + } + + [Test] + public async Task DisposeAsync_CalledMultipleTimes_ShouldNotThrow() + { + // Arrange + var redisFacade = CreateRedisFacade(); + await redisFacade.SetString(Guid.NewGuid().ToString(), "test", TimeSpan.FromMinutes(1), CancellationToken); + + // Act & Assert - Multiple dispose calls should not throw + await redisFacade.DisposeAsync(); + + Func secondDisposeAction = async () => await redisFacade.DisposeAsync(); + await secondDisposeAction.Should().NotThrowAsync(); + } + + } } \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/RedisFacade.cs b/source/Halibut/Queue/Redis/RedisFacade.cs index 24c5657cf..90a4ea73a 100644 --- a/source/Halibut/Queue/Redis/RedisFacade.cs +++ b/source/Halibut/Queue/Redis/RedisFacade.cs @@ -221,7 +221,7 @@ public async Task SubscribeToChannel(string channelName, Func< var disposable = new FuncAsyncDisposable(async () => { Interlocked.Decrement(ref TotalSubscribers); - await channel.UnsubscribeAsync(); + await Try.IgnoringError(async () => await channel.UnsubscribeAsync()); }); Interlocked.Increment(ref TotalSubscribers); From 8ac0d0cbe6ee6f49a535d218443f71f2003be8ca Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Thu, 14 Aug 2025 10:05:40 +1000 Subject: [PATCH 046/137] . --- .../Halibut.Tests/Support/ShouldEventually.cs | 113 ++++++++++++++++++ 1 file changed, 113 insertions(+) create mode 100644 source/Halibut.Tests/Support/ShouldEventually.cs diff --git a/source/Halibut.Tests/Support/ShouldEventually.cs b/source/Halibut.Tests/Support/ShouldEventually.cs new file mode 100644 index 000000000..94e0f1bc5 --- /dev/null +++ b/source/Halibut.Tests/Support/ShouldEventually.cs @@ -0,0 +1,113 @@ +using System; +using System.Diagnostics; +using System.Threading; +using System.Threading.Tasks; + +namespace Halibut.Tests.Support +{ + public static class ShouldEventually + { + /// + /// Keeps executing the given task until it completes without throwing an exception or the timeout is reached. + /// + /// The task to execute repeatedly until it succeeds + /// The maximum time to keep retrying + /// Optional cancellation token + /// A task that completes when the given task succeeds or throws when timeout is reached + public static async Task Eventually(Func task, TimeSpan timeout, CancellationToken cancellationToken = default) + { + using var cts = CancellationTokenSource.CreateLinkedTokenSource(cancellationToken); + cts.CancelAfter(timeout); + + var stopwatch = Stopwatch.StartNew(); + Exception? lastException = null; + + while (!cts.Token.IsCancellationRequested) + { + try + { + await task(); + return; // Success! + } + catch (Exception ex) + { + lastException = ex; + + // Short delay between retries + try + { + await Task.Delay(TimeSpan.FromMilliseconds(20), cts.Token); + } + catch (OperationCanceledException) + { + // Timeout reached + break; + } + } + } + + // If we get here, we've timed out + var timeoutMessage = $"Task did not complete successfully within {timeout.TotalSeconds:F1} seconds (elapsed: {stopwatch.Elapsed.TotalSeconds:F1}s)"; + if (lastException != null) + { + throw new TimeoutException($"{timeoutMessage}. Last exception: {lastException.Message}", lastException); + } + throw new TimeoutException(timeoutMessage); + } + + /// + /// Keeps executing the given action until it completes without throwing an exception or the timeout is reached. + /// + /// The action to execute repeatedly until it succeeds + /// The maximum time to keep retrying + /// Optional cancellation token + public static async Task Eventually(Action action, TimeSpan timeout, CancellationToken cancellationToken = default) + { + await Eventually(() => + { + action(); + return Task.CompletedTask; + }, timeout, cancellationToken); + } + + /// + /// Keeps executing the given function until it returns a result without throwing an exception or the timeout is reached. + /// + /// The return type of the function + /// The function to execute repeatedly until it succeeds + /// The maximum time to keep retrying + /// Optional cancellation token + /// The result of the function when it succeeds + public static async Task Eventually(Func> function, TimeSpan timeout, CancellationToken cancellationToken = default) + { + T result = default(T)!; + + await Eventually(async () => + { + result = await function(); + }, timeout, cancellationToken); + + return result; + } + + /// + /// Keeps executing the given function until it returns a result without throwing an exception or the timeout is reached. + /// + /// The return type of the function + /// The function to execute repeatedly until it succeeds + /// The maximum time to keep retrying + /// Optional cancellation token + /// The result of the function when it succeeds + public static async Task Eventually(Func function, TimeSpan timeout, CancellationToken cancellationToken = default) + { + T result = default(T)!; + + await Eventually(() => + { + result = function(); + }, timeout, cancellationToken); + + return result; + } + } +} \ No newline at end of file From b40616bf3978a76022be5446323935b39ff98b40 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Thu, 14 Aug 2025 10:18:32 +1000 Subject: [PATCH 047/137] Test we wait for the request to be collected before timing out on heart beats --- .../Queue/Redis/RedisFacadeFixture.cs | 2 +- .../Redis/RedisPendingRequestQueueFixture.cs | 94 +++++++------------ .../Queue/Redis/RedisQueuePerformanceTest.cs | 89 ++++++++++++++++++ .../Queue/Redis/NodeHeartBeatSender.cs | 1 - .../Queue/Redis/RedisPendingRequestQueue.cs | 11 ++- 5 files changed, 132 insertions(+), 65 deletions(-) create mode 100644 source/Halibut.Tests/Queue/Redis/RedisQueuePerformanceTest.cs diff --git a/source/Halibut.Tests/Queue/Redis/RedisFacadeFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisFacadeFixture.cs index f340c1a10..7262fb87a 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisFacadeFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisFacadeFixture.cs @@ -474,7 +474,7 @@ public async Task ListRightPushAsync_WithShortTTL_ShouldExpire() var payload = "test-payload"; // Act - Push with very short TTL - await redisFacade.ListRightPushAsync(key, payload, TimeSpan.FromMilliseconds(3), CancellationToken); + await redisFacade.ListRightPushAsync(key, payload, TimeSpan.FromSeconds(3), CancellationToken); // Immediately verify it exists var immediateValue = await redisFacade.ListLeftPopAsync(key, CancellationToken); diff --git a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs index 079407785..1b424cb23 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs @@ -97,60 +97,7 @@ public async Task WhenThePickupTimeoutExpires_AnErrorsReturned_AndTheRequestCanN result.Should().BeNull(); } - //[Test] - public async Task When100kTentaclesAreSubscribed_TheQueueStillWorks() - { - // Arrange - var endpoint = new Uri("poll://" + Guid.NewGuid()); - var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); - var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); - - var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); - var messageSerializer = new QueueMessageSerializerBuilder().Build(); - var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); - - var halibutTimeoutsAndLimits = new HalibutTimeoutsAndLimits(); - - await using var disposableCollection = new DisposableCollection(); - for (int i = 0; i < 300000; i++) - { - disposableCollection.Add(new RedisPendingRequestQueue(new Uri("poll://" + Guid.NewGuid()), new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, halibutTimeoutsAndLimits)); - if (i % 10000 == 0) - { - Logger.Information("Up to: {i}", i); - } - } - - this.Logger.Fatal("Waiting"); - await Task.Delay(30000); - this.Logger.Fatal("Done"); - - for (int i = 0; i < 10; i++) - { - var request = new RequestMessageBuilder(endpoint.ToString()).Build(); - - await using var sut = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(CreateRedisFacade()), messageReaderWriter, halibutTimeoutsAndLimits); - - var resultTask = sut.DequeueAsync(CancellationToken); - - await Task.Delay(100); - - var sw = Stopwatch.StartNew(); - - var task = sut.QueueAndWaitAsync(request, CancellationToken.None); - - var result = await resultTask; - // Act - - // Assert - result.Should().NotBeNull(); - result!.RequestMessage.Id.Should().Be(request.Id); - result.RequestMessage.MethodName.Should().Be(request.MethodName); - result.RequestMessage.ServiceName.Should().Be(request.ServiceName); - Logger.Information("It took {F}", sw.Elapsed.TotalSeconds.ToString("0.00")); - } - } - + [Test] public async Task FullSendAndReceiveShouldWork() { @@ -186,7 +133,7 @@ public async Task FullSendAndReceiveShouldWork() } [Test] - public async Task WhenDataLostIsDetected_InFlightRequestShouldBeAbandoned() + public async Task WhenDataLostIsDetected_InFlightRequestShouldBeAbandoned_AndARetryableExceptionIsThrown() { // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); @@ -387,8 +334,6 @@ public async Task FullSendAndReceiveWithDataStreamShouldWork() (await returnObject.Payload2!.ReadAsString(CancellationToken)).Should().Be("bye"); } - // TODO when receiver (dequeue) can not connect to redis it should not throw an exception. - // Or should it to try to encourage a new TCP connection which could go to a different node. [Test] public async Task WhenTheReceiversConnectionToRedisIsInterruptedAndRestoredBeforeWorkIsPublished_TheReceiverShouldBeAbleToCollectThatWorkQuickly() @@ -439,6 +384,35 @@ public async Task WhenTheReceiversConnectionToRedisIsInterruptedAndRestoredBefor requestReceived.Should().NotBeNull(); requestReceived!.RequestMessage.ActivityId.Should().Be(request.ActivityId); } + + + [Test] + public async Task WhenTheReceiverDoesntCollectWorkImmediately_TheRequestCanSitOnTheQueueForSometime_AndBeOnTheQueueLongerThanTheHeartBeatTimeout() + { + // Arrange + var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); + var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); + await using var redisFacade = CreateRedisFacade(); + + var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); + var messageSerializer = new QueueMessageSerializerBuilder().Build(); + var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); + + var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(redisFacade), messageReaderWriter, new HalibutTimeoutsAndLimits()); + // We are testing that we don't expect heart beats before the request is collected. + node1Sender.RequestReceivingNodeIsOfflineHeartBeatTimeout = TimeSpan.FromSeconds(1); + await node1Sender.WaitUntilQueueIsSubscribedToReceiveMessages(); + + var request = new RequestMessageBuilder("poll://test-endpoint").Build(); + request.Destination.PollingRequestQueueTimeout = TimeSpan.FromHours(1); + await using var cts = new CancelOnDisposeCancellationToken(CancellationToken); + + var queueAndWaitAsync = node1Sender.QueueAndWaitAsync(request, cts.Token); + + await Task.WhenAny(Task.Delay(TimeSpan.FromSeconds(5), CancellationToken), queueAndWaitAsync); + + queueAndWaitAsync.IsCompleted.Should().BeFalse(); + } [Test] public async Task WhenTheSendersConnectionToRedisIsBrieflyInterruptedWhileSendingWork_TheWorkIsStillSent() @@ -512,8 +486,8 @@ public async Task WhenTheReceiverDequeuesWorkAndThenDisconnectsFromRedisForEver_ // Lower this to complete the test sooner. node1Sender.DelayBetweenHeartBeatsForRequestProcessor = TimeSpan.FromSeconds(1); node2Receiver.DelayBetweenHeartBeatsForRequestProcessor = TimeSpan.FromSeconds(1); - node1Sender.NodeIsOfflineHeartBeatTimeoutForRequestProcessor = TimeSpan.FromSeconds(10); - node2Receiver.NodeIsOfflineHeartBeatTimeoutForRequestProcessor = TimeSpan.FromSeconds(10); + node1Sender.RequestReceivingNodeIsOfflineHeartBeatTimeout = TimeSpan.FromSeconds(10); + node2Receiver.RequestReceivingNodeIsOfflineHeartBeatTimeout = TimeSpan.FromSeconds(10); var request = new RequestMessageBuilder("poll://test-endpoint").Build(); @@ -629,7 +603,7 @@ public async Task WhenTheSenderBrieflyDisconnectsFromRedisRightWhenTheReceiverSe } [Test] - public async Task WhenTheRequestReceiverDetectsRedisDataLose_AndTheRequestSenderDoesNot_TheSenderReceivesARetryableResponse() + public async Task WhenTheRequestReceiverDetectsRedisDataLose_AndTheRequestSenderDoesNotYetDetectDataLose_TheSenderReceivesARetryableResponse() { // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); diff --git a/source/Halibut.Tests/Queue/Redis/RedisQueuePerformanceTest.cs b/source/Halibut.Tests/Queue/Redis/RedisQueuePerformanceTest.cs new file mode 100644 index 000000000..b3d7b2114 --- /dev/null +++ b/source/Halibut.Tests/Queue/Redis/RedisQueuePerformanceTest.cs @@ -0,0 +1,89 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; +using System.Diagnostics; +using System.Threading.Tasks; +using FluentAssertions; +using Halibut.Diagnostics; +using Halibut.Logging; +using Halibut.Queue.QueuedDataStreams; +using Halibut.Queue.Redis; +using Halibut.Tests.Builders; +using Halibut.Tests.Support.Logging; +using Halibut.Util; + +namespace Halibut.Tests.Queue.Redis.Utils +{ + public class RedisQueuePerformanceTest : BaseTest + { + const int redisPort = 6379; + private static RedisFacade CreateRedisFacade(int? port = redisPort, Guid? guid = null) => new("localhost:" + port, (guid ?? Guid.NewGuid()).ToString(), new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix("")); + + + //[Test] + public async Task When100kTentaclesAreSubscribed_TheQueueStillWorks() + { + // Arrange + var endpoint = new Uri("poll://" + Guid.NewGuid()); + var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); + var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); + + var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); + var messageSerializer = new QueueMessageSerializerBuilder().Build(); + var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); + + var halibutTimeoutsAndLimits = new HalibutTimeoutsAndLimits(); + + await using var disposableCollection = new DisposableCollection(); + for (int i = 0; i < 300000; i++) + { + disposableCollection.Add(new RedisPendingRequestQueue(new Uri("poll://" + Guid.NewGuid()), new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, halibutTimeoutsAndLimits)); + if (i % 10000 == 0) + { + Logger.Information("Up to: {i}", i); + } + } + + this.Logger.Fatal("Waiting"); + await Task.Delay(30000); + this.Logger.Fatal("Done"); + + for (int i = 0; i < 10; i++) + { + var request = new RequestMessageBuilder(endpoint.ToString()).Build(); + + await using var sut = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(CreateRedisFacade()), messageReaderWriter, halibutTimeoutsAndLimits); + + var resultTask = sut.DequeueAsync(CancellationToken); + + await Task.Delay(100); + + var sw = Stopwatch.StartNew(); + + var task = sut.QueueAndWaitAsync(request, CancellationToken); + + var result = await resultTask; + // Act + + // Assert + result.Should().NotBeNull(); + result!.RequestMessage.Id.Should().Be(request.Id); + result.RequestMessage.MethodName.Should().Be(request.MethodName); + result.RequestMessage.ServiceName.Should().Be(request.ServiceName); + Logger.Information("It took {F}", sw.Elapsed.TotalSeconds.ToString("0.00")); + } + } + } +} \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs b/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs index 8f6edc9ea..a40fd1efd 100644 --- a/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs +++ b/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs @@ -109,7 +109,6 @@ public static async Task WatchThatNodeProces await using var cts = new CancelOnDisposeCancellationToken(watchCancellationToken, redisPending.PendingRequestCancellationToken); try { - // TODO: test this is indeed called first. await WaitForRequestToBeCollected(endpoint, request, redisPending, halibutRedisTransport, timeBetweenCheckingIfRequestWasCollected, log, cts.Token); return await WatchForPulsesFromNode(endpoint, request.ActivityId, halibutRedisTransport, log, maxTimeBetweenHeartBeetsBeforeProcessingNodeIsAssumedToBeOffline, HalibutQueueNodeSendingPulses.Receiver, cts.Token); diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index 963d2c733..82ae006aa 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -172,7 +172,7 @@ void WatchProcessingNodeIsStillConnectedInBackground(RequestMessage request, Red halibutRedisTransport, TimeBetweenCheckingIfRequestWasCollected, log, - NodeIsOfflineHeartBeatTimeoutForRequestProcessor, + RequestReceivingNodeIsOfflineHeartBeatTimeout, watcherCtsCancellationToken); if (!watcherCtsCancellationToken.IsCancellationRequested && disconnected == NodeHeartBeatSender.NodeProcessingRequestWatcherResult.NodeMayHaveDisconnected) { @@ -271,8 +271,13 @@ async Task SubscribeToResponse(Guid activityId, internal TimeSpan DelayBetweenHeartBeatsForRequestSender { get; set; } = TimeSpan.FromSeconds(15); - // Setting this too high means things above the RPC might not have time to retry. - public TimeSpan NodeIsOfflineHeartBeatTimeoutForRequestProcessor { get; set; } = TimeSpan.FromSeconds(60); + /// + /// The amount of time since the last heart beat from the node sending the request to Tentacle + /// before the node is assumed to be offline. + /// + /// Setting this too high means things above the RPC might not have time to retry. + /// + public TimeSpan RequestReceivingNodeIsOfflineHeartBeatTimeout { get; set; } = TimeSpan.FromSeconds(60); internal TimeSpan DelayBetweenHeartBeatsForRequestProcessor { get; set; } = TimeSpan.FromSeconds(15); From 34f60c342bbb5bfecba61b79726383fd2583ca8a Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Thu, 14 Aug 2025 10:50:40 +1000 Subject: [PATCH 048/137] . --- .../Redis/RedisPendingRequestQueueFixture.cs | 3 +- ...onReturnedByHalibutProxyExtensionMethod.cs | 3 +- .../Queue/Redis/NodeHeartBeatSender.cs | 32 +++++-------------- 3 files changed, 12 insertions(+), 26 deletions(-) diff --git a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs index 1b424cb23..f2bbbe653 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs @@ -385,7 +385,6 @@ public async Task WhenTheReceiversConnectionToRedisIsInterruptedAndRestoredBefor requestReceived!.RequestMessage.ActivityId.Should().Be(request.ActivityId); } - [Test] public async Task WhenTheReceiverDoesntCollectWorkImmediately_TheRequestCanSitOnTheQueueForSometime_AndBeOnTheQueueLongerThanTheHeartBeatTimeout() { @@ -508,6 +507,8 @@ public async Task WhenTheReceiverDequeuesWorkAndThenDisconnectsFromRedisForEver_ var response = await queueAndWaitTask; response.Error.Should().NotBeNull(); response.Error!.Message.Should().Contain("The node processing the request did not send a heartbeat for long enough, and so the node is now assumed to be offline."); + + CreateExceptionFromResponse(response, log).IsRetryableError().Should().Be(HalibutRetryableErrorType.IsRetryable); } [Test] diff --git a/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs b/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs index 91bdf5702..3a19a3cc2 100644 --- a/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs +++ b/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs @@ -51,7 +51,8 @@ public static HalibutNetworkExceptionType IsNetworkError(this Exception exceptio if (exception is HalibutClientException) { - if (exception.Message.Contains(RedisPendingRequestQueue.RequestAbandonedMessage)) return HalibutNetworkExceptionType.IsNetworkError; + if (exception.Message.Contains("The request was abandoned, possibly because the node processing the request shutdown or redis lost all of its data.")) return HalibutNetworkExceptionType.IsNetworkError; + if(exception.Message.Contains("The node processing the request did not send a heartbeat for long enough, and so the node is now assumed to be offline.")) return HalibutNetworkExceptionType.IsNetworkError; } // TODO end diff --git a/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs b/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs index a40fd1efd..13517c81a 100644 --- a/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs +++ b/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs @@ -13,14 +13,12 @@ // limitations under the License. using System; -using System.Security.Cryptography; using System.Threading; using System.Threading.Tasks; using Halibut.Diagnostics; using Halibut.ServiceModel; using Halibut.Transport.Protocol; using Halibut.Util; -using Nito.AsyncEx; namespace Halibut.Queue.Redis { @@ -36,11 +34,11 @@ public enum NodeProcessingRequestWatcherResult NodeMayHaveDisconnected, NoDisconnectSeen } - - private readonly Uri endpoint; - private readonly Guid requestActivityId; + + readonly Uri endpoint; + readonly Guid requestActivityId; private readonly HalibutRedisTransport halibutRedisTransport; - private readonly CancelOnDisposeCancellationToken cancellationToken; + private readonly CancelOnDisposeCancellationToken cts; private readonly ILog log; private readonly HalibutQueueNodeSendingPulses nodeSendingPulsesType; @@ -57,13 +55,13 @@ public NodeHeartBeatSender( this.requestActivityId = requestActivityId; this.halibutRedisTransport = halibutRedisTransport; this.nodeSendingPulsesType = nodeSendingPulsesType; - cancellationToken = new CancelOnDisposeCancellationToken(); + cts = new CancelOnDisposeCancellationToken(); this.log = log.ForContext(); this.log.Write(EventType.Diagnostic, "Starting NodeHeartBeatSender for {0} node, request {1}, endpoint {2}", nodeSendingPulsesType, requestActivityId, endpoint); - TaskSendingPulses = Task.Run(() => SendPulsesWhileProcessingRequest(defaultDelayBetweenPulses, cancellationToken.Token)); + TaskSendingPulses = Task.Run(() => SendPulsesWhileProcessingRequest(defaultDelayBetweenPulses, cts.Token)); } - private async Task SendPulsesWhileProcessingRequest(TimeSpan defaultDelayBetweenPulses, CancellationToken cancellationToken) + async Task SendPulsesWhileProcessingRequest(TimeSpan defaultDelayBetweenPulses, CancellationToken cancellationToken) { log.Write(EventType.Diagnostic, "Starting heartbeat pulse loop for {0} node, request {1}", nodeSendingPulsesType, requestActivityId); @@ -194,24 +192,11 @@ static async Task WaitForRequestToBeCollected(Uri endpoint, RequestMessage reque { log = log.ForContext(); log.Write(EventType.Diagnostic, "Waiting for request {0} to be collected from queue", request.ActivityId); - - // TODO: Is this worthwhile? - var asyncManualResetEvent = new AsyncManualResetEvent(false); - // await using var subscription = await halibutRedisTransport.SubscribeToNodeHeartBeatChannel( - // endpoint, - // request.ActivityId, - // HalibutQueueNodeSendingPulses.Receiver, async () => - // { - // await Task.CompletedTask; - // asyncManualResetEvent.Set(); - // log.Write(EventType.Diagnostic, "While waiting for request to be collected received heartbeat from {0} node, request {1}", HalibutQueueNodeSendingPulses.Receiver, request.ActivityId); - // }, cancellationToken); while (!cancellationToken.IsCancellationRequested) { try { - asyncManualResetEvent.Reset(); // Has something else determined the request was collected? if(redisPending.HasRequestBeenMarkedAsCollected) { @@ -238,7 +223,6 @@ await Try.IgnoringError(async () => { await Task.WhenAny( Task.Delay(timeBetweenCheckingIfRequestWasCollected, cancellationToken), - asyncManualResetEvent.WaitAsync(cancellationToken), redisPending.WaitForRequestToBeMarkedAsCollected(cancellationToken)); }); } @@ -250,7 +234,7 @@ public async ValueTask DisposeAsync() { log.Write(EventType.Diagnostic, "Disposing NodeHeartBeatSender for {0} node, request {1}", nodeSendingPulsesType, requestActivityId); - await Try.IgnoringError(async () => await cancellationToken.DisposeAsync()); + await Try.IgnoringError(async () => await cts.DisposeAsync()); log.Write(EventType.Diagnostic, "NodeHeartBeatSender disposed for {0} node, request {1}", nodeSendingPulsesType, requestActivityId); } From cbdccb2f0f85c7fda9c02581fab3a7a5ee980e5f Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Thu, 14 Aug 2025 14:00:34 +1000 Subject: [PATCH 049/137] Dont wait forever when an error occurs reading the response --- .../Queue/Redis/RedisFacadeFixture.cs | 6 +- .../Redis/RedisPendingRequestQueueFixture.cs | 32 ++++++- .../MessageReaderWriterExtensionsMethods.cs | 62 ++++++++++++ ...onReturnedByHalibutProxyExtensionMethod.cs | 3 +- .../Queue/Redis/IMessageReaderWriter.cs | 28 ++++++ .../Queue/Redis/MessageReaderWriter.cs | 2 +- .../Queue/Redis/PollAndSubscribeToResponse.cs | 3 +- .../Queue/Redis/RedisPendingRequestQueue.cs | 96 +++++++++++++------ 8 files changed, 191 insertions(+), 41 deletions(-) create mode 100644 source/Halibut.Tests/Queue/Redis/Utils/MessageReaderWriterExtensionsMethods.cs create mode 100644 source/Halibut/Queue/Redis/IMessageReaderWriter.cs diff --git a/source/Halibut.Tests/Queue/Redis/RedisFacadeFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisFacadeFixture.cs index 7262fb87a..3d8c7f7b6 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisFacadeFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisFacadeFixture.cs @@ -453,10 +453,6 @@ public async Task SetString_WithShortTTL_ShouldExpire() // Act - Set with very short TTL await redisFacade.SetString(key, value, TimeSpan.FromMilliseconds(3), CancellationToken); - // Immediately verify it exists - var immediateValue = await redisFacade.GetString(key, CancellationToken); - immediateValue.Should().Be(value); - // Assert - Should eventually expire await ShouldEventually.Eventually(async () => { @@ -474,7 +470,7 @@ public async Task ListRightPushAsync_WithShortTTL_ShouldExpire() var payload = "test-payload"; // Act - Push with very short TTL - await redisFacade.ListRightPushAsync(key, payload, TimeSpan.FromSeconds(3), CancellationToken); + await redisFacade.ListRightPushAsync(key, payload, TimeSpan.FromSeconds(30), CancellationToken); // Immediately verify it exists var immediateValue = await redisFacade.ListLeftPopAsync(key, CancellationToken); diff --git a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs index f2bbbe653..b362ff9f2 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs @@ -132,6 +132,35 @@ public async Task FullSendAndReceiveShouldWork() responseMessage.Result.Should().Be("Yay"); } + [Test] + public async Task WhenReadingTheResponseFromTheQueueFails_TheQueueAndWaitTaskReturnsAnUnknownError() + { + // Arrange + var endpoint = new Uri("poll://" + Guid.NewGuid()); + var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); + var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); + var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); + var messageSerializer = new QueueMessageSerializerBuilder().Build(); + var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore) + .ThrowsOnReadResponse(() => new OperationCanceledException()); + + var request = new RequestMessageBuilder("poll://test-endpoint").Build(); + + var queue = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + await queue.WaitUntilQueueIsSubscribedToReceiveMessages(); + + var queueAndWaitAsync = queue.QueueAndWaitAsync(request, CancellationToken.None); + + var requestMessageWithCancellationToken = await queue.DequeueAsync(CancellationToken); + await queue.ApplyResponse(ResponseMessage.FromResult(requestMessageWithCancellationToken! .RequestMessage, "Yay"), + requestMessageWithCancellationToken.RequestMessage.ActivityId); + + var responseMessage = await queueAndWaitAsync; + responseMessage.Error.Should().NotBeNull(); + + CreateExceptionFromResponse(responseMessage, log).IsRetryableError().Should().Be(HalibutRetryableErrorType.IsRetryable); + } + [Test] public async Task WhenDataLostIsDetected_InFlightRequestShouldBeAbandoned_AndARetryableExceptionIsThrown() { @@ -490,8 +519,7 @@ public async Task WhenTheReceiverDequeuesWorkAndThenDisconnectsFromRedisForEver_ var request = new RequestMessageBuilder("poll://test-endpoint").Build(); - // TODO: Setting this low shows we don't timeout because the request was not picked up in time. - // Could be its own test. + // Setting this low shows we don't timeout because the request was not picked up in time. request.Destination.PollingRequestQueueTimeout = TimeSpan.FromSeconds(5); var queueAndWaitTask = node1Sender.QueueAndWaitAsync(request, CancellationToken.None); diff --git a/source/Halibut.Tests/Queue/Redis/Utils/MessageReaderWriterExtensionsMethods.cs b/source/Halibut.Tests/Queue/Redis/Utils/MessageReaderWriterExtensionsMethods.cs new file mode 100644 index 000000000..fd05e7bb0 --- /dev/null +++ b/source/Halibut.Tests/Queue/Redis/Utils/MessageReaderWriterExtensionsMethods.cs @@ -0,0 +1,62 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; +using System.Threading; +using System.Threading.Tasks; +using Halibut.Queue.Redis; +using Halibut.Transport.Protocol; + +namespace Halibut.Tests.Queue.Redis.Utils +{ + public static class MessageReaderWriterExtensionsMethods + { + public static IMessageReaderWriter ThrowsOnReadResponse(this IMessageReaderWriter messageReaderWriter, Func exceptionFactory) + { + return new MessageReaderWriterThatThrowsWhenReadingResponse(messageReaderWriter, exceptionFactory); + } + } + + class MessageReaderWriterThatThrowsWhenReadingResponse : IMessageReaderWriter + { + readonly IMessageReaderWriter messageReaderWriter; + readonly Func exception; + + public MessageReaderWriterThatThrowsWhenReadingResponse(IMessageReaderWriter messageReaderWriter, Func exception) + { + this.messageReaderWriter = messageReaderWriter; + this.exception = exception; + } + + public Task PrepareRequest(RequestMessage request, CancellationToken cancellationToken) + { + return messageReaderWriter.PrepareRequest(request, cancellationToken); + } + + public Task ReadRequest(string jsonRequest, CancellationToken cancellationToken) + { + return messageReaderWriter.ReadRequest(jsonRequest, cancellationToken); + } + + public Task PrepareResponse(ResponseMessage response, CancellationToken cancellationToken) + { + return messageReaderWriter.PrepareResponse(response, cancellationToken); + } + + public Task ReadResponse(string jsonResponse, CancellationToken cancellationToken) + { + throw exception(); + } + } +} \ No newline at end of file diff --git a/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs b/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs index 3a19a3cc2..13255096a 100644 --- a/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs +++ b/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs @@ -52,7 +52,8 @@ public static HalibutNetworkExceptionType IsNetworkError(this Exception exceptio if (exception is HalibutClientException) { if (exception.Message.Contains("The request was abandoned, possibly because the node processing the request shutdown or redis lost all of its data.")) return HalibutNetworkExceptionType.IsNetworkError; - if(exception.Message.Contains("The node processing the request did not send a heartbeat for long enough, and so the node is now assumed to be offline.")) return HalibutNetworkExceptionType.IsNetworkError; + if (exception.Message.Contains("The node processing the request did not send a heartbeat for long enough, and so the node is now assumed to be offline.")) return HalibutNetworkExceptionType.IsNetworkError; + if (exception.Message.Contains("Error occured when reading data from the queue")) return HalibutNetworkExceptionType.IsNetworkError; } // TODO end diff --git a/source/Halibut/Queue/Redis/IMessageReaderWriter.cs b/source/Halibut/Queue/Redis/IMessageReaderWriter.cs new file mode 100644 index 000000000..9fd1b1c9e --- /dev/null +++ b/source/Halibut/Queue/Redis/IMessageReaderWriter.cs @@ -0,0 +1,28 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System.Threading; +using System.Threading.Tasks; +using Halibut.Transport.Protocol; + +namespace Halibut.Queue.Redis +{ + public interface IMessageReaderWriter + { + Task PrepareRequest(RequestMessage request, CancellationToken cancellationToken); + Task ReadRequest(string jsonRequest, CancellationToken cancellationToken); + Task PrepareResponse(ResponseMessage response, CancellationToken cancellationToken); + Task ReadResponse(string jsonResponse, CancellationToken cancellationToken); + } +} \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/MessageReaderWriter.cs b/source/Halibut/Queue/Redis/MessageReaderWriter.cs index 8b88062d4..00feb9c2b 100644 --- a/source/Halibut/Queue/Redis/MessageReaderWriter.cs +++ b/source/Halibut/Queue/Redis/MessageReaderWriter.cs @@ -19,7 +19,7 @@ namespace Halibut.Queue.Redis { - public class MessageReaderWriter + public class MessageReaderWriter : IMessageReaderWriter { readonly QueueMessageSerializer queueMessageSerializer; readonly IStoreDataStreamsForDistributedQueues storeDataStreamsForDistributedQueues; diff --git a/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs b/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs index fcd535a39..35b99e20d 100644 --- a/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs +++ b/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs @@ -181,9 +181,8 @@ public async ValueTask DisposeAsync() log.Write(EventType.Diagnostic, "Disposing GenericWatcher for response - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); await Try.IgnoringError(async () => await watcherToken.CancelAsync()); - //Try.IgnoringError(() => watcherTokenSource.Dispose()); - // If the message task is not yet complete, then complete if now with null since we have nothing for it. + // If the message task is not yet complete, then mark it as cancelled Try.IgnoringError(() => message.TrySetCanceled()); log.Write(EventType.Diagnostic, "Disposed GenericWatcher for response - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index 82ae006aa..8e248a83f 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -31,7 +31,7 @@ class RedisPendingRequestQueue : IPendingRequestQueue, IDisposable readonly ILog log; readonly HalibutRedisTransport halibutRedisTransport; readonly HalibutTimeoutsAndLimits halibutTimeoutsAndLimits; - readonly MessageReaderWriter messageReaderWriter; + readonly IMessageReaderWriter messageReaderWriter; readonly AsyncManualResetEvent hasItemsForEndpoint = new(); readonly CancelOnDisposeCancellationToken queueCts = new (); @@ -49,7 +49,7 @@ public RedisPendingRequestQueue( IWatchForRedisLosingAllItsData watchForRedisLosingAllItsData, ILog log, HalibutRedisTransport halibutRedisTransport, - MessageReaderWriter messageReaderWriter, + IMessageReaderWriter messageReaderWriter, HalibutTimeoutsAndLimits halibutTimeoutsAndLimits) { this.endpoint = endpoint; @@ -100,7 +100,7 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can var payload = await messageReaderWriter.PrepareRequest(request, cancellationToken); // Start listening for a response to the request, we don't want to miss the response. - await using var _ = await SubscribeToResponse(request.ActivityId, pending.SetResponse, cancellationToken); + await using var pollAndSubscribeToResponse = new PollAndSubscribeToResponse(endpoint, request.ActivityId, halibutRedisTransport, log); var tryClearRequestFromQueueAtMostOnce = new AsyncLazy(async () => await TryClearRequestFromQueue(request, pending)); try @@ -116,16 +116,47 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can await using var watcherCts = new CancelOnDisposeCancellationToken(cts.Token); WatchProcessingNodeIsStillConnectedInBackground(request, pending, watcherCts); - + // TODO: We need to ensure that no matter what exceptions are thrown we eventually exit. // For example can the subscription to the response, fail and never come back? // Can the WatchProcessProcessingNodeIsStillConnected fail and never come back? - await pending.WaitUntilComplete( + + var waitingForResponse = WaitForResponse(pollAndSubscribeToResponse, request, cancellationToken); + var pendingRequestWaitUntilComplete = pending.WaitUntilComplete( async () => await tryClearRequestFromQueueAtMostOnce.Task, () => dataLoseCt.IsCancellationRequested ? new RedisDataLoseHalibutClientException($"Request {request.ActivityId} was cancelled because we detected that redis lost all of its data.") : null, cancellationToken); + + await Task.WhenAny(waitingForResponse, pendingRequestWaitUntilComplete); + + if (pendingRequestWaitUntilComplete.IsCompleted || cancellationToken.IsCancellationRequested) + { + await pendingRequestWaitUntilComplete; + return pending.Response!; + } + + if (waitingForResponse.IsCompleted) + { + var response = await waitingForResponse; + if (response != null) + { + pending.SetResponse(response); + return pending.Response!; + } + else if(!cancellationToken.IsCancellationRequested) + { + // We are no longer waiting for a response and have no response. + // The cancellation token has not been set so the request is not going to be cancelled. + // It is unclear how we got into this state, but lets at least error out. + pending.SetResponse(ResponseMessage.FromError(request, "Queue unexpectedly stopped waiting for a response")); + return pending.Response!; + } + } + + pending.SetResponse(ResponseMessage.FromError(request, "Impossible queue state reached")); + return pending.Response!; } finally { @@ -142,7 +173,7 @@ await pending.WaitUntilComplete( }); } - return pending.Response!; + } async Task SendCancellationIfRequestWasCancelled(RequestMessage request, RedisPendingRequest redisPending) @@ -232,33 +263,38 @@ async Task TryClearRequestFromQueue(RequestMessage request, RedisPendingRe const string ResponseMessageSubscriptionName = "ResponseMessage"; - async Task SubscribeToResponse(Guid activityId, - Action onResponse, + async Task WaitForResponse( + PollAndSubscribeToResponse pollAndSubscribeToResponse, + RequestMessage requestMessage, CancellationToken cancellationToken) { - await Task.CompletedTask; - var sub = new PollAndSubscribeToResponse(endpoint, activityId, halibutRedisTransport, log); - var _ = Task.Run(async () => + await Task.Yield(); + var activityId = requestMessage.ActivityId; + string responseJson; + try { - try - { - log.Write(EventType.Diagnostic, "Waiting for response for request {0}", activityId); - var responseJson = await sub.ResultTask; - log.Write(EventType.Diagnostic, "Received response JSON for request {0}, deserializing", activityId); - var response = await messageReaderWriter.ReadResponse(responseJson, cancellationToken); - log.Write(EventType.Diagnostic, "Successfully deserialized response for request {0}, invoking callback", activityId); - onResponse(response); - } - catch (OperationCanceledException) - { - log.Write(EventType.Diagnostic, "Response subscription cancelled for request {0}", activityId); - } - catch (Exception ex) - { - log.WriteException(EventType.Error, "Error while processing response for request {0}", ex, activityId); - } - }); - return sub; + log.Write(EventType.Diagnostic, "Waiting for response for request {0}", activityId); + responseJson = await pollAndSubscribeToResponse.ResultTask; + log.Write(EventType.Diagnostic, "Received response JSON for request {0}, deserializing", activityId); + } + catch (Exception ex) + { + log.WriteException(EventType.Error, "Error while processing response for request {0}", ex, activityId); + return null; + } + + try + { + var response = await messageReaderWriter.ReadResponse(responseJson, cancellationToken); + log.Write(EventType.Diagnostic, "Successfully deserialized response for request {0}", activityId); + return response; + } + catch (Exception ex) + { + log.Write(EventType.Error, "Error deserializeing response for request {0}", activityId); + return ResponseMessage.FromException(requestMessage, new Exception("Error occured when reading data from the queue", ex)); + } + } public bool IsEmpty => Count == 0; From 1dc6145d76f32654ddd39896498914e3a769c0bc Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Thu, 14 Aug 2025 16:14:03 +1000 Subject: [PATCH 050/137] Fix error types so that RPC will be retried --- .../Redis/RedisPendingRequestQueueFixture.cs | 91 ++++++- .../HalibutRedisTransportWithVirtuals.cs | 137 +++++++++++ .../MessageReaderWriterExtensionsMethods.cs | 51 +++- source/Halibut.Tests/Util/AssertThrowsAny.cs | 38 +++ ...onReturnedByHalibutProxyExtensionMethod.cs | 8 +- ...taLoseTokenInTimeHalibutClientException.cs | 25 ++ ...DataIntoRedisHalibutPendingRequestQueue.cs | 25 ++ ...ngRequestForQueueHalibutClientException.cs | 25 ++ .../RedisDataLoseHalibutClientException.cs | 0 .../RedisQueueShutdownClientException.cs | 23 ++ .../Queue/Redis/HalibutRedisTransport.cs | 7 +- .../Queue/Redis/IHalibutRedisTransport.cs | 66 ++++++ .../Queue/Redis/NodeHeartBeatSender.cs | 18 +- .../Queue/Redis/PollAndSubscribeToResponse.cs | 98 ++++---- source/Halibut/Queue/Redis/RedisFacade.cs | 7 +- .../Queue/Redis/RedisPendingRequestQueue.cs | 183 ++++++++------ .../Redis/WatchForRequestCancellation.cs | 6 +- ...orRequestCancellationOrSenderDisconnect.cs | 4 +- .../ServiceModel/PendingRequestQueueAsync.cs | 223 +++++++++++++++++- .../Halibut/ServiceModel/PendingRequestV2.cs | 26 +- 20 files changed, 892 insertions(+), 169 deletions(-) create mode 100644 source/Halibut.Tests/Queue/Redis/Utils/HalibutRedisTransportWithVirtuals.cs create mode 100644 source/Halibut.Tests/Util/AssertThrowsAny.cs create mode 100644 source/Halibut/Queue/Redis/Exceptions/CouldNotGetDataLoseTokenInTimeHalibutClientException.cs create mode 100644 source/Halibut/Queue/Redis/Exceptions/ErrorOccuredWhenInsertingDataIntoRedisHalibutPendingRequestQueue.cs create mode 100644 source/Halibut/Queue/Redis/Exceptions/ErrorWhilePreparingRequestForQueueHalibutClientException.cs rename source/Halibut/Queue/Redis/{ => Exceptions}/RedisDataLoseHalibutClientException.cs (100%) create mode 100644 source/Halibut/Queue/Redis/Exceptions/RedisQueueShutdownClientException.cs create mode 100644 source/Halibut/Queue/Redis/IHalibutRedisTransport.cs diff --git a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs index b362ff9f2..bfbb9b66d 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs @@ -27,6 +27,7 @@ using Halibut.Util; using Nito.AsyncEx; using NSubstitute; +using NSubstitute.Extensions; using NUnit.Framework; using Octopus.TestPortForwarder; using Serilog; @@ -161,6 +162,94 @@ await queue.ApplyResponse(ResponseMessage.FromResult(requestMessageWithCancellat CreateExceptionFromResponse(responseMessage, log).IsRetryableError().Should().Be(HalibutRetryableErrorType.IsRetryable); } + + [Test] + public async Task WhenEnteringTheQueue_AndRedisIsUnavailable_ARetryableExceptionIsThrown() + { + // Arrange + var endpoint = new Uri("poll://" + Guid.NewGuid()); + var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); + + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); + await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); + redisFacade.MaxDurationToRetryFor = TimeSpan.FromSeconds(1); + + var redisTransport = new HalibutRedisTransport(redisFacade); + var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); + var messageSerializer = new QueueMessageSerializerBuilder().Build(); + var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); + + var request = new RequestMessageBuilder("poll://test-endpoint").Build(); + var queue = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + portForwarder.EnterKillNewAndExistingConnectionsMode(); + + // Act Assert + var exception = await AssertThrowsAny.Exception(async () => await queue.QueueAndWaitAsync(request, CancellationToken.None)); + exception.IsRetryableError().Should().Be(HalibutRetryableErrorType.IsRetryable); + exception.Message.Should().Contain("ailed since an error occured inserting the data into the queue"); + } + + [Test] + public async Task WhenEnteringTheQueue_AndRedisIsUnavailableAndDataLoseOccurs_ARetryableExceptionIsThrown() + { + // Arrange + var endpoint = new Uri("poll://" + Guid.NewGuid()); + var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); + + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); + await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); + redisFacade.MaxDurationToRetryFor = TimeSpan.FromSeconds(1); + + var redisDataLoseDetector = new CancellableDataLossWatchForRedisLosingAllItsData(); + + var redisTransport = Substitute.ForPartsOf(new HalibutRedisTransport(redisFacade)); + redisTransport.Configure().PutRequest(Arg.Any(), Arg.Any(), Arg.Any(), Arg.Any(), Arg.Any()) + .Returns(async callInfo => + { + await redisDataLoseDetector.DataLossHasOccured(); + throw new OperationCanceledException(); + }); + + var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); + var messageSerializer = new QueueMessageSerializerBuilder().Build(); + var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); + + var request = new RequestMessageBuilder("poll://test-endpoint").Build(); + + var queue = new RedisPendingRequestQueue(endpoint, redisDataLoseDetector, log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + + + // Act Assert + var exception = await AssertThrowsAny.Exception(async () => await queue.QueueAndWaitAsync(request, CancellationToken.None)); + exception.IsRetryableError().Should().Be(HalibutRetryableErrorType.IsRetryable); + exception.Message.Should().Contain("was cancelled because we detected that redis lost all of its data."); + } + + [Test] + public async Task WhenPreparingRequestFails_ARetryableExceptionIsThrown() + { + // Arrange + var endpoint = new Uri("poll://" + Guid.NewGuid()); + var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); + + + await using var redisFacade = CreateRedisFacade(); + + var redisTransport = new HalibutRedisTransport(redisFacade); + var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); + var messageSerializer = new QueueMessageSerializerBuilder().Build(); + var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore) + .ThrowsOnPrepareRequest(() => new OperationCanceledException()); + + var request = new RequestMessageBuilder("poll://test-endpoint").Build(); + var queue = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + + // Act Assert + var exception = await AssertThrowsAny.Exception(async () => await queue.QueueAndWaitAsync(request, CancellationToken.None)); + exception.IsRetryableError().Should().Be(HalibutRetryableErrorType.IsRetryable); + exception.Message.Should().Contain("error occured when preparing request for queue"); + } + [Test] public async Task WhenDataLostIsDetected_InFlightRequestShouldBeAbandoned_AndARetryableExceptionIsThrown() { @@ -234,7 +323,7 @@ public async Task OnceARequestIsComplete_NoInflightDisposableShouldExist() responseMessage.Result.Should().Be("Yay"); // Assert - queue.disposablesForInFlightRequests.Should().BeEmpty(); + queue.DisposablesForInFlightRequests.Should().BeEmpty(); } [Test] diff --git a/source/Halibut.Tests/Queue/Redis/Utils/HalibutRedisTransportWithVirtuals.cs b/source/Halibut.Tests/Queue/Redis/Utils/HalibutRedisTransportWithVirtuals.cs new file mode 100644 index 000000000..c2e504050 --- /dev/null +++ b/source/Halibut.Tests/Queue/Redis/Utils/HalibutRedisTransportWithVirtuals.cs @@ -0,0 +1,137 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; +using System.Threading; +using System.Threading.Tasks; +using Halibut.Queue.Redis; +using StackExchange.Redis; + +namespace Halibut.Tests.Queue.Redis.Utils +{ + public class HalibutRedisTransportWithVirtuals : IHalibutRedisTransport + { + readonly IHalibutRedisTransport halibutRedisTransport; + + public HalibutRedisTransportWithVirtuals(IHalibutRedisTransport halibutRedisTransport) + { + this.halibutRedisTransport = halibutRedisTransport; + } + + public Task SubscribeToRequestMessagePulseChannel(Uri endpoint, Action onRequestMessagePulse, CancellationToken cancellationToken) + { + return halibutRedisTransport.SubscribeToRequestMessagePulseChannel(endpoint, onRequestMessagePulse, cancellationToken); + } + + public Task PulseRequestPushedToEndpoint(Uri endpoint, CancellationToken cancellationToken) + { + return halibutRedisTransport.PulseRequestPushedToEndpoint(endpoint, cancellationToken); + } + + public Task PushRequestGuidOnToQueue(Uri endpoint, Guid guid, CancellationToken cancellationToken) + { + return halibutRedisTransport.PushRequestGuidOnToQueue(endpoint, guid, cancellationToken); + } + + public Task TryPopNextRequestGuid(Uri endpoint, CancellationToken cancellationToken) + { + return halibutRedisTransport.TryPopNextRequestGuid(endpoint, cancellationToken); + } + + public virtual Task PutRequest(Uri endpoint, Guid requestId, string payload, TimeSpan requestPickupTimeout, CancellationToken cancellationToken) + { + return halibutRedisTransport.PutRequest(endpoint, requestId, payload, requestPickupTimeout, cancellationToken); + } + + public Task TryGetAndRemoveRequest(Uri endpoint, Guid requestId, CancellationToken cancellationToken) + { + return halibutRedisTransport.TryGetAndRemoveRequest(endpoint, requestId, cancellationToken); + } + + public Task IsRequestStillOnQueue(Uri endpoint, Guid requestId, CancellationToken cancellationToken) + { + return halibutRedisTransport.IsRequestStillOnQueue(endpoint, requestId, cancellationToken); + } + + public Task SubscribeToRequestCancellation(Uri endpoint, Guid request, Func onCancellationReceived, CancellationToken cancellationToken) + { + return halibutRedisTransport.SubscribeToRequestCancellation(endpoint, request, onCancellationReceived, cancellationToken); + } + + public Task PublishCancellation(Uri endpoint, Guid requestId, CancellationToken cancellationToken) + { + return halibutRedisTransport.PublishCancellation(endpoint, requestId, cancellationToken); + } + + public string RequestCancelledMarkerKey(Uri endpoint, Guid requestId) + { + return halibutRedisTransport.RequestCancelledMarkerKey(endpoint, requestId); + } + + public Task MarkRequestAsCancelled(Uri endpoint, Guid requestId, TimeSpan ttl, CancellationToken cancellationToken) + { + return halibutRedisTransport.MarkRequestAsCancelled(endpoint, requestId, ttl, cancellationToken); + } + + public Task IsRequestMarkedAsCancelled(Uri endpoint, Guid requestId, CancellationToken cancellationToken) + { + return halibutRedisTransport.IsRequestMarkedAsCancelled(endpoint, requestId, cancellationToken); + } + + public Task SubscribeToNodeHeartBeatChannel(Uri endpoint, Guid request, HalibutQueueNodeSendingPulses nodeSendingPulsesType, Func onHeartBeat, CancellationToken cancellationToken) + { + return halibutRedisTransport.SubscribeToNodeHeartBeatChannel(endpoint, request, nodeSendingPulsesType, onHeartBeat, cancellationToken); + } + + public Task SendHeartBeatFromNodeProcessingTheRequest(Uri endpoint, Guid requestId, HalibutQueueNodeSendingPulses nodeSendingPulsesType, CancellationToken cancellationToken) + { + return halibutRedisTransport.SendHeartBeatFromNodeProcessingTheRequest(endpoint, requestId, nodeSendingPulsesType, cancellationToken); + } + + public Task SendHeartBeatFromNodeProcessingTheRequest(Uri endpoint, Guid requestId, CancellationToken cancellationToken) + { + return halibutRedisTransport.SendHeartBeatFromNodeProcessingTheRequest(endpoint, requestId, cancellationToken); + } + + public Task SubscribeToNodeProcessingTheRequestHeartBeatChannel(Uri endpoint, Guid request, Func onHeartBeat, CancellationToken cancellationToken) + { + return halibutRedisTransport.SubscribeToNodeProcessingTheRequestHeartBeatChannel(endpoint, request, onHeartBeat, cancellationToken); + } + + public Task SubscribeToResponseChannel(Uri endpoint, Guid identifier, Func onValueReceived, CancellationToken cancellationToken) + { + return halibutRedisTransport.SubscribeToResponseChannel(endpoint, identifier, onValueReceived, cancellationToken); + } + + public Task PublishThatResponseIsAvailable(Uri endpoint, Guid identifier, string value, CancellationToken cancellationToken) + { + return halibutRedisTransport.PublishThatResponseIsAvailable(endpoint, identifier, value, cancellationToken); + } + + public Task MarkThatResponseIsSet(Uri endpoint, Guid identifier, string value, TimeSpan ttl, CancellationToken cancellationToken) + { + return halibutRedisTransport.MarkThatResponseIsSet(endpoint, identifier, value, ttl, cancellationToken); + } + + public Task GetResponseMessage(Uri endpoint, Guid identifier, CancellationToken cancellationToken) + { + return halibutRedisTransport.GetResponseMessage(endpoint, identifier, cancellationToken); + } + + public Task DeleteResponse(Uri endpoint, Guid identifier, CancellationToken cancellationToken) + { + return halibutRedisTransport.DeleteResponse(endpoint, identifier, cancellationToken); + } + } +} \ No newline at end of file diff --git a/source/Halibut.Tests/Queue/Redis/Utils/MessageReaderWriterExtensionsMethods.cs b/source/Halibut.Tests/Queue/Redis/Utils/MessageReaderWriterExtensionsMethods.cs index fd05e7bb0..60489c56e 100644 --- a/source/Halibut.Tests/Queue/Redis/Utils/MessageReaderWriterExtensionsMethods.cs +++ b/source/Halibut.Tests/Queue/Redis/Utils/MessageReaderWriterExtensionsMethods.cs @@ -26,35 +26,68 @@ public static IMessageReaderWriter ThrowsOnReadResponse(this IMessageReaderWrite { return new MessageReaderWriterThatThrowsWhenReadingResponse(messageReaderWriter, exceptionFactory); } - } - class MessageReaderWriterThatThrowsWhenReadingResponse : IMessageReaderWriter + public static IMessageReaderWriter ThrowsOnPrepareRequest(this IMessageReaderWriter messageReaderWriter, Func exception) + { + return new MessageReaderWriterThatThrowsOnPrepareRequest(messageReaderWriter, exception); + } +} + + class MessageReaderWriterWithVirtualMethods : IMessageReaderWriter { readonly IMessageReaderWriter messageReaderWriter; - readonly Func exception; - public MessageReaderWriterThatThrowsWhenReadingResponse(IMessageReaderWriter messageReaderWriter, Func exception) + public MessageReaderWriterWithVirtualMethods(IMessageReaderWriter messageReaderWriter) { this.messageReaderWriter = messageReaderWriter; - this.exception = exception; } - public Task PrepareRequest(RequestMessage request, CancellationToken cancellationToken) + public virtual Task PrepareRequest(RequestMessage request, CancellationToken cancellationToken) { return messageReaderWriter.PrepareRequest(request, cancellationToken); } - public Task ReadRequest(string jsonRequest, CancellationToken cancellationToken) + public virtual Task ReadRequest(string jsonRequest, CancellationToken cancellationToken) { return messageReaderWriter.ReadRequest(jsonRequest, cancellationToken); } - public Task PrepareResponse(ResponseMessage response, CancellationToken cancellationToken) + public virtual Task PrepareResponse(ResponseMessage response, CancellationToken cancellationToken) { return messageReaderWriter.PrepareResponse(response, cancellationToken); } - public Task ReadResponse(string jsonResponse, CancellationToken cancellationToken) + public virtual Task ReadResponse(string jsonResponse, CancellationToken cancellationToken) + { + return messageReaderWriter.ReadResponse(jsonResponse, cancellationToken); + } + } + + class MessageReaderWriterThatThrowsWhenReadingResponse : MessageReaderWriterWithVirtualMethods + { + readonly Func exception; + + public MessageReaderWriterThatThrowsWhenReadingResponse(IMessageReaderWriter messageReaderWriter, Func exception) : base(messageReaderWriter) + { + this.exception = exception; + } + + public override Task ReadResponse(string jsonResponse, CancellationToken cancellationToken) + { + throw exception(); + } + } + + class MessageReaderWriterThatThrowsOnPrepareRequest : MessageReaderWriterWithVirtualMethods + { + readonly Func exception; + + public MessageReaderWriterThatThrowsOnPrepareRequest(IMessageReaderWriter messageReaderWriter, Func exception) : base(messageReaderWriter) + { + this.exception = exception; + } + + public override Task PrepareRequest(RequestMessage request, CancellationToken cancellationToken) { throw exception(); } diff --git a/source/Halibut.Tests/Util/AssertThrowsAny.cs b/source/Halibut.Tests/Util/AssertThrowsAny.cs new file mode 100644 index 000000000..04ea81f09 --- /dev/null +++ b/source/Halibut.Tests/Util/AssertThrowsAny.cs @@ -0,0 +1,38 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; +using System.Threading.Tasks; +using NUnit.Framework; + +namespace Halibut.Tests.Util +{ + public static class AssertThrowsAny + { + public static async Task Exception(Func action) + { + try + { + await action(); + Assert.Fail("Should have thrown an exception."); + } + catch (Exception exception) + { + return exception; + } + + throw new Exception("Impossible?"); + } + } +} \ No newline at end of file diff --git a/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs b/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs index 13255096a..d0b5f3fc9 100644 --- a/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs +++ b/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs @@ -3,6 +3,7 @@ using System.Net.Sockets; using Halibut.Exceptions; using Halibut.Queue.Redis; +using Halibut.Queue.Redis.Exceptions; using Halibut.Transport; using Halibut.Transport.Protocol; using Halibut.Transport.Proxy.Exceptions; @@ -44,7 +45,11 @@ public static HalibutRetryableErrorType IsRetryableError(this Exception exceptio public static HalibutNetworkExceptionType IsNetworkError(this Exception exception) { // TODO: This should be in is retryable but for now it needs to be here to work with tentacle client. - if (exception is RedisDataLoseHalibutClientException) + if (exception is RedisDataLoseHalibutClientException + || exception is RedisQueueShutdownClientException + || exception is CouldNotGetDataLoseTokenInTimeHalibutClientException + || exception is ErrorWhilePreparingRequestForQueueHalibutClientException + || exception is ErrorOccuredWhenInsertingDataIntoRedisHalibutPendingRequestQueue) { return HalibutNetworkExceptionType.IsNetworkError; } @@ -54,6 +59,7 @@ public static HalibutNetworkExceptionType IsNetworkError(this Exception exceptio if (exception.Message.Contains("The request was abandoned, possibly because the node processing the request shutdown or redis lost all of its data.")) return HalibutNetworkExceptionType.IsNetworkError; if (exception.Message.Contains("The node processing the request did not send a heartbeat for long enough, and so the node is now assumed to be offline.")) return HalibutNetworkExceptionType.IsNetworkError; if (exception.Message.Contains("Error occured when reading data from the queue")) return HalibutNetworkExceptionType.IsNetworkError; + if(exception.Message.Contains("error occured when preparing request for queue")) return HalibutNetworkExceptionType.IsNetworkError; } // TODO end diff --git a/source/Halibut/Queue/Redis/Exceptions/CouldNotGetDataLoseTokenInTimeHalibutClientException.cs b/source/Halibut/Queue/Redis/Exceptions/CouldNotGetDataLoseTokenInTimeHalibutClientException.cs new file mode 100644 index 000000000..23c71c2c9 --- /dev/null +++ b/source/Halibut/Queue/Redis/Exceptions/CouldNotGetDataLoseTokenInTimeHalibutClientException.cs @@ -0,0 +1,25 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; + +namespace Halibut.Queue.Redis +{ + public class CouldNotGetDataLoseTokenInTimeHalibutClientException : HalibutClientException + { + public CouldNotGetDataLoseTokenInTimeHalibutClientException(string message, Exception inner) : base(message, inner) + { + } + } +} \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/Exceptions/ErrorOccuredWhenInsertingDataIntoRedisHalibutPendingRequestQueue.cs b/source/Halibut/Queue/Redis/Exceptions/ErrorOccuredWhenInsertingDataIntoRedisHalibutPendingRequestQueue.cs new file mode 100644 index 000000000..b61d10b97 --- /dev/null +++ b/source/Halibut/Queue/Redis/Exceptions/ErrorOccuredWhenInsertingDataIntoRedisHalibutPendingRequestQueue.cs @@ -0,0 +1,25 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; + +namespace Halibut.Queue.Redis.Exceptions +{ + public class ErrorOccuredWhenInsertingDataIntoRedisHalibutPendingRequestQueue : HalibutClientException + { + public ErrorOccuredWhenInsertingDataIntoRedisHalibutPendingRequestQueue(string message, Exception inner) : base(message, inner) + { + } + } +} \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/Exceptions/ErrorWhilePreparingRequestForQueueHalibutClientException.cs b/source/Halibut/Queue/Redis/Exceptions/ErrorWhilePreparingRequestForQueueHalibutClientException.cs new file mode 100644 index 000000000..11a99d35a --- /dev/null +++ b/source/Halibut/Queue/Redis/Exceptions/ErrorWhilePreparingRequestForQueueHalibutClientException.cs @@ -0,0 +1,25 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; + +namespace Halibut.Queue.Redis.Exceptions +{ + public class ErrorWhilePreparingRequestForQueueHalibutClientException : HalibutClientException + { + public ErrorWhilePreparingRequestForQueueHalibutClientException(string message, Exception inner) : base(message, inner) + { + } + } +} \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/RedisDataLoseHalibutClientException.cs b/source/Halibut/Queue/Redis/Exceptions/RedisDataLoseHalibutClientException.cs similarity index 100% rename from source/Halibut/Queue/Redis/RedisDataLoseHalibutClientException.cs rename to source/Halibut/Queue/Redis/Exceptions/RedisDataLoseHalibutClientException.cs diff --git a/source/Halibut/Queue/Redis/Exceptions/RedisQueueShutdownClientException.cs b/source/Halibut/Queue/Redis/Exceptions/RedisQueueShutdownClientException.cs new file mode 100644 index 000000000..efede93f3 --- /dev/null +++ b/source/Halibut/Queue/Redis/Exceptions/RedisQueueShutdownClientException.cs @@ -0,0 +1,23 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +namespace Halibut.Queue.Redis +{ + public class RedisQueueShutdownClientException : HalibutClientException + { + public RedisQueueShutdownClientException(string message) : base(message) + { + } + } +} \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/HalibutRedisTransport.cs b/source/Halibut/Queue/Redis/HalibutRedisTransport.cs index 067434d5e..99b2f5ca2 100644 --- a/source/Halibut/Queue/Redis/HalibutRedisTransport.cs +++ b/source/Halibut/Queue/Redis/HalibutRedisTransport.cs @@ -18,12 +18,11 @@ using System.Threading.Tasks; using Halibut.Util; using Newtonsoft.Json; -using Nito.AsyncEx; using StackExchange.Redis; namespace Halibut.Queue.Redis { - public class HalibutRedisTransport + public class HalibutRedisTransport : IHalibutRedisTransport { const string Namespace = "octopus:server:halibut"; @@ -238,13 +237,13 @@ public async Task MarkThatResponseIsSet(Uri endpoint, Guid identifier, string va await facade.SetString(key, value, ttl, cancellationToken); } - public async Task GetGenericMarker(Uri endpoint, Guid identifier, CancellationToken cancellationToken) + public async Task GetResponseMessage(Uri endpoint, Guid identifier, CancellationToken cancellationToken) { var key = ResponseMarkerKey(endpoint, identifier); return await facade.GetString(key, cancellationToken); } - public async Task DeleteResponseMarker(Uri endpoint, Guid identifier, CancellationToken cancellationToken) + public async Task DeleteResponse(Uri endpoint, Guid identifier, CancellationToken cancellationToken) { var key = ResponseMarkerKey(endpoint, identifier); return await facade.DeleteString(key, cancellationToken); diff --git a/source/Halibut/Queue/Redis/IHalibutRedisTransport.cs b/source/Halibut/Queue/Redis/IHalibutRedisTransport.cs new file mode 100644 index 000000000..9646e6bd8 --- /dev/null +++ b/source/Halibut/Queue/Redis/IHalibutRedisTransport.cs @@ -0,0 +1,66 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; +using System.Threading; +using System.Threading.Tasks; +using StackExchange.Redis; + +namespace Halibut.Queue.Redis +{ + public interface IHalibutRedisTransport + { + Task SubscribeToRequestMessagePulseChannel(Uri endpoint, Action onRequestMessagePulse, CancellationToken cancellationToken); + Task PulseRequestPushedToEndpoint(Uri endpoint, CancellationToken cancellationToken); + Task PushRequestGuidOnToQueue(Uri endpoint, Guid guid, CancellationToken cancellationToken); + Task TryPopNextRequestGuid(Uri endpoint, CancellationToken cancellationToken); + Task PutRequest(Uri endpoint, Guid requestId, string payload, TimeSpan requestPickupTimeout, CancellationToken cancellationToken); + Task TryGetAndRemoveRequest(Uri endpoint, Guid requestId, CancellationToken cancellationToken); + Task IsRequestStillOnQueue(Uri endpoint, Guid requestId, CancellationToken cancellationToken); + + Task SubscribeToRequestCancellation(Uri endpoint, Guid request, + Func onCancellationReceived, + CancellationToken cancellationToken); + + Task PublishCancellation(Uri endpoint, Guid requestId, CancellationToken cancellationToken); + string RequestCancelledMarkerKey(Uri endpoint, Guid requestId); + Task MarkRequestAsCancelled(Uri endpoint, Guid requestId, TimeSpan ttl, CancellationToken cancellationToken); + Task IsRequestMarkedAsCancelled(Uri endpoint, Guid requestId, CancellationToken cancellationToken); + + Task SubscribeToNodeHeartBeatChannel( + Uri endpoint, + Guid request, + HalibutQueueNodeSendingPulses nodeSendingPulsesType, + Func onHeartBeat, + CancellationToken cancellationToken); + + Task SendHeartBeatFromNodeProcessingTheRequest(Uri endpoint, Guid requestId, HalibutQueueNodeSendingPulses nodeSendingPulsesType, CancellationToken cancellationToken); + Task SendHeartBeatFromNodeProcessingTheRequest(Uri endpoint, Guid requestId, CancellationToken cancellationToken); + + Task SubscribeToNodeProcessingTheRequestHeartBeatChannel( + Uri endpoint, + Guid request, + Func onHeartBeat, + CancellationToken cancellationToken); + + Task SubscribeToResponseChannel(Uri endpoint, Guid identifier, + Func onValueReceived, + CancellationToken cancellationToken); + + Task PublishThatResponseIsAvailable(Uri endpoint, Guid identifier, string value, CancellationToken cancellationToken); + Task MarkThatResponseIsSet(Uri endpoint, Guid identifier, string value, TimeSpan ttl, CancellationToken cancellationToken); + Task GetResponseMessage(Uri endpoint, Guid identifier, CancellationToken cancellationToken); + Task DeleteResponse(Uri endpoint, Guid identifier, CancellationToken cancellationToken); + } +} \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs b/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs index 13517c81a..77b7dae4a 100644 --- a/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs +++ b/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs @@ -37,7 +37,7 @@ public enum NodeProcessingRequestWatcherResult readonly Uri endpoint; readonly Guid requestActivityId; - private readonly HalibutRedisTransport halibutRedisTransport; + private readonly IHalibutRedisTransport halibutRedisTransport; private readonly CancelOnDisposeCancellationToken cts; private readonly ILog log; private readonly HalibutQueueNodeSendingPulses nodeSendingPulsesType; @@ -46,7 +46,7 @@ public enum NodeProcessingRequestWatcherResult public NodeHeartBeatSender( Uri endpoint, Guid requestActivityId, - HalibutRedisTransport halibutRedisTransport, + IHalibutRedisTransport halibutRedisTransport, ILog log, HalibutQueueNodeSendingPulses nodeSendingPulsesType, TimeSpan defaultDelayBetweenPulses) @@ -96,7 +96,7 @@ public static async Task WatchThatNodeProces Uri endpoint, RequestMessage request, RedisPendingRequest redisPending, - HalibutRedisTransport halibutRedisTransport, + IHalibutRedisTransport halibutRedisTransport, TimeSpan timeBetweenCheckingIfRequestWasCollected, ILog log, TimeSpan maxTimeBetweenHeartBeetsBeforeProcessingNodeIsAssumedToBeOffline, @@ -111,11 +111,7 @@ public static async Task WatchThatNodeProces return await WatchForPulsesFromNode(endpoint, request.ActivityId, halibutRedisTransport, log, maxTimeBetweenHeartBeetsBeforeProcessingNodeIsAssumedToBeOffline, HalibutQueueNodeSendingPulses.Receiver, cts.Token); } - catch (Exception) when (!cts.Token.IsCancellationRequested) - { - throw; - } - catch (Exception) + catch (Exception) when (cts.Token.IsCancellationRequested) { return NodeProcessingRequestWatcherResult.NoDisconnectSeen; } @@ -124,7 +120,7 @@ public static async Task WatchThatNodeProces public static async Task WatchThatNodeWhichSentTheRequestIsStillAlive( Uri endpoint, Guid requestActivityId, - HalibutRedisTransport halibutRedisTransport, + IHalibutRedisTransport halibutRedisTransport, ILog log, TimeSpan maxTimeBetweenSenderHeartBeetsBeforeSenderIsAssumedToBeOffline, CancellationToken watchCancellationToken) @@ -135,7 +131,7 @@ public static async Task WatchThatNodeWhichS private static async Task WatchForPulsesFromNode( Uri endpoint, Guid requestActivityId, - HalibutRedisTransport halibutRedisTransport, + IHalibutRedisTransport halibutRedisTransport, ILog log, TimeSpan maxTimeBetweenHeartBeetsBeforeNodeIsAssumedToBeOffline, HalibutQueueNodeSendingPulses watchingForPulsesFrom, @@ -186,7 +182,7 @@ private static async Task WatchForPulsesFrom } } - static async Task WaitForRequestToBeCollected(Uri endpoint, RequestMessage request, RedisPendingRequest redisPending, HalibutRedisTransport halibutRedisTransport, + static async Task WaitForRequestToBeCollected(Uri endpoint, RequestMessage request, RedisPendingRequest redisPending, IHalibutRedisTransport halibutRedisTransport, TimeSpan timeBetweenCheckingIfRequestWasCollected, ILog log, CancellationToken cancellationToken) { diff --git a/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs b/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs index 35b99e20d..b2c028001 100644 --- a/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs +++ b/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs @@ -24,7 +24,7 @@ namespace Halibut.Queue.Redis public class PollAndSubscribeToResponse : IAsyncDisposable { public static async Task TrySendMessage( - HalibutRedisTransport halibutRedisTransport, + IHalibutRedisTransport halibutRedisTransport, Uri endpoint, Guid activityId, string value, @@ -60,16 +60,16 @@ public static async Task TrySendMessage( readonly ILog log; - readonly HalibutRedisTransport halibutRedisTransport; + readonly IHalibutRedisTransport halibutRedisTransport; readonly Uri endpoint; readonly Guid activityId; readonly LinearBackoffStrategy pollBackoffStrategy; - TaskCompletionSource message = new(); + TaskCompletionSource ResponseJsonCompletionSource = new(); - public Task ResultTask => message.Task; + public Task ResponseJson => ResponseJsonCompletionSource.Task; - public PollAndSubscribeToResponse(Uri endpoint, Guid activityId, HalibutRedisTransport halibutRedisTransport, ILog log) + public PollAndSubscribeToResponse(Uri endpoint, Guid activityId, IHalibutRedisTransport halibutRedisTransport, ILog log) { this.log = log.ForContext(); @@ -90,28 +90,6 @@ public PollAndSubscribeToResponse(Uri endpoint, Guid activityId, HalibutRedisTra readonly SemaphoreSlim trySetResultSemaphore = new SemaphoreSlim(1, 1); - public async Task TrySetResultAndRemoveValueFromRedis(string value, CancellationToken cancellationToken) - { - using var l = await trySetResultSemaphore.LockAsync(cancellationToken); - try - { - if(!message.Task.IsCompleted) message.TrySetResult(value); - } - catch (Exception) - { - // TODO log we could not set result. - } - - try - { - await halibutRedisTransport.DeleteResponseMarker(endpoint, activityId, cancellationToken); - } - catch (Exception) - { - // TODO log we could not remove value - } - - } async Task WaitForResponse(CancellationToken token) { try @@ -123,16 +101,7 @@ async Task WaitForResponse(CancellationToken token) { log.Write(EventType.Diagnostic, "Received response notification via subscription - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); - - var value = await halibutRedisTransport.GetGenericMarker(endpoint, activityId, token); - if (value != null) - { - await TrySetResultAndRemoveValueFromRedis(value, token); - } - - log.Write(EventType.Diagnostic, "Cancelling polling loop for response - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); - - await Try.IgnoringError(async () => await watcherToken.CancelAsync()); + await TryGetResponseFromRedis("subscription", token); }, token); @@ -144,12 +113,8 @@ async Task WaitForResponse(CancellationToken token) { try { - var value = await halibutRedisTransport.GetGenericMarker(endpoint, activityId, token); - if (value != null) + if (await TryGetResponseFromRedis("polling", token)) { - log.Write(EventType.Diagnostic, "Response detected via polling - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); - await TrySetResultAndRemoveValueFromRedis(value, token); - await watcherToken.CancelAsync(); break; } } @@ -176,6 +141,53 @@ async Task WaitForResponse(CancellationToken token) } } + /// + /// Makes an attempt to get the response from redis. + /// + /// + /// + /// true if a response message is available. + public async Task TryGetResponseFromRedis(string detectedBy, CancellationToken token) + { + using var l = await trySetResultSemaphore.LockAsync(token); + + if (ResponseJsonCompletionSource.Task.IsCompleted) return true; + + var responseJson = await halibutRedisTransport.GetResponseMessage(endpoint, activityId, token); + if (responseJson != null) + { + log.Write(EventType.Diagnostic, "Response detected via {0} - Endpoint: {1}, ActivityId: {2}", detectedBy, endpoint, activityId); + await TrySetResponse(responseJson, token); + await Try.IgnoringError(async () => await watcherToken.CancelAsync()); + log.Write(EventType.Diagnostic, "Cancelling polling loop for response - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); + return true; + } + + return false; + } + + async Task TrySetResponse(string value, CancellationToken cancellationToken) + { + try + { + ResponseJsonCompletionSource.TrySetResult(value); + } + catch (Exception ex) + { + log.Write(EventType.Error, "Failed to set response - Endpoint: {0}, ActivityId: {1}, Error: {2}", endpoint, activityId, ex.Message); + } + + try + { + await halibutRedisTransport.DeleteResponse(endpoint, activityId, cancellationToken); + } + catch (Exception ex) + { + log.Write(EventType.Error, "Failed to delete response from Redis - Endpoint: {0}, ActivityId: {1}, Error: {2}", endpoint, activityId, ex.Message); + } + + } + public async ValueTask DisposeAsync() { log.Write(EventType.Diagnostic, "Disposing GenericWatcher for response - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); @@ -183,7 +195,7 @@ public async ValueTask DisposeAsync() await Try.IgnoringError(async () => await watcherToken.CancelAsync()); // If the message task is not yet complete, then mark it as cancelled - Try.IgnoringError(() => message.TrySetCanceled()); + Try.IgnoringError(() => ResponseJsonCompletionSource.TrySetCanceled()); log.Write(EventType.Diagnostic, "Disposed GenericWatcher for response - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); } diff --git a/source/Halibut/Queue/Redis/RedisFacade.cs b/source/Halibut/Queue/Redis/RedisFacade.cs index 90a4ea73a..1c8549d9f 100644 --- a/source/Halibut/Queue/Redis/RedisFacade.cs +++ b/source/Halibut/Queue/Redis/RedisFacade.cs @@ -57,7 +57,9 @@ public class RedisFacade : IAsyncDisposable { readonly Lazy connection; readonly ILog log; - + // We can survive redis being unavailable for this amount of time. + internal TimeSpan MaxDurationToRetryFor = TimeSpan.FromSeconds(30); + ConnectionMultiplexer Connection => connection.Value; string keyPrefix; @@ -122,9 +124,6 @@ private void OnConnectionRestored(object? sender, ConnectionFailedEventArgs e) var message = $"Redis connection restored - EndPoint: {e.EndPoint}"; log?.Write(EventType.Diagnostic, message); } - - // We can survive redis being unavailable for this amount of time. - static readonly TimeSpan MaxDurationToRetryFor = TimeSpan.FromSeconds(30); /// /// Executes an operation with retry logic. Retries for up to 12 seconds with 1-second intervals. diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index 8e248a83f..ad6947294 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -17,6 +17,7 @@ using System.Threading; using System.Threading.Tasks; using Halibut.Diagnostics; +using Halibut.Queue.Redis.Exceptions; using Halibut.ServiceModel; using Halibut.Transport.Protocol; using Halibut.Util; @@ -29,13 +30,13 @@ class RedisPendingRequestQueue : IPendingRequestQueue, IDisposable readonly Uri endpoint; readonly IWatchForRedisLosingAllItsData watchForRedisLosingAllItsData; readonly ILog log; - readonly HalibutRedisTransport halibutRedisTransport; + readonly IHalibutRedisTransport halibutRedisTransport; readonly HalibutTimeoutsAndLimits halibutTimeoutsAndLimits; readonly IMessageReaderWriter messageReaderWriter; readonly AsyncManualResetEvent hasItemsForEndpoint = new(); readonly CancelOnDisposeCancellationToken queueCts = new (); - internal ConcurrentDictionary disposablesForInFlightRequests = new(); + internal ConcurrentDictionary DisposablesForInFlightRequests = new(); // TODO: this needs to be used in all public methods. readonly CancellationToken queueToken; @@ -48,7 +49,7 @@ public RedisPendingRequestQueue( Uri endpoint, IWatchForRedisLosingAllItsData watchForRedisLosingAllItsData, ILog log, - HalibutRedisTransport halibutRedisTransport, + IHalibutRedisTransport halibutRedisTransport, IMessageReaderWriter messageReaderWriter, HalibutTimeoutsAndLimits halibutTimeoutsAndLimits) { @@ -85,19 +86,47 @@ private async Task DataLossCancellationToken(CancellationToke public async Task QueueAndWaitAsync(RequestMessage request, CancellationToken requestCancellationToken) { + CancellationToken dataLoseCt; + try + { + dataLoseCt = await DataLossCancellationToken(requestCancellationToken); + } + catch (Exception ex) + { + if (requestCancellationToken.IsCancellationRequested) throw RedisPendingRequest.CreateExceptionForRequestWasCancelledBeforeCollected(request, log); + throw new CouldNotGetDataLoseTokenInTimeHalibutClientException("Unable to reconnect to redis to get data loss detection CT", ex); + } + + Exception? CancellationReason() + { + if (dataLoseCt.IsCancellationRequested) return new RedisDataLoseHalibutClientException($"Request {request.ActivityId} was cancelled because we detected that redis lost all of its data."); + if (queueToken.IsCancellationRequested) return new RedisQueueShutdownClientException($"Request {request.ActivityId} was cancelled because the queue is shutting down."); + return null; + } + + Exception? CreateCancellationExceptionIfCancelled() + { + if (requestCancellationToken.IsCancellationRequested) return RedisPendingRequest.CreateExceptionForRequestWasCancelledBeforeCollected(request, log); + return CancellationReason(); + } - var dataLoseCt = await DataLossCancellationToken(requestCancellationToken); - + await using var cts = new CancelOnDisposeCancellationToken(queueCts.Token, requestCancellationToken, dataLoseCt); - var cancellationToken = cts.Token; - // TODO RedisConnectionException can be raised out of here, what should the queue do? - // TODO it must raise an exception that supports being retried. + using var pending = new RedisPendingRequest(request, log); + + string payload; + try + { + payload = await messageReaderWriter.PrepareRequest(request, cancellationToken); + } + catch (Exception ex) + { + throw CreateCancellationExceptionIfCancelled() + ?? new ErrorWhilePreparingRequestForQueueHalibutClientException($"Request {request.ActivityId} failed since an error occured when preparing request for queue", ex); + } - // TODO: What if this payload was gigantic - // TODO: Do we need to encrypt this? - var payload = await messageReaderWriter.PrepareRequest(request, cancellationToken); // Start listening for a response to the request, we don't want to miss the response. await using var pollAndSubscribeToResponse = new PollAndSubscribeToResponse(endpoint, request.ActivityId, halibutRedisTransport, log); @@ -107,29 +136,31 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can { await using var senderPulse = new NodeHeartBeatSender(endpoint, request.ActivityId, halibutRedisTransport, log, HalibutQueueNodeSendingPulses.Sender, DelayBetweenHeartBeatsForRequestSender); // Make the request available before we tell people it is available. - await halibutRedisTransport.PutRequest(endpoint, request.ActivityId, payload, request.Destination.PollingRequestQueueTimeout, cancellationToken); - await halibutRedisTransport.PushRequestGuidOnToQueue(endpoint, request.ActivityId, cancellationToken); - await halibutRedisTransport.PulseRequestPushedToEndpoint(endpoint, cancellationToken); - Interlocked.Increment(ref numberOfInFlightRequestsThatHaveReachedTheStageOfBeingReadyForCollection); try { + await halibutRedisTransport.PutRequest(endpoint, request.ActivityId, payload, request.Destination.PollingRequestQueueTimeout, cancellationToken); + await halibutRedisTransport.PushRequestGuidOnToQueue(endpoint, request.ActivityId, cancellationToken); + await halibutRedisTransport.PulseRequestPushedToEndpoint(endpoint, cancellationToken); + } + catch (Exception ex) + { + throw CreateCancellationExceptionIfCancelled() + ?? new ErrorOccuredWhenInsertingDataIntoRedisHalibutPendingRequestQueue($"Request {request.ActivityId} failed since an error occured inserting the data into the queue", ex); + } - await using var watcherCts = new CancelOnDisposeCancellationToken(cts.Token); - WatchProcessingNodeIsStillConnectedInBackground(request, pending, watcherCts); - - // TODO: We need to ensure that no matter what exceptions are thrown we eventually exit. - // For example can the subscription to the response, fail and never come back? - // Can the WatchProcessProcessingNodeIsStillConnected fail and never come back? + Interlocked.Increment(ref numberOfInFlightRequestsThatHaveReachedTheStageOfBeingReadyForCollection); + try + { + // We must be careful here to ensure we will always return. + var watchProcessingNodeStillHasHeartBeat = WatchProcessingNodeIsStillConnectedInBackground(request, pending, cancellationToken); var waitingForResponse = WaitForResponse(pollAndSubscribeToResponse, request, cancellationToken); var pendingRequestWaitUntilComplete = pending.WaitUntilComplete( async () => await tryClearRequestFromQueueAtMostOnce.Task, - () => dataLoseCt.IsCancellationRequested ? - new RedisDataLoseHalibutClientException($"Request {request.ActivityId} was cancelled because we detected that redis lost all of its data.") - : null, + CancellationReason, cancellationToken); - await Task.WhenAny(waitingForResponse, pendingRequestWaitUntilComplete); + await Task.WhenAny(waitingForResponse, pendingRequestWaitUntilComplete, watchProcessingNodeStillHasHeartBeat); if (pendingRequestWaitUntilComplete.IsCompleted || cancellationToken.IsCancellationRequested) { @@ -142,21 +173,37 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can var response = await waitingForResponse; if (response != null) { - pending.SetResponse(response); - return pending.Response!; + return await pending.SetResponse(response); } else if(!cancellationToken.IsCancellationRequested) { // We are no longer waiting for a response and have no response. // The cancellation token has not been set so the request is not going to be cancelled. // It is unclear how we got into this state, but lets at least error out. - pending.SetResponse(ResponseMessage.FromError(request, "Queue unexpectedly stopped waiting for a response")); - return pending.Response!; + return await pending.SetResponse(ResponseMessage.FromError(request, "Queue unexpectedly stopped waiting for a response")); } } - pending.SetResponse(ResponseMessage.FromError(request, "Impossible queue state reached")); - return pending.Response!; + if (watchProcessingNodeStillHasHeartBeat.IsCompleted) + { + var watcherResult = await watchProcessingNodeStillHasHeartBeat; + if (watcherResult == NodeHeartBeatSender.NodeProcessingRequestWatcherResult.NodeMayHaveDisconnected) + { + // Make a list ditch effort to check if a response exists now. + if (await pollAndSubscribeToResponse.TryGetResponseFromRedis("Watcher", cancellationToken)) + { + var response = await waitingForResponse; + if (response != null) + { + return await pending.SetResponse(response); + } + } + + return await pending.SetResponse(ResponseMessage.FromError(request, "The node processing the request did not send a heartbeat for long enough, and so the node is now assumed to be offline.")); + } + } + + return await pending.SetResponse(ResponseMessage.FromError(request, "Impossible queue state reached")); } finally { @@ -165,23 +212,28 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can } finally { - // Make an attempt to ensure the request is removed from redis. - var background = Task.Run(async () => await Try.IgnoringError(async () => await tryClearRequestFromQueueAtMostOnce.Task)); - var backgroundCancellation = Task.Run(async () => + InBackgroundSendCancellationIfRequestWasCancelled(request, pending); + // Make an attempt to ensure the request is removed from redis, if we are unsure it was removed. + var background = Task.Run(async () => await Try.IgnoringError(async () => { - if(requestCancellationToken.IsCancellationRequested) await SendCancellationIfRequestWasCancelled(request, pending); - }); + if (pending.HasRequestBeenMarkedAsCollected + || !pollAndSubscribeToResponse.ResponseJson.IsCompletedSuccessfully) + { + await tryClearRequestFromQueueAtMostOnce.Task; + } + })); } } - async Task SendCancellationIfRequestWasCancelled(RequestMessage request, RedisPendingRequest redisPending) + + void InBackgroundSendCancellationIfRequestWasCancelled(RequestMessage request, RedisPendingRequest redisPending) { if (redisPending.PendingRequestCancellationToken.IsCancellationRequested) { log.Write(EventType.Diagnostic, "Request {0} was cancelled, sending cancellation to endpoint {1}", request.ActivityId, endpoint); - await WatchForRequestCancellation.TrySendCancellation(halibutRedisTransport, endpoint, request, log); + Task.Run(async () => await WatchForRequestCancellation.TrySendCancellation(halibutRedisTransport, endpoint, request, log)); } else { @@ -189,38 +241,27 @@ async Task SendCancellationIfRequestWasCancelled(RequestMessage request, RedisPe } } - void WatchProcessingNodeIsStillConnectedInBackground(RequestMessage request, RedisPendingRequest redisPending, CancelOnDisposeCancellationToken watcherCts) + async Task WatchProcessingNodeIsStillConnectedInBackground(RequestMessage request, RedisPendingRequest redisPending, CancellationToken cancellationToken) { - Task.Run(async () => + await Task.Yield(); + try { - var watcherCtsCancellationToken = watcherCts.Token; - try - { - var disconnected = await NodeHeartBeatSender.WatchThatNodeProcessingTheRequestIsStillAlive( - endpoint, - request, - redisPending, - halibutRedisTransport, - TimeBetweenCheckingIfRequestWasCollected, - log, - RequestReceivingNodeIsOfflineHeartBeatTimeout, - watcherCtsCancellationToken); - if (!watcherCtsCancellationToken.IsCancellationRequested && disconnected == NodeHeartBeatSender.NodeProcessingRequestWatcherResult.NodeMayHaveDisconnected) - { - // TODO: if(responseWatcher.CheckForResponseNow() == ResponseNotFound) { - redisPending.SetResponse(ResponseMessage.FromError(request, "The node processing the request did not send a heartbeat for long enough, and so the node is now assumed to be offline.")); - //} - } - } - catch (Exception) when (watcherCtsCancellationToken.IsCancellationRequested) - { - log.Write(EventType.Diagnostic, "Processing node watcher cancelled for request {0}, endpoint {1}", request.ActivityId, endpoint); - } - catch (Exception ex) - { - log.WriteException(EventType.Error, "Error watching processing node for request {0}, endpoint {1}", ex, request.ActivityId, endpoint); - } - }); + return await NodeHeartBeatSender.WatchThatNodeProcessingTheRequestIsStillAlive( + endpoint, + request, + redisPending, + halibutRedisTransport, + TimeBetweenCheckingIfRequestWasCollected, + log, + RequestReceivingNodeIsOfflineHeartBeatTimeout, + cancellationToken); + } + catch (Exception ex) + { + log.WriteException(EventType.Error, "Error watching processing node for request {0}, endpoint {1}", ex, request.ActivityId, endpoint); + } + + return null; } async Task TryClearRequestFromQueue(RequestMessage request, RedisPendingRequest redisPending) @@ -274,7 +315,7 @@ async Task TryClearRequestFromQueue(RequestMessage request, RedisPendingRe try { log.Write(EventType.Diagnostic, "Waiting for response for request {0}", activityId); - responseJson = await pollAndSubscribeToResponse.ResultTask; + responseJson = await pollAndSubscribeToResponse.ResponseJson; log.Write(EventType.Diagnostic, "Received response JSON for request {0}, deserializing", activityId); } catch (Exception ex) @@ -348,7 +389,7 @@ async Task TryClearRequestFromQueue(RequestMessage request, RedisPendingRe disposables.AddAsyncDisposable(cts); var response = new RequestMessageWithCancellationToken(pending, cts.Token); - disposablesForInFlightRequests[pending.ActivityId] = new WatcherAndDisposables(disposables, cts.Token, watcher); + DisposablesForInFlightRequests[pending.ActivityId] = new WatcherAndDisposables(disposables, cts.Token, watcher); return response; } catch (Exception) @@ -382,7 +423,7 @@ public async Task ApplyResponse(ResponseMessage response, Guid requestActivityId { log.Write(EventType.MessageExchange, "Applying response for request {0}", requestActivityId); WatcherAndDisposables? watcherAndDisposables = null; - if (!disposablesForInFlightRequests.TryRemove(requestActivityId, out watcherAndDisposables)) + if (!DisposablesForInFlightRequests.TryRemove(requestActivityId, out watcherAndDisposables)) { log.Write(EventType.Diagnostic, "No in-flight request resources found to dispose for request {0}", requestActivityId); } diff --git a/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs b/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs index b8dd13e4a..026b2cd8b 100644 --- a/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs +++ b/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs @@ -24,7 +24,7 @@ namespace Halibut.Queue.Redis public class WatchForRequestCancellation : IAsyncDisposable { public static async Task TrySendCancellation( - HalibutRedisTransport halibutRedisTransport, + IHalibutRedisTransport halibutRedisTransport, Uri endpoint, RequestMessage request, ILog log) @@ -67,7 +67,7 @@ public static async Task TrySendCancellation( readonly ILog log; - public WatchForRequestCancellation(Uri endpoint, Guid requestActivityId, HalibutRedisTransport halibutRedisTransport, ILog log) + public WatchForRequestCancellation(Uri endpoint, Guid requestActivityId, IHalibutRedisTransport halibutRedisTransport, ILog log) { this.log = log; log.Write(EventType.Diagnostic, "Starting to watch for request cancellation - Endpoint: {0}, ActivityId: {1}", endpoint, requestActivityId); @@ -76,7 +76,7 @@ public WatchForRequestCancellation(Uri endpoint, Guid requestActivityId, Halibut var _ = Task.Run(async () => await WatchForCancellation(endpoint, requestActivityId, halibutRedisTransport, token)); } - async Task WatchForCancellation(Uri endpoint, Guid requestActivityId, HalibutRedisTransport halibutRedisTransport, CancellationToken token) + async Task WatchForCancellation(Uri endpoint, Guid requestActivityId, IHalibutRedisTransport halibutRedisTransport, CancellationToken token) { try { diff --git a/source/Halibut/Queue/Redis/WatchForRequestCancellationOrSenderDisconnect.cs b/source/Halibut/Queue/Redis/WatchForRequestCancellationOrSenderDisconnect.cs index f9e6b3d1e..6a5d4247c 100644 --- a/source/Halibut/Queue/Redis/WatchForRequestCancellationOrSenderDisconnect.cs +++ b/source/Halibut/Queue/Redis/WatchForRequestCancellationOrSenderDisconnect.cs @@ -37,7 +37,7 @@ public class WatchForRequestCancellationOrSenderDisconnect : IAsyncDisposable public WatchForRequestCancellationOrSenderDisconnect( Uri endpoint, Guid requestActivityId, - HalibutRedisTransport halibutRedisTransport, + IHalibutRedisTransport halibutRedisTransport, TimeSpan nodeOfflineTimeoutBetweenHeartBeatsFromSender, ILog log) { @@ -62,7 +62,7 @@ public WatchForRequestCancellationOrSenderDisconnect( } } - async Task WatchThatNodeWhichSentTheRequestIsStillAlive(Uri endpoint, Guid requestActivityId, HalibutRedisTransport halibutRedisTransport, TimeSpan nodeOfflineTimeoutBetweenHeartBeatsFromSender, ILog log) + async Task WatchThatNodeWhichSentTheRequestIsStillAlive(Uri endpoint, Guid requestActivityId, IHalibutRedisTransport halibutRedisTransport, TimeSpan nodeOfflineTimeoutBetweenHeartBeatsFromSender, ILog log) { var watchCancellationToken = keepWatchingCancellationToken.Token; try diff --git a/source/Halibut/ServiceModel/PendingRequestQueueAsync.cs b/source/Halibut/ServiceModel/PendingRequestQueueAsync.cs index ba2b190db..1ea71b85d 100644 --- a/source/Halibut/ServiceModel/PendingRequestQueueAsync.cs +++ b/source/Halibut/ServiceModel/PendingRequestQueueAsync.cs @@ -5,6 +5,7 @@ using System.Threading.Tasks; using Halibut.Diagnostics; using Halibut.Exceptions; +using Halibut.Transport; using Halibut.Transport.Protocol; using Nito.AsyncEx; @@ -12,8 +13,8 @@ namespace Halibut.ServiceModel { public class PendingRequestQueueAsync : IPendingRequestQueue, IAsyncDisposable { - readonly List queue = new(); - readonly Dictionary inProgress = new(); + readonly List queue = new(); + readonly Dictionary inProgress = new(); readonly SemaphoreSlim queueLock = new(1, 1); readonly AsyncManualResetEvent itemAddedToQueue = new(false); readonly ILog log; @@ -35,12 +36,10 @@ public PendingRequestQueueAsync(ILog log, TimeSpan pollingQueueWaitTimeout) public async Task QueueAndWaitAsync(RequestMessage request, CancellationToken cancellationToken) { - //cancellationToken = CancellationToken.None; - using var cancellationTokenSource = CancellationTokenSource.CreateLinkedTokenSource(cancellationToken, this.entireQueueCancellationTokenSource.Token); cancellationToken = cancellationTokenSource.Token; - using var pending = new RedisPendingRequest(request, log); + using var pending = new PendingRequest(request, log); try { @@ -58,7 +57,7 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can try { - await pending.WaitUntilComplete(() => Task.CompletedTask, () => null, cancellationToken); + await pending.WaitUntilComplete(cancellationToken); } finally { @@ -108,7 +107,7 @@ public int Count return null; } - var result = await pending.RequestHasBeenCollectedAndWillBeTransferred(); + var result = await pending.BeginTransfer(); if (result) { return new (pending.Request, pending.PendingRequestCancellationToken); @@ -116,7 +115,7 @@ public int Count } } - async Task DequeueNextAsync(TimeSpan timeout, CancellationToken cancellationToken) + async Task DequeueNextAsync(TimeSpan timeout, CancellationToken cancellationToken) { var first = await TakeFirst(cancellationToken); if (first != null || timeout <= TimeSpan.Zero) @@ -139,7 +138,7 @@ await Task.WhenAny( return await TakeFirst(cancellationToken); } - async Task TakeFirst(CancellationToken cancellationToken) + async Task TakeFirst(CancellationToken cancellationToken) { using (await queueLock.LockAsync(cancellationToken)) { @@ -160,7 +159,7 @@ await Task.WhenAny( } } - public async Task ApplyResponse(ResponseMessage response, Guid requestActivityId) + public async Task ApplyResponse(ResponseMessage response, Guid destination) { if (response == null) { @@ -176,6 +175,210 @@ public async Task ApplyResponse(ResponseMessage response, Guid requestActivityId } } + class PendingRequest : IDisposable + { + readonly RequestMessage request; + readonly ILog log; + readonly AsyncManualResetEvent responseWaiter = new(false); + readonly SemaphoreSlim transferLock = new(1, 1); + bool transferBegun; + bool completed; + readonly CancellationTokenSource pendingRequestCancellationTokenSource; + ResponseMessage? response; + + public PendingRequest(RequestMessage request, ILog log) + { + this.request = request; + this.log = log; + + pendingRequestCancellationTokenSource = new CancellationTokenSource(); + PendingRequestCancellationToken = pendingRequestCancellationTokenSource.Token; + } + + public RequestMessage Request => request; + + public async Task WaitUntilComplete(CancellationToken cancellationToken) + { + log.Write(EventType.MessageExchange, "Request {0} was queued", request); + + bool responseSet; + var cancelled = false; + + try + { + responseSet = await WaitForResponseToBeSet( + request.Destination.PollingRequestQueueTimeout, + // Don't cancel a dequeued request as we need to wait PollingRequestMaximumMessageProcessingTimeout for it to complete + cancelTheRequestWhenTransferHasBegun: false, + cancellationToken); + + if (responseSet) + { + log.Write(EventType.MessageExchange, "Request {0} was collected by the polling endpoint", request); + return; + } + } + catch (RequestCancelledException) + { + cancelled = true; + + using (await transferLock.LockAsync(CancellationToken.None)) + { + if (!transferBegun) + { + completed = true; + log.Write(EventType.MessageExchange, "Request {0} was cancelled before it could be collected by the polling endpoint", request); + throw; + } + } + } + + var waitForTransferToComplete = false; + using (await transferLock.LockAsync(CancellationToken.None)) + { + if (transferBegun) + { + waitForTransferToComplete = true; + } + else + { + completed = true; + } + } + + if (waitForTransferToComplete) + { + responseSet = await WaitForResponseToBeSet( + null, + // Cancel the dequeued request to force Reads and Writes to be cancelled + cancelTheRequestWhenTransferHasBegun: true, + cancellationToken); + + if (responseSet) + { + // We end up here when the request is cancelled but already being transferred so we need to adjust the log message accordingly + if (cancelled) + { + log.Write(EventType.MessageExchange, "Request {0} was collected by the polling endpoint", request); + } + else + { + log.Write(EventType.MessageExchange, "Request {0} was eventually collected by the polling endpoint", request); + } + } + else + { + if (cancellationToken.IsCancellationRequested) + { + log.Write(EventType.MessageExchange, "Request {0} was cancelled before a response was received", request); + SetResponse(ResponseMessage.FromException( + request, + new TimeoutException($"A request was sent to a polling endpoint, the polling endpoint collected it but the request was cancelled before the polling endpoint responded."), + ConnectionState.Connecting)); + } + else + { + // This should never happen. + log.Write(EventType.MessageExchange, "Request {0} had an internal error, unexpectedly stopped waiting for the response.", request); + SetResponse(ResponseMessage.FromException( + request, + new PendingRequestQueueInternalException($"Request {request.Id} had an internal error, unexpectedly stopped waiting for the response."))); + } + } + } + else + { + log.Write(EventType.MessageExchange, "Request {0} timed out before it could be collected by the polling endpoint", request); + SetResponse(ResponseMessage.FromException( + request, + new TimeoutException($"A request was sent to a polling endpoint, but the polling endpoint did not collect the request within the allowed time ({request.Destination.PollingRequestQueueTimeout}), so the request timed out."), + ConnectionState.Connecting)); + } + } + + async Task WaitForResponseToBeSet( + TimeSpan? timeout, + bool cancelTheRequestWhenTransferHasBegun, + CancellationToken cancellationToken) + { + using var timeoutCancellationTokenSource = timeout.HasValue ? new CancellationTokenSource(timeout.Value) : new CancellationTokenSource(); + using var linkedTokenSource = CancellationTokenSource.CreateLinkedTokenSource(timeoutCancellationTokenSource.Token, cancellationToken); + + try + { + await responseWaiter.WaitAsync(linkedTokenSource.Token); + } + catch (OperationCanceledException ex) + { + using (await transferLock.LockAsync(CancellationToken.None)) + { + if (transferBegun && cancelTheRequestWhenTransferHasBegun) + { + // Cancel the dequeued request. This will cause co-operative cancellation on the thread dequeuing the request + pendingRequestCancellationTokenSource.Cancel(); + } + else if (!transferBegun) + { + // Cancel the queued request. This will flag the request as cancelled to stop it being dequeued + pendingRequestCancellationTokenSource.Cancel(); + } + + if (timeoutCancellationTokenSource.IsCancellationRequested) + { + return false; + } + + throw transferBegun ? new TransferringRequestCancelledException(ex) : new ConnectingRequestCancelledException(ex); + } + } + + return true; + } + + public async Task BeginTransfer() + { + // The PendingRequest is Disposed at the end of QueueAndWaitAsync but a race condition + // exists in the current approach that means DequeueAsync could pick this request up after + // it has been disposed. At that point we are no longer interested in the PendingRequest so + // this is "ok" and wrapping BeginTransfer in a try..catch.. ensures we don't error if the + // race condition occurs and also stops the polling tentacle dequeuing the request successfully. + try + { + using (await transferLock.LockAsync(CancellationToken.None)) + { + // Check if the request has already been completed or if the request has been cancelled + // to ensure we don't dequeue an already completed or already cancelled request + if (completed || pendingRequestCancellationTokenSource.IsCancellationRequested) + { + return false; + } + + transferBegun = true; + return true; + } + } + catch (ObjectDisposedException) + { + return false; + } + } + + public ResponseMessage Response => response ?? throw new InvalidOperationException("Response has not been set."); + public CancellationToken PendingRequestCancellationToken { get; } + + public void SetResponse(ResponseMessage response) + { + this.response = response; + responseWaiter.Set(); + } + + public void Dispose() + { + pendingRequestCancellationTokenSource?.Dispose(); + transferLock?.Dispose(); + } + } + public ValueTask DisposeAsync() { entireQueueCancellationTokenSource.Cancel(); diff --git a/source/Halibut/ServiceModel/PendingRequestV2.cs b/source/Halibut/ServiceModel/PendingRequestV2.cs index f4dcc261f..c1922721a 100644 --- a/source/Halibut/ServiceModel/PendingRequestV2.cs +++ b/source/Halibut/ServiceModel/PendingRequestV2.cs @@ -104,8 +104,7 @@ public async Task WaitUntilComplete(Func checkIfPendingRequestWasCollected OperationCanceledException operationCanceledException; if (!requestCollected.IsSet) { - log.Write(EventType.MessageExchange, "Request {0} was cancelled before it could be collected by the polling endpoint", request); - operationCanceledException = new OperationCanceledException($"Request {request} was cancelled before it could be collected by the polling endpoint"); + operationCanceledException = CreateExceptionForRequestWasCancelledBeforeCollected(request, log); } else { @@ -174,8 +173,14 @@ await SetResponseAsync(ResponseMessage.FromException( false); } } - - + + public static OperationCanceledException CreateExceptionForRequestWasCancelledBeforeCollected(RequestMessage request, ILog log) + { + + log.Write(EventType.MessageExchange, "Request {0} was cancelled before it could be collected by the polling endpoint", request); + return new OperationCanceledException($"Request {request} was cancelled before it could be collected by the polling endpoint"); + } + public async Task RequestHasBeenCollectedAndWillBeTransferred() { // The PendingRequest is Disposed at the end of QueueAndWaitAsync but a race condition @@ -206,26 +211,27 @@ public async Task RequestHasBeenCollectedAndWillBeTransferred() public ResponseMessage Response => response ?? throw new InvalidOperationException("Response has not been set."); public CancellationToken PendingRequestCancellationToken { get; } - public void SetResponse(ResponseMessage response) + public async Task SetResponse(ResponseMessage response) { // If someone is calling this then we know for sure they collected the request - this.SetResponseAsync(response, true).GetAwaiter().GetResult(); + return await SetResponseAsync(response, true); } - async Task SetResponseAsync(ResponseMessage response, bool requestWasCollected) + async Task SetResponseAsync(ResponseMessage response, bool requestWasCollected) { using (await transferLock.LockAsync(CancellationToken.None)) { - SetResponseNoLock(response, requestWasCollected); + return SetResponseNoLock(response, requestWasCollected); } } - void SetResponseNoLock(ResponseMessage response, bool requestWasCollected) + ResponseMessage SetResponseNoLock(ResponseMessage response, bool requestWasCollected) { - if(this.response != null) return; + if(this.response != null) return this.response; this.response = response; responseWaiter.Set(); if(requestWasCollected) requestCollected.Set(); // Also the request has been collected, if we have a response. + return this.response; } public void Dispose() From c926903d31deed2055af743623e24a81cca96c1e Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Thu, 14 Aug 2025 16:24:38 +1000 Subject: [PATCH 051/137] Random delay on error --- .../Redis/RedisPendingRequest.cs} | 5 ++--- .../Queue/Redis/RedisPendingRequestQueue.cs | 22 +++++++++++++------ 2 files changed, 17 insertions(+), 10 deletions(-) rename source/Halibut/{ServiceModel/PendingRequestV2.cs => Queue/Redis/RedisPendingRequest.cs} (98%) diff --git a/source/Halibut/ServiceModel/PendingRequestV2.cs b/source/Halibut/Queue/Redis/RedisPendingRequest.cs similarity index 98% rename from source/Halibut/ServiceModel/PendingRequestV2.cs rename to source/Halibut/Queue/Redis/RedisPendingRequest.cs index c1922721a..21c866f8a 100644 --- a/source/Halibut/ServiceModel/PendingRequestV2.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequest.cs @@ -17,12 +17,13 @@ using System.Threading.Tasks; using Halibut.Diagnostics; using Halibut.Exceptions; +using Halibut.ServiceModel; using Halibut.Transport; using Halibut.Transport.Protocol; using Halibut.Util; using Nito.AsyncEx; -namespace Halibut.ServiceModel +namespace Halibut.Queue.Redis { public class RedisPendingRequest : IDisposable { @@ -91,7 +92,6 @@ public async Task WaitUntilComplete(Func checkIfPendingRequestWasCollected if (cancellationToken.IsCancellationRequested) { - // TODO: This seems sus, we throw here but we don't throw below. This should be straightened out. await Try.IgnoringError(async () => await pendingRequestCancellationTokenSource.CancelAsync()); var cancellationException = overrideCancellationReason(); @@ -236,7 +236,6 @@ ResponseMessage SetResponseNoLock(ResponseMessage response, bool requestWasColle public void Dispose() { - pendingRequestCancellationTokenSource?.Dispose(); transferLock?.Dispose(); } } diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index ad6947294..7d509556a 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -478,17 +478,14 @@ public async Task ApplyResponse(ResponseMessage response, Guid requestActivityId await using var cts = new CancelOnDisposeCancellationToken(queueToken); try { - // TODO can we avoid going to redis here? - // TODO: Does this work well for multiple clients? We might go round before we collect work. - // TODO: test this. hasItemsForEndpoint.Reset(); - + var first = await TryRemoveNextItemFromQueue(cts.Token); if (first != null) return first; - + await Task.WhenAny( - hasItemsForEndpoint.WaitAsync(cts.Token), + hasItemsForEndpoint.WaitAsync(cts.Token), Task.Delay(halibutTimeoutsAndLimits.PollingQueueWaitTimeout, cts.Token)); if (!hasItemsForEndpoint.IsSet) @@ -497,9 +494,20 @@ await Task.WhenAny( // to keep the connection healthy. return null; } - + return await TryRemoveNextItemFromQueue(cts.Token); } + catch (Exception ex) + { + if (!queueToken.IsCancellationRequested) + { + log.WriteException(EventType.Error, "Error occured dequeuing from the queue", ex); + // It is very likely a queue error means every tentacle will return an error. + // Add a random delay to help avoid every client coming back at exactly the same time. + await Task.Delay(TimeSpan.FromSeconds(new Random().Next(15)), cts.Token); + throw; + } + } finally { await cts.CancelAsync(); From 1999a4a735f3169b057c3deccc2a887d03a1ba5f Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Thu, 14 Aug 2025 20:39:04 +1000 Subject: [PATCH 052/137] . --- source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index 7d509556a..8bedb1c21 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -505,8 +505,8 @@ await Task.WhenAny( // It is very likely a queue error means every tentacle will return an error. // Add a random delay to help avoid every client coming back at exactly the same time. await Task.Delay(TimeSpan.FromSeconds(new Random().Next(15)), cts.Token); - throw; } + throw; } finally { From b7a335414e5650159cc90fe0a163bf6c26b42c86 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Fri, 15 Aug 2025 09:28:06 +1000 Subject: [PATCH 053/137] One must dispose CTS less mem leak, but also not while anything is still using it --- .../Queue/Redis/PollAndSubscribeToResponse.cs | 2 +- .../Queue/Redis/RedisPendingRequestQueue.cs | 4 +- .../Util/AwaitAllAndIgnoreException.cs | 40 +++++++++++++++++++ .../Util/CancelOnDisposeCancellationToken.cs | 16 ++++++-- 4 files changed, 57 insertions(+), 5 deletions(-) create mode 100644 source/Halibut/Util/AwaitAllAndIgnoreException.cs diff --git a/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs b/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs index b2c028001..6cd1ac7e2 100644 --- a/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs +++ b/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs @@ -85,7 +85,7 @@ public PollAndSubscribeToResponse(Uri endpoint, Guid activityId, IHalibutRedisTr watcherToken = new CancelOnDisposeCancellationToken(); var token = watcherToken.Token; - var _ = Task.Run(async () => await WaitForResponse(token)); + watcherToken.AwaitTasksBeforeCTSDispose(Task.Run(async () => await WaitForResponse(token))); } readonly SemaphoreSlim trySetResultSemaphore = new SemaphoreSlim(1, 1); diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index 8bedb1c21..9b6b45f15 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -160,6 +160,8 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can CancellationReason, cancellationToken); + cts.AwaitTasksBeforeCTSDispose(watchProcessingNodeStillHasHeartBeat, waitingForResponse, pendingRequestWaitUntilComplete); + await Task.WhenAny(waitingForResponse, pendingRequestWaitUntilComplete, watchProcessingNodeStillHasHeartBeat); if (pendingRequestWaitUntilComplete.IsCompleted || cancellationToken.IsCancellationRequested) @@ -315,7 +317,7 @@ async Task TryClearRequestFromQueue(RequestMessage request, RedisPendingRe try { log.Write(EventType.Diagnostic, "Waiting for response for request {0}", activityId); - responseJson = await pollAndSubscribeToResponse.ResponseJson; + responseJson = await pollAndSubscribeToResponse.ResponseJson.WaitAsync(cancellationToken); log.Write(EventType.Diagnostic, "Received response JSON for request {0}, deserializing", activityId); } catch (Exception ex) diff --git a/source/Halibut/Util/AwaitAllAndIgnoreException.cs b/source/Halibut/Util/AwaitAllAndIgnoreException.cs new file mode 100644 index 000000000..03dd83822 --- /dev/null +++ b/source/Halibut/Util/AwaitAllAndIgnoreException.cs @@ -0,0 +1,40 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; +using System.Collections.Generic; +using System.Threading.Tasks; + +namespace Halibut.Util +{ + public class AwaitAllAndIgnoreException : IAsyncDisposable + { + List tasks = new List(); + + public void AddTasks(params Task[] tasksToAdd) + { + foreach (var task in tasksToAdd) + { +#pragma warning disable VSTHRD003 + tasks.Add(Halibut.Util.Try.IgnoringError(async () => await task)); +#pragma warning restore VSTHRD003 + } + } + + public async ValueTask DisposeAsync() + { + await Task.WhenAll(tasks); + } + } +} \ No newline at end of file diff --git a/source/Halibut/Util/CancelOnDisposeCancellationToken.cs b/source/Halibut/Util/CancelOnDisposeCancellationToken.cs index d41730768..a06dc756b 100644 --- a/source/Halibut/Util/CancelOnDisposeCancellationToken.cs +++ b/source/Halibut/Util/CancelOnDisposeCancellationToken.cs @@ -2,6 +2,8 @@ using System; using System.Threading; using System.Threading.Tasks; +using Halibut.Queue.Redis; +using Halibut.Transport.Protocol; namespace Halibut.Util { @@ -13,6 +15,8 @@ public sealed class CancelOnDisposeCancellationToken : IAsyncDisposable { readonly CancellationTokenSource cancellationTokenSource; bool disposed; + + AwaitAllAndIgnoreException awaitAllAndIgnoreException = new AwaitAllAndIgnoreException(); public CancelOnDisposeCancellationToken(params CancellationToken[] token) : this(CancellationTokenSource.CreateLinkedTokenSource(token)) @@ -39,9 +43,10 @@ public async ValueTask DisposeAsync() await Try.IgnoringError(async () => await CancelAsync()); - // And then don't dispose the CancellationTokenSource. - // Since doing so WILL result in race conditions where - // callbacks will be silently not executed. + // Wait for any tasks that are using the token, before disposal + await Try.IgnoringError(async () => await awaitAllAndIgnoreException.DisposeAsync()); + + Try.IgnoringError(() => cancellationTokenSource.Dispose()); } public async Task CancelAsync() @@ -57,5 +62,10 @@ public void CancelAfter(TimeSpan timeSpan) { cancellationTokenSource.CancelAfter(timeSpan); } + + public void AwaitTasksBeforeCTSDispose(params Task[] tasksUsingToken) + { + awaitAllAndIgnoreException.AddTasks(tasksUsingToken); + } } } \ No newline at end of file From 647dd08d0f135a746b0dc7bdcac95ff1320ae586 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Fri, 15 Aug 2025 11:26:42 +1000 Subject: [PATCH 054/137] Fix issue where we dispose the PollAndSubscribeToResponse before we delete the response from redis --- .../Queue/Redis/PollAndSubscribeToResponse.cs | 25 ++++++++++++------- .../Util/CancelOnDisposeCancellationToken.cs | 2 +- 2 files changed, 17 insertions(+), 10 deletions(-) diff --git a/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs b/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs index 6cd1ac7e2..7f202d94c 100644 --- a/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs +++ b/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs @@ -153,11 +153,16 @@ public async Task TryGetResponseFromRedis(string detectedBy, CancellationT if (ResponseJsonCompletionSource.Task.IsCompleted) return true; + // TODO wrap in try var responseJson = await halibutRedisTransport.GetResponseMessage(endpoint, activityId, token); + if (responseJson != null) { - log.Write(EventType.Diagnostic, "Response detected via {0} - Endpoint: {1}, ActivityId: {2}", detectedBy, endpoint, activityId); - await TrySetResponse(responseJson, token); + log.Write(EventType.Diagnostic, "Response detected via {0} - Endpoint: {1}, ActivityId: {2}", detectedBy, endpoint, activityId); + + await DeleteResponseFromRedis(detectedBy, token); + + TrySetResponse(responseJson, token); await Try.IgnoringError(async () => await watcherToken.CancelAsync()); log.Write(EventType.Diagnostic, "Cancelling polling loop for response - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); return true; @@ -165,27 +170,29 @@ public async Task TryGetResponseFromRedis(string detectedBy, CancellationT return false; } - - async Task TrySetResponse(string value, CancellationToken cancellationToken) + + async Task DeleteResponseFromRedis(string detectedBy, CancellationToken token) { try { - ResponseJsonCompletionSource.TrySetResult(value); + await halibutRedisTransport.DeleteResponse(endpoint, activityId, token); } catch (Exception ex) { - log.Write(EventType.Error, "Failed to set response - Endpoint: {0}, ActivityId: {1}, Error: {2}", endpoint, activityId, ex.Message); + log.Write(EventType.Error, "Failed to delete response from Redis via {0} - Endpoint: {1}, ActivityId: {2}, Error: {3}", detectedBy, endpoint, activityId, ex.Message); } + } + void TrySetResponse(string value, CancellationToken cancellationToken) + { try { - await halibutRedisTransport.DeleteResponse(endpoint, activityId, cancellationToken); + ResponseJsonCompletionSource.TrySetResult(value); } catch (Exception ex) { - log.Write(EventType.Error, "Failed to delete response from Redis - Endpoint: {0}, ActivityId: {1}, Error: {2}", endpoint, activityId, ex.Message); + log.Write(EventType.Error, "Failed to set response - Endpoint: {0}, ActivityId: {1}, Error: {2}", endpoint, activityId, ex.Message); } - } public async ValueTask DisposeAsync() diff --git a/source/Halibut/Util/CancelOnDisposeCancellationToken.cs b/source/Halibut/Util/CancelOnDisposeCancellationToken.cs index a06dc756b..b20c8acb0 100644 --- a/source/Halibut/Util/CancelOnDisposeCancellationToken.cs +++ b/source/Halibut/Util/CancelOnDisposeCancellationToken.cs @@ -16,7 +16,7 @@ public sealed class CancelOnDisposeCancellationToken : IAsyncDisposable readonly CancellationTokenSource cancellationTokenSource; bool disposed; - AwaitAllAndIgnoreException awaitAllAndIgnoreException = new AwaitAllAndIgnoreException(); + readonly AwaitAllAndIgnoreException awaitAllAndIgnoreException = new(); public CancelOnDisposeCancellationToken(params CancellationToken[] token) : this(CancellationTokenSource.CreateLinkedTokenSource(token)) From a8976b6385efe70a6bb899ce24de12ba7da8dc07 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Fri, 15 Aug 2025 15:47:07 +1000 Subject: [PATCH 055/137] It compiles on windows --- .../Builders/IPendingRequestQueueBuilder.cs | 27 ++++++ .../Builders/PendingRequestQueueBuilder.cs | 78 +---------------- .../RedisPendingRequestQueueBuilder.cs | 87 +++++++++++++++++++ .../Builders/ServiceEndPointBuilder.cs | 3 +- .../Halibut.Tests/ManyPollingTentacleTests.cs | 5 +- .../Queue/QueueMessageSerializerFixture.cs | 4 +- .../Queue/Redis/NodeHeartBeatSenderFixture.cs | 4 +- .../Queue/Redis/RedisFacadeFixture.cs | 4 +- .../RedisFacadeWhenRedisGoesDownAwayTests.cs | 5 +- .../Redis/RedisPendingRequestQueueFixture.cs | 4 +- .../Queue/Redis/RedisQueuePerformanceTest.cs | 8 +- .../TestRedisPendingRequestQueueFactory.cs | 5 +- .../HalibutRedisTransportWithVirtuals.cs | 5 +- ...LosingDataWatchForRedisLosingAllItsData.cs | 5 +- .../WatchForRedisLosingAllItsDataFixture.cs | 4 +- .../AllQueuesTestCasesAttribute.cs | 2 + ...oryStoreDataStreamsForDistributedQueues.cs | 1 + .../Queue/Redis/HalibutRedisTransport.cs | 5 +- .../Queue/Redis/IHalibutRedisTransport.cs | 5 +- .../Redis/IWatchForRedisLosingAllItsData.cs | 31 +++++++ .../Queue/Redis/NodeHeartBeatSender.cs | 5 +- .../Queue/Redis/PollAndSubscribeToResponse.cs | 5 +- source/Halibut/Queue/Redis/RedisFacade.cs | 10 --- .../Queue/Redis/RedisPendingRequest.cs | 4 +- .../Queue/Redis/RedisPendingRequestQueue.cs | 5 +- .../Redis/RedisPendingRequestQueueFactory.cs | 6 +- .../Redis/WatchForRedisLosingAllItsData.cs | 15 +--- .../Redis/WatchForRequestCancellation.cs | 5 +- ...orRequestCancellationOrSenderDisconnect.cs | 5 +- .../Util/CancelOnDisposeCancellationToken.cs | 3 +- .../Util/net48Helpers/NotNullWhenAttribute.cs | 27 ++++++ 31 files changed, 247 insertions(+), 135 deletions(-) create mode 100644 source/Halibut.Tests/Builders/IPendingRequestQueueBuilder.cs create mode 100644 source/Halibut.Tests/Builders/RedisPendingRequestQueueBuilder.cs create mode 100644 source/Halibut/Queue/Redis/IWatchForRedisLosingAllItsData.cs create mode 100644 source/Halibut/Util/net48Helpers/NotNullWhenAttribute.cs diff --git a/source/Halibut.Tests/Builders/IPendingRequestQueueBuilder.cs b/source/Halibut.Tests/Builders/IPendingRequestQueueBuilder.cs new file mode 100644 index 000000000..78c7459e1 --- /dev/null +++ b/source/Halibut.Tests/Builders/IPendingRequestQueueBuilder.cs @@ -0,0 +1,27 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; +using Halibut.Diagnostics; + +namespace Halibut.Tests.Builders +{ + public interface IPendingRequestQueueBuilder + { + public IPendingRequestQueueBuilder WithEndpoint(string endpoint); + public IPendingRequestQueueBuilder WithLog(ILog log); + public IPendingRequestQueueBuilder WithPollingQueueWaitTimeout(TimeSpan? pollingQueueWaitTimeout); + public QueueHolder Build(); + } +} \ No newline at end of file diff --git a/source/Halibut.Tests/Builders/PendingRequestQueueBuilder.cs b/source/Halibut.Tests/Builders/PendingRequestQueueBuilder.cs index 78a3dd699..a0bd9e96a 100644 --- a/source/Halibut.Tests/Builders/PendingRequestQueueBuilder.cs +++ b/source/Halibut.Tests/Builders/PendingRequestQueueBuilder.cs @@ -1,88 +1,14 @@ -using System; + +using System; using System.Threading.Tasks; using Halibut.Diagnostics; -using Halibut.Logging; -using Halibut.Queue.QueuedDataStreams; -using Halibut.Queue.Redis; using Halibut.ServiceModel; -using Halibut.Tests.Queue; -using Halibut.Tests.Queue.Redis.Utils; using Halibut.Tests.Support; -using Halibut.Tests.Support.Logging; using DisposableCollection = Halibut.Util.DisposableCollection; using ILog = Halibut.Diagnostics.ILog; namespace Halibut.Tests.Builders { - - public interface IPendingRequestQueueBuilder - { - public IPendingRequestQueueBuilder WithEndpoint(string endpoint); - public IPendingRequestQueueBuilder WithLog(ILog log); - public IPendingRequestQueueBuilder WithPollingQueueWaitTimeout(TimeSpan? pollingQueueWaitTimeout); - public QueueHolder Build(); - } - - public class RedisPendingRequestQueueBuilder : IPendingRequestQueueBuilder - { - - const int redisPort = 6379; - - ILog? log; - string? endpoint; - TimeSpan? pollingQueueWaitTimeout; - - public IPendingRequestQueueBuilder WithEndpoint(string endpoint) - { - this.endpoint = endpoint; - return this; - } - - public IPendingRequestQueueBuilder WithLog(ILog log) - { - this.log = log; - return this; - } - - public IPendingRequestQueueBuilder WithPollingQueueWaitTimeout(TimeSpan? pollingQueueWaitTimeout) - { - this.pollingQueueWaitTimeout = pollingQueueWaitTimeout; - return this; - } - - public QueueHolder Build() - { - var endpoint = new Uri(this.endpoint ?? "poll://endpoint001"); - var halibutTimeoutsAndLimits = new HalibutTimeoutsAndLimitsForTestsBuilder().Build(); - var log = this.log ?? new TestContextLogCreator("Queue", LogLevel.Trace).CreateNewForPrefix(""); - - if (this.pollingQueueWaitTimeout != null) - { - halibutTimeoutsAndLimits.PollingQueueWaitTimeout = pollingQueueWaitTimeout.Value; - } - - var disposableCollection = new DisposableCollection(); - - var redisFacade = new RedisFacade("localhost:" + redisPort, (Guid.NewGuid()).ToString(), log); - disposableCollection.AddAsyncDisposable(redisFacade); - - var redisTransport = new HalibutRedisTransport(redisFacade); - var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); - var messageSerializer = new QueueMessageSerializerBuilder().Build(); - var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); - - var request = new RequestMessageBuilder("poll://test-endpoint").Build(); - - var queue = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, halibutTimeoutsAndLimits); - -#pragma warning disable VSTHRD002 - queue.WaitUntilQueueIsSubscribedToReceiveMessages().GetAwaiter().GetResult(); -#pragma warning restore VSTHRD002 - - return new QueueHolder(queue, disposableCollection); - } - } - public class PendingRequestQueueBuilder : IPendingRequestQueueBuilder { ILog? log; diff --git a/source/Halibut.Tests/Builders/RedisPendingRequestQueueBuilder.cs b/source/Halibut.Tests/Builders/RedisPendingRequestQueueBuilder.cs new file mode 100644 index 000000000..39f4a23e9 --- /dev/null +++ b/source/Halibut.Tests/Builders/RedisPendingRequestQueueBuilder.cs @@ -0,0 +1,87 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +#if NET8_0_OR_GREATER +using System; +using Halibut.Logging; +using Halibut.Queue.QueuedDataStreams; +using Halibut.Queue.Redis; +using Halibut.Tests.Queue; +using Halibut.Tests.Queue.Redis.Utils; +using Halibut.Tests.Support.Logging; +using Halibut.Util; +using ILog = Halibut.Diagnostics.ILog; + +namespace Halibut.Tests.Builders +{ + public class RedisPendingRequestQueueBuilder : IPendingRequestQueueBuilder + { + + const int redisPort = 6379; + + ILog? log; + string? endpoint; + TimeSpan? pollingQueueWaitTimeout; + + public IPendingRequestQueueBuilder WithEndpoint(string endpoint) + { + this.endpoint = endpoint; + return this; + } + + public IPendingRequestQueueBuilder WithLog(ILog log) + { + this.log = log; + return this; + } + + public IPendingRequestQueueBuilder WithPollingQueueWaitTimeout(TimeSpan? pollingQueueWaitTimeout) + { + this.pollingQueueWaitTimeout = pollingQueueWaitTimeout; + return this; + } + + public QueueHolder Build() + { + var endpoint = new Uri(this.endpoint ?? "poll://endpoint001"); + var halibutTimeoutsAndLimits = new HalibutTimeoutsAndLimitsForTestsBuilder().Build(); + var log = this.log ?? new TestContextLogCreator("Queue", LogLevel.Trace).CreateNewForPrefix(""); + + if (this.pollingQueueWaitTimeout != null) + { + halibutTimeoutsAndLimits.PollingQueueWaitTimeout = pollingQueueWaitTimeout.Value; + } + + var disposableCollection = new DisposableCollection(); + + var redisFacade = new RedisFacade("localhost:" + redisPort, (Guid.NewGuid()).ToString(), log); + disposableCollection.AddAsyncDisposable(redisFacade); + + var redisTransport = new HalibutRedisTransport(redisFacade); + var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); + var messageSerializer = new QueueMessageSerializerBuilder().Build(); + var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); + + var request = new RequestMessageBuilder("poll://test-endpoint").Build(); + + var queue = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, halibutTimeoutsAndLimits); + +#pragma warning disable VSTHRD002 + queue.WaitUntilQueueIsSubscribedToReceiveMessages().GetAwaiter().GetResult(); +#pragma warning restore VSTHRD002 + + return new QueueHolder(queue, disposableCollection); + } + } +} +#endif \ No newline at end of file diff --git a/source/Halibut.Tests/Builders/ServiceEndPointBuilder.cs b/source/Halibut.Tests/Builders/ServiceEndPointBuilder.cs index e79c0323c..bfee5f055 100644 --- a/source/Halibut.Tests/Builders/ServiceEndPointBuilder.cs +++ b/source/Halibut.Tests/Builders/ServiceEndPointBuilder.cs @@ -1,4 +1,5 @@ -using Halibut.Transport.Protocol; + +using Halibut.Transport.Protocol; using System; using Halibut.Diagnostics; diff --git a/source/Halibut.Tests/ManyPollingTentacleTests.cs b/source/Halibut.Tests/ManyPollingTentacleTests.cs index e92898e83..0e03f44a1 100644 --- a/source/Halibut.Tests/ManyPollingTentacleTests.cs +++ b/source/Halibut.Tests/ManyPollingTentacleTests.cs @@ -11,7 +11,7 @@ // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. - +#if NET8_0_OR_GREATER using System; using System.Collections.Generic; using System.Diagnostics; @@ -190,4 +190,5 @@ public async Task CountBytesAsync(DataStream dataStream, CancellationToken throw new Exception(); } } -} \ No newline at end of file +} +#endif \ No newline at end of file diff --git a/source/Halibut.Tests/Queue/QueueMessageSerializerFixture.cs b/source/Halibut.Tests/Queue/QueueMessageSerializerFixture.cs index 4838512e5..aab0e0229 100644 --- a/source/Halibut.Tests/Queue/QueueMessageSerializerFixture.cs +++ b/source/Halibut.Tests/Queue/QueueMessageSerializerFixture.cs @@ -1,3 +1,4 @@ +#if NET8_0_OR_GREATER using System; using System.IO; using System.Linq; @@ -137,4 +138,5 @@ static Func WriteRepeatedStringsAsync(string to } } } -} \ No newline at end of file +} +#endif \ No newline at end of file diff --git a/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs b/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs index 5dbf959e8..bc6d9cd32 100644 --- a/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs @@ -1,3 +1,4 @@ +#if NET8_0_OR_GREATER using System; using System.Collections.Concurrent; using System.Collections.Generic; @@ -444,4 +445,5 @@ public async Task SenderNodeHeartbeats_ShouldNotBeReceivedByReceiverSubscription receiverHeartbeatsReceived.IsSet.Should().BeFalse("Should not have received sender heartbeat on receiver subscription"); } } -} \ No newline at end of file +} +#endif \ No newline at end of file diff --git a/source/Halibut.Tests/Queue/Redis/RedisFacadeFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisFacadeFixture.cs index 3d8c7f7b6..776d7770d 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisFacadeFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisFacadeFixture.cs @@ -1,3 +1,4 @@ +#if NET8_0_OR_GREATER using System; using System.Collections.Generic; using System.Threading.Tasks; @@ -593,4 +594,5 @@ public async Task DisposeAsync_CalledMultipleTimes_ShouldNotThrow() } -} \ No newline at end of file +} +#endif \ No newline at end of file diff --git a/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs b/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs index 735703303..d53ca9b55 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs @@ -11,7 +11,7 @@ // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. - +#if NET8_0_OR_GREATER using System; using System.Collections.Concurrent; using System.Collections.Generic; @@ -349,4 +349,5 @@ public async Task WhenSubscribedAndTheConnectionGoesDown_WhenTheConnectionIsRest } -} \ No newline at end of file +} +#endif \ No newline at end of file diff --git a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs index bfbb9b66d..c2f306056 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs @@ -1,3 +1,4 @@ +#if NET8_0_OR_GREATER using System; using System.Collections.Generic; using System.Diagnostics; @@ -863,4 +864,5 @@ StreamCapturingJsonSerializer StreamCapturingSerializer() } } } -} \ No newline at end of file +} +#endif \ No newline at end of file diff --git a/source/Halibut.Tests/Queue/Redis/RedisQueuePerformanceTest.cs b/source/Halibut.Tests/Queue/Redis/RedisQueuePerformanceTest.cs index b3d7b2114..1159800b8 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisQueuePerformanceTest.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisQueuePerformanceTest.cs @@ -11,7 +11,7 @@ // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. - +#if NET8_0_OR_GREATER using System; using System.Diagnostics; using System.Threading.Tasks; @@ -21,10 +21,11 @@ using Halibut.Queue.QueuedDataStreams; using Halibut.Queue.Redis; using Halibut.Tests.Builders; +using Halibut.Tests.Queue.Redis.Utils; using Halibut.Tests.Support.Logging; using Halibut.Util; -namespace Halibut.Tests.Queue.Redis.Utils +namespace Halibut.Tests.Queue.Redis { public class RedisQueuePerformanceTest : BaseTest { @@ -86,4 +87,5 @@ public async Task When100kTentaclesAreSubscribed_TheQueueStillWorks() } } } -} \ No newline at end of file +} +#endif \ No newline at end of file diff --git a/source/Halibut.Tests/Queue/Redis/TestRedisPendingRequestQueueFactory.cs b/source/Halibut.Tests/Queue/Redis/TestRedisPendingRequestQueueFactory.cs index cc846b25f..751bdfb3a 100644 --- a/source/Halibut.Tests/Queue/Redis/TestRedisPendingRequestQueueFactory.cs +++ b/source/Halibut.Tests/Queue/Redis/TestRedisPendingRequestQueueFactory.cs @@ -11,7 +11,7 @@ // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. - +#if NET8_0_OR_GREATER using System; using Halibut.Queue.Redis; using Halibut.ServiceModel; @@ -44,4 +44,5 @@ public static IPendingRequestQueueFactory WithWaitForReceiverToBeReady(this Redi return new TestRedisPendingRequestQueueFactory(redisPendingRequestQueueFactory); } } -} \ No newline at end of file +} +#endif \ No newline at end of file diff --git a/source/Halibut.Tests/Queue/Redis/Utils/HalibutRedisTransportWithVirtuals.cs b/source/Halibut.Tests/Queue/Redis/Utils/HalibutRedisTransportWithVirtuals.cs index c2e504050..58a74ef57 100644 --- a/source/Halibut.Tests/Queue/Redis/Utils/HalibutRedisTransportWithVirtuals.cs +++ b/source/Halibut.Tests/Queue/Redis/Utils/HalibutRedisTransportWithVirtuals.cs @@ -11,7 +11,7 @@ // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. - +#if NET8_0_OR_GREATER using System; using System.Threading; using System.Threading.Tasks; @@ -134,4 +134,5 @@ public Task DeleteResponse(Uri endpoint, Guid identifier, CancellationToke return halibutRedisTransport.DeleteResponse(endpoint, identifier, cancellationToken); } } -} \ No newline at end of file +} +#endif \ No newline at end of file diff --git a/source/Halibut.Tests/Queue/Redis/Utils/NeverLosingDataWatchForRedisLosingAllItsData.cs b/source/Halibut.Tests/Queue/Redis/Utils/NeverLosingDataWatchForRedisLosingAllItsData.cs index 12c366755..efb7c334a 100644 --- a/source/Halibut.Tests/Queue/Redis/Utils/NeverLosingDataWatchForRedisLosingAllItsData.cs +++ b/source/Halibut.Tests/Queue/Redis/Utils/NeverLosingDataWatchForRedisLosingAllItsData.cs @@ -11,7 +11,7 @@ // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. - +#if NET8_0_OR_GREATER using System; using System.Threading; using System.Threading.Tasks; @@ -35,4 +35,5 @@ public ValueTask DisposeAsync() return ValueTask.CompletedTask; } } -} \ No newline at end of file +} +#endif \ No newline at end of file diff --git a/source/Halibut.Tests/Queue/Redis/WatchForRedisLosingAllItsDataFixture.cs b/source/Halibut.Tests/Queue/Redis/WatchForRedisLosingAllItsDataFixture.cs index ba0af11b6..8c00b40ab 100644 --- a/source/Halibut.Tests/Queue/Redis/WatchForRedisLosingAllItsDataFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/WatchForRedisLosingAllItsDataFixture.cs @@ -1,3 +1,4 @@ +#if NET8_0_OR_GREATER using System; using System.Threading.Tasks; using FluentAssertions; @@ -115,4 +116,5 @@ public async Task WhenRedisLosesAllOfIts_TheWatcherShouldDetectTheDataLose() nextToken.IsCancellationRequested.Should().BeFalse("The new token should have no data loss"); } } -} \ No newline at end of file +} +#endif \ No newline at end of file diff --git a/source/Halibut.Tests/Support/TestAttributes/AllQueuesTestCasesAttribute.cs b/source/Halibut.Tests/Support/TestAttributes/AllQueuesTestCasesAttribute.cs index 34d7f91e0..e5c75cc85 100644 --- a/source/Halibut.Tests/Support/TestAttributes/AllQueuesTestCasesAttribute.cs +++ b/source/Halibut.Tests/Support/TestAttributes/AllQueuesTestCasesAttribute.cs @@ -25,7 +25,9 @@ static class PendingRequestQueueFactories public static IEnumerable GetEnumerator() { var factories = new List(); +#if NET8_0_OR_GREATER factories.Add(new PendingRequestQueueTestCase("Redis", () => new RedisPendingRequestQueueBuilder())); +#endif factories.Add(new PendingRequestQueueTestCase("InMemory", () => new PendingRequestQueueBuilder())); return factories; diff --git a/source/Halibut/Queue/QueuedDataStreams/InMemoryStoreDataStreamsForDistributedQueues.cs b/source/Halibut/Queue/QueuedDataStreams/InMemoryStoreDataStreamsForDistributedQueues.cs index b09413336..2ab9625f7 100644 --- a/source/Halibut/Queue/QueuedDataStreams/InMemoryStoreDataStreamsForDistributedQueues.cs +++ b/source/Halibut/Queue/QueuedDataStreams/InMemoryStoreDataStreamsForDistributedQueues.cs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. + using System; using System.Collections.Generic; using System.IO; diff --git a/source/Halibut/Queue/Redis/HalibutRedisTransport.cs b/source/Halibut/Queue/Redis/HalibutRedisTransport.cs index 99b2f5ca2..58febff34 100644 --- a/source/Halibut/Queue/Redis/HalibutRedisTransport.cs +++ b/source/Halibut/Queue/Redis/HalibutRedisTransport.cs @@ -11,7 +11,7 @@ // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. - +#if NET8_0_OR_GREATER using System; using System.Diagnostics; using System.Threading; @@ -262,4 +262,5 @@ public RedisHalibutQueueItem2(Guid requestId, string payloadJson) public Guid RequestId { get; protected set; } public string PayloadJson { get; protected set; } } -} \ No newline at end of file +} +#endif \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/IHalibutRedisTransport.cs b/source/Halibut/Queue/Redis/IHalibutRedisTransport.cs index 9646e6bd8..098ebca6f 100644 --- a/source/Halibut/Queue/Redis/IHalibutRedisTransport.cs +++ b/source/Halibut/Queue/Redis/IHalibutRedisTransport.cs @@ -11,7 +11,7 @@ // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. - +#if NET8_0_OR_GREATER using System; using System.Threading; using System.Threading.Tasks; @@ -63,4 +63,5 @@ Task SubscribeToResponseChannel(Uri endpoint, Guid identifier, Task GetResponseMessage(Uri endpoint, Guid identifier, CancellationToken cancellationToken); Task DeleteResponse(Uri endpoint, Guid identifier, CancellationToken cancellationToken); } -} \ No newline at end of file +} +#endif \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/IWatchForRedisLosingAllItsData.cs b/source/Halibut/Queue/Redis/IWatchForRedisLosingAllItsData.cs new file mode 100644 index 000000000..0c8f8f8fd --- /dev/null +++ b/source/Halibut/Queue/Redis/IWatchForRedisLosingAllItsData.cs @@ -0,0 +1,31 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; +using System.Threading; +using System.Threading.Tasks; + +namespace Halibut.Queue.Redis +{ + public interface IWatchForRedisLosingAllItsData : IAsyncDisposable + { + /// + /// Will cause the caller to wait until we are connected to redis and so can detect datalose. + /// + /// Time to wait for this to reach a state where it can detect datalose + /// + /// A cancellation token which is triggered when data lose occurs. + Task GetTokenForDataLoseDetection(TimeSpan timeToWait, CancellationToken cancellationToken); + } +} \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs b/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs index 77b7dae4a..c46e76de9 100644 --- a/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs +++ b/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs @@ -11,7 +11,7 @@ // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. - +#if NET8_0_OR_GREATER using System; using System.Threading; using System.Threading.Tasks; @@ -235,4 +235,5 @@ public async ValueTask DisposeAsync() log.Write(EventType.Diagnostic, "NodeHeartBeatSender disposed for {0} node, request {1}", nodeSendingPulsesType, requestActivityId); } } -} \ No newline at end of file +} +#endif \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs b/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs index 7f202d94c..a8ea76f2c 100644 --- a/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs +++ b/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs @@ -11,7 +11,7 @@ // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. - +#if NET8_0_OR_GREATER using System; using System.Threading; using System.Threading.Tasks; @@ -207,4 +207,5 @@ public async ValueTask DisposeAsync() log.Write(EventType.Diagnostic, "Disposed GenericWatcher for response - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); } } -} \ No newline at end of file +} +#endif \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/RedisFacade.cs b/source/Halibut/Queue/Redis/RedisFacade.cs index 1c8549d9f..001e6e878 100644 --- a/source/Halibut/Queue/Redis/RedisFacade.cs +++ b/source/Halibut/Queue/Redis/RedisFacade.cs @@ -97,16 +97,6 @@ public RedisFacade(ConfigurationOptions redisOptions, string? keyPrefix, ILog lo }); } - public class ShouldAbandonAndReconnectHelper - { - readonly TaskCompletionSource connectionInError = new TaskCompletionSource(); - - public bool IsConnectionInError => connectionInError.Task.IsCompleted; - - public void SetReconnectionIsAdvised() => connectionInError.SetResult(); - - public Task WaitUntilShouldReSubscribeTask => connectionInError.Task; - } private void OnConnectionFailed(object? sender, ConnectionFailedEventArgs e) { var message = $"Redis connection failed - EndPoint: {e.EndPoint}, Failure: {e.FailureType}, Exception: {e.Exception?.Message}"; diff --git a/source/Halibut/Queue/Redis/RedisPendingRequest.cs b/source/Halibut/Queue/Redis/RedisPendingRequest.cs index 21c866f8a..f58fe40d2 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequest.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequest.cs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#if NET8_0_OR_GREATER using System; using System.Threading; using System.Threading.Tasks; @@ -240,4 +241,5 @@ public void Dispose() } } -} \ No newline at end of file +} +#endif \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index 9b6b45f15..90f6058a4 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -11,7 +11,7 @@ // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. - +#if NET8_0_OR_GREATER using System; using System.Collections.Concurrent; using System.Threading; @@ -550,4 +550,5 @@ public void Dispose() DisposeAsync().GetAwaiter().GetResult(); } } -} \ No newline at end of file +} +#endif \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueueFactory.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueueFactory.cs index 65fe38582..855b9b099 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueueFactory.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueueFactory.cs @@ -11,7 +11,7 @@ // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. - +#if NET8_0_OR_GREATER using System; using System.Threading.Tasks; using Halibut.Diagnostics; @@ -58,4 +58,6 @@ public IPendingRequestQueue CreateQueue(Uri endpoint) } } -} \ No newline at end of file +} + +#endif \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/WatchForRedisLosingAllItsData.cs b/source/Halibut/Queue/Redis/WatchForRedisLosingAllItsData.cs index 16fc90794..b53e34c8e 100644 --- a/source/Halibut/Queue/Redis/WatchForRedisLosingAllItsData.cs +++ b/source/Halibut/Queue/Redis/WatchForRedisLosingAllItsData.cs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#if NET8_0_OR_GREATER using System; using System.Threading; using System.Threading.Tasks; @@ -20,17 +21,6 @@ namespace Halibut.Queue.Redis { - public interface IWatchForRedisLosingAllItsData : IAsyncDisposable - { - /// - /// Will cause the caller to wait until we are connected to redis and so can detect datalose. - /// - /// Time to wait for this to reach a state where it can detect datalose - /// - /// A cancellation token which is triggered when data lose occurs. - Task GetTokenForDataLoseDetection(TimeSpan timeToWait, CancellationToken cancellationToken); - } - public class WatchForRedisLosingAllItsData : IWatchForRedisLosingAllItsData { RedisFacade redisFacade; @@ -139,4 +129,5 @@ public async ValueTask DisposeAsync() await cts.DisposeAsync(); } } -} \ No newline at end of file +} +#endif \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs b/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs index 026b2cd8b..1502252fb 100644 --- a/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs +++ b/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs @@ -11,7 +11,7 @@ // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. - +#if NET8_0_OR_GREATER using System; using System.Threading; using System.Threading.Tasks; @@ -142,4 +142,5 @@ public async ValueTask DisposeAsync() log.Write(EventType.Diagnostic, "WatchForRequestCancellation disposed"); } } -} \ No newline at end of file +} +#endif \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/WatchForRequestCancellationOrSenderDisconnect.cs b/source/Halibut/Queue/Redis/WatchForRequestCancellationOrSenderDisconnect.cs index 6a5d4247c..54ce63ba2 100644 --- a/source/Halibut/Queue/Redis/WatchForRequestCancellationOrSenderDisconnect.cs +++ b/source/Halibut/Queue/Redis/WatchForRequestCancellationOrSenderDisconnect.cs @@ -11,7 +11,7 @@ // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. - +#if NET8_0_OR_GREATER using System; using System.Security.Policy; using System.Threading; @@ -89,4 +89,5 @@ public async ValueTask DisposeAsync() await disposableCollection.DisposeAsync(); } } -} \ No newline at end of file +} +#endif \ No newline at end of file diff --git a/source/Halibut/Util/CancelOnDisposeCancellationToken.cs b/source/Halibut/Util/CancelOnDisposeCancellationToken.cs index b20c8acb0..0e3ef0372 100644 --- a/source/Halibut/Util/CancelOnDisposeCancellationToken.cs +++ b/source/Halibut/Util/CancelOnDisposeCancellationToken.cs @@ -54,7 +54,8 @@ public async Task CancelAsync() #if NET8_0_OR_GREATER await cancellationTokenSource.CancelAsync(); #else - CancellationTokenSource.Cancel(); + await Task.CompletedTask; + cancellationTokenSource.Cancel(); #endif } diff --git a/source/Halibut/Util/net48Helpers/NotNullWhenAttribute.cs b/source/Halibut/Util/net48Helpers/NotNullWhenAttribute.cs new file mode 100644 index 000000000..60798dc05 --- /dev/null +++ b/source/Halibut/Util/net48Helpers/NotNullWhenAttribute.cs @@ -0,0 +1,27 @@ + + + +// This is not available in net48 + +#if NETFRAMEWORK +using System; + +#nullable enable +namespace System.Diagnostics.CodeAnalysis +{ + /// Specifies that the output will be non-null if the named parameter is non-null. + [AttributeUsage(AttributeTargets.Property | AttributeTargets.Parameter | AttributeTargets.ReturnValue, AllowMultiple = true, Inherited = false)] + public sealed class NotNullIfNotNullAttribute : Attribute + { + /// Initializes the attribute with the associated parameter name. + /// The associated parameter name. The output will be non-null if the argument to the parameter specified is non-null. + public NotNullIfNotNullAttribute(string parameterName) => this.ParameterName = parameterName; + + /// Gets the associated parameter name. + /// The associated parameter name. The output will be non-null if the argument to the parameter specified is non-null. + public string ParameterName { get; } + } +} + + +#endif \ No newline at end of file From a40e45c559b65468447da69fc559e651ed1df7c7 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Fri, 15 Aug 2025 16:07:36 +1000 Subject: [PATCH 056/137] Ignore most redis tests for now --- source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs | 1 + .../Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs | 1 + .../Queue/Redis/RedisPendingRequestQueueFixture.cs | 1 + source/Halibut.Tests/Queue/Redis/RedisQueuePerformanceTest.cs | 2 ++ .../Queue/Redis/TestRedisPendingRequestQueueFactory.cs | 2 ++ source/Halibut.Tests/ServiceModel/PendingRequestQueueFixture.cs | 2 +- 6 files changed, 8 insertions(+), 1 deletion(-) diff --git a/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs b/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs index bc6d9cd32..8cf91bfea 100644 --- a/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs @@ -21,6 +21,7 @@ namespace Halibut.Tests.Queue.Redis { + [Ignore("REDISTODO")] [SuppressMessage("Usage", "VSTHRD003:Avoid awaiting foreign Tasks")] public class NodeHeartBeatSenderFixture : BaseTest { diff --git a/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs b/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs index d53ca9b55..d55cea1c9 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs @@ -30,6 +30,7 @@ namespace Halibut.Tests.Queue.Redis { + [Ignore("REDISTODO")] [SuppressMessage("Usage", "VSTHRD003:Avoid awaiting foreign Tasks")] public class RedisFacadeWhenRedisGoesDownAwayTests : BaseTest { diff --git a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs index c2f306056..99b9c2726 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs @@ -38,6 +38,7 @@ namespace Halibut.Tests.Queue.Redis { + [Ignore("REDISTODO")] public class RedisPendingRequestQueueFixture : BaseTest { const int redisPort = 6379; diff --git a/source/Halibut.Tests/Queue/Redis/RedisQueuePerformanceTest.cs b/source/Halibut.Tests/Queue/Redis/RedisQueuePerformanceTest.cs index 1159800b8..b0e71beab 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisQueuePerformanceTest.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisQueuePerformanceTest.cs @@ -24,9 +24,11 @@ using Halibut.Tests.Queue.Redis.Utils; using Halibut.Tests.Support.Logging; using Halibut.Util; +using NUnit.Framework; namespace Halibut.Tests.Queue.Redis { + [Ignore("REDISTODO")] public class RedisQueuePerformanceTest : BaseTest { const int redisPort = 6379; diff --git a/source/Halibut.Tests/Queue/Redis/TestRedisPendingRequestQueueFactory.cs b/source/Halibut.Tests/Queue/Redis/TestRedisPendingRequestQueueFactory.cs index 751bdfb3a..7ef865f7f 100644 --- a/source/Halibut.Tests/Queue/Redis/TestRedisPendingRequestQueueFactory.cs +++ b/source/Halibut.Tests/Queue/Redis/TestRedisPendingRequestQueueFactory.cs @@ -15,9 +15,11 @@ using System; using Halibut.Queue.Redis; using Halibut.ServiceModel; +using NUnit.Framework; namespace Halibut.Tests.Queue.Redis { + [Ignore("REDISTODO")] public class TestRedisPendingRequestQueueFactory : IPendingRequestQueueFactory { RedisPendingRequestQueueFactory redisPendingRequestQueueFactory; diff --git a/source/Halibut.Tests/ServiceModel/PendingRequestQueueFixture.cs b/source/Halibut.Tests/ServiceModel/PendingRequestQueueFixture.cs index 448d51aa3..c3f05bf32 100644 --- a/source/Halibut.Tests/ServiceModel/PendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/ServiceModel/PendingRequestQueueFixture.cs @@ -15,7 +15,7 @@ namespace Halibut.Tests.ServiceModel { - + [Ignore("REDISTODO")] public class PendingRequestQueueFixture : BaseTest { [Test] From f6b9346bd02b30845600df8bbbc528b8ed95f69e Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Fri, 15 Aug 2025 16:53:17 +1000 Subject: [PATCH 057/137] Start redis if not already started --- .../BindCertificatesForAllTests.cs | 20 ------ .../BumpThreadPoolForAllTests.cs | 23 ------- source/Halibut.Tests/Halibut.Tests.csproj | 1 + .../WatchForRedisLosingAllItsDataFixture.cs | 15 +++-- .../TestSetup/BumpThreadPoolForAllTests.cs | 37 ++++++++++ .../Halibut.Tests/TestSetup/ISetupFixture.cs | 25 +++++++ .../CreateRedisDockerContainerForTests.cs | 57 ++++++++++++++++ .../EnsureRedisIsAvailableSetupFixture.cs | 67 +++++++++++++++++++ .../TestSetup/Redis/RedisPort.cs | 37 ++++++++++ source/Halibut.Tests/TestsSetupClass.cs | 45 +++++++++++++ 10 files changed, 279 insertions(+), 48 deletions(-) delete mode 100644 source/Halibut.Tests/BindCertificatesForAllTests.cs delete mode 100644 source/Halibut.Tests/BumpThreadPoolForAllTests.cs create mode 100644 source/Halibut.Tests/TestSetup/BumpThreadPoolForAllTests.cs create mode 100644 source/Halibut.Tests/TestSetup/ISetupFixture.cs create mode 100644 source/Halibut.Tests/TestSetup/Redis/CreateRedisDockerContainerForTests.cs create mode 100644 source/Halibut.Tests/TestSetup/Redis/EnsureRedisIsAvailableSetupFixture.cs create mode 100644 source/Halibut.Tests/TestSetup/Redis/RedisPort.cs create mode 100644 source/Halibut.Tests/TestsSetupClass.cs diff --git a/source/Halibut.Tests/BindCertificatesForAllTests.cs b/source/Halibut.Tests/BindCertificatesForAllTests.cs deleted file mode 100644 index 52317e20e..000000000 --- a/source/Halibut.Tests/BindCertificatesForAllTests.cs +++ /dev/null @@ -1,20 +0,0 @@ -using Halibut.Tests.Support; -using NUnit.Framework; - -namespace Halibut.Tests -{ - public class BindCertificatesForAllTests - { - [SetUpFixture] - public class TestsSetupClass - { - [OneTimeSetUp] - public void GlobalSetup() - { -#if SUPPORTS_WEB_SOCKET_CLIENT - WebSocketSslCertificateHelper.AddSslCertToLocalStore(); -#endif - } - } - } -} \ No newline at end of file diff --git a/source/Halibut.Tests/BumpThreadPoolForAllTests.cs b/source/Halibut.Tests/BumpThreadPoolForAllTests.cs deleted file mode 100644 index 50a268740..000000000 --- a/source/Halibut.Tests/BumpThreadPoolForAllTests.cs +++ /dev/null @@ -1,23 +0,0 @@ -using System; -using System.Threading; -using NUnit.Framework; - -namespace Halibut.Tests -{ - public class BumpThreadPoolForAllTests - { - [SetUpFixture] - public class TestsSetupClass - { - [OneTimeSetUp] - public void GlobalSetup() - { - var minWorkerPoolThreads = 400; - ThreadPool.GetMinThreads(out _, out var minCompletionPortThreads); - ThreadPool.GetMaxThreads(out var maxWorkerThreads, out var maxCompletionPortThreads); - ThreadPool.SetMaxThreads(Math.Max(minWorkerPoolThreads, maxWorkerThreads), Math.Max(minCompletionPortThreads, maxCompletionPortThreads)); - ThreadPool.SetMinThreads(minWorkerPoolThreads, minCompletionPortThreads); - } - } - } -} \ No newline at end of file diff --git a/source/Halibut.Tests/Halibut.Tests.csproj b/source/Halibut.Tests/Halibut.Tests.csproj index c08878e60..8a9df14f2 100644 --- a/source/Halibut.Tests/Halibut.Tests.csproj +++ b/source/Halibut.Tests/Halibut.Tests.csproj @@ -7,6 +7,7 @@ true false 9.0 + VSTHRD002 enable true diff --git a/source/Halibut.Tests/Queue/Redis/WatchForRedisLosingAllItsDataFixture.cs b/source/Halibut.Tests/Queue/Redis/WatchForRedisLosingAllItsDataFixture.cs index 8c00b40ab..35e61bc37 100644 --- a/source/Halibut.Tests/Queue/Redis/WatchForRedisLosingAllItsDataFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/WatchForRedisLosingAllItsDataFixture.cs @@ -7,6 +7,7 @@ using Halibut.Tests.Queue.Redis.Utils; using Halibut.Tests.Support; using Halibut.Tests.Support.Logging; +using Halibut.Tests.TestSetup.Redis; using NUnit.Framework; using Octopus.TestPortForwarder; @@ -14,13 +15,17 @@ namespace Halibut.Tests.Queue.Redis { public class WatchForRedisLosingAllItsDataFixture : BaseTest { - const int redisPort = 6379; - private static RedisFacade CreateRedisFacade(int? port = redisPort, Guid? guid = null) => new("localhost:" + port, (guid ?? Guid.NewGuid()).ToString(), new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix("")); + + private static RedisFacade CreateRedisFacade(int? port = 0, Guid? guid = null) + { + port = port == 0 ? RedisPort.Port() : port; + return new RedisFacade("localhost:" + port, (guid ?? Guid.NewGuid()).ToString(), new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix("")); + } [Test] public async Task WhenTheConnectionToRedisCanNotBeCreated_WhenAskingForALostDataCancellationToken_ATimeoutOccurs() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); portForwarder.EnterKillNewAndExistingConnectionsMode(); await using var redisFacade = CreateRedisFacade(port: portForwarder.ListeningPort); await using var watcher = new WatchForRedisLosingAllItsData(redisFacade, HalibutLog, watchInterval:TimeSpan.FromSeconds(1)); @@ -32,7 +37,7 @@ public async Task WhenTheConnectionToRedisCanNotBeCreated_WhenAskingForALostData [Test] public async Task WhenTheConnectionToRedisIsInitiallyDown_WhenAskingForALostDataCancellationToken_AndTheConnectionToRedisReturns_TheCTIsReturned() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); portForwarder.EnterKillNewAndExistingConnectionsMode(); await using var redisFacade = CreateRedisFacade(port: portForwarder.ListeningPort); await using var watcher = new WatchForRedisLosingAllItsData(redisFacade, HalibutLog, watchInterval:TimeSpan.FromSeconds(1)); @@ -50,7 +55,7 @@ public async Task WhenTheConnectionToRedisIsInitiallyDown_WhenAskingForALostData [Test] public async Task WatchForARealRedisLosingAllOfItsData_TimesOutWhenWaitingForCTWhenNoConnectionToRedisCanBeEstablished() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); portForwarder.EnterKillNewAndExistingConnectionsMode(); await using var redisFacade = CreateRedisFacade(port: portForwarder.ListeningPort); await using var watcher = new WatchForRedisLosingAllItsData(redisFacade, HalibutLog, watchInterval:TimeSpan.FromSeconds(1)); diff --git a/source/Halibut.Tests/TestSetup/BumpThreadPoolForAllTests.cs b/source/Halibut.Tests/TestSetup/BumpThreadPoolForAllTests.cs new file mode 100644 index 000000000..f01eb3676 --- /dev/null +++ b/source/Halibut.Tests/TestSetup/BumpThreadPoolForAllTests.cs @@ -0,0 +1,37 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; +using System.Threading; +using Serilog; + +namespace Halibut.Tests.TestSetup +{ + public class BumpThreadPoolForAllTests : ISetupFixture + { + public void OneTimeSetUp(ILogger logger) + { + logger.Information("Bumping thread pool"); + var minWorkerPoolThreads = 400; + ThreadPool.GetMinThreads(out _, out var minCompletionPortThreads); + ThreadPool.GetMaxThreads(out var maxWorkerThreads, out var maxCompletionPortThreads); + ThreadPool.SetMaxThreads(Math.Max(minWorkerPoolThreads, maxWorkerThreads), Math.Max(minCompletionPortThreads, maxCompletionPortThreads)); + ThreadPool.SetMinThreads(minWorkerPoolThreads, minCompletionPortThreads); + } + + public void OneTimeTearDown(ILogger logger) + { + } + } +} \ No newline at end of file diff --git a/source/Halibut.Tests/TestSetup/ISetupFixture.cs b/source/Halibut.Tests/TestSetup/ISetupFixture.cs new file mode 100644 index 000000000..4c8713ded --- /dev/null +++ b/source/Halibut.Tests/TestSetup/ISetupFixture.cs @@ -0,0 +1,25 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using Serilog; + +namespace Halibut.Tests.TestSetup +{ + public interface ISetupFixture + { + public void OneTimeSetUp(ILogger logger); + + public void OneTimeTearDown(ILogger logger); + } +} \ No newline at end of file diff --git a/source/Halibut.Tests/TestSetup/Redis/CreateRedisDockerContainerForTests.cs b/source/Halibut.Tests/TestSetup/Redis/CreateRedisDockerContainerForTests.cs new file mode 100644 index 000000000..2d46bf201 --- /dev/null +++ b/source/Halibut.Tests/TestSetup/Redis/CreateRedisDockerContainerForTests.cs @@ -0,0 +1,57 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; +using System.Threading.Tasks; +using Halibut.Tests.Queue.Redis.Utils; +using Serilog; + +namespace Halibut.Tests.TestSetup +{ + public class CreateRedisDockerContainerForTests : IAsyncDisposable + { + readonly ILogger logger; + public RedisContainer? container = null; + + public CreateRedisDockerContainerForTests(ILogger logger) + { + this.logger = logger; + } + + public async Task InitializeAsync() + { + logger.Information("Creating Redis container"); + container = new RedisContainerBuilder().Build(); + + logger.Information("Starting Redis container"); + await container.StartAsync(); + logger.Information("Redis container started successfully with connection string: {ConnectionString}", container.ConnectionString); + + } + public async ValueTask DisposeAsync() + { + if (container != null) + { + try + { + await container.DisposeAsync(); + } + catch (Exception e) + { + logger.Error(e, "Error while disposing Redis container"); + } + } + } + } +} \ No newline at end of file diff --git a/source/Halibut.Tests/TestSetup/Redis/EnsureRedisIsAvailableSetupFixture.cs b/source/Halibut.Tests/TestSetup/Redis/EnsureRedisIsAvailableSetupFixture.cs new file mode 100644 index 000000000..4cbd3061c --- /dev/null +++ b/source/Halibut.Tests/TestSetup/Redis/EnsureRedisIsAvailableSetupFixture.cs @@ -0,0 +1,67 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; +using System.Runtime.InteropServices; +using Halibut.Tests.Support; +using Serilog; +using StackExchange.Redis; + +namespace Halibut.Tests.TestSetup.Redis +{ + public class EnsureRedisIsAvailableSetupFixture : ISetupFixture + { + CreateRedisDockerContainerForTests? redisContainer = null; + public void OneTimeSetUp(ILogger logger) + { +#if NETFRAMEWORK + return; +#endif + var isWindows = RuntimeInformation.IsOSPlatform(OSPlatform.Windows); + bool shouldCreateRedis = !isWindows && TeamCityDetection.IsRunningInTeamCity(); + + if (!TeamCityDetection.IsRunningInTeamCity()) + { + if (!isWindows) + { + // Does the user already have redis running on the normal port? + try + { + using var multiplexer = ConnectionMultiplexer.Connect("localhost:6379"); + var ts = multiplexer.GetDatabase().Ping(); + RedisPort.SetPort(6379); + } + catch + { + shouldCreateRedis = true; + } + } + } + + if (shouldCreateRedis) + { + redisContainer = new CreateRedisDockerContainerForTests(logger); + redisContainer.InitializeAsync().GetAwaiter().GetResult(); + RedisPort.SetPort(redisContainer.container!.RedisPort); + logger.Information("RedisPort is: {RedisPort}", RedisPort.Port()); + } + } + + public void OneTimeTearDown(ILogger logger) + { + + if(redisContainer != null) redisContainer.DisposeAsync().GetAwaiter().GetResult(); + } + } +} \ No newline at end of file diff --git a/source/Halibut.Tests/TestSetup/Redis/RedisPort.cs b/source/Halibut.Tests/TestSetup/Redis/RedisPort.cs new file mode 100644 index 000000000..7f47f4f38 --- /dev/null +++ b/source/Halibut.Tests/TestSetup/Redis/RedisPort.cs @@ -0,0 +1,37 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; + +namespace Halibut.Tests.TestSetup.Redis +{ + public static class RedisPort + { + static int redisPort = 0; + public static void SetPort(int value) + { + redisPort = value; + } + + public static int Port() + { + if (redisPort == 0) + { + throw new Exception("Redis is unavailable"); + } + + return redisPort; + } + } +} \ No newline at end of file diff --git a/source/Halibut.Tests/TestsSetupClass.cs b/source/Halibut.Tests/TestsSetupClass.cs new file mode 100644 index 000000000..12e235308 --- /dev/null +++ b/source/Halibut.Tests/TestsSetupClass.cs @@ -0,0 +1,45 @@ +using System; +using System.Text; +using Halibut.Tests.Support; +using Halibut.Tests.TestSetup; +using Halibut.Tests.TestSetup.Redis; +using NUnit.Framework; + +namespace Halibut.Tests +{ + /// + /// We will have only one of these for Nunit, since multiple of these can + /// result in pain e.g. test runners being confused, no logs showing. + /// + [SetUpFixture] + public class TestsSetupClass + { + private ISetupFixture[] setupFixtures = new ISetupFixture[] + { + new EnsureRedisIsAvailableSetupFixture(), + new BumpThreadPoolForAllTests() + }; + + [OneTimeSetUp] + public void OneTimeSetup() + { + var sb = new StringBuilder(); + var logger = new SerilogLoggerBuilder() + // Ideally we would write the logs to a static string and have a test print those out. + // So that we can see what is happening in start up in teamcity. + // However, the logging is a bit of a mess in halibut currently. + .Build() + .ForContext(); + foreach (var setupFixture in setupFixtures) + { + setupFixture.OneTimeSetUp(logger.ForContext(setupFixture.GetType())); + } + +#if SUPPORTS_WEB_SOCKET_CLIENT + WebSocketSslCertificateHelper.AddSslCertToLocalStore(); +#endif + } + + } + +} \ No newline at end of file From 2f9964d5015e47dbaa30835ba5acacf89de3e7e4 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Fri, 15 Aug 2025 16:59:44 +1000 Subject: [PATCH 058/137] . --- .../RedisPendingRequestQueueBuilder.cs | 6 +-- .../Queue/Redis/NodeHeartBeatSenderFixture.cs | 51 ++++++++++--------- .../Queue/Redis/RedisFacadeFixture.cs | 51 ++++++++++--------- .../Redis/RedisPendingRequestQueueFixture.cs | 37 +++++++++----- .../Queue/Redis/RedisQueuePerformanceTest.cs | 6 +-- .../Queue/Redis/Utils/RedisFacadeBuilder.cs | 31 +++++++++++ .../WatchForRedisLosingAllItsDataFixture.cs | 2 +- .../EnsureRedisIsAvailableSetupFixture.cs | 3 +- 8 files changed, 116 insertions(+), 71 deletions(-) create mode 100644 source/Halibut.Tests/Queue/Redis/Utils/RedisFacadeBuilder.cs diff --git a/source/Halibut.Tests/Builders/RedisPendingRequestQueueBuilder.cs b/source/Halibut.Tests/Builders/RedisPendingRequestQueueBuilder.cs index 39f4a23e9..7b84d48e8 100644 --- a/source/Halibut.Tests/Builders/RedisPendingRequestQueueBuilder.cs +++ b/source/Halibut.Tests/Builders/RedisPendingRequestQueueBuilder.cs @@ -19,6 +19,7 @@ using Halibut.Tests.Queue; using Halibut.Tests.Queue.Redis.Utils; using Halibut.Tests.Support.Logging; +using Halibut.Tests.TestSetup.Redis; using Halibut.Util; using ILog = Halibut.Diagnostics.ILog; @@ -26,9 +27,6 @@ namespace Halibut.Tests.Builders { public class RedisPendingRequestQueueBuilder : IPendingRequestQueueBuilder { - - const int redisPort = 6379; - ILog? log; string? endpoint; TimeSpan? pollingQueueWaitTimeout; @@ -64,7 +62,7 @@ public QueueHolder Build() var disposableCollection = new DisposableCollection(); - var redisFacade = new RedisFacade("localhost:" + redisPort, (Guid.NewGuid()).ToString(), log); + var redisFacade = new RedisFacade("localhost:" + RedisPort.Port(), (Guid.NewGuid()).ToString(), log); disposableCollection.AddAsyncDisposable(redisFacade); var redisTransport = new HalibutRedisTransport(redisFacade); diff --git a/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs b/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs index 8cf91bfea..1038328ea 100644 --- a/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs @@ -1,20 +1,17 @@ #if NET8_0_OR_GREATER using System; using System.Collections.Concurrent; -using System.Collections.Generic; using System.Diagnostics.CodeAnalysis; using System.Linq; using System.Threading; using System.Threading.Tasks; using FluentAssertions; -using Halibut.Diagnostics; using Halibut.Logging; using Halibut.Queue.Redis; -using Halibut.ServiceModel; using Halibut.Tests.Builders; +using Halibut.Tests.Queue.Redis.Utils; using Halibut.Tests.Support.Logging; -using Halibut.Transport.Protocol; -using Halibut.Util; +using Halibut.Tests.TestSetup.Redis; using Nito.AsyncEx; using NUnit.Framework; using Octopus.TestPortForwarder; @@ -25,11 +22,11 @@ namespace Halibut.Tests.Queue.Redis [SuppressMessage("Usage", "VSTHRD003:Avoid awaiting foreign Tasks")] public class NodeHeartBeatSenderFixture : BaseTest { - const int redisPort = 6379; - - private static RedisFacade CreateRedisFacade(int? port = redisPort, Guid? guid = null) => - new("localhost:" + port, (guid ?? Guid.NewGuid()).ToString(), - new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix("")); + private static RedisFacade CreateRedisFacade(int? port = 0, Guid? guid = null) + { + port = port == 0 ? RedisPort.Port() : port; + return new RedisFacade("localhost:" + port, (guid ?? Guid.NewGuid()).ToString(), new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix("")); + } // TODO: ai tests need review [Test] @@ -39,7 +36,8 @@ public async Task WhenCreated_ShouldStartSendingHeartbeats() var endpoint = new Uri("poll://" + Guid.NewGuid()); var requestActivityId = Guid.NewGuid(); var log = new TestContextLogCreator("NodeHeartBeat", LogLevel.Trace).CreateNewForPrefix(""); - var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); + var redisTransport = new HalibutRedisTransport(redisFacade); var anyHeartBeatReceived = new AsyncManualResetEvent(false); @@ -71,9 +69,9 @@ public async Task WhenRedisConnectionIsInterrupted_ShouldSwitchToPanicMode() var log = new TestContextLogCreator("NodeHeartBeat", LogLevel.Trace).CreateNewForPrefix(""); var guid = Guid.NewGuid(); - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); await using var unstableRedisFacade = CreateRedisFacade(portForwarder.ListeningPort, guid); - await using var stableRedisFacade = CreateRedisFacade(redisPort, guid); + await using var stableRedisFacade = CreateRedisFacade(RedisPort.Port(), guid); var redisTransport = new HalibutRedisTransport(unstableRedisFacade); @@ -122,7 +120,8 @@ public async Task WhenDisposed_ShouldStopSendingHeartbeats() var endpoint = new Uri("poll://" + Guid.NewGuid()); var requestActivityId = Guid.NewGuid(); var log = new TestContextLogCreator("NodeHeartBeat", LogLevel.Trace).CreateNewForPrefix(""); - var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); + var redisTransport = new HalibutRedisTransport(redisFacade); var heartbeatsReceived = new ConcurrentBag(); var anyHeartBeatReceived = new AsyncManualResetEvent(false); @@ -162,9 +161,9 @@ public async Task WaitUntilNodeProcessingRequestFlatLines_WhenHeartbeatsStop_Sho var log = new TestContextLogCreator("NodeHeartBeat", LogLevel.Trace).CreateNewForPrefix(""); var guid = Guid.NewGuid(); - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); await using var unstableRedisFacade = CreateRedisFacade(portForwarder.ListeningPort, guid); - await using var stableRedisFacade = CreateRedisFacade(redisPort, guid); + await using var stableRedisFacade = CreateRedisFacade(RedisPort.Port(), guid); var unstableRedisTransport = new HalibutRedisTransport(unstableRedisFacade); var stableRedisTransport = new HalibutRedisTransport(stableRedisFacade); @@ -209,7 +208,8 @@ public async Task NoIssueReturnedWhenNodeProcessingRequestIsNotSeenToGoOffline() var endpoint = new Uri("poll://" + Guid.NewGuid()); var requestActivityId = Guid.NewGuid(); var log = new TestContextLogCreator("NodeHeartBeat", LogLevel.Trace).CreateNewForPrefix(""); - var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); + var redisTransport = new HalibutRedisTransport(redisFacade); var request = new RequestMessageBuilder(endpoint.ToString()) .WithActivityId(requestActivityId) @@ -248,9 +248,9 @@ public async Task WaitUntilNodeProcessingRequestFlatLines_WhenConnectionInterrup var log = new TestContextLogCreator("NodeHeartBeat", LogLevel.Trace).CreateNewForPrefix(""); var guid = Guid.NewGuid(); - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); await using var unstableRedisFacade = CreateRedisFacade(portForwarder.ListeningPort, guid); - await using var stableRedisFacade = CreateRedisFacade(redisPort, guid); + await using var stableRedisFacade = CreateRedisFacade(RedisPort.Port(), guid); var unstableRedisTransport = new HalibutRedisTransport(unstableRedisFacade); var stableRedisTransport = new HalibutRedisTransport(stableRedisFacade); @@ -295,7 +295,8 @@ public async Task WhenMultipleHeartBeatSendersForSameRequest_OnlyOneSetOfHeartbe var endpoint = new Uri("poll://" + Guid.NewGuid()); var requestActivityId = Guid.NewGuid(); var log = new TestContextLogCreator("NodeHeartBeat", LogLevel.Trace).CreateNewForPrefix(""); - var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); + var redisTransport = new HalibutRedisTransport(redisFacade); var heartbeatsReceived = new ConcurrentBag(); @@ -328,9 +329,9 @@ public async Task WhenHeartBeatSenderConnectionRecovery_ShouldResumeNormalHeartb var log = new TestContextLogCreator("NodeHeartBeat", LogLevel.Trace).CreateNewForPrefix(""); var guid = Guid.NewGuid(); - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); await using var unstableRedisFacade = CreateRedisFacade(portForwarder.ListeningPort, guid); - await using var stableRedisFacade = CreateRedisFacade(redisPort, guid); + await using var stableRedisFacade = CreateRedisFacade(RedisPort.Port(), guid); var unstableRedisTransport = new HalibutRedisTransport(unstableRedisFacade); var stableRedisTransport = new HalibutRedisTransport(stableRedisFacade); @@ -379,7 +380,8 @@ public async Task SenderAndReceiverNodeTypes_ShouldUseDistinctChannels() var endpoint = new Uri("poll://" + Guid.NewGuid()); var requestActivityId = Guid.NewGuid(); var log = new TestContextLogCreator("NodeHeartBeat", LogLevel.Trace).CreateNewForPrefix(""); - var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); + var redisTransport = new HalibutRedisTransport(redisFacade); var senderHeartbeatsReceived = new AsyncManualResetEvent(false); var receiverHeartbeatsReceived = new AsyncManualResetEvent(false); @@ -424,7 +426,8 @@ public async Task SenderNodeHeartbeats_ShouldNotBeReceivedByReceiverSubscription var endpoint = new Uri("poll://" + Guid.NewGuid()); var requestActivityId = Guid.NewGuid(); var log = new TestContextLogCreator("NodeHeartBeat", LogLevel.Trace).CreateNewForPrefix(""); - var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); + var redisTransport = new HalibutRedisTransport(redisFacade); var receiverHeartbeatsReceived = new AsyncManualResetEvent(false); diff --git a/source/Halibut.Tests/Queue/Redis/RedisFacadeFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisFacadeFixture.cs index 776d7770d..d3dd1772b 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisFacadeFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisFacadeFixture.cs @@ -5,8 +5,10 @@ using FluentAssertions; using Halibut.Logging; using Halibut.Queue.Redis; +using Halibut.Tests.Queue.Redis.Utils; using Halibut.Tests.Support; using Halibut.Tests.Support.Logging; +using Halibut.Tests.TestSetup.Redis; using Halibut.Util.AsyncEx; using NUnit.Framework; @@ -14,7 +16,6 @@ namespace Halibut.Tests.Queue.Redis { public class RedisFacadeFixture : BaseTest { - private static RedisFacade CreateRedisFacade() => new("localhost", Guid.NewGuid().ToString(), new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix("")); [Test] public void Constructor_WithRedisHostAndKeyPrefix_ShouldCreateInstance() @@ -40,7 +41,7 @@ public void Constructor_WithNullKeyPrefix_ShouldUseDefaultPrefix() public async Task SetString_AndGetString_ShouldStoreAndRetrieveValue() { // Arrange - await using var redisFacade = CreateRedisFacade(); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); var key = Guid.NewGuid().ToString(); var value = "test-value"; @@ -56,7 +57,7 @@ public async Task SetString_AndGetString_ShouldStoreAndRetrieveValue() public async Task GetString_WithNonExistentKey_ShouldReturnNull() { // Arrange - await using var redisFacade = CreateRedisFacade(); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); var nonExistentKey = Guid.NewGuid().ToString(); // Act @@ -70,7 +71,7 @@ public async Task GetString_WithNonExistentKey_ShouldReturnNull() public async Task SetInHash_ShouldStoreValueInHash() { // Arrange - await using var redisFacade = CreateRedisFacade(); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); var key = Guid.NewGuid().ToString(); var field = "test-field"; var payload = "test-payload"; @@ -87,7 +88,7 @@ public async Task SetInHash_ShouldStoreValueInHash() public async Task TryGetAndDeleteFromHash_WithExistingValue_ShouldReturnValueAndDelete() { // Arrange - await using var redisFacade = CreateRedisFacade(); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); var key = Guid.NewGuid().ToString(); var field = "test-field"; var payload = "test-payload"; @@ -105,7 +106,7 @@ public async Task TryGetAndDeleteFromHash_WithExistingValue_ShouldReturnValueAnd public async Task HashContainsKey_WithExistingField_ShouldReturnTrue() { // Arrange - await using var redisFacade = CreateRedisFacade(); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); var key = Guid.NewGuid().ToString(); var field = "test-field"; var payload = "test-payload"; @@ -123,7 +124,7 @@ public async Task HashContainsKey_WithExistingField_ShouldReturnTrue() public async Task HashContainsKey_WithNonExistentField_ShouldReturnFalse() { // Arrange - await using var redisFacade = CreateRedisFacade(); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); var key = Guid.NewGuid().ToString(); var nonExistentField = "non-existent-field"; @@ -138,7 +139,7 @@ public async Task HashContainsKey_WithNonExistentField_ShouldReturnFalse() public async Task HashContainsKey_WithNonExistentKey_ShouldReturnFalse() { // Arrange - await using var redisFacade = CreateRedisFacade(); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); var nonExistentKey = Guid.NewGuid().ToString(); var field = "test-field"; @@ -153,7 +154,7 @@ public async Task HashContainsKey_WithNonExistentKey_ShouldReturnFalse() public async Task TryGetAndDeleteFromHash_ShouldDeleteTheEntireKey() { // Arrange - await using var redisFacade = CreateRedisFacade(); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); var key = Guid.NewGuid().ToString(); var field = "test-field"; var payload = "test-payload"; @@ -183,7 +184,7 @@ public async Task TryGetAndDeleteFromHash_ShouldDeleteTheEntireKey() public async Task ListRightPushAsync_AndListLeftPopAsync_ShouldWorkAsQueue() { // Arrange - await using var redisFacade = CreateRedisFacade(); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); var key = Guid.NewGuid().ToString(); var payload1 = "first-item"; var payload2 = "second-item"; @@ -205,7 +206,7 @@ public async Task ListRightPushAsync_AndListLeftPopAsync_ShouldWorkAsQueue() public async Task ListLeftPopAsync_WithEmptyList_ShouldReturnNull() { // Arrange - await using var redisFacade = CreateRedisFacade(); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); var emptyListKey = Guid.NewGuid().ToString(); // Act @@ -219,7 +220,7 @@ public async Task ListLeftPopAsync_WithEmptyList_ShouldReturnNull() public async Task PublishToChannel_AndSubscribeToChannel_ShouldDeliverMessage() { // Arrange - await using var redisFacade = CreateRedisFacade(); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); var channelName = Guid.NewGuid().ToString(); var testMessage = "test-message"; var receivedMessages = new List(); @@ -251,7 +252,7 @@ public async Task PublishToChannel_AndSubscribeToChannel_ShouldDeliverMessage() public async Task PublishToChannel_WithMultipleMessages_ShouldDeliverAllMessages() { // Arrange - await using var redisFacade = CreateRedisFacade(); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); var channelName = Guid.NewGuid().ToString(); var messages = new[] { "message1", "message2", "message3" }; var receivedMessages = new List(); @@ -289,7 +290,7 @@ public async Task PublishToChannel_WithMultipleMessages_ShouldDeliverAllMessages public async Task SubscribeToChannel_WhenDisposed_ShouldUnsubscribe() { // Arrange - await using var redisFacade = CreateRedisFacade(); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); var channelName = Guid.NewGuid().ToString(); var receivedMessages = new List(); @@ -346,7 +347,7 @@ public async Task KeyPrefixing_ShouldIsolateDataBetweenDifferentPrefixes() public async Task SetInHash_WithTTL_ShouldExpireAfterSpecifiedTime() { // Arrange - await using var redisFacade = CreateRedisFacade(); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); var key = Guid.NewGuid().ToString(); var field = "test-field"; var payload = "test-payload"; @@ -381,7 +382,7 @@ await ShouldEventually.Eventually(async () => public async Task DeleteString_WithExistingKey_ShouldReturnTrueAndDeleteValue() { // Arrange - await using var redisFacade = CreateRedisFacade(); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); var key = Guid.NewGuid().ToString(); var value = "test-value"; @@ -407,7 +408,7 @@ public async Task DeleteString_WithExistingKey_ShouldReturnTrueAndDeleteValue() public async Task DeleteString_WithNonExistentKey_ShouldReturnFalse() { // Arrange - await using var redisFacade = CreateRedisFacade(); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); var nonExistentKey = Guid.NewGuid().ToString(); // Act @@ -421,7 +422,7 @@ public async Task DeleteString_WithNonExistentKey_ShouldReturnFalse() public async Task SetTtlForString_WithExistingKey_ShouldUpdateTTL() { // Arrange - await using var redisFacade = CreateRedisFacade(); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); var key = Guid.NewGuid().ToString(); var value = "test-value"; @@ -447,7 +448,7 @@ public async Task SetTtlForString_WithExistingKey_ShouldUpdateTTL() public async Task SetString_WithShortTTL_ShouldExpire() { // Arrange - await using var redisFacade = CreateRedisFacade(); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); var key = Guid.NewGuid().ToString(); var value = "test-value"; @@ -466,7 +467,7 @@ await ShouldEventually.Eventually(async () => public async Task ListRightPushAsync_WithShortTTL_ShouldExpire() { // Arrange - await using var redisFacade = CreateRedisFacade(); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); var key = Guid.NewGuid().ToString(); var payload = "test-payload"; @@ -502,7 +503,7 @@ public void IsConnected_WhenNotInitialized_ShouldReturnFalse() public async Task IsConnected_AfterSuccessfulOperation_ShouldReturnTrue() { // Arrange - await using var redisFacade = CreateRedisFacade(); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); // Act - Perform an operation to initialize connection await redisFacade.SetString(Guid.NewGuid().ToString(), "test", TimeSpan.FromMinutes(1), CancellationToken); @@ -515,7 +516,7 @@ public async Task IsConnected_AfterSuccessfulOperation_ShouldReturnTrue() public async Task TotalSubscribers_ShouldTrackActiveSubscriptions() { // Arrange - await using var redisFacade = CreateRedisFacade(); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); var channelName = Guid.NewGuid().ToString(); // Act & Assert - Initially no subscribers @@ -541,7 +542,7 @@ public async Task TotalSubscribers_ShouldTrackActiveSubscriptions() public async Task MultipleSetString_WithDifferentTTLs_ShouldRespectIndividualTTLs() { // Arrange - await using var redisFacade = CreateRedisFacade(); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); var key1 = Guid.NewGuid().ToString(); var key2 = Guid.NewGuid().ToString(); var value1 = "value1"; @@ -567,7 +568,7 @@ await ShouldEventually.Eventually(async () => public async Task DisposeAsync_ShouldCleanupResourcesAndNotThrow() { // Arrange - var redisFacade = CreateRedisFacade(); + var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); // Perform some operations to initialize resources await redisFacade.SetString(Guid.NewGuid().ToString(), "test", TimeSpan.FromMinutes(1), CancellationToken); @@ -582,7 +583,7 @@ public async Task DisposeAsync_ShouldCleanupResourcesAndNotThrow() public async Task DisposeAsync_CalledMultipleTimes_ShouldNotThrow() { // Arrange - var redisFacade = CreateRedisFacade(); + var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); await redisFacade.SetString(Guid.NewGuid().ToString(), "test", TimeSpan.FromMinutes(1), CancellationToken); // Act & Assert - Multiple dispose calls should not throw diff --git a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs index 99b9c2726..8728b2624 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs @@ -50,7 +50,8 @@ public async Task DequeueAsync_ShouldReturnRequestFromRedis() // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid()); var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); - var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); + var redisTransport = new HalibutRedisTransport(redisFacade); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); @@ -78,7 +79,8 @@ public async Task WhenThePickupTimeoutExpires_AnErrorsReturned_AndTheRequestCanN // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid()); var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); - var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); + var redisTransport = new HalibutRedisTransport(redisFacade); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); @@ -107,7 +109,8 @@ public async Task FullSendAndReceiveShouldWork() // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); - var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); + var redisTransport = new HalibutRedisTransport(redisFacade); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); @@ -141,7 +144,8 @@ public async Task WhenReadingTheResponseFromTheQueueFails_TheQueueAndWaitTaskRet // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid()); var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); - var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); + var redisTransport = new HalibutRedisTransport(redisFacade); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore) @@ -235,7 +239,7 @@ public async Task WhenPreparingRequestFails_ARetryableExceptionIsThrown() var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); - await using var redisFacade = CreateRedisFacade(); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); var redisTransport = new HalibutRedisTransport(redisFacade); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); @@ -258,7 +262,8 @@ public async Task WhenDataLostIsDetected_InFlightRequestShouldBeAbandoned_AndARe // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); - var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); + var redisTransport = new HalibutRedisTransport(redisFacade); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); @@ -302,7 +307,8 @@ public async Task OnceARequestIsComplete_NoInflightDisposableShouldExist() // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); - var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); + var redisTransport = new HalibutRedisTransport(redisFacade); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); @@ -334,7 +340,8 @@ public async Task OnceARequestIsComplete_NoSenderHeartBeatsShouldBeSent() // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); - var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); + var redisTransport = new HalibutRedisTransport(redisFacade); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); @@ -378,7 +385,8 @@ public async Task OnceARequestIsComplete_NoReceiverHeartBeatsShouldBeSent() // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); - var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); + var redisTransport = new HalibutRedisTransport(redisFacade); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); @@ -422,7 +430,8 @@ public async Task FullSendAndReceiveWithDataStreamShouldWork() var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); var senderLog = new TestContextLogCreator("QueueSender", LogLevel.Trace).CreateNewForPrefix(""); var receiverLog = new TestContextLogCreator("ReceiverLog", LogLevel.Trace).CreateNewForPrefix(""); - var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); + var redisTransport = new HalibutRedisTransport(redisFacade); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); @@ -511,7 +520,7 @@ public async Task WhenTheReceiverDoesntCollectWorkImmediately_TheRequestCanSitOn // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); - await using var redisFacade = CreateRedisFacade(); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); @@ -781,7 +790,8 @@ static Exception CreateExceptionFromResponse(ResponseMessage responseThatWouldNo [LatestClientAndLatestServiceTestCases(testNetworkConditions: false, testListening: false, testWebSocket: false)] public async Task WhenUsingTheRedisQueue_ASimpleEchoServiceCanBeCalled(ClientAndServiceTestCase clientAndServiceTestCase) { - var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); + var redisTransport = new HalibutRedisTransport(redisFacade); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); await using (var clientAndService = await clientAndServiceTestCase.CreateTestCaseBuilder() @@ -817,7 +827,8 @@ public async Task CancellingARequestShouldResultInTheDequeuedResponseTokenBeingC // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); - var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); + var redisTransport = new HalibutRedisTransport(redisFacade); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); diff --git a/source/Halibut.Tests/Queue/Redis/RedisQueuePerformanceTest.cs b/source/Halibut.Tests/Queue/Redis/RedisQueuePerformanceTest.cs index b0e71beab..6f79589c2 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisQueuePerformanceTest.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisQueuePerformanceTest.cs @@ -41,7 +41,8 @@ public async Task When100kTentaclesAreSubscribed_TheQueueStillWorks() // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid()); var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); - var redisTransport = new HalibutRedisTransport(CreateRedisFacade()); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); + var redisTransport = new HalibutRedisTransport(redisFacade); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); @@ -66,8 +67,7 @@ public async Task When100kTentaclesAreSubscribed_TheQueueStillWorks() for (int i = 0; i < 10; i++) { var request = new RequestMessageBuilder(endpoint.ToString()).Build(); - - await using var sut = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(CreateRedisFacade()), messageReaderWriter, halibutTimeoutsAndLimits); + await using var sut = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, halibutTimeoutsAndLimits); var resultTask = sut.DequeueAsync(CancellationToken); diff --git a/source/Halibut.Tests/Queue/Redis/Utils/RedisFacadeBuilder.cs b/source/Halibut.Tests/Queue/Redis/Utils/RedisFacadeBuilder.cs new file mode 100644 index 000000000..64e6f1ec3 --- /dev/null +++ b/source/Halibut.Tests/Queue/Redis/Utils/RedisFacadeBuilder.cs @@ -0,0 +1,31 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; +using Halibut.Logging; +using Halibut.Queue.Redis; +using Halibut.Tests.Support.Logging; +using Halibut.Tests.TestSetup.Redis; + +namespace Halibut.Tests.Queue.Redis.Utils +{ + public class RedisFacadeBuilder + { + public static RedisFacade CreateRedisFacade(int? port = 0, Guid? guid = null) + { + port = port == 0 ? RedisPort.Port() : port; + return new RedisFacade("localhost:" + port, (guid ?? Guid.NewGuid()).ToString(), new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix("")); + } + } +} \ No newline at end of file diff --git a/source/Halibut.Tests/Queue/Redis/WatchForRedisLosingAllItsDataFixture.cs b/source/Halibut.Tests/Queue/Redis/WatchForRedisLosingAllItsDataFixture.cs index 35e61bc37..49d27d863 100644 --- a/source/Halibut.Tests/Queue/Redis/WatchForRedisLosingAllItsDataFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/WatchForRedisLosingAllItsDataFixture.cs @@ -67,7 +67,7 @@ public async Task WatchForARealRedisLosingAllOfItsData_TimesOutWhenWaitingForCTW [Test] public async Task WhenRedisRunsForLongerThanTheKeyTTL_NoDataLoseShouldBeDetected() { - await using var redisFacade = CreateRedisFacade(); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); await using var watcher = new WatchForRedisLosingAllItsData(redisFacade, HalibutLog, watchInterval:TimeSpan.FromMilliseconds(100), keyTTL: TimeSpan.FromSeconds(2)); var watcherCt = await watcher.GetTokenForDataLoseDetection(TimeSpan.FromSeconds(20), CancellationToken); diff --git a/source/Halibut.Tests/TestSetup/Redis/EnsureRedisIsAvailableSetupFixture.cs b/source/Halibut.Tests/TestSetup/Redis/EnsureRedisIsAvailableSetupFixture.cs index 4cbd3061c..2c823e11a 100644 --- a/source/Halibut.Tests/TestSetup/Redis/EnsureRedisIsAvailableSetupFixture.cs +++ b/source/Halibut.Tests/TestSetup/Redis/EnsureRedisIsAvailableSetupFixture.cs @@ -27,7 +27,7 @@ public void OneTimeSetUp(ILogger logger) { #if NETFRAMEWORK return; -#endif +#else var isWindows = RuntimeInformation.IsOSPlatform(OSPlatform.Windows); bool shouldCreateRedis = !isWindows && TeamCityDetection.IsRunningInTeamCity(); @@ -56,6 +56,7 @@ public void OneTimeSetUp(ILogger logger) RedisPort.SetPort(redisContainer.container!.RedisPort); logger.Information("RedisPort is: {RedisPort}", RedisPort.Port()); } +#endif } public void OneTimeTearDown(ILogger logger) From 606fcaf7bf4bfcd82b0475c9eba5f827cfcc4dbe Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Sun, 17 Aug 2025 06:34:52 +1000 Subject: [PATCH 059/137] Add redis test attribute --- source/Halibut.Tests/Halibut.Tests.csproj | 2 +- .../Halibut.Tests/ManyPollingTentacleTests.cs | 4 +- .../Queue/Redis/NodeHeartBeatSenderFixture.cs | 26 +- .../Queue/Redis/RedisFacadeFixture.cs | 30 +- .../RedisFacadeWhenRedisGoesDownAwayTests.cs | 151 +++++---- .../Redis/RedisPendingRequestQueueFixture.cs | 295 ++++++++---------- .../Queue/Redis/RedisQueuePerformanceTest.cs | 6 +- .../Queue/Redis/Utils/RedisFacadeBuilder.cs | 4 +- .../WatchForRedisLosingAllItsDataFixture.cs | 16 +- .../TestAttributes/RedisTestAttribute.cs | 23 ++ .../EnsureRedisIsAvailableSetupFixture.cs | 35 ++- source/Halibut/Queue/Redis/RedisFacade.cs | 4 +- 12 files changed, 278 insertions(+), 318 deletions(-) create mode 100644 source/Halibut.Tests/Support/TestAttributes/RedisTestAttribute.cs diff --git a/source/Halibut.Tests/Halibut.Tests.csproj b/source/Halibut.Tests/Halibut.Tests.csproj index 8a9df14f2..9eb552ceb 100644 --- a/source/Halibut.Tests/Halibut.Tests.csproj +++ b/source/Halibut.Tests/Halibut.Tests.csproj @@ -7,7 +7,7 @@ true false 9.0 - VSTHRD002 + VSTHRD002,VSTHRD003 enable true diff --git a/source/Halibut.Tests/ManyPollingTentacleTests.cs b/source/Halibut.Tests/ManyPollingTentacleTests.cs index 0e03f44a1..495c40f2f 100644 --- a/source/Halibut.Tests/ManyPollingTentacleTests.cs +++ b/source/Halibut.Tests/ManyPollingTentacleTests.cs @@ -25,6 +25,7 @@ using Halibut.Queue.QueuedDataStreams; using Halibut.Queue.Redis; using Halibut.ServiceModel; +using Halibut.Tests.Queue.Redis.Utils; using Halibut.Tests.Support; using Halibut.Tests.Support.Logging; using Halibut.Tests.Support.TestAttributes; @@ -37,6 +38,7 @@ namespace Halibut.Tests { [NonParallelizable] + [RedisTest] public class ManyPollingTentacleTests : BaseTest { [Test] @@ -49,7 +51,7 @@ public async Task ManyRequestToPollingTentacles_Works_AndDoesNotUseTooManyResour await using var disposables = new DisposableCollection(); var isRedis = queueTestCase.ToString().ToLower().Contains("redis"); var log = new TestContextLogCreator("Redis", LogLevel.Fatal); - await using var redisFacade = new RedisFacade("localhost:6379", Guid.NewGuid().ToString(), log.CreateNewForPrefix("")); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); await using (var octopus = new HalibutRuntimeBuilder() .WithServerCertificate(Certificates.Octopus) .WithPendingRequestQueueFactory(msgSer => diff --git a/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs b/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs index 1038328ea..3af8cd306 100644 --- a/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs @@ -1,7 +1,6 @@ #if NET8_0_OR_GREATER using System; using System.Collections.Concurrent; -using System.Diagnostics.CodeAnalysis; using System.Linq; using System.Threading; using System.Threading.Tasks; @@ -18,16 +17,9 @@ namespace Halibut.Tests.Queue.Redis { - [Ignore("REDISTODO")] - [SuppressMessage("Usage", "VSTHRD003:Avoid awaiting foreign Tasks")] + [RedisTest] public class NodeHeartBeatSenderFixture : BaseTest { - private static RedisFacade CreateRedisFacade(int? port = 0, Guid? guid = null) - { - port = port == 0 ? RedisPort.Port() : port; - return new RedisFacade("localhost:" + port, (guid ?? Guid.NewGuid()).ToString(), new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix("")); - } - // TODO: ai tests need review [Test] public async Task WhenCreated_ShouldStartSendingHeartbeats() @@ -70,8 +62,8 @@ public async Task WhenRedisConnectionIsInterrupted_ShouldSwitchToPanicMode() var guid = Guid.NewGuid(); using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); - await using var unstableRedisFacade = CreateRedisFacade(portForwarder.ListeningPort, guid); - await using var stableRedisFacade = CreateRedisFacade(RedisPort.Port(), guid); + await using var unstableRedisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder.ListeningPort, guid); + await using var stableRedisFacade = RedisFacadeBuilder.CreateRedisFacade(RedisPort.Port(), guid); var redisTransport = new HalibutRedisTransport(unstableRedisFacade); @@ -162,8 +154,8 @@ public async Task WaitUntilNodeProcessingRequestFlatLines_WhenHeartbeatsStop_Sho var guid = Guid.NewGuid(); using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); - await using var unstableRedisFacade = CreateRedisFacade(portForwarder.ListeningPort, guid); - await using var stableRedisFacade = CreateRedisFacade(RedisPort.Port(), guid); + await using var unstableRedisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder.ListeningPort, guid); + await using var stableRedisFacade = RedisFacadeBuilder.CreateRedisFacade(RedisPort.Port(), guid); var unstableRedisTransport = new HalibutRedisTransport(unstableRedisFacade); var stableRedisTransport = new HalibutRedisTransport(stableRedisFacade); @@ -249,8 +241,8 @@ public async Task WaitUntilNodeProcessingRequestFlatLines_WhenConnectionInterrup var guid = Guid.NewGuid(); using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); - await using var unstableRedisFacade = CreateRedisFacade(portForwarder.ListeningPort, guid); - await using var stableRedisFacade = CreateRedisFacade(RedisPort.Port(), guid); + await using var unstableRedisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder.ListeningPort, guid); + await using var stableRedisFacade = RedisFacadeBuilder.CreateRedisFacade(RedisPort.Port(), guid); var unstableRedisTransport = new HalibutRedisTransport(unstableRedisFacade); var stableRedisTransport = new HalibutRedisTransport(stableRedisFacade); @@ -330,8 +322,8 @@ public async Task WhenHeartBeatSenderConnectionRecovery_ShouldResumeNormalHeartb var guid = Guid.NewGuid(); using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); - await using var unstableRedisFacade = CreateRedisFacade(portForwarder.ListeningPort, guid); - await using var stableRedisFacade = CreateRedisFacade(RedisPort.Port(), guid); + await using var unstableRedisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder.ListeningPort, guid); + await using var stableRedisFacade = RedisFacadeBuilder.CreateRedisFacade(RedisPort.Port(), guid); var unstableRedisTransport = new HalibutRedisTransport(unstableRedisFacade); var stableRedisTransport = new HalibutRedisTransport(stableRedisFacade); diff --git a/source/Halibut.Tests/Queue/Redis/RedisFacadeFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisFacadeFixture.cs index d3dd1772b..fcadd70ea 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisFacadeFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisFacadeFixture.cs @@ -8,35 +8,15 @@ using Halibut.Tests.Queue.Redis.Utils; using Halibut.Tests.Support; using Halibut.Tests.Support.Logging; -using Halibut.Tests.TestSetup.Redis; using Halibut.Util.AsyncEx; using NUnit.Framework; namespace Halibut.Tests.Queue.Redis { + [RedisTest] public class RedisFacadeFixture : BaseTest { - [Test] - public void Constructor_WithRedisHostAndKeyPrefix_ShouldCreateInstance() - { - // Arrange & Act - var redisFacade = new RedisFacade("localhost", "test-prefix", new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix("")); - - // Assert - redisFacade.Should().NotBeNull(); - } - - [Test] - public void Constructor_WithNullKeyPrefix_ShouldUseDefaultPrefix() - { - // Arrange & Act - var redisFacade = new RedisFacade("localhost", null, new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix("")); - - // Assert - redisFacade.Should().NotBeNull(); - } - [Test] public async Task SetString_AndGetString_ShouldStoreAndRetrieveValue() { @@ -320,11 +300,11 @@ public async Task SubscribeToChannel_WhenDisposed_ShouldUnsubscribe() public async Task KeyPrefixing_ShouldIsolateDataBetweenDifferentPrefixes() { // Arrange - var prefix1 = Guid.NewGuid().ToString(); - var prefix2 = Guid.NewGuid().ToString(); + var prefix1 = Guid.NewGuid(); + var prefix2 = Guid.NewGuid(); - await using var redisFacade1 = new RedisFacade("localhost", prefix1, new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix("")); - await using var redisFacade2 = new RedisFacade("localhost", prefix2, new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix("")); + await using var redisFacade1 = RedisFacadeBuilder.CreateRedisFacade(prefix: prefix1); + await using var redisFacade2 = RedisFacadeBuilder.CreateRedisFacade(prefix: prefix2); var key = "shared-key"; var value1 = "value-from-facade1"; diff --git a/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs b/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs index d55cea1c9..c74b7e610 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs @@ -11,87 +11,85 @@ // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. + #if NET8_0_OR_GREATER using System; using System.Collections.Concurrent; -using System.Collections.Generic; using System.Diagnostics.CodeAnalysis; -using System.Globalization; using System.Threading.Tasks; using FluentAssertions; -using Halibut.Diagnostics; using Halibut.Logging; using Halibut.Queue.Redis; using Halibut.Tests.Support.Logging; +using Halibut.Tests.TestSetup.Redis; using Halibut.Util; using NUnit.Framework; using Octopus.TestPortForwarder; -using StackExchange.Redis; namespace Halibut.Tests.Queue.Redis { - [Ignore("REDISTODO")] + [RedisTest] [SuppressMessage("Usage", "VSTHRD003:Avoid awaiting foreign Tasks")] public class RedisFacadeWhenRedisGoesDownAwayTests : BaseTest { - private static RedisFacade CreateRedisFacade(int port) => new("localhost:" + port, Guid.NewGuid().ToString(), new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix("")); + static RedisFacade CreateRedisFacade(int? port = 0, Guid? guid = null) + { + port = port == 0 ? RedisPort.Port() : port; + return new RedisFacade("localhost:" + port, (guid ?? Guid.NewGuid()).ToString(), new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix("")); + } - - int redisPort = 6379; - [Test] public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_SometimeLaterOnWeCanDoBasicCalls() { - - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); - + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); await redisFacade.SetString("foo", "bar", TimeSpan.FromMinutes(1), CancellationToken); (await redisFacade.GetString("foo", CancellationToken)).Should().Be("bar"); - + portForwarder.EnterKillNewAndExistingConnectionsMode(); portForwarder.ReturnToNormalMode(); // After a short delay it does seem to work again. await Task.Delay(1000); - + await redisFacade.GetString("foo", CancellationToken); } - + [Test] public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelyDoBasicCalls() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); - + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); await redisFacade.SetString("foo", "bar", TimeSpan.FromMinutes(1), CancellationToken); (await redisFacade.GetString("foo", CancellationToken)).Should().Be("bar"); - + portForwarder.EnterKillNewAndExistingConnectionsMode(); portForwarder.ReturnToNormalMode(); - + // No delay here - + await redisFacade.GetString("foo", CancellationToken); } [Test] public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelyPublishToChannel() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); - + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); // Establish connection first await redisFacade.SetString("connection", "established", TimeSpan.FromMinutes(1), CancellationToken); - + portForwarder.EnterKillNewAndExistingConnectionsMode(); portForwarder.ReturnToNormalMode(); - + // No delay here - should retry and succeed await redisFacade.PublishToChannel("test-channel", "test-message", CancellationToken); } @@ -99,16 +97,16 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe [Test] public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelySetInHash() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); - + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); // Establish connection first await redisFacade.SetString("connection", "established", TimeSpan.FromMinutes(1), CancellationToken); - + portForwarder.EnterKillNewAndExistingConnectionsMode(); portForwarder.ReturnToNormalMode(); - + // No delay here - should retry and succeed await redisFacade.SetInHash("test-hash", "test-field", "test-value", TimeSpan.FromMinutes(1), CancellationToken); } @@ -116,16 +114,16 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe [Test] public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelyTryGetAndDeleteFromHash() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); - + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); // Establish connection and set up test data await redisFacade.SetInHash("test-hash", "test-field", "test-value", TimeSpan.FromMinutes(1), CancellationToken); - + portForwarder.EnterKillNewAndExistingConnectionsMode(); portForwarder.ReturnToNormalMode(); - + // No delay here - should retry and succeed var result = await redisFacade.TryGetAndDeleteFromHash("test-hash", "test-field", CancellationToken); result.Should().Be("test-value"); @@ -134,33 +132,33 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe [Test] public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelyListRightPush() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); - + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); // Establish connection first await redisFacade.SetString("connection", "established", TimeSpan.FromMinutes(1), CancellationToken); - + portForwarder.EnterKillNewAndExistingConnectionsMode(); portForwarder.ReturnToNormalMode(); - + // No delay here - should retry and succeed - await redisFacade.ListRightPushAsync("test-list", "test-item", TimeSpan.FromMinutes(1), CancellationToken ); + await redisFacade.ListRightPushAsync("test-list", "test-item", TimeSpan.FromMinutes(1), CancellationToken); } [Test] public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelyListLeftPop() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); - + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); // Establish connection and set up test data await redisFacade.ListRightPushAsync("test-list", "test-item", TimeSpan.FromMinutes(1), CancellationToken); - + portForwarder.EnterKillNewAndExistingConnectionsMode(); portForwarder.ReturnToNormalMode(); - + // No delay here - should retry and succeed var result = await redisFacade.ListLeftPopAsync("test-list", CancellationToken); result.Should().Be("test-item"); @@ -169,16 +167,16 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe [Test] public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelySetString() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); - + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); // Establish connection first await redisFacade.SetString("connection", "established", TimeSpan.FromMinutes(1), CancellationToken); - + portForwarder.EnterKillNewAndExistingConnectionsMode(); portForwarder.ReturnToNormalMode(); - + // No delay here - should retry and succeed await redisFacade.SetString("test-key", "test-value", TimeSpan.FromMinutes(1), CancellationToken); } @@ -186,16 +184,16 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe [Test] public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelyGetString() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); - + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); // Establish connection and set up test data await redisFacade.SetString("test-key", "test-value", TimeSpan.FromMinutes(1), CancellationToken); - + portForwarder.EnterKillNewAndExistingConnectionsMode(); portForwarder.ReturnToNormalMode(); - + // No delay here - should retry and succeed var result = await redisFacade.GetString("test-key", CancellationToken); result.Should().Be("test-value"); @@ -204,16 +202,16 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe [Test] public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelyHashContainsKey() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); - + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); // Establish connection and set up test data await redisFacade.SetInHash("test-hash", "test-field", "test-value", TimeSpan.FromMinutes(1), CancellationToken); - + portForwarder.EnterKillNewAndExistingConnectionsMode(); portForwarder.ReturnToNormalMode(); - + // No delay here - should retry and succeed var exists = await redisFacade.HashContainsKey("test-hash", "test-field", CancellationToken); exists.Should().BeTrue(); @@ -222,31 +220,30 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe [Test] public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndWeTryToSubscribe_WhenTheConnectionIsRestored_WeCanReceiveMessages() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); var redisLogCreator = new TestContextLogCreator("Redis", LogLevel.Trace); var guid = Guid.NewGuid().ToString(); await using var redisViaPortForwarder = new RedisFacade("localhost:" + portForwarder.ListeningPort, guid, redisLogCreator.CreateNewForPrefix("Unstable")); - await using var redisStableConnection = new RedisFacade("localhost:" + redisPort, guid, redisLogCreator.CreateNewForPrefix("Stable")); + await using var redisStableConnection = new RedisFacade("localhost:" + RedisPort.Port(), guid, redisLogCreator.CreateNewForPrefix("Stable")); await redisViaPortForwarder.SetString("Establish connection", "before we subscribe", TimeSpan.FromMinutes(1), CancellationToken); - + portForwarder.EnterKillNewAndExistingConnectionsMode(); - + var msgs = new ConcurrentBag(); var subscribeToChannelTask = redisViaPortForwarder.SubscribeToChannel("bob", async message => { await Task.CompletedTask; msgs.Add(message.Message!); }, CancellationToken); - // Give everything enough time to have a crack at trying to subscribe to messages. await Task.Delay(2000); await redisStableConnection.PublishToChannel("bob", "MISSED", CancellationToken); - + // Just in case the subscriber reconnects faster than the publish call. await Task.Delay(2000); - + portForwarder.ReturnToNormalMode(); // Keep going around the loop until we recieve something @@ -261,34 +258,34 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndWeTryT msgs.Should().NotContain("MISSED", "Since this was sent when the subscriber could not have been connected. " + "If this is seen maybe the test itself has a bug."); } - + [Test] public async Task WhenTheConnectionIsNeverEstablished_AndWeTryToSubscribe_WhenTheConnectionIsRestored_WeCanReceiveMessages() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); var redisLogCreator = new TestContextLogCreator("Redis", LogLevel.Trace); var guid = Guid.NewGuid().ToString(); await using var redisViaPortForwarder = new RedisFacade("localhost:" + portForwarder.ListeningPort, guid, redisLogCreator.CreateNewForPrefix("Unstable")); - await using var redisStableConnection = new RedisFacade("localhost:" + redisPort, guid, redisLogCreator.CreateNewForPrefix("Stable")); + await using var redisStableConnection = new RedisFacade("localhost:" + RedisPort.Port(), guid, redisLogCreator.CreateNewForPrefix("Stable")); portForwarder.EnterKillNewAndExistingConnectionsMode(); - + var msgs = new ConcurrentBag(); var subscribeToChannelTask = redisViaPortForwarder.SubscribeToChannel("bob", async message => { await Task.CompletedTask; msgs.Add(message.Message!); }, CancellationToken); - + await using var _ = new FuncAsyncDisposable(() => Try.IgnoringError(async () => await (await subscribeToChannelTask).DisposeAsync())); // Give everything enough time to have a crack at trying to subscribe to messages. await Task.Delay(2000); await redisStableConnection.PublishToChannel("bob", "MISSED", CancellationToken); - + // Just in case the subscriber reconnects faster than the publish call. await Task.Delay(2000); - + portForwarder.ReturnToNormalMode(); // Keep going around the loop until we recieve something @@ -303,32 +300,32 @@ public async Task WhenTheConnectionIsNeverEstablished_AndWeTryToSubscribe_WhenTh msgs.Should().NotContain("MISSED", "Since this was sent when the subscriber could not have been connected. " + "If this is seen maybe the test itself has a bug."); } - - [Test] + + [WindowsTest] public async Task WhenSubscribedAndTheConnectionGoesDown_WhenTheConnectionIsRestored_MessagesCanEventuallyBeSentToTheSubscriberAgain() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); var redisLogCreator = new TestContextLogCreator("Redis", LogLevel.Trace); - + var guid = Guid.NewGuid().ToString(); await using var redisViaPortForwarder = new RedisFacade("localhost:" + portForwarder.ListeningPort, guid, redisLogCreator.CreateNewForPrefix("Unstable")); - - await using var redisStableConnection = new RedisFacade("localhost:" + redisPort, guid, redisLogCreator.CreateNewForPrefix("Stable")); + + await using var redisStableConnection = new RedisFacade("localhost:" + RedisPort.Port(), guid, redisLogCreator.CreateNewForPrefix("Stable")); var msgs = new ConcurrentBag(); await using var channel = await redisViaPortForwarder.SubscribeToChannel("bob", async message => { await Task.CompletedTask; msgs.Add(message.Message!); - }, CancellationToken); - + }, CancellationToken); + // Check both sides can publish. await redisViaPortForwarder.PublishToChannel("bob", "hello unstable", CancellationToken); await redisStableConnection.PublishToChannel("bob", "hello stable", CancellationToken); await Task.Delay(1000); // TODO better msgs.Should().BeEquivalentTo("hello unstable", "hello stable"); - + portForwarder.EnterKillNewAndExistingConnectionsMode(); // The stable connection should still be able to publish to redis. // But the subscriber on the unstable connection will not got the message. @@ -347,8 +344,6 @@ public async Task WhenSubscribedAndTheConnectionGoesDown_WhenTheConnectionIsRest msgs.Should().NotContain("MISSED", "Since this was sent when the subscriber could not have been connected. " + "If this is seen maybe the test itself has a bug."); } - - } } #endif \ No newline at end of file diff --git a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs index 8728b2624..238317d6a 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs @@ -1,10 +1,5 @@ #if NET8_0_OR_GREATER using System; -using System.Collections.Generic; -using System.Diagnostics; -using System.Globalization; -using System.IO; -using System.IO.Compression; using System.Threading; using System.Threading.Tasks; using FluentAssertions; @@ -22,28 +17,22 @@ using Halibut.Tests.Support.TestAttributes; using Halibut.Tests.Support.TestCases; using Halibut.Tests.TestServices.Async; +using Halibut.Tests.TestSetup.Redis; using Halibut.Tests.Util; using Halibut.TestUtils.Contracts; using Halibut.Transport.Protocol; using Halibut.Util; -using Nito.AsyncEx; using NSubstitute; using NSubstitute.Extensions; using NUnit.Framework; using Octopus.TestPortForwarder; -using Serilog; -using DisposableCollection = Halibut.Util.DisposableCollection; using ILog = Halibut.Diagnostics.ILog; -using Try = Halibut.Tests.Support.Try; namespace Halibut.Tests.Queue.Redis { - [Ignore("REDISTODO")] + [RedisTest] public class RedisPendingRequestQueueFixture : BaseTest { - const int redisPort = 6379; - private static RedisFacade CreateRedisFacade(int? port = redisPort, Guid? guid = null) => new("localhost:" + port, (guid ?? Guid.NewGuid()).ToString(), new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix("")); - [Test] public async Task DequeueAsync_ShouldReturnRequestFromRedis() { @@ -72,7 +61,7 @@ public async Task DequeueAsync_ShouldReturnRequestFromRedis() result.RequestMessage.MethodName.Should().Be(request.MethodName); result.RequestMessage.ServiceName.Should().Be(request.ServiceName); } - + [Test] public async Task WhenThePickupTimeoutExpires_AnErrorsReturned_AndTheRequestCanNotBeCollected() { @@ -102,12 +91,11 @@ public async Task WhenThePickupTimeoutExpires_AnErrorsReturned_AndTheRequestCanN result.Should().BeNull(); } - [Test] public async Task FullSendAndReceiveShouldWork() { // Arrange - var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); + var endpoint = new Uri("poll://" + Guid.NewGuid()); var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); var redisTransport = new HalibutRedisTransport(redisFacade); @@ -137,7 +125,7 @@ public async Task FullSendAndReceiveShouldWork() responseMessage.Result.Should().Be("Yay"); } - + [Test] public async Task WhenReadingTheResponseFromTheQueueFails_TheQueueAndWaitTaskReturnsAnUnknownError() { @@ -159,7 +147,7 @@ public async Task WhenReadingTheResponseFromTheQueueFails_TheQueueAndWaitTaskRet var queueAndWaitAsync = queue.QueueAndWaitAsync(request, CancellationToken.None); var requestMessageWithCancellationToken = await queue.DequeueAsync(CancellationToken); - await queue.ApplyResponse(ResponseMessage.FromResult(requestMessageWithCancellationToken! .RequestMessage, "Yay"), + await queue.ApplyResponse(ResponseMessage.FromResult(requestMessageWithCancellationToken!.RequestMessage, "Yay"), requestMessageWithCancellationToken.RequestMessage.ActivityId); var responseMessage = await queueAndWaitAsync; @@ -167,8 +155,7 @@ await queue.ApplyResponse(ResponseMessage.FromResult(requestMessageWithCancellat CreateExceptionFromResponse(responseMessage, log).IsRetryableError().Should().Be(HalibutRetryableErrorType.IsRetryable); } - - + [Test] public async Task WhenEnteringTheQueue_AndRedisIsUnavailable_ARetryableExceptionIsThrown() { @@ -176,10 +163,10 @@ public async Task WhenEnteringTheQueue_AndRedisIsUnavailable_ARetryableException var endpoint = new Uri("poll://" + Guid.NewGuid()); var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); - await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder.ListeningPort, null); redisFacade.MaxDurationToRetryFor = TimeSpan.FromSeconds(1); - + var redisTransport = new HalibutRedisTransport(redisFacade); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); @@ -188,13 +175,13 @@ public async Task WhenEnteringTheQueue_AndRedisIsUnavailable_ARetryableException var request = new RequestMessageBuilder("poll://test-endpoint").Build(); var queue = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); portForwarder.EnterKillNewAndExistingConnectionsMode(); - + // Act Assert var exception = await AssertThrowsAny.Exception(async () => await queue.QueueAndWaitAsync(request, CancellationToken.None)); exception.IsRetryableError().Should().Be(HalibutRetryableErrorType.IsRetryable); exception.Message.Should().Contain("ailed since an error occured inserting the data into the queue"); } - + [Test] public async Task WhenEnteringTheQueue_AndRedisIsUnavailableAndDataLoseOccurs_ARetryableExceptionIsThrown() { @@ -202,35 +189,34 @@ public async Task WhenEnteringTheQueue_AndRedisIsUnavailableAndDataLoseOccurs_AR var endpoint = new Uri("poll://" + Guid.NewGuid()); var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); - await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder.ListeningPort, null); redisFacade.MaxDurationToRetryFor = TimeSpan.FromSeconds(1); - + var redisDataLoseDetector = new CancellableDataLossWatchForRedisLosingAllItsData(); - + var redisTransport = Substitute.ForPartsOf(new HalibutRedisTransport(redisFacade)); redisTransport.Configure().PutRequest(Arg.Any(), Arg.Any(), Arg.Any(), Arg.Any(), Arg.Any()) - .Returns(async callInfo => + .Returns(async callInfo => { await redisDataLoseDetector.DataLossHasOccured(); throw new OperationCanceledException(); }); - + var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); var request = new RequestMessageBuilder("poll://test-endpoint").Build(); - + var queue = new RedisPendingRequestQueue(endpoint, redisDataLoseDetector, log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); - - + // Act Assert var exception = await AssertThrowsAny.Exception(async () => await queue.QueueAndWaitAsync(request, CancellationToken.None)); exception.IsRetryableError().Should().Be(HalibutRetryableErrorType.IsRetryable); exception.Message.Should().Contain("was cancelled because we detected that redis lost all of its data."); } - + [Test] public async Task WhenPreparingRequestFails_ARetryableExceptionIsThrown() { @@ -238,9 +224,8 @@ public async Task WhenPreparingRequestFails_ARetryableExceptionIsThrown() var endpoint = new Uri("poll://" + Guid.NewGuid()); var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); - await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); - + var redisTransport = new HalibutRedisTransport(redisFacade); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); @@ -249,18 +234,18 @@ public async Task WhenPreparingRequestFails_ARetryableExceptionIsThrown() var request = new RequestMessageBuilder("poll://test-endpoint").Build(); var queue = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); - + // Act Assert var exception = await AssertThrowsAny.Exception(async () => await queue.QueueAndWaitAsync(request, CancellationToken.None)); exception.IsRetryableError().Should().Be(HalibutRetryableErrorType.IsRetryable); exception.Message.Should().Contain("error occured when preparing request for queue"); } - + [Test] public async Task WhenDataLostIsDetected_InFlightRequestShouldBeAbandoned_AndARetryableExceptionIsThrown() { // Arrange - var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); + var endpoint = new Uri("poll://" + Guid.NewGuid()); var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); var redisTransport = new HalibutRedisTransport(redisFacade); @@ -298,14 +283,13 @@ public async Task WhenDataLostIsDetected_InFlightRequestShouldBeAbandoned_AndARe var e = await AssertException.Throws(queueAndWaitAsync); e.And.IsRetryableError().Should().Be(HalibutRetryableErrorType.IsRetryable); e.And.Should().BeOfType(); - } - + [Test] public async Task OnceARequestIsComplete_NoInflightDisposableShouldExist() { // Arrange - var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); + var endpoint = new Uri("poll://" + Guid.NewGuid()); var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); var redisTransport = new HalibutRedisTransport(redisFacade); @@ -329,16 +313,16 @@ public async Task OnceARequestIsComplete_NoInflightDisposableShouldExist() var responseMessage = await queueAndWaitAsync; responseMessage.Result.Should().Be("Yay"); - + // Assert queue.DisposablesForInFlightRequests.Should().BeEmpty(); } - + [Test] public async Task OnceARequestIsComplete_NoSenderHeartBeatsShouldBeSent() { // Arrange - var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); + var endpoint = new Uri("poll://" + Guid.NewGuid()); var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); var redisTransport = new HalibutRedisTransport(redisFacade); @@ -351,7 +335,7 @@ public async Task OnceARequestIsComplete_NoSenderHeartBeatsShouldBeSent() var queue = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); await queue.WaitUntilQueueIsSubscribedToReceiveMessages(); queue.DelayBetweenHeartBeatsForRequestSender = TimeSpan.FromSeconds(1); - + // Act var queueAndWaitAsync = queue.QueueAndWaitAsync(request, CancellationToken.None); @@ -360,30 +344,29 @@ public async Task OnceARequestIsComplete_NoSenderHeartBeatsShouldBeSent() var response = ResponseMessage.FromResult(requestMessageWithCancellationToken!.RequestMessage, "Yay"); await queue.ApplyResponse(response, requestMessageWithCancellationToken.RequestMessage.ActivityId); - var responseMessage = await queueAndWaitAsync; responseMessage.Result.Should().Be("Yay"); - + // Assert - bool heartBeatSent = false; + var heartBeatSent = false; var cts = new CancelOnDisposeCancellationToken(); using var _ = redisTransport.SubscribeToNodeHeartBeatChannel(endpoint, request.ActivityId, HalibutQueueNodeSendingPulses.Sender, async () => - { - await Task.CompletedTask; - heartBeatSent = true; - }, + { + await Task.CompletedTask; + heartBeatSent = true; + }, cts.Token); - + await Task.Delay(5000); heartBeatSent.Should().BeFalse(); } - + [Test] public async Task OnceARequestIsComplete_NoReceiverHeartBeatsShouldBeSent() { // Arrange - var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); + var endpoint = new Uri("poll://" + Guid.NewGuid()); var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); var redisTransport = new HalibutRedisTransport(redisFacade); @@ -408,17 +391,17 @@ public async Task OnceARequestIsComplete_NoReceiverHeartBeatsShouldBeSent() var responseMessage = await queueAndWaitAsync; responseMessage.Result.Should().Be("Yay"); - + // Assert - bool heartBeatSent = false; + var heartBeatSent = false; var cts = new CancelOnDisposeCancellationToken(); using var _ = redisTransport.SubscribeToNodeHeartBeatChannel(endpoint, request.ActivityId, HalibutQueueNodeSendingPulses.Receiver, async () => - { - await Task.CompletedTask; - heartBeatSent = true; - }, + { + await Task.CompletedTask; + heartBeatSent = true; + }, cts.Token); - + await Task.Delay(5000); heartBeatSent.Should().BeFalse(); } @@ -427,7 +410,7 @@ public async Task OnceARequestIsComplete_NoReceiverHeartBeatsShouldBeSent() public async Task FullSendAndReceiveWithDataStreamShouldWork() { // Arrange - var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); + var endpoint = new Uri("poll://" + Guid.NewGuid()); var senderLog = new TestContextLogCreator("QueueSender", LogLevel.Trace).CreateNewForPrefix(""); var receiverLog = new TestContextLogCreator("ReceiverLog", LogLevel.Trace).CreateNewForPrefix(""); await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); @@ -462,20 +445,19 @@ public async Task FullSendAndReceiveWithDataStreamShouldWork() (await returnObject.Payload1!.ReadAsString(CancellationToken)).Should().Be("good"); (await returnObject.Payload2!.ReadAsString(CancellationToken)).Should().Be("bye"); } - - + [Test] public async Task WhenTheReceiversConnectionToRedisIsInterruptedAndRestoredBeforeWorkIsPublished_TheReceiverShouldBeAbleToCollectThatWorkQuickly() { // Arrange - var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); + var endpoint = new Uri("poll://" + Guid.NewGuid()); var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); var guid = Guid.NewGuid(); - await using var redisFacadeSender = CreateRedisFacade(guid: guid); + await using var redisFacadeSender = RedisFacadeBuilder.CreateRedisFacade(prefix: guid); + + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + await using var redisFacadeReceiver = RedisFacadeBuilder.CreateRedisFacade(portForwarder.ListeningPort, guid); - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); - await using var redisFacadeReceiver = CreateRedisFacade(portForwarder.ListeningPort, guid: guid); - var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); @@ -484,24 +466,24 @@ public async Task WhenTheReceiversConnectionToRedisIsInterruptedAndRestoredBefor var halibutTimeoutAndLimits = new HalibutTimeoutsAndLimits(); halibutTimeoutAndLimits.PollingQueueWaitTimeout = TimeSpan.FromDays(1); // We should not need to rely on the timeout working for very short disconnects. - + var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(redisFacadeSender), messageReaderWriter, halibutTimeoutAndLimits); var node2Receiver = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(redisFacadeReceiver), messageReaderWriter, halibutTimeoutAndLimits); await node2Receiver.WaitUntilQueueIsSubscribedToReceiveMessages(); var dequeueTask = node2Receiver.DequeueAsync(CancellationToken); - + await Task.Delay(5000, CancellationToken); // Allow some time for the receiver to subscribe to work. dequeueTask.IsCompleted.Should().BeFalse("Dequeue should not have "); - + portForwarder.EnterKillNewAndExistingConnectionsMode(); await Task.Delay(1000, CancellationToken); // The network outage continues! - + portForwarder.ReturnToNormalMode(); // The network outage gets all fixed up :D Logger.Information("Network restored!"); - + // The receiver should be able to get itself back into a state where it can collect messages quickly, within this time. await Task.Delay(TimeSpan.FromSeconds(30), CancellationToken); - + var queueAndWaitAsync = node1Sender.QueueAndWaitAsync(request, CancellationToken.None); // Surely it will be done in 25s, it should take less than 1s. @@ -513,15 +495,15 @@ public async Task WhenTheReceiversConnectionToRedisIsInterruptedAndRestoredBefor requestReceived.Should().NotBeNull(); requestReceived!.RequestMessage.ActivityId.Should().Be(request.ActivityId); } - + [Test] public async Task WhenTheReceiverDoesntCollectWorkImmediately_TheRequestCanSitOnTheQueueForSometime_AndBeOnTheQueueLongerThanTheHeartBeatTimeout() { // Arrange - var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); + var endpoint = new Uri("poll://" + Guid.NewGuid()); var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); - + var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); @@ -530,13 +512,13 @@ public async Task WhenTheReceiverDoesntCollectWorkImmediately_TheRequestCanSitOn // We are testing that we don't expect heart beats before the request is collected. node1Sender.RequestReceivingNodeIsOfflineHeartBeatTimeout = TimeSpan.FromSeconds(1); await node1Sender.WaitUntilQueueIsSubscribedToReceiveMessages(); - + var request = new RequestMessageBuilder("poll://test-endpoint").Build(); request.Destination.PollingRequestQueueTimeout = TimeSpan.FromHours(1); await using var cts = new CancelOnDisposeCancellationToken(CancellationToken); - + var queueAndWaitAsync = node1Sender.QueueAndWaitAsync(request, cts.Token); - + await Task.WhenAny(Task.Delay(TimeSpan.FromSeconds(5), CancellationToken), queueAndWaitAsync); queueAndWaitAsync.IsCompleted.Should().BeFalse(); @@ -546,14 +528,14 @@ public async Task WhenTheReceiverDoesntCollectWorkImmediately_TheRequestCanSitOn public async Task WhenTheSendersConnectionToRedisIsBrieflyInterruptedWhileSendingWork_TheWorkIsStillSent() { // Arrange - var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); + var endpoint = new Uri("poll://" + Guid.NewGuid()); var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); var guid = Guid.NewGuid(); - await using var redisFacadeReceiver = CreateRedisFacade(guid: guid); + await using var redisFacadeReceiver = RedisFacadeBuilder.CreateRedisFacade(prefix: guid); + + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + await using var redisFacadeSender = RedisFacadeBuilder.CreateRedisFacade(portForwarder.ListeningPort, guid); - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); - await using var redisFacadeSender = CreateRedisFacade(portForwarder.ListeningPort, guid: guid); - var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); @@ -563,37 +545,34 @@ public async Task WhenTheSendersConnectionToRedisIsBrieflyInterruptedWhileSendin var halibutTimeoutAndLimits = new HalibutTimeoutsAndLimits(); halibutTimeoutAndLimits.PollingQueueWaitTimeout = TimeSpan.FromDays(1); // We should not need to rely on the timeout working for very short disconnects. - var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(redisFacadeSender), messageReaderWriter, halibutTimeoutAndLimits); var node2Receiver = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(redisFacadeSender), messageReaderWriter, halibutTimeoutAndLimits); await node2Receiver.WaitUntilQueueIsSubscribedToReceiveMessages(); - + portForwarder.EnterKillNewAndExistingConnectionsMode(); var networkRestoreTask = Task.Run(async () => { - await Task.Delay(TimeSpan.FromSeconds(3), CancellationToken); portForwarder.ReturnToNormalMode(); }); - + var queueAndWaitAsync = node1Sender.QueueAndWaitAsync(request, CancellationToken.None); - + var dequeuedRequest = await node2Receiver.DequeueAsync(CancellationToken); dequeuedRequest.Should().NotBeNull(); dequeuedRequest!.RequestMessage.ActivityId.Should().Be(request.ActivityId); } - [Test] public async Task WhenTheReceiverDequeuesWorkAndThenDisconnectsFromRedisForEver_TheSenderEventuallyTimesOut() { // Arrange - var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); + var endpoint = new Uri("poll://" + Guid.NewGuid()); var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); var guid = Guid.NewGuid(); - + var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); @@ -602,29 +581,29 @@ public async Task WhenTheReceiverDequeuesWorkAndThenDisconnectsFromRedisForEver_ halibutTimeoutAndLimits.PollingRequestQueueTimeout = TimeSpan.FromDays(1); halibutTimeoutAndLimits.PollingQueueWaitTimeout = TimeSpan.FromDays(1); // We should not need to rely on the timeout working for very short disconnects. - await using var stableRedisConnection = CreateRedisFacade(guid: guid); + await using var stableRedisConnection = RedisFacadeBuilder.CreateRedisFacade(prefix: guid); + + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + await using var unstableRedisConnection = RedisFacadeBuilder.CreateRedisFacade(portForwarder.ListeningPort, guid); - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); - await using var unstableRedisConnection = CreateRedisFacade(portForwarder.ListeningPort, guid: guid); - var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(stableRedisConnection), messageReaderWriter, halibutTimeoutAndLimits); var node2Receiver = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(unstableRedisConnection), messageReaderWriter, halibutTimeoutAndLimits); await node2Receiver.WaitUntilQueueIsSubscribedToReceiveMessages(); - + // Lower this to complete the test sooner. node1Sender.DelayBetweenHeartBeatsForRequestProcessor = TimeSpan.FromSeconds(1); node2Receiver.DelayBetweenHeartBeatsForRequestProcessor = TimeSpan.FromSeconds(1); node1Sender.RequestReceivingNodeIsOfflineHeartBeatTimeout = TimeSpan.FromSeconds(10); node2Receiver.RequestReceivingNodeIsOfflineHeartBeatTimeout = TimeSpan.FromSeconds(10); - + var request = new RequestMessageBuilder("poll://test-endpoint").Build(); - + // Setting this low shows we don't timeout because the request was not picked up in time. request.Destination.PollingRequestQueueTimeout = TimeSpan.FromSeconds(5); var queueAndWaitTask = node1Sender.QueueAndWaitAsync(request, CancellationToken.None); - + var dequeuedRequest = await node2Receiver.DequeueAsync(CancellationToken); - + // Now disconnect the receiver from redis. portForwarder.EnterKillNewAndExistingConnectionsMode(); @@ -638,15 +617,15 @@ public async Task WhenTheReceiverDequeuesWorkAndThenDisconnectsFromRedisForEver_ CreateExceptionFromResponse(response, log).IsRetryableError().Should().Be(HalibutRetryableErrorType.IsRetryable); } - + [Test] public async Task WhenTheReceiverDequeuesWork_AndTheSenderDisconnects_AndNeverReconnects_TheDequeuedWorkIsEventuallyCancelled() { // Arrange - var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); + var endpoint = new Uri("poll://" + Guid.NewGuid()); var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); var guid = Guid.NewGuid(); - + var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); @@ -655,69 +634,68 @@ public async Task WhenTheReceiverDequeuesWork_AndTheSenderDisconnects_AndNeverRe halibutTimeoutAndLimits.PollingRequestQueueTimeout = TimeSpan.FromDays(1); halibutTimeoutAndLimits.PollingQueueWaitTimeout = TimeSpan.FromDays(1); // We should not need to rely on the timeout working for very short disconnects. - await using var stableRedisConnection = CreateRedisFacade(guid: guid); + await using var stableRedisConnection = RedisFacadeBuilder.CreateRedisFacade(prefix: guid); + + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + await using var unstableRedisConnection = RedisFacadeBuilder.CreateRedisFacade(portForwarder.ListeningPort, guid); - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); - await using var unstableRedisConnection = CreateRedisFacade(portForwarder.ListeningPort, guid: guid); - var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(unstableRedisConnection), messageReaderWriter, halibutTimeoutAndLimits); var node2Receiver = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(stableRedisConnection), messageReaderWriter, halibutTimeoutAndLimits); await node2Receiver.WaitUntilQueueIsSubscribedToReceiveMessages(); - - node1Sender.DelayBetweenHeartBeatsForRequestSender= TimeSpan.FromSeconds(1); - node2Receiver.DelayBetweenHeartBeatsForRequestSender= TimeSpan.FromSeconds(1); + + node1Sender.DelayBetweenHeartBeatsForRequestSender = TimeSpan.FromSeconds(1); + node2Receiver.DelayBetweenHeartBeatsForRequestSender = TimeSpan.FromSeconds(1); node1Sender.NodeIsOfflineHeartBeatTimeoutForRequestSender = TimeSpan.FromSeconds(15); node2Receiver.NodeIsOfflineHeartBeatTimeoutForRequestSender = TimeSpan.FromSeconds(15); node1Sender.TimeBetweenCheckingIfRequestWasCollected = TimeSpan.FromSeconds(1); node2Receiver.TimeBetweenCheckingIfRequestWasCollected = TimeSpan.FromSeconds(1); - + var request = new RequestMessageBuilder("poll://test-endpoint").Build(); - + var queueAndWaitTask = node1Sender.QueueAndWaitAsync(request, CancellationToken.None); - + var dequeuedRequest = await node2Receiver.DequeueAsync(CancellationToken); dequeuedRequest!.CancellationToken.IsCancellationRequested.Should().BeFalse(); - + // Now disconnect the sender from redis. portForwarder.EnterKillNewAndExistingConnectionsMode(); await Task.WhenAny(Task.Delay(TimeSpan.FromMinutes(2), dequeuedRequest.CancellationToken)); - dequeuedRequest.CancellationToken.IsCancellationRequested.Should().BeTrue(); } - + [Test] public async Task WhenTheSenderBrieflyDisconnectsFromRedisRightWhenTheReceiverSendsTheResponseBack_TheSenderStillGetsTheResponse() { // Arrange - var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); + var endpoint = new Uri("poll://" + Guid.NewGuid()); var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); var guid = Guid.NewGuid(); - + var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); - await using var stableConnection = CreateRedisFacade(guid: guid); - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(redisPort, Logger).Build(); - await using var unreliableConnection = CreateRedisFacade(portForwarder.ListeningPort, guid: guid); - + await using var stableConnection = RedisFacadeBuilder.CreateRedisFacade(prefix: guid); + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + await using var unreliableConnection = RedisFacadeBuilder.CreateRedisFacade(portForwarder.ListeningPort, guid); + var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(unreliableConnection), messageReaderWriter, new HalibutTimeoutsAndLimits()); var node2Receiver = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(stableConnection), messageReaderWriter, new HalibutTimeoutsAndLimits()); await node2Receiver.WaitUntilQueueIsSubscribedToReceiveMessages(); - + var request = new RequestMessageBuilder("poll://test-endpoint").Build(); // TODO: This only needs to be set because we do not detect the work was collected as soon as it has been collected. request.Destination.PollingRequestQueueTimeout = TimeSpan.FromDays(1); var queueAndWaitTask = node1Sender.QueueAndWaitAsync(request, CancellationToken.None); - + var dequeuedRequest = await node2Receiver.DequeueAsync(CancellationToken); - + // Just before we send the response, disconnect the sender. portForwarder.EnterKillNewAndExistingConnectionsMode(); await node2Receiver.ApplyResponse(ResponseMessage.FromResult(dequeuedRequest!.RequestMessage, "Yay"), dequeuedRequest!.RequestMessage.ActivityId); - + await Task.Delay(TimeSpan.FromSeconds(2), CancellationToken); portForwarder.ReturnToNormalMode(); @@ -728,45 +706,44 @@ public async Task WhenTheSenderBrieflyDisconnectsFromRedisRightWhenTheReceiverSe var response = await queueAndWaitTask; response.Error.Should().BeNull(); response.Result.Should().Be("Yay"); - } - + [Test] public async Task WhenTheRequestReceiverDetectsRedisDataLose_AndTheRequestSenderDoesNotYetDetectDataLose_TheSenderReceivesARetryableResponse() { // Arrange - var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); + var endpoint = new Uri("poll://" + Guid.NewGuid()); var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); var guid = Guid.NewGuid(); - + var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); - await using var stableConnection = CreateRedisFacade(guid: guid); + await using var stableConnection = RedisFacadeBuilder.CreateRedisFacade(prefix: guid); var redisDataLoseDetectorOnReceiver = new CancellableDataLossWatchForRedisLosingAllItsData(); var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(stableConnection), messageReaderWriter, new HalibutTimeoutsAndLimits()); var node2Receiver = new RedisPendingRequestQueue(endpoint, redisDataLoseDetectorOnReceiver, log, new HalibutRedisTransport(stableConnection), messageReaderWriter, new HalibutTimeoutsAndLimits()); await node2Receiver.WaitUntilQueueIsSubscribedToReceiveMessages(); - + var request = new RequestMessageBuilder("poll://test-endpoint").Build(); var queueAndWaitTask = node1Sender.QueueAndWaitAsync(request, CancellationToken.None); - + var dequeuedRequest = await node2Receiver.DequeueAsync(CancellationToken); - + // Act await redisDataLoseDetectorOnReceiver.DataLossHasOccured(); var responseThatWouldNotBeRetried = ResponseMessage.FromException(dequeuedRequest!.RequestMessage, new NoMatchingServiceOrMethodHalibutClientException("")); CreateExceptionFromResponse(responseThatWouldNotBeRetried, log) .IsRetryableError().Should().Be(HalibutRetryableErrorType.NotRetryable); - + await node2Receiver.ApplyResponse(ResponseMessage.FromResult(dequeuedRequest!.RequestMessage, "Yay"), dequeuedRequest!.RequestMessage.ActivityId); var response = await queueAndWaitTask; response.Error.Should().NotBeNull(); - + // Assert CreateExceptionFromResponse(response, log) .IsRetryableError().Should().Be(HalibutRetryableErrorType.IsRetryable); @@ -782,6 +759,7 @@ static Exception CreateExceptionFromResponse(ResponseMessage responseThatWouldNo { return e; } + Assert.Fail("Excpected an exception in the response message"); throw new Exception("it failed"); } @@ -799,33 +777,27 @@ public async Task WhenUsingTheRedisQueue_ASimpleEchoServiceCanBeCalled(ClientAnd .AsLatestClientAndLatestServiceBuilder() .WithPendingRequestQueueFactory((queueMessageSerializer, logFactory) => new RedisPendingRequestQueueFactory( - queueMessageSerializer, - dataStreamStore, - new NeverLosingDataWatchForRedisLosingAllItsData(), - redisTransport, - new HalibutTimeoutsAndLimits(), - logFactory) + queueMessageSerializer, + dataStreamStore, + new NeverLosingDataWatchForRedisLosingAllItsData(), + redisTransport, + new HalibutTimeoutsAndLimits(), + logFactory) .WithWaitForReceiverToBeReady()) .Build(CancellationToken)) { var echo = clientAndService.CreateAsyncClient(); (await echo.SayHelloAsync("Deploy package A")).Should().Be("Deploy package A..."); - for (var i = 0; i < clientAndServiceTestCase.RecommendedIterations; i++) - { - (await echo.SayHelloAsync($"Deploy package A {i}")).Should().Be($"Deploy package A {i}..."); - } + for (var i = 0; i < clientAndServiceTestCase.RecommendedIterations; i++) (await echo.SayHelloAsync($"Deploy package A {i}")).Should().Be($"Deploy package A {i}..."); } } - - - [Test] public async Task CancellingARequestShouldResultInTheDequeuedResponseTokenBeingCancelled() { // Arrange - var endpoint = new Uri("poll://" + Guid.NewGuid().ToString()); + var endpoint = new Uri("poll://" + Guid.NewGuid()); var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); var redisTransport = new HalibutRedisTransport(redisFacade); @@ -840,11 +812,11 @@ public async Task CancellingARequestShouldResultInTheDequeuedResponseTokenBeingC await node1Sender.WaitUntilQueueIsSubscribedToReceiveMessages(); using var cts = new CancellationTokenSource(); - + var queueAndWaitAsync = node1Sender.QueueAndWaitAsync(request, cts.Token); var requestMessageWithCancellationToken = await node1Sender.DequeueAsync(CancellationToken); - + requestMessageWithCancellationToken!.CancellationToken.IsCancellationRequested.Should().BeFalse(); await cts.CancelAsync(); @@ -852,8 +824,11 @@ public async Task CancellingARequestShouldResultInTheDequeuedResponseTokenBeingC try { await Task.Delay(TimeSpan.FromSeconds(10), requestMessageWithCancellationToken.CancellationToken); - } catch (TaskCanceledException){} - + } + catch (TaskCanceledException) + { + } + requestMessageWithCancellationToken!.CancellationToken.IsCancellationRequested.Should().BeTrue(); } diff --git a/source/Halibut.Tests/Queue/Redis/RedisQueuePerformanceTest.cs b/source/Halibut.Tests/Queue/Redis/RedisQueuePerformanceTest.cs index 6f79589c2..ca6b1179f 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisQueuePerformanceTest.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisQueuePerformanceTest.cs @@ -28,13 +28,9 @@ namespace Halibut.Tests.Queue.Redis { - [Ignore("REDISTODO")] + [RedisTest] public class RedisQueuePerformanceTest : BaseTest { - const int redisPort = 6379; - private static RedisFacade CreateRedisFacade(int? port = redisPort, Guid? guid = null) => new("localhost:" + port, (guid ?? Guid.NewGuid()).ToString(), new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix("")); - - //[Test] public async Task When100kTentaclesAreSubscribed_TheQueueStillWorks() { diff --git a/source/Halibut.Tests/Queue/Redis/Utils/RedisFacadeBuilder.cs b/source/Halibut.Tests/Queue/Redis/Utils/RedisFacadeBuilder.cs index 64e6f1ec3..90a6a2047 100644 --- a/source/Halibut.Tests/Queue/Redis/Utils/RedisFacadeBuilder.cs +++ b/source/Halibut.Tests/Queue/Redis/Utils/RedisFacadeBuilder.cs @@ -22,10 +22,10 @@ namespace Halibut.Tests.Queue.Redis.Utils { public class RedisFacadeBuilder { - public static RedisFacade CreateRedisFacade(int? port = 0, Guid? guid = null) + public static RedisFacade CreateRedisFacade(int? port = 0, Guid? prefix = null) { port = port == 0 ? RedisPort.Port() : port; - return new RedisFacade("localhost:" + port, (guid ?? Guid.NewGuid()).ToString(), new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix("")); + return new RedisFacade("localhost:" + port, (prefix ?? Guid.NewGuid()).ToString(), new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix("")); } } } \ No newline at end of file diff --git a/source/Halibut.Tests/Queue/Redis/WatchForRedisLosingAllItsDataFixture.cs b/source/Halibut.Tests/Queue/Redis/WatchForRedisLosingAllItsDataFixture.cs index 49d27d863..6196a3b9a 100644 --- a/source/Halibut.Tests/Queue/Redis/WatchForRedisLosingAllItsDataFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/WatchForRedisLosingAllItsDataFixture.cs @@ -13,21 +13,15 @@ namespace Halibut.Tests.Queue.Redis { + [RedisTest] public class WatchForRedisLosingAllItsDataFixture : BaseTest { - - private static RedisFacade CreateRedisFacade(int? port = 0, Guid? guid = null) - { - port = port == 0 ? RedisPort.Port() : port; - return new RedisFacade("localhost:" + port, (guid ?? Guid.NewGuid()).ToString(), new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix("")); - } - [Test] public async Task WhenTheConnectionToRedisCanNotBeCreated_WhenAskingForALostDataCancellationToken_ATimeoutOccurs() { using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); portForwarder.EnterKillNewAndExistingConnectionsMode(); - await using var redisFacade = CreateRedisFacade(port: portForwarder.ListeningPort); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder.ListeningPort, null); await using var watcher = new WatchForRedisLosingAllItsData(redisFacade, HalibutLog, watchInterval:TimeSpan.FromSeconds(1)); @@ -39,7 +33,7 @@ public async Task WhenTheConnectionToRedisIsInitiallyDown_WhenAskingForALostData { using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); portForwarder.EnterKillNewAndExistingConnectionsMode(); - await using var redisFacade = CreateRedisFacade(port: portForwarder.ListeningPort); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder.ListeningPort, null); await using var watcher = new WatchForRedisLosingAllItsData(redisFacade, HalibutLog, watchInterval:TimeSpan.FromSeconds(1)); var _ = Task.Run(async () => @@ -57,7 +51,7 @@ public async Task WatchForARealRedisLosingAllOfItsData_TimesOutWhenWaitingForCTW { using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); portForwarder.EnterKillNewAndExistingConnectionsMode(); - await using var redisFacade = CreateRedisFacade(port: portForwarder.ListeningPort); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder.ListeningPort, null); await using var watcher = new WatchForRedisLosingAllItsData(redisFacade, HalibutLog, watchInterval:TimeSpan.FromSeconds(1)); @@ -90,7 +84,7 @@ public async Task WhenRedisLosesAllOfIts_TheWatcherShouldDetectTheDataLose() Logger.Information("Redis container started successfully with connection string: {ConnectionString}", container.ConnectionString); // Create RedisFacade connected to the containerized Redis - await using var redisFacade = CreateRedisFacade(port: container.RedisPort); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(container.RedisPort, null); await using var watcher = new WatchForRedisLosingAllItsData(redisFacade, HalibutLog, watchInterval:TimeSpan.FromSeconds(1)); diff --git a/source/Halibut.Tests/Support/TestAttributes/RedisTestAttribute.cs b/source/Halibut.Tests/Support/TestAttributes/RedisTestAttribute.cs new file mode 100644 index 000000000..66bc39355 --- /dev/null +++ b/source/Halibut.Tests/Support/TestAttributes/RedisTestAttribute.cs @@ -0,0 +1,23 @@ +using System; +using Halibut.Tests.TestSetup.Redis; +using NUnit.Framework; +using NUnit.Framework.Interfaces; +using NUnit.Framework.Internal; + +[AttributeUsage(AttributeTargets.Class | AttributeTargets.Method)] +public class RedisTestAttribute : NUnitAttribute, IApplyToTest +{ + public void ApplyToTest(Test test) + { + if (test.RunState == RunState.NotRunnable || test.RunState == RunState.Ignored) + { + return; + } + + if (!EnsureRedisIsAvailableSetupFixture.WillRunRedisTests) + { + test.RunState = RunState.Skipped; + test.Properties.Add("_SKIPREASON", "Redis tests are not yet supported on this OS or dotnet version."); + } + } +} diff --git a/source/Halibut.Tests/TestSetup/Redis/EnsureRedisIsAvailableSetupFixture.cs b/source/Halibut.Tests/TestSetup/Redis/EnsureRedisIsAvailableSetupFixture.cs index 2c823e11a..47be34ab4 100644 --- a/source/Halibut.Tests/TestSetup/Redis/EnsureRedisIsAvailableSetupFixture.cs +++ b/source/Halibut.Tests/TestSetup/Redis/EnsureRedisIsAvailableSetupFixture.cs @@ -22,30 +22,33 @@ namespace Halibut.Tests.TestSetup.Redis { public class EnsureRedisIsAvailableSetupFixture : ISetupFixture { + public static bool WillRunRedisTests => +#if NETFRAMEWORK + false; +#else + !RuntimeInformation.IsOSPlatform(OSPlatform.Windows) + || !TeamCityDetection.IsRunningInTeamCity(); + CreateRedisDockerContainerForTests? redisContainer = null; public void OneTimeSetUp(ILogger logger) { -#if NETFRAMEWORK - return; -#else + if (!WillRunRedisTests) return; + var isWindows = RuntimeInformation.IsOSPlatform(OSPlatform.Windows); - bool shouldCreateRedis = !isWindows && TeamCityDetection.IsRunningInTeamCity(); + bool shouldCreateRedis = WillRunRedisTests; if (!TeamCityDetection.IsRunningInTeamCity()) { - if (!isWindows) + // Does the user already have redis running on the normal port? + try + { + using var multiplexer = ConnectionMultiplexer.Connect("localhost:6379"); + var ts = multiplexer.GetDatabase().Ping(); + RedisPort.SetPort(6379); + } + catch { - // Does the user already have redis running on the normal port? - try - { - using var multiplexer = ConnectionMultiplexer.Connect("localhost:6379"); - var ts = multiplexer.GetDatabase().Ping(); - RedisPort.SetPort(6379); - } - catch - { - shouldCreateRedis = true; - } + shouldCreateRedis = true; } } diff --git a/source/Halibut/Queue/Redis/RedisFacade.cs b/source/Halibut/Queue/Redis/RedisFacade.cs index 001e6e878..84a19ec71 100644 --- a/source/Halibut/Queue/Redis/RedisFacade.cs +++ b/source/Halibut/Queue/Redis/RedisFacade.cs @@ -68,11 +68,11 @@ public class RedisFacade : IAsyncDisposable CancelOnDisposeCancellationToken cts; CancellationToken facadeCancellationToken; - public RedisFacade(string configuration, string? keyPrefix, ILog log) : this(ConfigurationOptions.Parse(configuration), keyPrefix, log) + public RedisFacade(string configuration, string keyPrefix, ILog log) : this(ConfigurationOptions.Parse(configuration), keyPrefix, log) { } - public RedisFacade(ConfigurationOptions redisOptions, string? keyPrefix, ILog log) + public RedisFacade(ConfigurationOptions redisOptions, string keyPrefix, ILog log) { this.keyPrefix = keyPrefix ?? "halibut"; this.log = log; From bda4d4fa1bef5914e40b09650b6853cedc5d553d Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Sun, 17 Aug 2025 06:44:20 +1000 Subject: [PATCH 060/137] Log setup fixture to standard location --- source/Halibut.Tests/BaseTest.cs | 2 +- source/Halibut.Tests/TestsSetupClass.cs | 7 ++++--- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/source/Halibut.Tests/BaseTest.cs b/source/Halibut.Tests/BaseTest.cs index 1d9ac4c1a..1ae8d199d 100644 --- a/source/Halibut.Tests/BaseTest.cs +++ b/source/Halibut.Tests/BaseTest.cs @@ -30,7 +30,7 @@ public void SetUp() .Build() .ForContext(GetType()); - HalibutLog = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); + HalibutLog = new TestContextLogCreator("", LogLevel.Trace).CreateNewForPrefix(""); Logger.Information("Trace log file {LogFile}", traceLogFileLogger.logFilePath); diff --git a/source/Halibut.Tests/TestsSetupClass.cs b/source/Halibut.Tests/TestsSetupClass.cs index 12e235308..6e7203f74 100644 --- a/source/Halibut.Tests/TestsSetupClass.cs +++ b/source/Halibut.Tests/TestsSetupClass.cs @@ -24,12 +24,13 @@ public class TestsSetupClass public void OneTimeSetup() { var sb = new StringBuilder(); + var traceLogFileLogger = new TraceLogFileLogger("TestsSetupClass"); + var logger = new SerilogLoggerBuilder() - // Ideally we would write the logs to a static string and have a test print those out. - // So that we can see what is happening in start up in teamcity. - // However, the logging is a bit of a mess in halibut currently. + .SetTraceLogFileLogger(traceLogFileLogger) .Build() .ForContext(); + logger.Information("Trace log file {LogFile}", traceLogFileLogger.logFilePath); foreach (var setupFixture in setupFixtures) { setupFixture.OneTimeSetUp(logger.ForContext(setupFixture.GetType())); From 04ac59978faed46231e90296374537eae07a0f83 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Sun, 17 Aug 2025 06:51:43 +1000 Subject: [PATCH 061/137] . --- .../TestSetup/Redis/EnsureRedisIsAvailableSetupFixture.cs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/Halibut.Tests/TestSetup/Redis/EnsureRedisIsAvailableSetupFixture.cs b/source/Halibut.Tests/TestSetup/Redis/EnsureRedisIsAvailableSetupFixture.cs index 47be34ab4..e99f7561b 100644 --- a/source/Halibut.Tests/TestSetup/Redis/EnsureRedisIsAvailableSetupFixture.cs +++ b/source/Halibut.Tests/TestSetup/Redis/EnsureRedisIsAvailableSetupFixture.cs @@ -28,6 +28,7 @@ public class EnsureRedisIsAvailableSetupFixture : ISetupFixture #else !RuntimeInformation.IsOSPlatform(OSPlatform.Windows) || !TeamCityDetection.IsRunningInTeamCity(); +#endif CreateRedisDockerContainerForTests? redisContainer = null; public void OneTimeSetUp(ILogger logger) @@ -59,7 +60,6 @@ public void OneTimeSetUp(ILogger logger) RedisPort.SetPort(redisContainer.container!.RedisPort); logger.Information("RedisPort is: {RedisPort}", RedisPort.Port()); } -#endif } public void OneTimeTearDown(ILogger logger) From eadfee5580265107fbd3040dc822a98e82833cde Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Sun, 17 Aug 2025 06:54:27 +1000 Subject: [PATCH 062/137] . --- .../Support/EnvironmentVariableReaderHelper.cs | 16 ++++++++++++++++ .../Redis/EnsureRedisIsAvailableSetupFixture.cs | 6 ++++-- 2 files changed, 20 insertions(+), 2 deletions(-) diff --git a/source/Halibut.Tests/Support/EnvironmentVariableReaderHelper.cs b/source/Halibut.Tests/Support/EnvironmentVariableReaderHelper.cs index c48051ada..7d286a4c6 100644 --- a/source/Halibut.Tests/Support/EnvironmentVariableReaderHelper.cs +++ b/source/Halibut.Tests/Support/EnvironmentVariableReaderHelper.cs @@ -15,5 +15,21 @@ public static bool EnvironmentVariableAsBool(string envVar, bool defaultValue) return value!.Equals("true"); } + + public static int? TryReadIntFromEnvironmentVariable(string envVar) + { + var value = Environment.GetEnvironmentVariable(envVar); + if (string.IsNullOrWhiteSpace(value)) + { + return null; + } + + if (int.TryParse(value, out var result)) + { + return result; + } + + return null; + } } } \ No newline at end of file diff --git a/source/Halibut.Tests/TestSetup/Redis/EnsureRedisIsAvailableSetupFixture.cs b/source/Halibut.Tests/TestSetup/Redis/EnsureRedisIsAvailableSetupFixture.cs index e99f7561b..fcd03d583 100644 --- a/source/Halibut.Tests/TestSetup/Redis/EnsureRedisIsAvailableSetupFixture.cs +++ b/source/Halibut.Tests/TestSetup/Redis/EnsureRedisIsAvailableSetupFixture.cs @@ -30,6 +30,8 @@ public class EnsureRedisIsAvailableSetupFixture : ISetupFixture || !TeamCityDetection.IsRunningInTeamCity(); #endif + static readonly int RedisPortToTry = EnvironmentVariableReaderHelper.TryReadIntFromEnvironmentVariable("HALIBUT_REDIS_PORT") ?? 6379; + static readonly string RedisHost = Environment.GetEnvironmentVariable("HALIBUT_REDIS_HOST") ?? "localhost"; CreateRedisDockerContainerForTests? redisContainer = null; public void OneTimeSetUp(ILogger logger) { @@ -43,9 +45,9 @@ public void OneTimeSetUp(ILogger logger) // Does the user already have redis running on the normal port? try { - using var multiplexer = ConnectionMultiplexer.Connect("localhost:6379"); + using var multiplexer = ConnectionMultiplexer.Connect(RedisHost + ":" + RedisPortToTry); var ts = multiplexer.GetDatabase().Ping(); - RedisPort.SetPort(6379); + RedisPort.SetPort(RedisPortToTry); } catch { From 899c03593fd47976cd4ec29b07590972ab2b0ec5 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Sun, 17 Aug 2025 07:04:11 +1000 Subject: [PATCH 063/137] Support setting redis host --- .../RedisPendingRequestQueueBuilder.cs | 2 +- source/Halibut.Tests/Halibut.Tests.csproj | 2 +- .../Queue/Redis/NodeHeartBeatSenderFixture.cs | 16 ++++----- .../RedisFacadeWhenRedisGoesDownAwayTests.cs | 34 +++++++++---------- .../Redis/RedisPendingRequestQueueFixture.cs | 14 ++++---- .../Queue/Redis/Utils/RedisFacadeBuilder.cs | 4 +-- .../WatchForRedisLosingAllItsDataFixture.cs | 6 ++-- .../EnsureRedisIsAvailableSetupFixture.cs | 8 +++-- .../Redis/{RedisPort.cs => RedisTestHost.cs} | 4 ++- 9 files changed, 47 insertions(+), 43 deletions(-) rename source/Halibut.Tests/TestSetup/Redis/{RedisPort.cs => RedisTestHost.cs} (90%) diff --git a/source/Halibut.Tests/Builders/RedisPendingRequestQueueBuilder.cs b/source/Halibut.Tests/Builders/RedisPendingRequestQueueBuilder.cs index 7b84d48e8..22c055bee 100644 --- a/source/Halibut.Tests/Builders/RedisPendingRequestQueueBuilder.cs +++ b/source/Halibut.Tests/Builders/RedisPendingRequestQueueBuilder.cs @@ -62,7 +62,7 @@ public QueueHolder Build() var disposableCollection = new DisposableCollection(); - var redisFacade = new RedisFacade("localhost:" + RedisPort.Port(), (Guid.NewGuid()).ToString(), log); + var redisFacade = new RedisFacade("localhost:" + RedisTestHost.Port(), (Guid.NewGuid()).ToString(), log); disposableCollection.AddAsyncDisposable(redisFacade); var redisTransport = new HalibutRedisTransport(redisFacade); diff --git a/source/Halibut.Tests/Halibut.Tests.csproj b/source/Halibut.Tests/Halibut.Tests.csproj index 9eb552ceb..0e31f3e3e 100644 --- a/source/Halibut.Tests/Halibut.Tests.csproj +++ b/source/Halibut.Tests/Halibut.Tests.csproj @@ -12,7 +12,7 @@ true - net48;net8.0 + net8.0 net8.0 diff --git a/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs b/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs index 3af8cd306..001a3495a 100644 --- a/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs @@ -61,9 +61,9 @@ public async Task WhenRedisConnectionIsInterrupted_ShouldSwitchToPanicMode() var log = new TestContextLogCreator("NodeHeartBeat", LogLevel.Trace).CreateNewForPrefix(""); var guid = Guid.NewGuid(); - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); await using var unstableRedisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder.ListeningPort, guid); - await using var stableRedisFacade = RedisFacadeBuilder.CreateRedisFacade(RedisPort.Port(), guid); + await using var stableRedisFacade = RedisFacadeBuilder.CreateRedisFacade(RedisTestHost.Port(), guid); var redisTransport = new HalibutRedisTransport(unstableRedisFacade); @@ -153,9 +153,9 @@ public async Task WaitUntilNodeProcessingRequestFlatLines_WhenHeartbeatsStop_Sho var log = new TestContextLogCreator("NodeHeartBeat", LogLevel.Trace).CreateNewForPrefix(""); var guid = Guid.NewGuid(); - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); await using var unstableRedisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder.ListeningPort, guid); - await using var stableRedisFacade = RedisFacadeBuilder.CreateRedisFacade(RedisPort.Port(), guid); + await using var stableRedisFacade = RedisFacadeBuilder.CreateRedisFacade(RedisTestHost.Port(), guid); var unstableRedisTransport = new HalibutRedisTransport(unstableRedisFacade); var stableRedisTransport = new HalibutRedisTransport(stableRedisFacade); @@ -240,9 +240,9 @@ public async Task WaitUntilNodeProcessingRequestFlatLines_WhenConnectionInterrup var log = new TestContextLogCreator("NodeHeartBeat", LogLevel.Trace).CreateNewForPrefix(""); var guid = Guid.NewGuid(); - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); await using var unstableRedisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder.ListeningPort, guid); - await using var stableRedisFacade = RedisFacadeBuilder.CreateRedisFacade(RedisPort.Port(), guid); + await using var stableRedisFacade = RedisFacadeBuilder.CreateRedisFacade(RedisTestHost.Port(), guid); var unstableRedisTransport = new HalibutRedisTransport(unstableRedisFacade); var stableRedisTransport = new HalibutRedisTransport(stableRedisFacade); @@ -321,9 +321,9 @@ public async Task WhenHeartBeatSenderConnectionRecovery_ShouldResumeNormalHeartb var log = new TestContextLogCreator("NodeHeartBeat", LogLevel.Trace).CreateNewForPrefix(""); var guid = Guid.NewGuid(); - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); await using var unstableRedisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder.ListeningPort, guid); - await using var stableRedisFacade = RedisFacadeBuilder.CreateRedisFacade(RedisPort.Port(), guid); + await using var stableRedisFacade = RedisFacadeBuilder.CreateRedisFacade(RedisTestHost.Port(), guid); var unstableRedisTransport = new HalibutRedisTransport(unstableRedisFacade); var stableRedisTransport = new HalibutRedisTransport(stableRedisFacade); diff --git a/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs b/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs index c74b7e610..5dfec09de 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs @@ -34,14 +34,14 @@ public class RedisFacadeWhenRedisGoesDownAwayTests : BaseTest { static RedisFacade CreateRedisFacade(int? port = 0, Guid? guid = null) { - port = port == 0 ? RedisPort.Port() : port; + port = port == 0 ? RedisTestHost.Port() : port; return new RedisFacade("localhost:" + port, (guid ?? Guid.NewGuid()).ToString(), new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix("")); } [Test] public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_SometimeLaterOnWeCanDoBasicCalls() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); @@ -61,7 +61,7 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe [Test] public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelyDoBasicCalls() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); @@ -80,7 +80,7 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe [Test] public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelyPublishToChannel() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); @@ -97,7 +97,7 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe [Test] public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelySetInHash() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); @@ -114,7 +114,7 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe [Test] public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelyTryGetAndDeleteFromHash() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); @@ -132,7 +132,7 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe [Test] public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelyListRightPush() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); @@ -149,7 +149,7 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe [Test] public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelyListLeftPop() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); @@ -167,7 +167,7 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe [Test] public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelySetString() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); @@ -184,7 +184,7 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe [Test] public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelyGetString() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); @@ -202,7 +202,7 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe [Test] public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelyHashContainsKey() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); @@ -220,11 +220,11 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe [Test] public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndWeTryToSubscribe_WhenTheConnectionIsRestored_WeCanReceiveMessages() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); var redisLogCreator = new TestContextLogCreator("Redis", LogLevel.Trace); var guid = Guid.NewGuid().ToString(); await using var redisViaPortForwarder = new RedisFacade("localhost:" + portForwarder.ListeningPort, guid, redisLogCreator.CreateNewForPrefix("Unstable")); - await using var redisStableConnection = new RedisFacade("localhost:" + RedisPort.Port(), guid, redisLogCreator.CreateNewForPrefix("Stable")); + await using var redisStableConnection = new RedisFacade("localhost:" + RedisTestHost.Port(), guid, redisLogCreator.CreateNewForPrefix("Stable")); await redisViaPortForwarder.SetString("Establish connection", "before we subscribe", TimeSpan.FromMinutes(1), CancellationToken); @@ -262,11 +262,11 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndWeTryT [Test] public async Task WhenTheConnectionIsNeverEstablished_AndWeTryToSubscribe_WhenTheConnectionIsRestored_WeCanReceiveMessages() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); var redisLogCreator = new TestContextLogCreator("Redis", LogLevel.Trace); var guid = Guid.NewGuid().ToString(); await using var redisViaPortForwarder = new RedisFacade("localhost:" + portForwarder.ListeningPort, guid, redisLogCreator.CreateNewForPrefix("Unstable")); - await using var redisStableConnection = new RedisFacade("localhost:" + RedisPort.Port(), guid, redisLogCreator.CreateNewForPrefix("Stable")); + await using var redisStableConnection = new RedisFacade("localhost:" + RedisTestHost.Port(), guid, redisLogCreator.CreateNewForPrefix("Stable")); portForwarder.EnterKillNewAndExistingConnectionsMode(); @@ -304,14 +304,14 @@ public async Task WhenTheConnectionIsNeverEstablished_AndWeTryToSubscribe_WhenTh [WindowsTest] public async Task WhenSubscribedAndTheConnectionGoesDown_WhenTheConnectionIsRestored_MessagesCanEventuallyBeSentToTheSubscriberAgain() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); var redisLogCreator = new TestContextLogCreator("Redis", LogLevel.Trace); var guid = Guid.NewGuid().ToString(); await using var redisViaPortForwarder = new RedisFacade("localhost:" + portForwarder.ListeningPort, guid, redisLogCreator.CreateNewForPrefix("Unstable")); - await using var redisStableConnection = new RedisFacade("localhost:" + RedisPort.Port(), guid, redisLogCreator.CreateNewForPrefix("Stable")); + await using var redisStableConnection = new RedisFacade("localhost:" + RedisTestHost.Port(), guid, redisLogCreator.CreateNewForPrefix("Stable")); var msgs = new ConcurrentBag(); await using var channel = await redisViaPortForwarder.SubscribeToChannel("bob", async message => diff --git a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs index 238317d6a..de67c6d77 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs @@ -163,7 +163,7 @@ public async Task WhenEnteringTheQueue_AndRedisIsUnavailable_ARetryableException var endpoint = new Uri("poll://" + Guid.NewGuid()); var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder.ListeningPort, null); redisFacade.MaxDurationToRetryFor = TimeSpan.FromSeconds(1); @@ -189,7 +189,7 @@ public async Task WhenEnteringTheQueue_AndRedisIsUnavailableAndDataLoseOccurs_AR var endpoint = new Uri("poll://" + Guid.NewGuid()); var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder.ListeningPort, null); redisFacade.MaxDurationToRetryFor = TimeSpan.FromSeconds(1); @@ -455,7 +455,7 @@ public async Task WhenTheReceiversConnectionToRedisIsInterruptedAndRestoredBefor var guid = Guid.NewGuid(); await using var redisFacadeSender = RedisFacadeBuilder.CreateRedisFacade(prefix: guid); - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); await using var redisFacadeReceiver = RedisFacadeBuilder.CreateRedisFacade(portForwarder.ListeningPort, guid); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); @@ -533,7 +533,7 @@ public async Task WhenTheSendersConnectionToRedisIsBrieflyInterruptedWhileSendin var guid = Guid.NewGuid(); await using var redisFacadeReceiver = RedisFacadeBuilder.CreateRedisFacade(prefix: guid); - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); await using var redisFacadeSender = RedisFacadeBuilder.CreateRedisFacade(portForwarder.ListeningPort, guid); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); @@ -583,7 +583,7 @@ public async Task WhenTheReceiverDequeuesWorkAndThenDisconnectsFromRedisForEver_ await using var stableRedisConnection = RedisFacadeBuilder.CreateRedisFacade(prefix: guid); - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); await using var unstableRedisConnection = RedisFacadeBuilder.CreateRedisFacade(portForwarder.ListeningPort, guid); var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(stableRedisConnection), messageReaderWriter, halibutTimeoutAndLimits); @@ -636,7 +636,7 @@ public async Task WhenTheReceiverDequeuesWork_AndTheSenderDisconnects_AndNeverRe await using var stableRedisConnection = RedisFacadeBuilder.CreateRedisFacade(prefix: guid); - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); await using var unstableRedisConnection = RedisFacadeBuilder.CreateRedisFacade(portForwarder.ListeningPort, guid); var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(unstableRedisConnection), messageReaderWriter, halibutTimeoutAndLimits); @@ -678,7 +678,7 @@ public async Task WhenTheSenderBrieflyDisconnectsFromRedisRightWhenTheReceiverSe var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); await using var stableConnection = RedisFacadeBuilder.CreateRedisFacade(prefix: guid); - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); await using var unreliableConnection = RedisFacadeBuilder.CreateRedisFacade(portForwarder.ListeningPort, guid); var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(unreliableConnection), messageReaderWriter, new HalibutTimeoutsAndLimits()); diff --git a/source/Halibut.Tests/Queue/Redis/Utils/RedisFacadeBuilder.cs b/source/Halibut.Tests/Queue/Redis/Utils/RedisFacadeBuilder.cs index 90a6a2047..cc1b264d0 100644 --- a/source/Halibut.Tests/Queue/Redis/Utils/RedisFacadeBuilder.cs +++ b/source/Halibut.Tests/Queue/Redis/Utils/RedisFacadeBuilder.cs @@ -24,8 +24,8 @@ public class RedisFacadeBuilder { public static RedisFacade CreateRedisFacade(int? port = 0, Guid? prefix = null) { - port = port == 0 ? RedisPort.Port() : port; - return new RedisFacade("localhost:" + port, (prefix ?? Guid.NewGuid()).ToString(), new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix("")); + port = port == 0 ? RedisTestHost.Port() : port; + return new RedisFacade(RedisTestHost.RedisHost + ":" + port, (prefix ?? Guid.NewGuid()).ToString(), new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix("")); } } } \ No newline at end of file diff --git a/source/Halibut.Tests/Queue/Redis/WatchForRedisLosingAllItsDataFixture.cs b/source/Halibut.Tests/Queue/Redis/WatchForRedisLosingAllItsDataFixture.cs index 6196a3b9a..2a55caab9 100644 --- a/source/Halibut.Tests/Queue/Redis/WatchForRedisLosingAllItsDataFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/WatchForRedisLosingAllItsDataFixture.cs @@ -19,7 +19,7 @@ public class WatchForRedisLosingAllItsDataFixture : BaseTest [Test] public async Task WhenTheConnectionToRedisCanNotBeCreated_WhenAskingForALostDataCancellationToken_ATimeoutOccurs() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); portForwarder.EnterKillNewAndExistingConnectionsMode(); await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder.ListeningPort, null); await using var watcher = new WatchForRedisLosingAllItsData(redisFacade, HalibutLog, watchInterval:TimeSpan.FromSeconds(1)); @@ -31,7 +31,7 @@ public async Task WhenTheConnectionToRedisCanNotBeCreated_WhenAskingForALostData [Test] public async Task WhenTheConnectionToRedisIsInitiallyDown_WhenAskingForALostDataCancellationToken_AndTheConnectionToRedisReturns_TheCTIsReturned() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); portForwarder.EnterKillNewAndExistingConnectionsMode(); await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder.ListeningPort, null); await using var watcher = new WatchForRedisLosingAllItsData(redisFacade, HalibutLog, watchInterval:TimeSpan.FromSeconds(1)); @@ -49,7 +49,7 @@ public async Task WhenTheConnectionToRedisIsInitiallyDown_WhenAskingForALostData [Test] public async Task WatchForARealRedisLosingAllOfItsData_TimesOutWhenWaitingForCTWhenNoConnectionToRedisCanBeEstablished() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisPort.Port(), Logger).Build(); + using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); portForwarder.EnterKillNewAndExistingConnectionsMode(); await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder.ListeningPort, null); await using var watcher = new WatchForRedisLosingAllItsData(redisFacade, HalibutLog, watchInterval:TimeSpan.FromSeconds(1)); diff --git a/source/Halibut.Tests/TestSetup/Redis/EnsureRedisIsAvailableSetupFixture.cs b/source/Halibut.Tests/TestSetup/Redis/EnsureRedisIsAvailableSetupFixture.cs index fcd03d583..0c85fe08a 100644 --- a/source/Halibut.Tests/TestSetup/Redis/EnsureRedisIsAvailableSetupFixture.cs +++ b/source/Halibut.Tests/TestSetup/Redis/EnsureRedisIsAvailableSetupFixture.cs @@ -47,7 +47,9 @@ public void OneTimeSetUp(ILogger logger) { using var multiplexer = ConnectionMultiplexer.Connect(RedisHost + ":" + RedisPortToTry); var ts = multiplexer.GetDatabase().Ping(); - RedisPort.SetPort(RedisPortToTry); + RedisTestHost.SetPort(RedisPortToTry); + RedisTestHost.RedisHost = RedisHost; + return; } catch { @@ -59,8 +61,8 @@ public void OneTimeSetUp(ILogger logger) { redisContainer = new CreateRedisDockerContainerForTests(logger); redisContainer.InitializeAsync().GetAwaiter().GetResult(); - RedisPort.SetPort(redisContainer.container!.RedisPort); - logger.Information("RedisPort is: {RedisPort}", RedisPort.Port()); + RedisTestHost.SetPort(redisContainer.container!.RedisPort); + logger.Information("RedisPort is: {RedisPort}", RedisTestHost.Port()); } } diff --git a/source/Halibut.Tests/TestSetup/Redis/RedisPort.cs b/source/Halibut.Tests/TestSetup/Redis/RedisTestHost.cs similarity index 90% rename from source/Halibut.Tests/TestSetup/Redis/RedisPort.cs rename to source/Halibut.Tests/TestSetup/Redis/RedisTestHost.cs index 7f47f4f38..11855a8e6 100644 --- a/source/Halibut.Tests/TestSetup/Redis/RedisPort.cs +++ b/source/Halibut.Tests/TestSetup/Redis/RedisTestHost.cs @@ -16,7 +16,7 @@ namespace Halibut.Tests.TestSetup.Redis { - public static class RedisPort + public static class RedisTestHost { static int redisPort = 0; public static void SetPort(int value) @@ -33,5 +33,7 @@ public static int Port() return redisPort; } + + public static string RedisHost { get; set; } = "localhost"; } } \ No newline at end of file From d1a1527b8119e0c4d0d0e42931d6eeb3a270c9de Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Mon, 18 Aug 2025 09:37:49 +1000 Subject: [PATCH 064/137] Finally the host is respected --- .../Queue/Redis/NodeHeartBeatSenderFixture.cs | 45 ++++++++----------- .../RedisFacadeWhenRedisGoesDownAwayTests.cs | 42 ++++++++--------- .../Redis/RedisPendingRequestQueueFixture.cs | 28 ++++++------ .../Queue/Redis/Utils/RedisFacadeBuilder.cs | 12 ++++- .../WatchForRedisLosingAllItsDataFixture.cs | 14 +++--- .../EnsureRedisIsAvailableSetupFixture.cs | 1 + 6 files changed, 73 insertions(+), 69 deletions(-) diff --git a/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs b/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs index 001a3495a..a993f506d 100644 --- a/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs @@ -9,6 +9,7 @@ using Halibut.Queue.Redis; using Halibut.Tests.Builders; using Halibut.Tests.Queue.Redis.Utils; +using Halibut.Tests.Support; using Halibut.Tests.Support.Logging; using Halibut.Tests.TestSetup.Redis; using Nito.AsyncEx; @@ -61,9 +62,9 @@ public async Task WhenRedisConnectionIsInterrupted_ShouldSwitchToPanicMode() var log = new TestContextLogCreator("NodeHeartBeat", LogLevel.Trace).CreateNewForPrefix(""); var guid = Guid.NewGuid(); - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); - await using var unstableRedisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder.ListeningPort, guid); - await using var stableRedisFacade = RedisFacadeBuilder.CreateRedisFacade(RedisTestHost.Port(), guid); + using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); + await using var unstableRedisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder, guid); + await using var stableRedisFacade = RedisFacadeBuilder.CreateRedisFacade(port: RedisTestHost.Port(), prefix: guid); var redisTransport = new HalibutRedisTransport(unstableRedisFacade); @@ -153,9 +154,9 @@ public async Task WaitUntilNodeProcessingRequestFlatLines_WhenHeartbeatsStop_Sho var log = new TestContextLogCreator("NodeHeartBeat", LogLevel.Trace).CreateNewForPrefix(""); var guid = Guid.NewGuid(); - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); - await using var unstableRedisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder.ListeningPort, guid); - await using var stableRedisFacade = RedisFacadeBuilder.CreateRedisFacade(RedisTestHost.Port(), guid); + using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); + await using var unstableRedisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder, guid); + await using var stableRedisFacade = RedisFacadeBuilder.CreateRedisFacade(prefix: guid); var unstableRedisTransport = new HalibutRedisTransport(unstableRedisFacade); var stableRedisTransport = new HalibutRedisTransport(stableRedisFacade); @@ -240,9 +241,9 @@ public async Task WaitUntilNodeProcessingRequestFlatLines_WhenConnectionInterrup var log = new TestContextLogCreator("NodeHeartBeat", LogLevel.Trace).CreateNewForPrefix(""); var guid = Guid.NewGuid(); - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); - await using var unstableRedisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder.ListeningPort, guid); - await using var stableRedisFacade = RedisFacadeBuilder.CreateRedisFacade(RedisTestHost.Port(), guid); + using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); + await using var unstableRedisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder, guid); + await using var stableRedisFacade = RedisFacadeBuilder.CreateRedisFacade(prefix: guid); var unstableRedisTransport = new HalibutRedisTransport(unstableRedisFacade); var stableRedisTransport = new HalibutRedisTransport(stableRedisFacade); @@ -321,48 +322,40 @@ public async Task WhenHeartBeatSenderConnectionRecovery_ShouldResumeNormalHeartb var log = new TestContextLogCreator("NodeHeartBeat", LogLevel.Trace).CreateNewForPrefix(""); var guid = Guid.NewGuid(); - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); - await using var unstableRedisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder.ListeningPort, guid); - await using var stableRedisFacade = RedisFacadeBuilder.CreateRedisFacade(RedisTestHost.Port(), guid); + using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); + await using var unstableRedisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder, guid); + await using var stableRedisFacade = RedisFacadeBuilder.CreateRedisFacade(prefix: guid); var unstableRedisTransport = new HalibutRedisTransport(unstableRedisFacade); var stableRedisTransport = new HalibutRedisTransport(stableRedisFacade); var heartbeatTimestamps = new ConcurrentBag(); + var heartBeatsReceived = new AsyncManualResetEvent(false); + await using var subscription = await stableRedisTransport.SubscribeToNodeHeartBeatChannel( endpoint, requestActivityId, HalibutQueueNodeSendingPulses.Receiver, async () => { await Task.CompletedTask; heartbeatTimestamps.Add(DateTimeOffset.Now); + heartBeatsReceived.Set(); }, CancellationToken); // Act await using var heartBeatSender = new NodeHeartBeatSender(endpoint, requestActivityId, unstableRedisTransport, log, HalibutQueueNodeSendingPulses.Receiver, TimeSpan.FromSeconds(1)); // Wait for initial heartbeats (normal 15s interval) - await Task.Delay(TimeSpan.FromSeconds(5), CancellationToken); + await heartBeatsReceived.WaitAsync(CancellationToken); // Interrupt connection to trigger panic mode (7s interval) portForwarder.EnterKillNewAndExistingConnectionsMode(); await Task.Delay(TimeSpan.FromSeconds(2), CancellationToken); + heartBeatsReceived.Reset(); // Restore connection portForwarder.ReturnToNormalMode(); - // Wait for recovery and return to normal intervals - await Task.Delay(TimeSpan.FromSeconds(20), CancellationToken); - - // Assert - heartbeatTimestamps.Should().NotBeEmpty("Should have received heartbeats after recovery"); - - // Verify we have heartbeats spanning the recovery period - var timestamps = heartbeatTimestamps.ToArray(); - if (timestamps.Length > 1) - { - var timeSpan = timestamps.Max() - timestamps.Min(); - timeSpan.Should().BeGreaterThan(TimeSpan.FromSeconds(10), "Should have heartbeats over recovery period"); - } + await heartBeatsReceived.WaitAsync(CancellationToken); } [Test] diff --git a/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs b/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs index 5dfec09de..cbb2df980 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs @@ -20,11 +20,14 @@ using FluentAssertions; using Halibut.Logging; using Halibut.Queue.Redis; +using Halibut.Tests.Queue.Redis.Utils; +using Halibut.Tests.Support; using Halibut.Tests.Support.Logging; using Halibut.Tests.TestSetup.Redis; using Halibut.Util; using NUnit.Framework; using Octopus.TestPortForwarder; +using Try = Halibut.Util.Try; namespace Halibut.Tests.Queue.Redis { @@ -41,7 +44,7 @@ static RedisFacade CreateRedisFacade(int? port = 0, Guid? guid = null) [Test] public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_SometimeLaterOnWeCanDoBasicCalls() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); + using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); @@ -61,7 +64,7 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe [Test] public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelyDoBasicCalls() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); + using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); @@ -80,7 +83,7 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe [Test] public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelyPublishToChannel() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); + using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); @@ -97,7 +100,7 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe [Test] public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelySetInHash() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); + using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); @@ -114,7 +117,7 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe [Test] public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelyTryGetAndDeleteFromHash() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); + using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); @@ -132,7 +135,7 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe [Test] public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelyListRightPush() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); + using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); @@ -149,7 +152,7 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe [Test] public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelyListLeftPop() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); + using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); @@ -167,7 +170,7 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe [Test] public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelySetString() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); + using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); @@ -184,7 +187,7 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe [Test] public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelyGetString() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); + using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); @@ -202,7 +205,7 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe [Test] public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelyHashContainsKey() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); + using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); @@ -220,12 +223,11 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe [Test] public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndWeTryToSubscribe_WhenTheConnectionIsRestored_WeCanReceiveMessages() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); + using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); var redisLogCreator = new TestContextLogCreator("Redis", LogLevel.Trace); - var guid = Guid.NewGuid().ToString(); - await using var redisViaPortForwarder = new RedisFacade("localhost:" + portForwarder.ListeningPort, guid, redisLogCreator.CreateNewForPrefix("Unstable")); - await using var redisStableConnection = new RedisFacade("localhost:" + RedisTestHost.Port(), guid, redisLogCreator.CreateNewForPrefix("Stable")); - + var guid = Guid.NewGuid(); + await using var redisViaPortForwarder = RedisFacadeBuilder.CreateRedisFacade(portForwarder: portForwarder, prefix: guid); + await using var redisStableConnection = RedisFacadeBuilder.CreateRedisFacade(prefix: guid); await redisViaPortForwarder.SetString("Establish connection", "before we subscribe", TimeSpan.FromMinutes(1), CancellationToken); portForwarder.EnterKillNewAndExistingConnectionsMode(); @@ -262,11 +264,11 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndWeTryT [Test] public async Task WhenTheConnectionIsNeverEstablished_AndWeTryToSubscribe_WhenTheConnectionIsRestored_WeCanReceiveMessages() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); + using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); var redisLogCreator = new TestContextLogCreator("Redis", LogLevel.Trace); - var guid = Guid.NewGuid().ToString(); - await using var redisViaPortForwarder = new RedisFacade("localhost:" + portForwarder.ListeningPort, guid, redisLogCreator.CreateNewForPrefix("Unstable")); - await using var redisStableConnection = new RedisFacade("localhost:" + RedisTestHost.Port(), guid, redisLogCreator.CreateNewForPrefix("Stable")); + var guid = Guid.NewGuid(); + await using var redisViaPortForwarder = RedisFacadeBuilder.CreateRedisFacade(portForwarder: portForwarder, prefix: guid); + await using var redisStableConnection = RedisFacadeBuilder.CreateRedisFacade(prefix: guid); portForwarder.EnterKillNewAndExistingConnectionsMode(); @@ -304,7 +306,7 @@ public async Task WhenTheConnectionIsNeverEstablished_AndWeTryToSubscribe_WhenTh [WindowsTest] public async Task WhenSubscribedAndTheConnectionGoesDown_WhenTheConnectionIsRestored_MessagesCanEventuallyBeSentToTheSubscriberAgain() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); + using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); var redisLogCreator = new TestContextLogCreator("Redis", LogLevel.Trace); diff --git a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs index de67c6d77..894e150e8 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs @@ -163,8 +163,8 @@ public async Task WhenEnteringTheQueue_AndRedisIsUnavailable_ARetryableException var endpoint = new Uri("poll://" + Guid.NewGuid()); var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); - await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder.ListeningPort, null); + using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder, null); redisFacade.MaxDurationToRetryFor = TimeSpan.FromSeconds(1); var redisTransport = new HalibutRedisTransport(redisFacade); @@ -189,8 +189,8 @@ public async Task WhenEnteringTheQueue_AndRedisIsUnavailableAndDataLoseOccurs_AR var endpoint = new Uri("poll://" + Guid.NewGuid()); var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); - await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder.ListeningPort, null); + using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder, null); redisFacade.MaxDurationToRetryFor = TimeSpan.FromSeconds(1); var redisDataLoseDetector = new CancellableDataLossWatchForRedisLosingAllItsData(); @@ -455,8 +455,8 @@ public async Task WhenTheReceiversConnectionToRedisIsInterruptedAndRestoredBefor var guid = Guid.NewGuid(); await using var redisFacadeSender = RedisFacadeBuilder.CreateRedisFacade(prefix: guid); - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); - await using var redisFacadeReceiver = RedisFacadeBuilder.CreateRedisFacade(portForwarder.ListeningPort, guid); + using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); + await using var redisFacadeReceiver = RedisFacadeBuilder.CreateRedisFacade(portForwarder, guid); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); @@ -533,8 +533,8 @@ public async Task WhenTheSendersConnectionToRedisIsBrieflyInterruptedWhileSendin var guid = Guid.NewGuid(); await using var redisFacadeReceiver = RedisFacadeBuilder.CreateRedisFacade(prefix: guid); - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); - await using var redisFacadeSender = RedisFacadeBuilder.CreateRedisFacade(portForwarder.ListeningPort, guid); + using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); + await using var redisFacadeSender = RedisFacadeBuilder.CreateRedisFacade(portForwarder, guid); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); @@ -583,8 +583,8 @@ public async Task WhenTheReceiverDequeuesWorkAndThenDisconnectsFromRedisForEver_ await using var stableRedisConnection = RedisFacadeBuilder.CreateRedisFacade(prefix: guid); - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); - await using var unstableRedisConnection = RedisFacadeBuilder.CreateRedisFacade(portForwarder.ListeningPort, guid); + using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); + await using var unstableRedisConnection = RedisFacadeBuilder.CreateRedisFacade(portForwarder, guid); var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(stableRedisConnection), messageReaderWriter, halibutTimeoutAndLimits); var node2Receiver = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(unstableRedisConnection), messageReaderWriter, halibutTimeoutAndLimits); @@ -636,8 +636,8 @@ public async Task WhenTheReceiverDequeuesWork_AndTheSenderDisconnects_AndNeverRe await using var stableRedisConnection = RedisFacadeBuilder.CreateRedisFacade(prefix: guid); - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); - await using var unstableRedisConnection = RedisFacadeBuilder.CreateRedisFacade(portForwarder.ListeningPort, guid); + using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); + await using var unstableRedisConnection = RedisFacadeBuilder.CreateRedisFacade(portForwarder, guid); var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(unstableRedisConnection), messageReaderWriter, halibutTimeoutAndLimits); var node2Receiver = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(stableRedisConnection), messageReaderWriter, halibutTimeoutAndLimits); @@ -678,8 +678,8 @@ public async Task WhenTheSenderBrieflyDisconnectsFromRedisRightWhenTheReceiverSe var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); await using var stableConnection = RedisFacadeBuilder.CreateRedisFacade(prefix: guid); - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); - await using var unreliableConnection = RedisFacadeBuilder.CreateRedisFacade(portForwarder.ListeningPort, guid); + using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); + await using var unreliableConnection = RedisFacadeBuilder.CreateRedisFacade(portForwarder, guid); var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(unreliableConnection), messageReaderWriter, new HalibutTimeoutsAndLimits()); var node2Receiver = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(stableConnection), messageReaderWriter, new HalibutTimeoutsAndLimits()); diff --git a/source/Halibut.Tests/Queue/Redis/Utils/RedisFacadeBuilder.cs b/source/Halibut.Tests/Queue/Redis/Utils/RedisFacadeBuilder.cs index cc1b264d0..05094fbd1 100644 --- a/source/Halibut.Tests/Queue/Redis/Utils/RedisFacadeBuilder.cs +++ b/source/Halibut.Tests/Queue/Redis/Utils/RedisFacadeBuilder.cs @@ -17,15 +17,23 @@ using Halibut.Queue.Redis; using Halibut.Tests.Support.Logging; using Halibut.Tests.TestSetup.Redis; +using Octopus.TestPortForwarder; namespace Halibut.Tests.Queue.Redis.Utils { public class RedisFacadeBuilder { - public static RedisFacade CreateRedisFacade(int? port = 0, Guid? prefix = null) + public static RedisFacade CreateRedisFacade(string? host = null, int? port = 0, Guid? prefix = null) { port = port == 0 ? RedisTestHost.Port() : port; - return new RedisFacade(RedisTestHost.RedisHost + ":" + port, (prefix ?? Guid.NewGuid()).ToString(), new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix("")); + return new RedisFacade((host??RedisTestHost.RedisHost) + ":" + port, (prefix ?? Guid.NewGuid()).ToString(), new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix("")); + } + + public static RedisFacade CreateRedisFacade(PortForwarder portForwarder, Guid? prefix = null) + { + return CreateRedisFacade(host: portForwarder.PublicEndpoint.Host, + port: portForwarder.ListeningPort, + prefix: prefix); } } } \ No newline at end of file diff --git a/source/Halibut.Tests/Queue/Redis/WatchForRedisLosingAllItsDataFixture.cs b/source/Halibut.Tests/Queue/Redis/WatchForRedisLosingAllItsDataFixture.cs index 2a55caab9..d95058d31 100644 --- a/source/Halibut.Tests/Queue/Redis/WatchForRedisLosingAllItsDataFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/WatchForRedisLosingAllItsDataFixture.cs @@ -19,9 +19,9 @@ public class WatchForRedisLosingAllItsDataFixture : BaseTest [Test] public async Task WhenTheConnectionToRedisCanNotBeCreated_WhenAskingForALostDataCancellationToken_ATimeoutOccurs() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); + using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); portForwarder.EnterKillNewAndExistingConnectionsMode(); - await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder.ListeningPort, null); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder, null); await using var watcher = new WatchForRedisLosingAllItsData(redisFacade, HalibutLog, watchInterval:TimeSpan.FromSeconds(1)); @@ -31,9 +31,9 @@ public async Task WhenTheConnectionToRedisCanNotBeCreated_WhenAskingForALostData [Test] public async Task WhenTheConnectionToRedisIsInitiallyDown_WhenAskingForALostDataCancellationToken_AndTheConnectionToRedisReturns_TheCTIsReturned() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); + using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); portForwarder.EnterKillNewAndExistingConnectionsMode(); - await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder.ListeningPort, null); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder, null); await using var watcher = new WatchForRedisLosingAllItsData(redisFacade, HalibutLog, watchInterval:TimeSpan.FromSeconds(1)); var _ = Task.Run(async () => @@ -49,9 +49,9 @@ public async Task WhenTheConnectionToRedisIsInitiallyDown_WhenAskingForALostData [Test] public async Task WatchForARealRedisLosingAllOfItsData_TimesOutWhenWaitingForCTWhenNoConnectionToRedisCanBeEstablished() { - using var portForwarder = PortForwarderBuilder.ForwardingToLocalPort(RedisTestHost.Port(), Logger).Build(); + using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); portForwarder.EnterKillNewAndExistingConnectionsMode(); - await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder.ListeningPort, null); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder, null); await using var watcher = new WatchForRedisLosingAllItsData(redisFacade, HalibutLog, watchInterval:TimeSpan.FromSeconds(1)); @@ -84,7 +84,7 @@ public async Task WhenRedisLosesAllOfIts_TheWatcherShouldDetectTheDataLose() Logger.Information("Redis container started successfully with connection string: {ConnectionString}", container.ConnectionString); // Create RedisFacade connected to the containerized Redis - await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(container.RedisPort, null); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(host: "localhost", container.RedisPort); await using var watcher = new WatchForRedisLosingAllItsData(redisFacade, HalibutLog, watchInterval:TimeSpan.FromSeconds(1)); diff --git a/source/Halibut.Tests/TestSetup/Redis/EnsureRedisIsAvailableSetupFixture.cs b/source/Halibut.Tests/TestSetup/Redis/EnsureRedisIsAvailableSetupFixture.cs index 0c85fe08a..06a013404 100644 --- a/source/Halibut.Tests/TestSetup/Redis/EnsureRedisIsAvailableSetupFixture.cs +++ b/source/Halibut.Tests/TestSetup/Redis/EnsureRedisIsAvailableSetupFixture.cs @@ -49,6 +49,7 @@ public void OneTimeSetUp(ILogger logger) var ts = multiplexer.GetDatabase().Ping(); RedisTestHost.SetPort(RedisPortToTry); RedisTestHost.RedisHost = RedisHost; + logger.Information("Able to connect to redis using {Host}:{Port}", RedisHost, RedisPortToTry); return; } catch From d63c83d01caa0870ac976549f6d820db55e83895 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Mon, 18 Aug 2025 09:38:25 +1000 Subject: [PATCH 065/137] Add back net48 --- source/Halibut.Tests/Halibut.Tests.csproj | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/Halibut.Tests/Halibut.Tests.csproj b/source/Halibut.Tests/Halibut.Tests.csproj index 0e31f3e3e..9eb552ceb 100644 --- a/source/Halibut.Tests/Halibut.Tests.csproj +++ b/source/Halibut.Tests/Halibut.Tests.csproj @@ -12,7 +12,7 @@ true - net8.0 + net48;net8.0 net8.0 From b049d975d64fd71832390e703c9001e420d5edf1 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Mon, 18 Aug 2025 09:39:17 +1000 Subject: [PATCH 066/137] . --- .../Support/PortForwardingToRedisBuilder.cs | 15 +++++++++++++++ 1 file changed, 15 insertions(+) create mode 100644 source/Halibut.Tests/Support/PortForwardingToRedisBuilder.cs diff --git a/source/Halibut.Tests/Support/PortForwardingToRedisBuilder.cs b/source/Halibut.Tests/Support/PortForwardingToRedisBuilder.cs new file mode 100644 index 000000000..426835554 --- /dev/null +++ b/source/Halibut.Tests/Support/PortForwardingToRedisBuilder.cs @@ -0,0 +1,15 @@ +using System; +using Halibut.Tests.TestSetup.Redis; +using Octopus.TestPortForwarder; +using Serilog; + +namespace Halibut.Tests.Support +{ + public static class PortForwardingToRedisBuilder + { + public static PortForwarder ForwardingToRedis(ILogger logger) + { + return new PortForwarderBuilder(new Uri("http://" + RedisTestHost.RedisHost + ":" + RedisTestHost.Port()), logger).Build(); + } + } +} From 992a515d3b55fb4a7af4c5218bd18fce70b90f27 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Mon, 18 Aug 2025 10:30:23 +1000 Subject: [PATCH 067/137] . --- source/Halibut/ServiceModel/PendingRequest.cs | 249 ------------------ 1 file changed, 249 deletions(-) delete mode 100644 source/Halibut/ServiceModel/PendingRequest.cs diff --git a/source/Halibut/ServiceModel/PendingRequest.cs b/source/Halibut/ServiceModel/PendingRequest.cs deleted file mode 100644 index be9ec5baa..000000000 --- a/source/Halibut/ServiceModel/PendingRequest.cs +++ /dev/null @@ -1,249 +0,0 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -using System; -using System.Threading; -using System.Threading.Tasks; -using Halibut.Diagnostics; -using Halibut.Exceptions; -using Halibut.Transport; -using Halibut.Transport.Protocol; -using Nito.AsyncEx; - -namespace Halibut.ServiceModel -{ - public class PendingRequestOld : IDisposable - { - readonly RequestMessage request; - readonly ILog log; - readonly AsyncManualResetEvent responseWaiter = new(false); - readonly SemaphoreSlim transferLock = new(1, 1); - //bool transferBegun; - AsyncManualResetEvent requestCollected = new(false); - bool completed; - readonly CancellationTokenSource pendingRequestCancellationTokenSource; - ResponseMessage? response; - - public PendingRequestOld(RequestMessage request, ILog log) - { - this.request = request; - this.log = log; - - pendingRequestCancellationTokenSource = new CancellationTokenSource(); - PendingRequestCancellationToken = pendingRequestCancellationTokenSource.Token; - } - - public Task WaitForRequestToBeMarkedAsCollected(CancellationToken cancellationToken) => requestCollected.WaitAsync(cancellationToken); - - public bool HasRequestBeenMarkedAsCollected => requestCollected.IsSet; - - public RequestMessage Request => request; - - /// - /// - /// - /// - /// This will be called either when the pick-up timeout has elapsed OR if the Cancellation Token has been triggered. - /// This gives the user an opportunity to remove the pending request from shared places and optionally - /// call BeginTransfer - /// - /// - public async Task WaitUntilComplete(Func timePendingRequestCanBeOnTheQueueHasElapsed, CancellationToken cancellationToken) - { - log.Write(EventType.MessageExchange, "Request {0} was queued", request); - - bool responseSet; - var cancelled = false; - - try - { - responseSet = await WaitForResponseToBeSet( - request.Destination.PollingRequestQueueTimeout, - // Don't cancel a dequeued request as we need to wait PollingRequestMaximumMessageProcessingTimeout for it to complete - cancelTheRequestWhenTransferHasBegun: false, - cancellationToken); - - if (responseSet) - { - log.Write(EventType.MessageExchange, "Request {0} was collected by the polling endpoint", request); - return; - } - } - catch (RequestCancelledException) - { - cancelled = true; - if(!requestCollected.IsSet) await timePendingRequestCanBeOnTheQueueHasElapsed(); - using (await transferLock.LockAsync(CancellationToken.None)) - { - if (!requestCollected.IsSet) - { - completed = true; - log.Write(EventType.MessageExchange, "Request {0} was cancelled before it could be collected by the polling endpoint", request); - throw; - } - } - } - - if(!requestCollected.IsSet) await timePendingRequestCanBeOnTheQueueHasElapsed(); - var waitForTransferToComplete = false; - using (await transferLock.LockAsync(CancellationToken.None)) - { - if (requestCollected.IsSet) - { - waitForTransferToComplete = true; - } - else - { - completed = true; - } - } - - if (waitForTransferToComplete) - { - responseSet = await WaitForResponseToBeSet( - null, - // Cancel the dequeued request to force Reads and Writes to be cancelled - cancelTheRequestWhenTransferHasBegun: true, - cancellationToken); - - if (responseSet) - { - // We end up here when the request is cancelled but already being transferred so we need to adjust the log message accordingly - if (cancelled) - { - log.Write(EventType.MessageExchange, "Request {0} was collected by the polling endpoint", request); - } - else - { - log.Write(EventType.MessageExchange, "Request {0} was eventually collected by the polling endpoint", request); - } - } - else - { - if (cancellationToken.IsCancellationRequested) - { - log.Write(EventType.MessageExchange, "Request {0} was cancelled before a response was received", request); - SetResponse(ResponseMessage.FromException( - request, - new TimeoutException($"A request was sent to a polling endpoint, the polling endpoint collected it but the request was cancelled before the polling endpoint responded."), - ConnectionState.Connecting)); - } - else - { - // This should never happen. - log.Write(EventType.MessageExchange, "Request {0} had an internal error, unexpectedly stopped waiting for the response.", request); - SetResponse(ResponseMessage.FromException( - request, - new PendingRequestQueueInternalException($"Request {request.Id} had an internal error, unexpectedly stopped waiting for the response."))); - } - } - } - else - { - log.Write(EventType.MessageExchange, "Request {0} timed out before it could be collected by the polling endpoint", request); - SetResponse(ResponseMessage.FromException( - request, - new TimeoutException($"A request was sent to a polling endpoint, but the polling endpoint did not collect the request within the allowed time ({request.Destination.PollingRequestQueueTimeout}), so the request timed out."), - ConnectionState.Connecting)); - } - } - - async Task WaitForResponseToBeSet( - TimeSpan? timeout, - bool cancelTheRequestWhenTransferHasBegun, - CancellationToken cancellationToken) - { - using var timeoutCancellationTokenSource = timeout.HasValue ? new CancellationTokenSource(timeout.Value) : new CancellationTokenSource(); - using var linkedTokenSource = CancellationTokenSource.CreateLinkedTokenSource(timeoutCancellationTokenSource.Token, cancellationToken); - - try - { - await responseWaiter.WaitAsync(linkedTokenSource.Token); - } - catch (OperationCanceledException ex) - { - using (await transferLock.LockAsync(CancellationToken.None)) - { - if (requestCollected.IsSet && cancelTheRequestWhenTransferHasBegun) - { - // Cancel the dequeued request. This will cause co-operative cancellation on the thread dequeuing the request - pendingRequestCancellationTokenSource.Cancel(); - } - else if (!requestCollected.IsSet) - { - // Cancel the queued request. This will flag the request as cancelled to stop it being dequeued - //pendingRequestCancellationTokenSource.Cancel(); - } - - if (timeoutCancellationTokenSource.IsCancellationRequested) - { - return false; - } - - throw requestCollected.IsSet ? new TransferringRequestCancelledException(ex) : new ConnectingRequestCancelledException(ex); - } - } - - return true; - } - - public async Task RequestHasBeenCollectedAndWillBeTransferred() - { - // The PendingRequest is Disposed at the end of QueueAndWaitAsync but a race condition - // exists in the current approach that means DequeueAsync could pick this request up after - // it has been disposed. At that point we are no longer interested in the PendingRequest so - // this is "ok" and wrapping BeginTransfer in a try..catch.. ensures we don't error if the - // race condition occurs and also stops the polling tentacle dequeuing the request successfully. - try - { - using (await transferLock.LockAsync(CancellationToken.None)) - { - // Check if the request has already been completed or if the request has been cancelled - // to ensure we don't dequeue an already completed or already cancelled request - if (completed || pendingRequestCancellationTokenSource.IsCancellationRequested) - { - return false; - } - - requestCollected.Set(); - return true; - } - } - catch (ObjectDisposedException) - { - return false; - } - } - - public ResponseMessage Response => response ?? throw new InvalidOperationException("Response has not been set."); - public CancellationToken PendingRequestCancellationToken { get; } - - public void SetResponse(ResponseMessage response) - { - lock (responseWaiter) - { - if(this.response != null) return; - this.response = response; - responseWaiter.Set(); - } - } - - public void Dispose() - { - pendingRequestCancellationTokenSource?.Dispose(); - transferLock?.Dispose(); - } - } - -} \ No newline at end of file From a29f0d46e660eb64c642f58abde4f085edea70a0 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Mon, 18 Aug 2025 10:39:37 +1000 Subject: [PATCH 068/137] . --- .../Builders/IPendingRequestQueueBuilder.cs | 14 -------------- .../ServiceModel/PendingRequestQueueFixture.cs | 1 - source/Halibut/Util/DisposableCollection.cs | 14 -------------- 3 files changed, 29 deletions(-) diff --git a/source/Halibut.Tests/Builders/IPendingRequestQueueBuilder.cs b/source/Halibut.Tests/Builders/IPendingRequestQueueBuilder.cs index 78c7459e1..171076bbb 100644 --- a/source/Halibut.Tests/Builders/IPendingRequestQueueBuilder.cs +++ b/source/Halibut.Tests/Builders/IPendingRequestQueueBuilder.cs @@ -1,17 +1,3 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - using System; using Halibut.Diagnostics; diff --git a/source/Halibut.Tests/ServiceModel/PendingRequestQueueFixture.cs b/source/Halibut.Tests/ServiceModel/PendingRequestQueueFixture.cs index c3f05bf32..c9682a116 100644 --- a/source/Halibut.Tests/ServiceModel/PendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/ServiceModel/PendingRequestQueueFixture.cs @@ -15,7 +15,6 @@ namespace Halibut.Tests.ServiceModel { - [Ignore("REDISTODO")] public class PendingRequestQueueFixture : BaseTest { [Test] diff --git a/source/Halibut/Util/DisposableCollection.cs b/source/Halibut/Util/DisposableCollection.cs index f79ea1a06..61b900e3d 100644 --- a/source/Halibut/Util/DisposableCollection.cs +++ b/source/Halibut/Util/DisposableCollection.cs @@ -1,17 +1,3 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - using System; using System.Collections.Concurrent; using System.Collections.Generic; From ca4b5e2db6c57253db4b0e296a88aa896520f3c1 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Mon, 18 Aug 2025 10:44:31 +1000 Subject: [PATCH 069/137] . --- source/Halibut.Tests/Util/AssertThrowsAny.cs | 14 -------------- source/Halibut/Queue/QueueMessageSerializer.cs | 14 -------------- .../IStoreDataStreamsForDistributedQueues.cs | 14 -------------- ...InMemoryStoreDataStreamsForDistributedQueues.cs | 14 -------------- ...GetDataLoseTokenInTimeHalibutClientException.cs | 14 -------------- ...rtingDataIntoRedisHalibutPendingRequestQueue.cs | 14 -------------- ...eparingRequestForQueueHalibutClientException.cs | 14 -------------- .../RedisDataLoseHalibutClientException.cs | 14 -------------- .../RedisQueueShutdownClientException.cs | 14 -------------- source/Halibut/Queue/Redis/IMessageReaderWriter.cs | 14 -------------- .../Queue/Redis/IWatchForRedisLosingAllItsData.cs | 14 -------------- source/Halibut/Queue/Redis/MessageReaderWriter.cs | 14 -------------- source/Halibut/Queue/Redis/RedisFacade.cs | 14 -------------- source/Halibut/Queue/Redis/RedisPendingRequest.cs | 14 -------------- .../Queue/Redis/WatchForRedisLosingAllItsData.cs | 14 -------------- source/Halibut/Util/AwaitAllAndIgnoreException.cs | 14 -------------- source/Halibut/Util/FuncAsyncDisposable.cs | 14 -------------- source/Halibut/Util/StringExtensionMethods.cs | 14 -------------- 18 files changed, 252 deletions(-) diff --git a/source/Halibut.Tests/Util/AssertThrowsAny.cs b/source/Halibut.Tests/Util/AssertThrowsAny.cs index 04ea81f09..52a915e40 100644 --- a/source/Halibut.Tests/Util/AssertThrowsAny.cs +++ b/source/Halibut.Tests/Util/AssertThrowsAny.cs @@ -1,17 +1,3 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - using System; using System.Threading.Tasks; using NUnit.Framework; diff --git a/source/Halibut/Queue/QueueMessageSerializer.cs b/source/Halibut/Queue/QueueMessageSerializer.cs index 96bb5b491..75555f623 100644 --- a/source/Halibut/Queue/QueueMessageSerializer.cs +++ b/source/Halibut/Queue/QueueMessageSerializer.cs @@ -1,17 +1,3 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - using System; using System.Collections.Generic; using System.Globalization; diff --git a/source/Halibut/Queue/QueuedDataStreams/IStoreDataStreamsForDistributedQueues.cs b/source/Halibut/Queue/QueuedDataStreams/IStoreDataStreamsForDistributedQueues.cs index 950e61cc9..1dfdf2f88 100644 --- a/source/Halibut/Queue/QueuedDataStreams/IStoreDataStreamsForDistributedQueues.cs +++ b/source/Halibut/Queue/QueuedDataStreams/IStoreDataStreamsForDistributedQueues.cs @@ -1,17 +1,3 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - using System; using System.Collections.Generic; using System.Threading; diff --git a/source/Halibut/Queue/QueuedDataStreams/InMemoryStoreDataStreamsForDistributedQueues.cs b/source/Halibut/Queue/QueuedDataStreams/InMemoryStoreDataStreamsForDistributedQueues.cs index 2ab9625f7..e83a96684 100644 --- a/source/Halibut/Queue/QueuedDataStreams/InMemoryStoreDataStreamsForDistributedQueues.cs +++ b/source/Halibut/Queue/QueuedDataStreams/InMemoryStoreDataStreamsForDistributedQueues.cs @@ -1,17 +1,3 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - using System; using System.Collections.Generic; diff --git a/source/Halibut/Queue/Redis/Exceptions/CouldNotGetDataLoseTokenInTimeHalibutClientException.cs b/source/Halibut/Queue/Redis/Exceptions/CouldNotGetDataLoseTokenInTimeHalibutClientException.cs index 23c71c2c9..a77633875 100644 --- a/source/Halibut/Queue/Redis/Exceptions/CouldNotGetDataLoseTokenInTimeHalibutClientException.cs +++ b/source/Halibut/Queue/Redis/Exceptions/CouldNotGetDataLoseTokenInTimeHalibutClientException.cs @@ -1,17 +1,3 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - using System; namespace Halibut.Queue.Redis diff --git a/source/Halibut/Queue/Redis/Exceptions/ErrorOccuredWhenInsertingDataIntoRedisHalibutPendingRequestQueue.cs b/source/Halibut/Queue/Redis/Exceptions/ErrorOccuredWhenInsertingDataIntoRedisHalibutPendingRequestQueue.cs index b61d10b97..511bda9e3 100644 --- a/source/Halibut/Queue/Redis/Exceptions/ErrorOccuredWhenInsertingDataIntoRedisHalibutPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/Exceptions/ErrorOccuredWhenInsertingDataIntoRedisHalibutPendingRequestQueue.cs @@ -1,17 +1,3 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - using System; namespace Halibut.Queue.Redis.Exceptions diff --git a/source/Halibut/Queue/Redis/Exceptions/ErrorWhilePreparingRequestForQueueHalibutClientException.cs b/source/Halibut/Queue/Redis/Exceptions/ErrorWhilePreparingRequestForQueueHalibutClientException.cs index 11a99d35a..a4e50d769 100644 --- a/source/Halibut/Queue/Redis/Exceptions/ErrorWhilePreparingRequestForQueueHalibutClientException.cs +++ b/source/Halibut/Queue/Redis/Exceptions/ErrorWhilePreparingRequestForQueueHalibutClientException.cs @@ -1,17 +1,3 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - using System; namespace Halibut.Queue.Redis.Exceptions diff --git a/source/Halibut/Queue/Redis/Exceptions/RedisDataLoseHalibutClientException.cs b/source/Halibut/Queue/Redis/Exceptions/RedisDataLoseHalibutClientException.cs index 662d00ef9..3a321c938 100644 --- a/source/Halibut/Queue/Redis/Exceptions/RedisDataLoseHalibutClientException.cs +++ b/source/Halibut/Queue/Redis/Exceptions/RedisDataLoseHalibutClientException.cs @@ -1,17 +1,3 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - using System; namespace Halibut.Queue.Redis diff --git a/source/Halibut/Queue/Redis/Exceptions/RedisQueueShutdownClientException.cs b/source/Halibut/Queue/Redis/Exceptions/RedisQueueShutdownClientException.cs index efede93f3..da094c92a 100644 --- a/source/Halibut/Queue/Redis/Exceptions/RedisQueueShutdownClientException.cs +++ b/source/Halibut/Queue/Redis/Exceptions/RedisQueueShutdownClientException.cs @@ -1,17 +1,3 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - namespace Halibut.Queue.Redis { public class RedisQueueShutdownClientException : HalibutClientException diff --git a/source/Halibut/Queue/Redis/IMessageReaderWriter.cs b/source/Halibut/Queue/Redis/IMessageReaderWriter.cs index 9fd1b1c9e..f35b0583b 100644 --- a/source/Halibut/Queue/Redis/IMessageReaderWriter.cs +++ b/source/Halibut/Queue/Redis/IMessageReaderWriter.cs @@ -1,17 +1,3 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - using System.Threading; using System.Threading.Tasks; using Halibut.Transport.Protocol; diff --git a/source/Halibut/Queue/Redis/IWatchForRedisLosingAllItsData.cs b/source/Halibut/Queue/Redis/IWatchForRedisLosingAllItsData.cs index 0c8f8f8fd..1a0d7f1f2 100644 --- a/source/Halibut/Queue/Redis/IWatchForRedisLosingAllItsData.cs +++ b/source/Halibut/Queue/Redis/IWatchForRedisLosingAllItsData.cs @@ -1,17 +1,3 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - using System; using System.Threading; using System.Threading.Tasks; diff --git a/source/Halibut/Queue/Redis/MessageReaderWriter.cs b/source/Halibut/Queue/Redis/MessageReaderWriter.cs index 00feb9c2b..ff6e8fda5 100644 --- a/source/Halibut/Queue/Redis/MessageReaderWriter.cs +++ b/source/Halibut/Queue/Redis/MessageReaderWriter.cs @@ -1,17 +1,3 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - using System.Threading; using System.Threading.Tasks; using Halibut.Queue.QueuedDataStreams; diff --git a/source/Halibut/Queue/Redis/RedisFacade.cs b/source/Halibut/Queue/Redis/RedisFacade.cs index 84a19ec71..726efaead 100644 --- a/source/Halibut/Queue/Redis/RedisFacade.cs +++ b/source/Halibut/Queue/Redis/RedisFacade.cs @@ -1,17 +1,3 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - using System; using System.Diagnostics; using System.Threading; diff --git a/source/Halibut/Queue/Redis/RedisPendingRequest.cs b/source/Halibut/Queue/Redis/RedisPendingRequest.cs index f58fe40d2..e94955382 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequest.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequest.cs @@ -1,17 +1,3 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - #if NET8_0_OR_GREATER using System; using System.Threading; diff --git a/source/Halibut/Queue/Redis/WatchForRedisLosingAllItsData.cs b/source/Halibut/Queue/Redis/WatchForRedisLosingAllItsData.cs index b53e34c8e..e5d1bcb51 100644 --- a/source/Halibut/Queue/Redis/WatchForRedisLosingAllItsData.cs +++ b/source/Halibut/Queue/Redis/WatchForRedisLosingAllItsData.cs @@ -1,17 +1,3 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - #if NET8_0_OR_GREATER using System; using System.Threading; diff --git a/source/Halibut/Util/AwaitAllAndIgnoreException.cs b/source/Halibut/Util/AwaitAllAndIgnoreException.cs index 03dd83822..6891c5329 100644 --- a/source/Halibut/Util/AwaitAllAndIgnoreException.cs +++ b/source/Halibut/Util/AwaitAllAndIgnoreException.cs @@ -1,17 +1,3 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - using System; using System.Collections.Generic; using System.Threading.Tasks; diff --git a/source/Halibut/Util/FuncAsyncDisposable.cs b/source/Halibut/Util/FuncAsyncDisposable.cs index 0f1e752b5..23ff504b6 100644 --- a/source/Halibut/Util/FuncAsyncDisposable.cs +++ b/source/Halibut/Util/FuncAsyncDisposable.cs @@ -1,17 +1,3 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - using System; using System.Threading.Tasks; diff --git a/source/Halibut/Util/StringExtensionMethods.cs b/source/Halibut/Util/StringExtensionMethods.cs index 1e77ef3f6..ed6785a53 100644 --- a/source/Halibut/Util/StringExtensionMethods.cs +++ b/source/Halibut/Util/StringExtensionMethods.cs @@ -1,17 +1,3 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - using System; using System.Diagnostics.CodeAnalysis; From faabbeb0fe2e6d0303d80c02d2ba29f92ca8675e Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Mon, 18 Aug 2025 10:47:13 +1000 Subject: [PATCH 070/137] . --- .../Queue/QueueMessageSerializerBuilder.cs | 14 -------------- .../Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs | 14 -------------- ...ellableDataLossWatchForRedisLosingAllItsData.cs | 14 -------------- .../Utils/MessageReaderWriterExtensionsMethods.cs | 14 -------------- .../Queue/Redis/Utils/RedisFacadeBuilder.cs | 14 -------------- .../TestSetup/BumpThreadPoolForAllTests.cs | 14 -------------- source/Halibut.Tests/TestSetup/ISetupFixture.cs | 14 -------------- .../Redis/CreateRedisDockerContainerForTests.cs | 14 -------------- .../Redis/EnsureRedisIsAvailableSetupFixture.cs | 14 -------------- .../Halibut.Tests/TestSetup/Redis/RedisTestHost.cs | 14 -------------- 10 files changed, 140 deletions(-) diff --git a/source/Halibut.Tests/Queue/QueueMessageSerializerBuilder.cs b/source/Halibut.Tests/Queue/QueueMessageSerializerBuilder.cs index 5a0194b3f..35dece9d0 100644 --- a/source/Halibut.Tests/Queue/QueueMessageSerializerBuilder.cs +++ b/source/Halibut.Tests/Queue/QueueMessageSerializerBuilder.cs @@ -1,17 +1,3 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - using System; using Halibut.Diagnostics; using Halibut.Queue; diff --git a/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs b/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs index cbb2df980..7c75c317b 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs @@ -1,17 +1,3 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - #if NET8_0_OR_GREATER using System; using System.Collections.Concurrent; diff --git a/source/Halibut.Tests/Queue/Redis/Utils/CancellableDataLossWatchForRedisLosingAllItsData.cs b/source/Halibut.Tests/Queue/Redis/Utils/CancellableDataLossWatchForRedisLosingAllItsData.cs index 45abbd372..aa231fb91 100644 --- a/source/Halibut.Tests/Queue/Redis/Utils/CancellableDataLossWatchForRedisLosingAllItsData.cs +++ b/source/Halibut.Tests/Queue/Redis/Utils/CancellableDataLossWatchForRedisLosingAllItsData.cs @@ -1,17 +1,3 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - using System; using System.Threading; using System.Threading.Tasks; diff --git a/source/Halibut.Tests/Queue/Redis/Utils/MessageReaderWriterExtensionsMethods.cs b/source/Halibut.Tests/Queue/Redis/Utils/MessageReaderWriterExtensionsMethods.cs index 60489c56e..fb5700201 100644 --- a/source/Halibut.Tests/Queue/Redis/Utils/MessageReaderWriterExtensionsMethods.cs +++ b/source/Halibut.Tests/Queue/Redis/Utils/MessageReaderWriterExtensionsMethods.cs @@ -1,17 +1,3 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - using System; using System.Threading; using System.Threading.Tasks; diff --git a/source/Halibut.Tests/Queue/Redis/Utils/RedisFacadeBuilder.cs b/source/Halibut.Tests/Queue/Redis/Utils/RedisFacadeBuilder.cs index 05094fbd1..ea022b749 100644 --- a/source/Halibut.Tests/Queue/Redis/Utils/RedisFacadeBuilder.cs +++ b/source/Halibut.Tests/Queue/Redis/Utils/RedisFacadeBuilder.cs @@ -1,17 +1,3 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - using System; using Halibut.Logging; using Halibut.Queue.Redis; diff --git a/source/Halibut.Tests/TestSetup/BumpThreadPoolForAllTests.cs b/source/Halibut.Tests/TestSetup/BumpThreadPoolForAllTests.cs index f01eb3676..fd95af745 100644 --- a/source/Halibut.Tests/TestSetup/BumpThreadPoolForAllTests.cs +++ b/source/Halibut.Tests/TestSetup/BumpThreadPoolForAllTests.cs @@ -1,17 +1,3 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - using System; using System.Threading; using Serilog; diff --git a/source/Halibut.Tests/TestSetup/ISetupFixture.cs b/source/Halibut.Tests/TestSetup/ISetupFixture.cs index 4c8713ded..bb4a8f0b8 100644 --- a/source/Halibut.Tests/TestSetup/ISetupFixture.cs +++ b/source/Halibut.Tests/TestSetup/ISetupFixture.cs @@ -1,17 +1,3 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - using Serilog; namespace Halibut.Tests.TestSetup diff --git a/source/Halibut.Tests/TestSetup/Redis/CreateRedisDockerContainerForTests.cs b/source/Halibut.Tests/TestSetup/Redis/CreateRedisDockerContainerForTests.cs index 2d46bf201..cdd819553 100644 --- a/source/Halibut.Tests/TestSetup/Redis/CreateRedisDockerContainerForTests.cs +++ b/source/Halibut.Tests/TestSetup/Redis/CreateRedisDockerContainerForTests.cs @@ -1,17 +1,3 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - using System; using System.Threading.Tasks; using Halibut.Tests.Queue.Redis.Utils; diff --git a/source/Halibut.Tests/TestSetup/Redis/EnsureRedisIsAvailableSetupFixture.cs b/source/Halibut.Tests/TestSetup/Redis/EnsureRedisIsAvailableSetupFixture.cs index 06a013404..0b82d2b21 100644 --- a/source/Halibut.Tests/TestSetup/Redis/EnsureRedisIsAvailableSetupFixture.cs +++ b/source/Halibut.Tests/TestSetup/Redis/EnsureRedisIsAvailableSetupFixture.cs @@ -1,17 +1,3 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - using System; using System.Runtime.InteropServices; using Halibut.Tests.Support; diff --git a/source/Halibut.Tests/TestSetup/Redis/RedisTestHost.cs b/source/Halibut.Tests/TestSetup/Redis/RedisTestHost.cs index 11855a8e6..4e117dd9f 100644 --- a/source/Halibut.Tests/TestSetup/Redis/RedisTestHost.cs +++ b/source/Halibut.Tests/TestSetup/Redis/RedisTestHost.cs @@ -1,17 +1,3 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - using System; namespace Halibut.Tests.TestSetup.Redis From 0a82c2ba2f798cb627d802449646f85b46abb478 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Mon, 18 Aug 2025 11:08:02 +1000 Subject: [PATCH 071/137] . --- .../Builders/RedisPendingRequestQueueBuilder.cs | 14 +------------- source/Halibut.Tests/ManyPollingTentacleTests.cs | 14 +------------- .../Queue/Redis/RedisQueuePerformanceTest.cs | 14 +------------- .../Redis/TestRedisPendingRequestQueueFactory.cs | 14 +------------- .../Utils/HalibutRedisTransportWithVirtuals.cs | 14 +------------- ...NeverLosingDataWatchForRedisLosingAllItsData.cs | 14 +------------- .../Halibut/Queue/Redis/HalibutRedisTransport.cs | 14 +------------- .../Halibut/Queue/Redis/IHalibutRedisTransport.cs | 14 +------------- source/Halibut/Queue/Redis/NodeHeartBeatSender.cs | 14 +------------- .../Queue/Redis/PollAndSubscribeToResponse.cs | 14 +------------- .../Queue/Redis/RedisPendingRequestQueue.cs | 14 +------------- .../Queue/Redis/RedisPendingRequestQueueFactory.cs | 14 +------------- .../Queue/Redis/WatchForRequestCancellation.cs | 14 +------------- ...atchForRequestCancellationOrSenderDisconnect.cs | 14 +------------- 14 files changed, 14 insertions(+), 182 deletions(-) diff --git a/source/Halibut.Tests/Builders/RedisPendingRequestQueueBuilder.cs b/source/Halibut.Tests/Builders/RedisPendingRequestQueueBuilder.cs index 22c055bee..a3eeb311e 100644 --- a/source/Halibut.Tests/Builders/RedisPendingRequestQueueBuilder.cs +++ b/source/Halibut.Tests/Builders/RedisPendingRequestQueueBuilder.cs @@ -1,16 +1,4 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. + #if NET8_0_OR_GREATER using System; using Halibut.Logging; diff --git a/source/Halibut.Tests/ManyPollingTentacleTests.cs b/source/Halibut.Tests/ManyPollingTentacleTests.cs index 495c40f2f..602700119 100644 --- a/source/Halibut.Tests/ManyPollingTentacleTests.cs +++ b/source/Halibut.Tests/ManyPollingTentacleTests.cs @@ -1,16 +1,4 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. + #if NET8_0_OR_GREATER using System; using System.Collections.Generic; diff --git a/source/Halibut.Tests/Queue/Redis/RedisQueuePerformanceTest.cs b/source/Halibut.Tests/Queue/Redis/RedisQueuePerformanceTest.cs index ca6b1179f..ae3b9d9e5 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisQueuePerformanceTest.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisQueuePerformanceTest.cs @@ -1,16 +1,4 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. + #if NET8_0_OR_GREATER using System; using System.Diagnostics; diff --git a/source/Halibut.Tests/Queue/Redis/TestRedisPendingRequestQueueFactory.cs b/source/Halibut.Tests/Queue/Redis/TestRedisPendingRequestQueueFactory.cs index 7ef865f7f..e5ca47036 100644 --- a/source/Halibut.Tests/Queue/Redis/TestRedisPendingRequestQueueFactory.cs +++ b/source/Halibut.Tests/Queue/Redis/TestRedisPendingRequestQueueFactory.cs @@ -1,16 +1,4 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. + #if NET8_0_OR_GREATER using System; using Halibut.Queue.Redis; diff --git a/source/Halibut.Tests/Queue/Redis/Utils/HalibutRedisTransportWithVirtuals.cs b/source/Halibut.Tests/Queue/Redis/Utils/HalibutRedisTransportWithVirtuals.cs index 58a74ef57..02bcfa688 100644 --- a/source/Halibut.Tests/Queue/Redis/Utils/HalibutRedisTransportWithVirtuals.cs +++ b/source/Halibut.Tests/Queue/Redis/Utils/HalibutRedisTransportWithVirtuals.cs @@ -1,16 +1,4 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. + #if NET8_0_OR_GREATER using System; using System.Threading; diff --git a/source/Halibut.Tests/Queue/Redis/Utils/NeverLosingDataWatchForRedisLosingAllItsData.cs b/source/Halibut.Tests/Queue/Redis/Utils/NeverLosingDataWatchForRedisLosingAllItsData.cs index efb7c334a..b27e5c284 100644 --- a/source/Halibut.Tests/Queue/Redis/Utils/NeverLosingDataWatchForRedisLosingAllItsData.cs +++ b/source/Halibut.Tests/Queue/Redis/Utils/NeverLosingDataWatchForRedisLosingAllItsData.cs @@ -1,16 +1,4 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. + #if NET8_0_OR_GREATER using System; using System.Threading; diff --git a/source/Halibut/Queue/Redis/HalibutRedisTransport.cs b/source/Halibut/Queue/Redis/HalibutRedisTransport.cs index 58febff34..2f0e46a73 100644 --- a/source/Halibut/Queue/Redis/HalibutRedisTransport.cs +++ b/source/Halibut/Queue/Redis/HalibutRedisTransport.cs @@ -1,16 +1,4 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. + #if NET8_0_OR_GREATER using System; using System.Diagnostics; diff --git a/source/Halibut/Queue/Redis/IHalibutRedisTransport.cs b/source/Halibut/Queue/Redis/IHalibutRedisTransport.cs index 098ebca6f..7f983a6e0 100644 --- a/source/Halibut/Queue/Redis/IHalibutRedisTransport.cs +++ b/source/Halibut/Queue/Redis/IHalibutRedisTransport.cs @@ -1,16 +1,4 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. + #if NET8_0_OR_GREATER using System; using System.Threading; diff --git a/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs b/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs index c46e76de9..59c7ef511 100644 --- a/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs +++ b/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs @@ -1,16 +1,4 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. + #if NET8_0_OR_GREATER using System; using System.Threading; diff --git a/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs b/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs index a8ea76f2c..c9e7f2b38 100644 --- a/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs +++ b/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs @@ -1,16 +1,4 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. + #if NET8_0_OR_GREATER using System; using System.Threading; diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index 90f6058a4..0398a477d 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -1,16 +1,4 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. + #if NET8_0_OR_GREATER using System; using System.Collections.Concurrent; diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueueFactory.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueueFactory.cs index 855b9b099..a03a592a3 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueueFactory.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueueFactory.cs @@ -1,16 +1,4 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. + #if NET8_0_OR_GREATER using System; using System.Threading.Tasks; diff --git a/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs b/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs index 1502252fb..08e9c098b 100644 --- a/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs +++ b/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs @@ -1,16 +1,4 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. + #if NET8_0_OR_GREATER using System; using System.Threading; diff --git a/source/Halibut/Queue/Redis/WatchForRequestCancellationOrSenderDisconnect.cs b/source/Halibut/Queue/Redis/WatchForRequestCancellationOrSenderDisconnect.cs index 54ce63ba2..4ed7f9ee7 100644 --- a/source/Halibut/Queue/Redis/WatchForRequestCancellationOrSenderDisconnect.cs +++ b/source/Halibut/Queue/Redis/WatchForRequestCancellationOrSenderDisconnect.cs @@ -1,16 +1,4 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. + #if NET8_0_OR_GREATER using System; using System.Security.Policy; From 9ddbb7d94143a3c4199b220c03629b225008eaf1 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Mon, 18 Aug 2025 11:15:48 +1000 Subject: [PATCH 072/137] . --- Redis.md | 2 -- docs/RedisQueue.md | 25 +++++++++++++++++++++++++ 2 files changed, 25 insertions(+), 2 deletions(-) delete mode 100644 Redis.md create mode 100644 docs/RedisQueue.md diff --git a/Redis.md b/Redis.md deleted file mode 100644 index 8d621b663..000000000 --- a/Redis.md +++ /dev/null @@ -1,2 +0,0 @@ -docker run -v `pwd`/redis-conf:/usr/local/etc/redis -p 6379:6379 --name redis -d redis redis-server /usr/local/etc/redis/redis.conf - diff --git a/docs/RedisQueue.md b/docs/RedisQueue.md new file mode 100644 index 000000000..b992fbb1d --- /dev/null +++ b/docs/RedisQueue.md @@ -0,0 +1,25 @@ +# Redis Pending Request Queue Beta + +Halibut provides a Redis backed pending request queue for multi node setups. This solves the problem where +a cluster of multiple clients need to send commands to polling services which connect to only one of the +clients. + +For example if we have two clients ClientA and ClientB and the Service connects to B, yet A wants +to execute an RPC. Currently that wont work as the request will end up in the in memory queue for ClientA +but it needs to be accessible to ClientB. + +The Redis queue solves this, as the request is placed into Redis allowing ClientB to access the request and +so send it to the Service. + +## How to run Redis for this queue. + +Redis can be started by running the following command in the root of the directory: + +``` +docker run -v `pwd`/redis-conf:/usr/local/etc/redis -p 6379:6379 --name redis -d redis redis-server /usr/local/etc/redis/redis.conf +``` + +Note that Redis is configured to have no backup, everything must be in memory. The queue makes this assumption to function. + +## TODO design. + From a7c742e965f58a4413565a743ab769c5a252b7e5 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Mon, 18 Aug 2025 11:49:27 +1000 Subject: [PATCH 073/137] . --- source/Halibut/Queue/Redis/RedisFacade.cs | 31 ----------------------- 1 file changed, 31 deletions(-) diff --git a/source/Halibut/Queue/Redis/RedisFacade.cs b/source/Halibut/Queue/Redis/RedisFacade.cs index 726efaead..e43bb4a98 100644 --- a/source/Halibut/Queue/Redis/RedisFacade.cs +++ b/source/Halibut/Queue/Redis/RedisFacade.cs @@ -8,37 +8,6 @@ namespace Halibut.Queue.Redis { - /// - /// Facade for Redis operations with built-in connection monitoring and disconnect detection. - /// - /// Usage example for connection monitoring: - /// - /// var facade = new RedisFacade("localhost:6379", "myapp", logger); - /// - /// // Monitor overall connection events - /// facade.ConnectionFailed += message => Console.WriteLine($"Connection failed: {message}"); - /// facade.ConnectionRestored += message => Console.WriteLine($"Connection restored: {message}"); - /// facade.ErrorOccurred += message => Console.WriteLine($"Redis error: {message}"); - /// - /// // Subscribe with per-subscription monitoring - /// var subscription = await facade.SubscribeToChannel("my-channel", async message => { - /// Console.WriteLine($"Received: {message}"); - /// }); - /// - /// // Monitor individual subscription disconnects - /// if (subscription is RedisSubscriptionWrapper wrapper) - /// { - /// wrapper.SubscriptionDisconnected += message => Console.WriteLine($"Subscription lost: {message}"); - /// wrapper.SubscriptionReconnected += message => Console.WriteLine($"Subscription restored: {message}"); - /// } - /// - /// // Check connection status - /// if (!facade.IsConnected) - /// { - /// Console.WriteLine("Redis is not connected!"); - /// } - /// - /// public class RedisFacade : IAsyncDisposable { readonly Lazy connection; From e64b29f6275010ed6ee129cf4fe9accb8c095b97 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Mon, 18 Aug 2025 12:50:28 +1000 Subject: [PATCH 074/137] . --- .../Redis/RedisPendingRequestQueueFixture.cs | 1 + .../Queue/Redis/RedisQueuePerformanceTest.cs | 77 ------------------- .../TestRedisPendingRequestQueueFactory.cs | 2 - ...oryStoreDataStreamsForDistributedQueues.cs | 8 +- ...onReturnedByHalibutProxyExtensionMethod.cs | 7 -- .../IStoreDataStreamsForDistributedQueues.cs | 22 ++++++ ...taLoseTokenInTimeHalibutClientException.cs | 2 +- .../RedisDataLoseHalibutClientException.cs | 2 +- .../RedisQueueShutdownClientException.cs | 4 +- 9 files changed, 31 insertions(+), 94 deletions(-) delete mode 100644 source/Halibut.Tests/Queue/Redis/RedisQueuePerformanceTest.cs rename source/{Halibut/Queue/QueuedDataStreams => Halibut.Tests/Queue/Redis/Utils}/InMemoryStoreDataStreamsForDistributedQueues.cs (86%) diff --git a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs index 894e150e8..5a5f4a619 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs @@ -9,6 +9,7 @@ using Halibut.Queue; using Halibut.Queue.QueuedDataStreams; using Halibut.Queue.Redis; +using Halibut.Queue.Redis.Exceptions; using Halibut.ServiceModel; using Halibut.Tests.Builders; using Halibut.Tests.Queue.Redis.Utils; diff --git a/source/Halibut.Tests/Queue/Redis/RedisQueuePerformanceTest.cs b/source/Halibut.Tests/Queue/Redis/RedisQueuePerformanceTest.cs deleted file mode 100644 index ae3b9d9e5..000000000 --- a/source/Halibut.Tests/Queue/Redis/RedisQueuePerformanceTest.cs +++ /dev/null @@ -1,77 +0,0 @@ - -#if NET8_0_OR_GREATER -using System; -using System.Diagnostics; -using System.Threading.Tasks; -using FluentAssertions; -using Halibut.Diagnostics; -using Halibut.Logging; -using Halibut.Queue.QueuedDataStreams; -using Halibut.Queue.Redis; -using Halibut.Tests.Builders; -using Halibut.Tests.Queue.Redis.Utils; -using Halibut.Tests.Support.Logging; -using Halibut.Util; -using NUnit.Framework; - -namespace Halibut.Tests.Queue.Redis -{ - [RedisTest] - public class RedisQueuePerformanceTest : BaseTest - { - //[Test] - public async Task When100kTentaclesAreSubscribed_TheQueueStillWorks() - { - // Arrange - var endpoint = new Uri("poll://" + Guid.NewGuid()); - var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); - await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); - var redisTransport = new HalibutRedisTransport(redisFacade); - - var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); - var messageSerializer = new QueueMessageSerializerBuilder().Build(); - var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); - - var halibutTimeoutsAndLimits = new HalibutTimeoutsAndLimits(); - - await using var disposableCollection = new DisposableCollection(); - for (int i = 0; i < 300000; i++) - { - disposableCollection.Add(new RedisPendingRequestQueue(new Uri("poll://" + Guid.NewGuid()), new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, halibutTimeoutsAndLimits)); - if (i % 10000 == 0) - { - Logger.Information("Up to: {i}", i); - } - } - - this.Logger.Fatal("Waiting"); - await Task.Delay(30000); - this.Logger.Fatal("Done"); - - for (int i = 0; i < 10; i++) - { - var request = new RequestMessageBuilder(endpoint.ToString()).Build(); - await using var sut = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, halibutTimeoutsAndLimits); - - var resultTask = sut.DequeueAsync(CancellationToken); - - await Task.Delay(100); - - var sw = Stopwatch.StartNew(); - - var task = sut.QueueAndWaitAsync(request, CancellationToken); - - var result = await resultTask; - // Act - - // Assert - result.Should().NotBeNull(); - result!.RequestMessage.Id.Should().Be(request.Id); - result.RequestMessage.MethodName.Should().Be(request.MethodName); - result.RequestMessage.ServiceName.Should().Be(request.ServiceName); - Logger.Information("It took {F}", sw.Elapsed.TotalSeconds.ToString("0.00")); - } - } - } -} -#endif \ No newline at end of file diff --git a/source/Halibut.Tests/Queue/Redis/TestRedisPendingRequestQueueFactory.cs b/source/Halibut.Tests/Queue/Redis/TestRedisPendingRequestQueueFactory.cs index e5ca47036..d1d040e2d 100644 --- a/source/Halibut.Tests/Queue/Redis/TestRedisPendingRequestQueueFactory.cs +++ b/source/Halibut.Tests/Queue/Redis/TestRedisPendingRequestQueueFactory.cs @@ -3,11 +3,9 @@ using System; using Halibut.Queue.Redis; using Halibut.ServiceModel; -using NUnit.Framework; namespace Halibut.Tests.Queue.Redis { - [Ignore("REDISTODO")] public class TestRedisPendingRequestQueueFactory : IPendingRequestQueueFactory { RedisPendingRequestQueueFactory redisPendingRequestQueueFactory; diff --git a/source/Halibut/Queue/QueuedDataStreams/InMemoryStoreDataStreamsForDistributedQueues.cs b/source/Halibut.Tests/Queue/Redis/Utils/InMemoryStoreDataStreamsForDistributedQueues.cs similarity index 86% rename from source/Halibut/Queue/QueuedDataStreams/InMemoryStoreDataStreamsForDistributedQueues.cs rename to source/Halibut.Tests/Queue/Redis/Utils/InMemoryStoreDataStreamsForDistributedQueues.cs index e83a96684..9ab04540f 100644 --- a/source/Halibut/Queue/QueuedDataStreams/InMemoryStoreDataStreamsForDistributedQueues.cs +++ b/source/Halibut.Tests/Queue/Redis/Utils/InMemoryStoreDataStreamsForDistributedQueues.cs @@ -1,19 +1,17 @@ - using System; using System.Collections.Generic; using System.IO; using System.Threading; using System.Threading.Tasks; +using Halibut.Queue.QueuedDataStreams; -namespace Halibut.Queue.QueuedDataStreams +namespace Halibut.Tests.Queue.Redis.Utils { public class InMemoryStoreDataStreamsForDistributedQueues : IStoreDataStreamsForDistributedQueues { - public IDictionary dataStreamsStored = new Dictionary(); + readonly IDictionary dataStreamsStored = new Dictionary(); public async Task StoreDataStreams(IReadOnlyList dataStreams, CancellationToken cancellationToken) { - await Task.CompletedTask; - foreach (var dataStream in dataStreams) { using var memoryStream = new MemoryStream(); diff --git a/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs b/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs index d0b5f3fc9..cdc8d9dae 100644 --- a/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs +++ b/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs @@ -10,13 +10,6 @@ namespace Halibut.Diagnostics { - public enum HalibutRetryableErrorType - { - IsRetryable, - UnknownError, - NotRetryable - } - public static class ExceptionReturnedByHalibutProxyExtensionMethod { public static HalibutRetryableErrorType IsRetryableError(this Exception exception) diff --git a/source/Halibut/Queue/QueuedDataStreams/IStoreDataStreamsForDistributedQueues.cs b/source/Halibut/Queue/QueuedDataStreams/IStoreDataStreamsForDistributedQueues.cs index 1dfdf2f88..ab3b37a79 100644 --- a/source/Halibut/Queue/QueuedDataStreams/IStoreDataStreamsForDistributedQueues.cs +++ b/source/Halibut/Queue/QueuedDataStreams/IStoreDataStreamsForDistributedQueues.cs @@ -5,10 +5,32 @@ namespace Halibut.Queue.QueuedDataStreams { + /// + /// The Redis Queue requires that something else can store data streams. The + /// Redis Queue will call this interface for storage and retrieval of data streams. + /// + /// The ReHydrateDataStreams method will be called at most once, and each data stream passed to + /// ReHydrateDataStreams will be read at most once. Thus, it is safe to delete the DataStream from + /// storage once the DataStream `writerAsync` Func is called and will no longer return any more + /// data. This includes in the case the writerAsync method throws. + /// public interface IStoreDataStreamsForDistributedQueues { + /// + /// Must store the data for the given dataStreams. + /// + /// + /// + /// public Task StoreDataStreams(IReadOnlyList dataStreams, CancellationToken cancellationToken); + /// + /// Updates the dataStreams `writerAsync` to write the previously stored data. Using + /// the SetWriterAsync method. + /// + /// + /// + /// public Task ReHydrateDataStreams(IReadOnlyList dataStreams, CancellationToken cancellationToken); } } \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/Exceptions/CouldNotGetDataLoseTokenInTimeHalibutClientException.cs b/source/Halibut/Queue/Redis/Exceptions/CouldNotGetDataLoseTokenInTimeHalibutClientException.cs index a77633875..e60e52ed7 100644 --- a/source/Halibut/Queue/Redis/Exceptions/CouldNotGetDataLoseTokenInTimeHalibutClientException.cs +++ b/source/Halibut/Queue/Redis/Exceptions/CouldNotGetDataLoseTokenInTimeHalibutClientException.cs @@ -1,6 +1,6 @@ using System; -namespace Halibut.Queue.Redis +namespace Halibut.Queue.Redis.Exceptions { public class CouldNotGetDataLoseTokenInTimeHalibutClientException : HalibutClientException { diff --git a/source/Halibut/Queue/Redis/Exceptions/RedisDataLoseHalibutClientException.cs b/source/Halibut/Queue/Redis/Exceptions/RedisDataLoseHalibutClientException.cs index 3a321c938..e4bbaa10e 100644 --- a/source/Halibut/Queue/Redis/Exceptions/RedisDataLoseHalibutClientException.cs +++ b/source/Halibut/Queue/Redis/Exceptions/RedisDataLoseHalibutClientException.cs @@ -1,6 +1,6 @@ using System; -namespace Halibut.Queue.Redis +namespace Halibut.Queue.Redis.Exceptions { public class RedisDataLoseHalibutClientException : HalibutClientException { diff --git a/source/Halibut/Queue/Redis/Exceptions/RedisQueueShutdownClientException.cs b/source/Halibut/Queue/Redis/Exceptions/RedisQueueShutdownClientException.cs index da094c92a..c5227d689 100644 --- a/source/Halibut/Queue/Redis/Exceptions/RedisQueueShutdownClientException.cs +++ b/source/Halibut/Queue/Redis/Exceptions/RedisQueueShutdownClientException.cs @@ -1,4 +1,6 @@ -namespace Halibut.Queue.Redis +using System; + +namespace Halibut.Queue.Redis.Exceptions { public class RedisQueueShutdownClientException : HalibutClientException { From 52748cb16b1660eecc3eddc2173154b89dc7c165 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Mon, 18 Aug 2025 12:52:42 +1000 Subject: [PATCH 075/137] . --- .../Diagnostics/HalibutRetryableErrorType.cs | 25 +++++++++++++++++++ 1 file changed, 25 insertions(+) create mode 100644 source/Halibut/Diagnostics/HalibutRetryableErrorType.cs diff --git a/source/Halibut/Diagnostics/HalibutRetryableErrorType.cs b/source/Halibut/Diagnostics/HalibutRetryableErrorType.cs new file mode 100644 index 000000000..b7a7c0083 --- /dev/null +++ b/source/Halibut/Diagnostics/HalibutRetryableErrorType.cs @@ -0,0 +1,25 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System; + +namespace Halibut.Diagnostics +{ + public enum HalibutRetryableErrorType + { + IsRetryable, + UnknownError, + NotRetryable + } +} \ No newline at end of file From 0c86570ac311c1d1e03d9d4df0657554a71c5efa Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Mon, 18 Aug 2025 12:55:08 +1000 Subject: [PATCH 076/137] . --- .../Diagnostics/HalibutRetryableErrorType.cs | 14 -------------- 1 file changed, 14 deletions(-) diff --git a/source/Halibut/Diagnostics/HalibutRetryableErrorType.cs b/source/Halibut/Diagnostics/HalibutRetryableErrorType.cs index b7a7c0083..8818df5e2 100644 --- a/source/Halibut/Diagnostics/HalibutRetryableErrorType.cs +++ b/source/Halibut/Diagnostics/HalibutRetryableErrorType.cs @@ -1,17 +1,3 @@ -// Copyright 2012-2013 Octopus Deploy Pty. Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - using System; namespace Halibut.Diagnostics From 648f2ab668f8f9cf31cc0c0046cabd48fc573716 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Mon, 18 Aug 2025 13:58:43 +1000 Subject: [PATCH 077/137] Cleanup HalibutRedisTransport --- .../Queue/Redis/NodeHeartBeatSenderFixture.cs | 40 ++-- .../Redis/RedisPendingRequestQueueFixture.cs | 4 +- .../HalibutRedisTransportWithVirtuals.cs | 30 +-- .../Queue/Redis/HalibutRedisTransport.cs | 178 ++++++++++-------- .../Queue/Redis/IHalibutRedisTransport.cs | 19 +- .../Queue/Redis/NodeHeartBeatSender.cs | 15 +- .../Queue/Redis/PollAndSubscribeToResponse.cs | 14 +- .../Queue/Redis/RedisPendingRequestQueue.cs | 8 +- 8 files changed, 158 insertions(+), 150 deletions(-) diff --git a/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs b/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs index a993f506d..28375104e 100644 --- a/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs @@ -36,14 +36,14 @@ public async Task WhenCreated_ShouldStartSendingHeartbeats() // Subscribe to heartbeats before creating the sender await using var subscription = await redisTransport.SubscribeToNodeHeartBeatChannel( - endpoint, requestActivityId, HalibutQueueNodeSendingPulses.Receiver, async () => + endpoint, requestActivityId, HalibutQueueNodeSendingPulses.RequestProcessorNode, async () => { await Task.CompletedTask; anyHeartBeatReceived.Set(); }, CancellationToken); // Act - await using var heartBeatSender = new NodeHeartBeatSender(endpoint, requestActivityId, redisTransport, log, HalibutQueueNodeSendingPulses.Receiver, defaultDelayBetweenPulses: TimeSpan.FromSeconds(1)); + await using var heartBeatSender = new NodeHeartBeatSender(endpoint, requestActivityId, redisTransport, log, HalibutQueueNodeSendingPulses.RequestProcessorNode, defaultDelayBetweenPulses: TimeSpan.FromSeconds(1)); // Wait for a few heartbeats await Task.WhenAny(Task.Delay(TimeSpan.FromSeconds(20), CancellationToken), anyHeartBeatReceived.WaitAsync()); @@ -73,14 +73,14 @@ public async Task WhenRedisConnectionIsInterrupted_ShouldSwitchToPanicMode() // Subscribe with stable connection to monitor heartbeats await using var subscription = await new HalibutRedisTransport(stableRedisFacade) .SubscribeToNodeHeartBeatChannel( - endpoint, requestActivityId, HalibutQueueNodeSendingPulses.Receiver, async () => + endpoint, requestActivityId, HalibutQueueNodeSendingPulses.RequestProcessorNode, async () => { await Task.CompletedTask; heartbeatsReceived.Add(DateTimeOffset.Now); }, CancellationToken); // Act - await using var heartBeatSender = new NodeHeartBeatSender(endpoint, requestActivityId, redisTransport, log, HalibutQueueNodeSendingPulses.Receiver, TimeSpan.FromSeconds(1)); + await using var heartBeatSender = new NodeHeartBeatSender(endpoint, requestActivityId, redisTransport, log, HalibutQueueNodeSendingPulses.RequestProcessorNode, TimeSpan.FromSeconds(1)); // Wait for initial heartbeat await Task.Delay(TimeSpan.FromSeconds(2), CancellationToken); @@ -120,7 +120,7 @@ public async Task WhenDisposed_ShouldStopSendingHeartbeats() var anyHeartBeatReceived = new AsyncManualResetEvent(false); await using var subscription = await redisTransport.SubscribeToNodeHeartBeatChannel( - endpoint, requestActivityId, HalibutQueueNodeSendingPulses.Receiver, async () => + endpoint, requestActivityId, HalibutQueueNodeSendingPulses.RequestProcessorNode, async () => { await Task.CompletedTask; anyHeartBeatReceived.Set(); @@ -128,7 +128,7 @@ public async Task WhenDisposed_ShouldStopSendingHeartbeats() }, CancellationToken); // Act - var heartBeatSender = new NodeHeartBeatSender(endpoint, requestActivityId, redisTransport, log, HalibutQueueNodeSendingPulses.Receiver, defaultDelayBetweenPulses: TimeSpan.FromSeconds(1)); + var heartBeatSender = new NodeHeartBeatSender(endpoint, requestActivityId, redisTransport, log, HalibutQueueNodeSendingPulses.RequestProcessorNode, defaultDelayBetweenPulses: TimeSpan.FromSeconds(1)); // Wait for some heartbeats await anyHeartBeatReceived.WaitAsync(CancellationToken); @@ -167,7 +167,7 @@ public async Task WaitUntilNodeProcessingRequestFlatLines_WhenHeartbeatsStop_Sho var pendingRequest = new RedisPendingRequest(request, log); // Start heartbeat sender - await using var heartBeatSender = new NodeHeartBeatSender(endpoint, requestActivityId, unstableRedisTransport, log, HalibutQueueNodeSendingPulses.Receiver, TimeSpan.FromSeconds(1)); + await using var heartBeatSender = new NodeHeartBeatSender(endpoint, requestActivityId, unstableRedisTransport, log, HalibutQueueNodeSendingPulses.RequestProcessorNode, TimeSpan.FromSeconds(1)); // Mark request as collected so watcher proceeds to monitoring phase await pendingRequest.RequestHasBeenCollectedAndWillBeTransferred(); @@ -211,7 +211,7 @@ public async Task NoIssueReturnedWhenNodeProcessingRequestIsNotSeenToGoOffline() await pendingRequest.RequestHasBeenCollectedAndWillBeTransferred(); // Start heartbeat sender - await using var heartBeatSender = new NodeHeartBeatSender(endpoint, requestActivityId, redisTransport, log, HalibutQueueNodeSendingPulses.Receiver, TimeSpan.FromSeconds(1)); + await using var heartBeatSender = new NodeHeartBeatSender(endpoint, requestActivityId, redisTransport, log, HalibutQueueNodeSendingPulses.RequestProcessorNode, TimeSpan.FromSeconds(1)); // Start the watcher without marking request as collected using var cts = new CancellationTokenSource(); @@ -254,7 +254,7 @@ public async Task WaitUntilNodeProcessingRequestFlatLines_WhenConnectionInterrup var pendingRequest = new RedisPendingRequest(request, log); // Start heartbeat sender with unstable connection - await using var heartBeatSender = new NodeHeartBeatSender(endpoint, requestActivityId, unstableRedisTransport, log, HalibutQueueNodeSendingPulses.Receiver, TimeSpan.FromSeconds(1)); + await using var heartBeatSender = new NodeHeartBeatSender(endpoint, requestActivityId, unstableRedisTransport, log, HalibutQueueNodeSendingPulses.RequestProcessorNode, TimeSpan.FromSeconds(1)); // Mark request as collected await pendingRequest.RequestHasBeenCollectedAndWillBeTransferred(); @@ -294,15 +294,15 @@ public async Task WhenMultipleHeartBeatSendersForSameRequest_OnlyOneSetOfHeartbe var heartbeatsReceived = new ConcurrentBag(); await using var subscription = await redisTransport.SubscribeToNodeHeartBeatChannel( - endpoint, requestActivityId, HalibutQueueNodeSendingPulses.Receiver, async () => + endpoint, requestActivityId, HalibutQueueNodeSendingPulses.RequestProcessorNode, async () => { await Task.CompletedTask; heartbeatsReceived.Add(DateTimeOffset.Now); }, CancellationToken); // Act - Create multiple senders for the same request - await using var heartBeatSender1 = new NodeHeartBeatSender(endpoint, requestActivityId, redisTransport, log, HalibutQueueNodeSendingPulses.Receiver, TimeSpan.FromSeconds(1)); - await using var heartBeatSender2 = new NodeHeartBeatSender(endpoint, requestActivityId, redisTransport, log, HalibutQueueNodeSendingPulses.Receiver, TimeSpan.FromSeconds(1)); + await using var heartBeatSender1 = new NodeHeartBeatSender(endpoint, requestActivityId, redisTransport, log, HalibutQueueNodeSendingPulses.RequestProcessorNode, TimeSpan.FromSeconds(1)); + await using var heartBeatSender2 = new NodeHeartBeatSender(endpoint, requestActivityId, redisTransport, log, HalibutQueueNodeSendingPulses.RequestProcessorNode, TimeSpan.FromSeconds(1)); // Wait for heartbeats await Task.Delay(TimeSpan.FromSeconds(10), CancellationToken); @@ -334,7 +334,7 @@ public async Task WhenHeartBeatSenderConnectionRecovery_ShouldResumeNormalHeartb var heartBeatsReceived = new AsyncManualResetEvent(false); await using var subscription = await stableRedisTransport.SubscribeToNodeHeartBeatChannel( - endpoint, requestActivityId, HalibutQueueNodeSendingPulses.Receiver, async () => + endpoint, requestActivityId, HalibutQueueNodeSendingPulses.RequestProcessorNode, async () => { await Task.CompletedTask; heartbeatTimestamps.Add(DateTimeOffset.Now); @@ -342,7 +342,7 @@ public async Task WhenHeartBeatSenderConnectionRecovery_ShouldResumeNormalHeartb }, CancellationToken); // Act - await using var heartBeatSender = new NodeHeartBeatSender(endpoint, requestActivityId, unstableRedisTransport, log, HalibutQueueNodeSendingPulses.Receiver, TimeSpan.FromSeconds(1)); + await using var heartBeatSender = new NodeHeartBeatSender(endpoint, requestActivityId, unstableRedisTransport, log, HalibutQueueNodeSendingPulses.RequestProcessorNode, TimeSpan.FromSeconds(1)); // Wait for initial heartbeats (normal 15s interval) await heartBeatsReceived.WaitAsync(CancellationToken); @@ -373,7 +373,7 @@ public async Task SenderAndReceiverNodeTypes_ShouldUseDistinctChannels() // Subscribe to sender heartbeats await using var senderSubscription = await redisTransport.SubscribeToNodeHeartBeatChannel( - endpoint, requestActivityId, HalibutQueueNodeSendingPulses.Sender, async () => + endpoint, requestActivityId, HalibutQueueNodeSendingPulses.RequestSenderNode, async () => { await Task.CompletedTask; senderHeartbeatsReceived.Set(); @@ -381,20 +381,20 @@ public async Task SenderAndReceiverNodeTypes_ShouldUseDistinctChannels() // Subscribe to receiver heartbeats await using var receiverSubscription = await redisTransport.SubscribeToNodeHeartBeatChannel( - endpoint, requestActivityId, HalibutQueueNodeSendingPulses.Receiver, async () => + endpoint, requestActivityId, HalibutQueueNodeSendingPulses.RequestProcessorNode, async () => { await Task.CompletedTask; receiverHeartbeatsReceived.Set(); }, CancellationToken); // Act - Create sender node heartbeat sender - await using var senderHeartBeatSender = new NodeHeartBeatSender(endpoint, requestActivityId, redisTransport, log, HalibutQueueNodeSendingPulses.Sender, defaultDelayBetweenPulses: TimeSpan.FromSeconds(1)); + await using var senderHeartBeatSender = new NodeHeartBeatSender(endpoint, requestActivityId, redisTransport, log, HalibutQueueNodeSendingPulses.RequestSenderNode, defaultDelayBetweenPulses: TimeSpan.FromSeconds(1)); // Wait for sender heartbeat await Task.WhenAny(Task.Delay(TimeSpan.FromSeconds(5), CancellationToken), senderHeartbeatsReceived.WaitAsync()); // Create receiver node heartbeat sender - await using var receiverHeartBeatSender = new NodeHeartBeatSender(endpoint, requestActivityId, redisTransport, log, HalibutQueueNodeSendingPulses.Receiver, defaultDelayBetweenPulses: TimeSpan.FromSeconds(1)); + await using var receiverHeartBeatSender = new NodeHeartBeatSender(endpoint, requestActivityId, redisTransport, log, HalibutQueueNodeSendingPulses.RequestProcessorNode, defaultDelayBetweenPulses: TimeSpan.FromSeconds(1)); // Wait for receiver heartbeat await Task.WhenAny(Task.Delay(TimeSpan.FromSeconds(5), CancellationToken), receiverHeartbeatsReceived.WaitAsync()); @@ -418,14 +418,14 @@ public async Task SenderNodeHeartbeats_ShouldNotBeReceivedByReceiverSubscription // Subscribe only to receiver heartbeats await using var receiverSubscription = await redisTransport.SubscribeToNodeHeartBeatChannel( - endpoint, requestActivityId, HalibutQueueNodeSendingPulses.Receiver, async () => + endpoint, requestActivityId, HalibutQueueNodeSendingPulses.RequestProcessorNode, async () => { await Task.CompletedTask; receiverHeartbeatsReceived.Set(); }, CancellationToken); // Act - Create sender node heartbeat sender (should not trigger receiver subscription) - await using var senderHeartBeatSender = new NodeHeartBeatSender(endpoint, requestActivityId, redisTransport, log, HalibutQueueNodeSendingPulses.Sender, defaultDelayBetweenPulses: TimeSpan.FromSeconds(1)); + await using var senderHeartBeatSender = new NodeHeartBeatSender(endpoint, requestActivityId, redisTransport, log, HalibutQueueNodeSendingPulses.RequestSenderNode, defaultDelayBetweenPulses: TimeSpan.FromSeconds(1)); // Wait to see if receiver subscription gets triggered (it shouldn't) await Task.WhenAny(Task.Delay(TimeSpan.FromSeconds(3), CancellationToken), receiverHeartbeatsReceived.WaitAsync()); diff --git a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs index 5a5f4a619..2292e0f76 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs @@ -352,7 +352,7 @@ public async Task OnceARequestIsComplete_NoSenderHeartBeatsShouldBeSent() // Assert var heartBeatSent = false; var cts = new CancelOnDisposeCancellationToken(); - using var _ = redisTransport.SubscribeToNodeHeartBeatChannel(endpoint, request.ActivityId, HalibutQueueNodeSendingPulses.Sender, async () => + using var _ = redisTransport.SubscribeToNodeHeartBeatChannel(endpoint, request.ActivityId, HalibutQueueNodeSendingPulses.RequestSenderNode, async () => { await Task.CompletedTask; heartBeatSent = true; @@ -396,7 +396,7 @@ public async Task OnceARequestIsComplete_NoReceiverHeartBeatsShouldBeSent() // Assert var heartBeatSent = false; var cts = new CancelOnDisposeCancellationToken(); - using var _ = redisTransport.SubscribeToNodeHeartBeatChannel(endpoint, request.ActivityId, HalibutQueueNodeSendingPulses.Receiver, async () => + using var _ = redisTransport.SubscribeToNodeHeartBeatChannel(endpoint, request.ActivityId, HalibutQueueNodeSendingPulses.RequestProcessorNode, async () => { await Task.CompletedTask; heartBeatSent = true; diff --git a/source/Halibut.Tests/Queue/Redis/Utils/HalibutRedisTransportWithVirtuals.cs b/source/Halibut.Tests/Queue/Redis/Utils/HalibutRedisTransportWithVirtuals.cs index 02bcfa688..7c18f4832 100644 --- a/source/Halibut.Tests/Queue/Redis/Utils/HalibutRedisTransportWithVirtuals.cs +++ b/source/Halibut.Tests/Queue/Redis/Utils/HalibutRedisTransportWithVirtuals.cs @@ -37,9 +37,9 @@ public Task PushRequestGuidOnToQueue(Uri endpoint, Guid guid, CancellationToken return halibutRedisTransport.TryPopNextRequestGuid(endpoint, cancellationToken); } - public virtual Task PutRequest(Uri endpoint, Guid requestId, string payload, TimeSpan requestPickupTimeout, CancellationToken cancellationToken) + public virtual Task PutRequest(Uri endpoint, Guid requestId, string requestMessage, TimeSpan requestPickupTimeout, CancellationToken cancellationToken) { - return halibutRedisTransport.PutRequest(endpoint, requestId, payload, requestPickupTimeout, cancellationToken); + return halibutRedisTransport.PutRequest(endpoint, requestId, requestMessage, requestPickupTimeout, cancellationToken); } public Task TryGetAndRemoveRequest(Uri endpoint, Guid requestId, CancellationToken cancellationToken) @@ -52,9 +52,9 @@ public Task IsRequestStillOnQueue(Uri endpoint, Guid requestId, Cancellati return halibutRedisTransport.IsRequestStillOnQueue(endpoint, requestId, cancellationToken); } - public Task SubscribeToRequestCancellation(Uri endpoint, Guid request, Func onCancellationReceived, CancellationToken cancellationToken) + public Task SubscribeToRequestCancellation(Uri endpoint, Guid request, Func onRpcCancellation, CancellationToken cancellationToken) { - return halibutRedisTransport.SubscribeToRequestCancellation(endpoint, request, onCancellationReceived, cancellationToken); + return halibutRedisTransport.SubscribeToRequestCancellation(endpoint, request, onRpcCancellation, cancellationToken); } public Task PublishCancellation(Uri endpoint, Guid requestId, CancellationToken cancellationToken) @@ -82,19 +82,9 @@ public Task SubscribeToNodeHeartBeatChannel(Uri endpoint, Guid return halibutRedisTransport.SubscribeToNodeHeartBeatChannel(endpoint, request, nodeSendingPulsesType, onHeartBeat, cancellationToken); } - public Task SendHeartBeatFromNodeProcessingTheRequest(Uri endpoint, Guid requestId, HalibutQueueNodeSendingPulses nodeSendingPulsesType, CancellationToken cancellationToken) + public Task SendNodeHeartBeat(Uri endpoint, Guid requestId, HalibutQueueNodeSendingPulses nodeSendingPulsesType, CancellationToken cancellationToken) { - return halibutRedisTransport.SendHeartBeatFromNodeProcessingTheRequest(endpoint, requestId, nodeSendingPulsesType, cancellationToken); - } - - public Task SendHeartBeatFromNodeProcessingTheRequest(Uri endpoint, Guid requestId, CancellationToken cancellationToken) - { - return halibutRedisTransport.SendHeartBeatFromNodeProcessingTheRequest(endpoint, requestId, cancellationToken); - } - - public Task SubscribeToNodeProcessingTheRequestHeartBeatChannel(Uri endpoint, Guid request, Func onHeartBeat, CancellationToken cancellationToken) - { - return halibutRedisTransport.SubscribeToNodeProcessingTheRequestHeartBeatChannel(endpoint, request, onHeartBeat, cancellationToken); + return halibutRedisTransport.SendNodeHeartBeat(endpoint, requestId, nodeSendingPulsesType, cancellationToken); } public Task SubscribeToResponseChannel(Uri endpoint, Guid identifier, Func onValueReceived, CancellationToken cancellationToken) @@ -107,9 +97,9 @@ public Task PublishThatResponseIsAvailable(Uri endpoint, Guid identifier, string return halibutRedisTransport.PublishThatResponseIsAvailable(endpoint, identifier, value, cancellationToken); } - public Task MarkThatResponseIsSet(Uri endpoint, Guid identifier, string value, TimeSpan ttl, CancellationToken cancellationToken) + public Task SetResponseMessage(Uri endpoint, Guid identifier, string responseMessage, TimeSpan ttl, CancellationToken cancellationToken) { - return halibutRedisTransport.MarkThatResponseIsSet(endpoint, identifier, value, ttl, cancellationToken); + return halibutRedisTransport.SetResponseMessage(endpoint, identifier, responseMessage, ttl, cancellationToken); } public Task GetResponseMessage(Uri endpoint, Guid identifier, CancellationToken cancellationToken) @@ -117,9 +107,9 @@ public Task MarkThatResponseIsSet(Uri endpoint, Guid identifier, string value, T return halibutRedisTransport.GetResponseMessage(endpoint, identifier, cancellationToken); } - public Task DeleteResponse(Uri endpoint, Guid identifier, CancellationToken cancellationToken) + public Task DeleteResponseMessage(Uri endpoint, Guid identifier, CancellationToken cancellationToken) { - return halibutRedisTransport.DeleteResponse(endpoint, identifier, cancellationToken); + return halibutRedisTransport.DeleteResponseMessage(endpoint, identifier, cancellationToken); } } } diff --git a/source/Halibut/Queue/Redis/HalibutRedisTransport.cs b/source/Halibut/Queue/Redis/HalibutRedisTransport.cs index 2f0e46a73..0971adee3 100644 --- a/source/Halibut/Queue/Redis/HalibutRedisTransport.cs +++ b/source/Halibut/Queue/Redis/HalibutRedisTransport.cs @@ -12,7 +12,7 @@ namespace Halibut.Queue.Redis { public class HalibutRedisTransport : IHalibutRedisTransport { - const string Namespace = "octopus:server:halibut"; + const string Namespace = "octopus::server::halibut"; readonly RedisFacade facade; @@ -20,11 +20,14 @@ public HalibutRedisTransport(RedisFacade facade) { this.facade = facade; } - - // Request Pulse + + // Request pulse channel. + // Polling services will be notified of new request via this channel. + // The Service will subscribe to the channel, while the client will publish (pulse) + // the channel when a request is available. static string RequestMessagesPulseChannelName(Uri endpoint) { - return $"{Namespace}::RequestMessagesPulseChannelName::{endpoint}"; + return $"{Namespace}::RequestMessagesPulseChannel::{endpoint}"; } public async Task SubscribeToRequestMessagePulseChannel(Uri endpoint, Action onRequestMessagePulse, CancellationToken cancellationToken) @@ -45,93 +48,124 @@ public async Task PulseRequestPushedToEndpoint(Uri endpoint, CancellationToken c await facade.PublishToChannel(channelName, emptyJson, cancellationToken); } - // Request IDs list + // Pending Request IDs list + // A list in redis holding the set of available Pending Requests a Service can collect. + // The Service will Pop the Ids while the Client will Push new Pending Request Ids to the list. - static string KeyForNextRequestGuidInListForEndpoint(Uri endpoint) + static string PendingRequestGuidsQueueKey(Uri endpoint) { - return $"{Namespace}::NextRequestInListForEndpoint::{endpoint}"; + return $"{Namespace}::PendingRequestGuidsQueue::{endpoint}"; } public async Task PushRequestGuidOnToQueue(Uri endpoint, Guid guid, CancellationToken cancellationToken) { // TTL is high since it applies to all GUIDs in the queue. var ttlForAllRequestsGuidsInList = TimeSpan.FromDays(1); - await facade.ListRightPushAsync(KeyForNextRequestGuidInListForEndpoint(endpoint), guid.ToString(), ttlForAllRequestsGuidsInList, cancellationToken); + await facade.ListRightPushAsync(PendingRequestGuidsQueueKey(endpoint), guid.ToString(), ttlForAllRequestsGuidsInList, cancellationToken); } public async Task TryPopNextRequestGuid(Uri endpoint, CancellationToken cancellationToken) { - var result = await facade.ListLeftPopAsync(KeyForNextRequestGuidInListForEndpoint(endpoint), cancellationToken); + var result = await facade.ListLeftPopAsync(PendingRequestGuidsQueueKey(endpoint), cancellationToken); return result.ToGuid(); } - // Request Message + // Pending Request Message + // Stores the Pending Request Message for collection by the service. + // Note that the service will first need to TryPopNextRequestGuid to be able to + // fins the RequestMessage. static string RequestMessageKey(Uri endpoint, Guid requestId) { - return $"{Namespace}::RequestMessageKey::{endpoint}::{requestId}"; + return $"{Namespace}::RequestMessage::{endpoint}::{requestId}"; } - static string RequestField = "RequestField"; - - public async Task PutRequest(Uri endpoint, Guid requestId, string payload, TimeSpan requestPickupTimeout, CancellationToken cancellationToken) + static readonly string RequestMessageField = "RequestMessageField"; + + /// + /// The amount of time on top of the requestPickupTimout, the request will stay on the queue + /// before being automatically picked up. + /// The theory being we might need some grace period where it takes some time to collect + /// the request. It is not clear if we need this. This will be addressed in: + /// https://whimsical.com/under-some-circumstances-old-requests-can-still-be-sent-to-tenta-79CoT5PpvE1n5wApB6e2Zx + /// + static readonly TimeSpan AdditionalRequestMessageTtl = TimeSpan.FromMinutes(2); + + public async Task PutRequest(Uri endpoint, Guid requestId, string requestMessage, TimeSpan requestPickupTimeout, CancellationToken cancellationToken) { - var redisQueueItem = new RedisHalibutQueueItem2(requestId, payload); - - var serialisedQueueItem = JsonConvert.SerializeObject(redisQueueItem); - var requestKey = RequestMessageKey(endpoint, requestId); - var ttl = requestPickupTimeout + TimeSpan.FromMinutes(2); + var ttl = requestPickupTimeout + AdditionalRequestMessageTtl; - await facade.SetInHash(requestKey, RequestField, serialisedQueueItem, ttl, cancellationToken); + await facade.SetInHash(requestKey, RequestMessageField, requestMessage, ttl, cancellationToken); } - + + /// + /// Atomically Gets and removes the request from the queue. + /// Exactly up to one caller of this method will be given the RequestMessage, all + /// other calls will get null. + /// Note: currently a minor issue exists where redis disconnecting mid "Delete" call + /// can result in the Delete succeeding but no caller know if it succeeded. Thus, + /// it might be possible that no one Gets the request. In this case normal heart beat + /// timeouts will cause the request to be failed. + /// + /// + /// + /// + /// public async Task TryGetAndRemoveRequest(Uri endpoint, Guid requestId, CancellationToken cancellationToken) { var requestKey = RequestMessageKey(endpoint, requestId); - var requestMessage = await facade.TryGetAndDeleteFromHash(requestKey, RequestField, cancellationToken); - if (requestMessage == null) return null; - - var redisQueueItem = JsonConvert.DeserializeObject(requestMessage); - if (redisQueueItem is null) return null; - - return redisQueueItem.PayloadJson; + var requestMessage = await facade.TryGetAndDeleteFromHash(requestKey, RequestMessageField, cancellationToken); + return requestMessage; } public async Task IsRequestStillOnQueue(Uri endpoint, Guid requestId, CancellationToken cancellationToken) { var requestKey = RequestMessageKey(endpoint, requestId); - return await facade.HashContainsKey(requestKey, RequestField, cancellationToken); + return await facade.HashContainsKey(requestKey, RequestMessageField, cancellationToken); } // Cancellation channel - static string RequestCancelledChannel(Uri endpoint, Guid requestId) + // The node processing the request will subscribe to this channel, and the node + // sending the request will publish to this channel when the RPC has been cancelled. + static string RequestCancelledChannelName(Uri endpoint, Guid requestId) { return $"{Namespace}::RequestCancelledChannel::{endpoint}::{requestId}"; } - public async Task SubscribeToRequestCancellation(Uri endpoint, Guid request, - Func onCancellationReceived, - CancellationToken cancellationToken) + /// + /// + /// + /// + /// + /// Called when the RPC has been cancelled. + /// + /// + public async Task SubscribeToRequestCancellation(Uri endpoint, Guid request, Func onRpcCancellation, CancellationToken cancellationToken) { - var channelName = RequestCancelledChannel(endpoint, request); + var channelName = RequestCancelledChannelName(endpoint, request); return await facade.SubscribeToChannel(channelName, async foo => { string? response = foo.Message; - if (response is not null) await onCancellationReceived(); + if (response is not null) await onRpcCancellation(); }, cancellationToken); } public async Task PublishCancellation(Uri endpoint, Guid requestId, CancellationToken cancellationToken) { - var channelName = RequestCancelledChannel(endpoint, requestId); + var channelName = RequestCancelledChannelName(endpoint, requestId); await facade.PublishToChannel(channelName, "{}", cancellationToken); } + // Request cancellation + // Since pub/sub does not have guaranteed deliver, cancellation can also + // be detected by the RequestCancelledMarker. The node processing the request + // will poll for the existence of the RequestCancelledMarker, and if found + // it knows the RPC has been cancelled. public string RequestCancelledMarkerKey(Uri endpoint, Guid requestId) { - return $"{Namespace}::RequestCancelledMarkerKey::{endpoint}::{requestId}"; + return $"{Namespace}::RequestCancelledMarker::{endpoint}::{requestId}"; } public async Task MarkRequestAsCancelled(Uri endpoint, Guid requestId, TimeSpan ttl, CancellationToken cancellationToken) @@ -146,8 +180,17 @@ public async Task IsRequestMarkedAsCancelled(Uri endpoint, Guid requestId, return (await facade.GetString(key, cancellationToken)) != null; } + // Node heartbeat channels (per request). + // Each unique request has two node heart beat channels. + // One channel for the `RequestSenderNode` where the node that executes the RPC, + // publishes heart beats, for the duration of the time it is waiting for the RPC + // to be executed. + // Another channel for the `RequestProcessorNode` where the node that is sending the + // request to the service (e.g. Tentacle) is publishing heart beats, for the duration + // of processing the request. + // Both nodes are able to monitor to the heart beat channel of the other node to detect + // if the other node has gone offline. - // Node Processing the request heart beat channel static string NodeHeartBeatChannel(Uri endpoint, Guid requestId, HalibutQueueNodeSendingPulses nodeSendingPulsesType) { return $"{Namespace}::NodeHeartBeatChannel::{endpoint}::{requestId}::{nodeSendingPulsesType}"; @@ -168,30 +211,17 @@ public async Task SubscribeToNodeHeartBeatChannel( }, cancellationToken); } - public async Task SendHeartBeatFromNodeProcessingTheRequest(Uri endpoint, Guid requestId, HalibutQueueNodeSendingPulses nodeSendingPulsesType, CancellationToken cancellationToken) + public async Task SendNodeHeartBeat(Uri endpoint, Guid requestId, HalibutQueueNodeSendingPulses nodeSendingPulsesType, CancellationToken cancellationToken) { var channelName = NodeHeartBeatChannel(endpoint, requestId, nodeSendingPulsesType); await facade.PublishToChannel(channelName, "{}", cancellationToken); } - - // Backward compatibility methods (defaulting to Receiver for existing code) - public async Task SubscribeToNodeProcessingTheRequestHeartBeatChannel( - Uri endpoint, - Guid request, - Func onHeartBeat, - CancellationToken cancellationToken) - { - return await SubscribeToNodeHeartBeatChannel(endpoint, request, HalibutQueueNodeSendingPulses.Receiver, onHeartBeat, cancellationToken); - } - - public async Task SendHeartBeatFromNodeProcessingTheRequest(Uri endpoint, Guid requestId, CancellationToken cancellationToken) - { - await SendHeartBeatFromNodeProcessingTheRequest(endpoint, requestId, HalibutQueueNodeSendingPulses.Receiver, cancellationToken); - } - // Generic methods for watching for any string value being set + // Response channel. + // The node processing the request `RequestProcessorNode` will publish to this channel + // once the Response is available. - string ResponseAvailableChannel(Uri endpoint, Guid identifier) + string ResponseChannelName(Uri endpoint, Guid identifier) { return $"{Namespace}::ResponseAvailableChannel::{endpoint}::{identifier}"; } @@ -200,7 +230,7 @@ public async Task SubscribeToResponseChannel(Uri endpoint, Gui Func onValueReceived, CancellationToken cancellationToken) { - var channelName = ResponseAvailableChannel(endpoint, identifier); + var channelName = ResponseChannelName(endpoint, identifier); return await facade.SubscribeToChannel(channelName, async foo => { string? value = foo.Message; @@ -210,45 +240,35 @@ public async Task SubscribeToResponseChannel(Uri endpoint, Gui public async Task PublishThatResponseIsAvailable(Uri endpoint, Guid identifier, string value, CancellationToken cancellationToken) { - var channelName = ResponseAvailableChannel(endpoint, identifier); + var channelName = ResponseChannelName(endpoint, identifier); await facade.PublishToChannel(channelName, value, cancellationToken); } - string ResponseMarkerKey(Uri endpoint, Guid identifier) + // Response + // This is where the Response is placed in Redis. + + string ResponseMessageKey(Uri endpoint, Guid identifier) { - return $"{Namespace}::ResponseMarkerKey::{endpoint}::{identifier}"; + return $"{Namespace}::Response::{endpoint}::{identifier}"; } - public async Task MarkThatResponseIsSet(Uri endpoint, Guid identifier, string value, TimeSpan ttl, CancellationToken cancellationToken) + public async Task SetResponseMessage(Uri endpoint, Guid identifier, string responseMessage, TimeSpan ttl, CancellationToken cancellationToken) { - var key = ResponseMarkerKey(endpoint, identifier); - await facade.SetString(key, value, ttl, cancellationToken); + var key = ResponseMessageKey(endpoint, identifier); + await facade.SetString(key, responseMessage, ttl, cancellationToken); } public async Task GetResponseMessage(Uri endpoint, Guid identifier, CancellationToken cancellationToken) { - var key = ResponseMarkerKey(endpoint, identifier); + var key = ResponseMessageKey(endpoint, identifier); return await facade.GetString(key, cancellationToken); } - public async Task DeleteResponse(Uri endpoint, Guid identifier, CancellationToken cancellationToken) + public async Task DeleteResponseMessage(Uri endpoint, Guid identifier, CancellationToken cancellationToken) { - var key = ResponseMarkerKey(endpoint, identifier); + var key = ResponseMessageKey(endpoint, identifier); return await facade.DeleteString(key, cancellationToken); } } - - - public class RedisHalibutQueueItem2 - { - public RedisHalibutQueueItem2(Guid requestId, string payloadJson) - { - RequestId = requestId; - PayloadJson = payloadJson; - } - - public Guid RequestId { get; protected set; } - public string PayloadJson { get; protected set; } - } } #endif \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/IHalibutRedisTransport.cs b/source/Halibut/Queue/Redis/IHalibutRedisTransport.cs index 7f983a6e0..7ea6603f9 100644 --- a/source/Halibut/Queue/Redis/IHalibutRedisTransport.cs +++ b/source/Halibut/Queue/Redis/IHalibutRedisTransport.cs @@ -13,12 +13,12 @@ public interface IHalibutRedisTransport Task PulseRequestPushedToEndpoint(Uri endpoint, CancellationToken cancellationToken); Task PushRequestGuidOnToQueue(Uri endpoint, Guid guid, CancellationToken cancellationToken); Task TryPopNextRequestGuid(Uri endpoint, CancellationToken cancellationToken); - Task PutRequest(Uri endpoint, Guid requestId, string payload, TimeSpan requestPickupTimeout, CancellationToken cancellationToken); + Task PutRequest(Uri endpoint, Guid requestId, string requestMessage, TimeSpan requestPickupTimeout, CancellationToken cancellationToken); Task TryGetAndRemoveRequest(Uri endpoint, Guid requestId, CancellationToken cancellationToken); Task IsRequestStillOnQueue(Uri endpoint, Guid requestId, CancellationToken cancellationToken); Task SubscribeToRequestCancellation(Uri endpoint, Guid request, - Func onCancellationReceived, + Func onRpcCancellation, CancellationToken cancellationToken); Task PublishCancellation(Uri endpoint, Guid requestId, CancellationToken cancellationToken); @@ -33,23 +33,16 @@ Task SubscribeToNodeHeartBeatChannel( Func onHeartBeat, CancellationToken cancellationToken); - Task SendHeartBeatFromNodeProcessingTheRequest(Uri endpoint, Guid requestId, HalibutQueueNodeSendingPulses nodeSendingPulsesType, CancellationToken cancellationToken); - Task SendHeartBeatFromNodeProcessingTheRequest(Uri endpoint, Guid requestId, CancellationToken cancellationToken); - - Task SubscribeToNodeProcessingTheRequestHeartBeatChannel( - Uri endpoint, - Guid request, - Func onHeartBeat, - CancellationToken cancellationToken); - + Task SendNodeHeartBeat(Uri endpoint, Guid requestId, HalibutQueueNodeSendingPulses nodeSendingPulsesType, CancellationToken cancellationToken); + Task SubscribeToResponseChannel(Uri endpoint, Guid identifier, Func onValueReceived, CancellationToken cancellationToken); Task PublishThatResponseIsAvailable(Uri endpoint, Guid identifier, string value, CancellationToken cancellationToken); - Task MarkThatResponseIsSet(Uri endpoint, Guid identifier, string value, TimeSpan ttl, CancellationToken cancellationToken); + Task SetResponseMessage(Uri endpoint, Guid identifier, string responseMessage, TimeSpan ttl, CancellationToken cancellationToken); Task GetResponseMessage(Uri endpoint, Guid identifier, CancellationToken cancellationToken); - Task DeleteResponse(Uri endpoint, Guid identifier, CancellationToken cancellationToken); + Task DeleteResponseMessage(Uri endpoint, Guid identifier, CancellationToken cancellationToken); } } #endif \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs b/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs index 59c7ef511..95470767c 100644 --- a/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs +++ b/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs @@ -12,8 +12,13 @@ namespace Halibut.Queue.Redis { public enum HalibutQueueNodeSendingPulses { - Sender, - Receiver + // The node the RPC is executing on. + // The node that calls QueueAndWait + RequestSenderNode, + + // The node sending/receiving the Request to/from the service. + // The node that calls Dequeue and ApplyResponse. + RequestProcessorNode } public class NodeHeartBeatSender : IAsyncDisposable { @@ -58,7 +63,7 @@ async Task SendPulsesWhileProcessingRequest(TimeSpan defaultDelayBetweenPulses, { try { - await halibutRedisTransport.SendHeartBeatFromNodeProcessingTheRequest(endpoint, requestActivityId, nodeSendingPulsesType, cancellationToken); + await halibutRedisTransport.SendNodeHeartBeat(endpoint, requestActivityId, nodeSendingPulsesType, cancellationToken); delayBetweenPulse = defaultDelayBetweenPulses; log.Write(EventType.Diagnostic, "Successfully sent heartbeat for {0} node, request {1}, next pulse in {2} seconds", nodeSendingPulsesType, requestActivityId, delayBetweenPulse.TotalSeconds); } @@ -97,7 +102,7 @@ public static async Task WatchThatNodeProces { await WaitForRequestToBeCollected(endpoint, request, redisPending, halibutRedisTransport, timeBetweenCheckingIfRequestWasCollected, log, cts.Token); - return await WatchForPulsesFromNode(endpoint, request.ActivityId, halibutRedisTransport, log, maxTimeBetweenHeartBeetsBeforeProcessingNodeIsAssumedToBeOffline, HalibutQueueNodeSendingPulses.Receiver, cts.Token); + return await WatchForPulsesFromNode(endpoint, request.ActivityId, halibutRedisTransport, log, maxTimeBetweenHeartBeetsBeforeProcessingNodeIsAssumedToBeOffline, HalibutQueueNodeSendingPulses.RequestProcessorNode, cts.Token); } catch (Exception) when (cts.Token.IsCancellationRequested) { @@ -113,7 +118,7 @@ public static async Task WatchThatNodeWhichS TimeSpan maxTimeBetweenSenderHeartBeetsBeforeSenderIsAssumedToBeOffline, CancellationToken watchCancellationToken) { - return await WatchForPulsesFromNode(endpoint, requestActivityId, halibutRedisTransport, log, maxTimeBetweenSenderHeartBeetsBeforeSenderIsAssumedToBeOffline, HalibutQueueNodeSendingPulses.Sender, watchCancellationToken); + return await WatchForPulsesFromNode(endpoint, requestActivityId, halibutRedisTransport, log, maxTimeBetweenSenderHeartBeetsBeforeSenderIsAssumedToBeOffline, HalibutQueueNodeSendingPulses.RequestSenderNode, watchCancellationToken); } private static async Task WatchForPulsesFromNode( diff --git a/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs b/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs index c9e7f2b38..cef5449f5 100644 --- a/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs +++ b/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs @@ -11,26 +11,27 @@ namespace Halibut.Queue.Redis { public class PollAndSubscribeToResponse : IAsyncDisposable { - public static async Task TrySendMessage( + public static async Task SendResponse( IHalibutRedisTransport halibutRedisTransport, Uri endpoint, Guid activityId, - string value, + string responseMessage, TimeSpan ttl, ILog log) { log.Write(EventType.Diagnostic, "Attempting to set response for - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); await using var cts = new CancelOnDisposeCancellationToken(); - cts.CancelAfter(TimeSpan.FromMinutes(2)); // Best efforts. + // More than ten minutes to send the response to redis, seems sus. + cts.CancelAfter(TimeSpan.FromMinutes(10)); try { log.Write(EventType.Diagnostic, "Marking response as set - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); - await halibutRedisTransport.MarkThatResponseIsSet(endpoint, activityId, value, ttl, cts.Token); + await halibutRedisTransport.SetResponseMessage(endpoint, activityId, responseMessage, ttl, cts.Token); log.Write(EventType.Diagnostic, "Publishing response notification - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); - await halibutRedisTransport.PublishThatResponseIsAvailable(endpoint, activityId, value, cts.Token); + await halibutRedisTransport.PublishThatResponseIsAvailable(endpoint, activityId, responseMessage, cts.Token); log.Write(EventType.Diagnostic, "Successfully set response - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); } @@ -141,7 +142,6 @@ public async Task TryGetResponseFromRedis(string detectedBy, CancellationT if (ResponseJsonCompletionSource.Task.IsCompleted) return true; - // TODO wrap in try var responseJson = await halibutRedisTransport.GetResponseMessage(endpoint, activityId, token); if (responseJson != null) @@ -163,7 +163,7 @@ async Task DeleteResponseFromRedis(string detectedBy, CancellationToken token) { try { - await halibutRedisTransport.DeleteResponse(endpoint, activityId, token); + await halibutRedisTransport.DeleteResponseMessage(endpoint, activityId, token); } catch (Exception ex) { diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index 0398a477d..cb06e37b2 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -122,7 +122,7 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can var tryClearRequestFromQueueAtMostOnce = new AsyncLazy(async () => await TryClearRequestFromQueue(request, pending)); try { - await using var senderPulse = new NodeHeartBeatSender(endpoint, request.ActivityId, halibutRedisTransport, log, HalibutQueueNodeSendingPulses.Sender, DelayBetweenHeartBeatsForRequestSender); + await using var senderPulse = new NodeHeartBeatSender(endpoint, request.ActivityId, halibutRedisTransport, log, HalibutQueueNodeSendingPulses.RequestSenderNode, DelayBetweenHeartBeatsForRequestSender); // Make the request available before we tell people it is available. try { @@ -371,7 +371,7 @@ async Task TryClearRequestFromQueue(RequestMessage request, RedisPendingRe // In that case we will just time out because of the lack of heart beats. var dataLossCT = await this.watchForRedisLosingAllItsData.GetTokenForDataLoseDetection(TimeSpan.FromSeconds(30), queueToken); - disposables.AddAsyncDisposable(new NodeHeartBeatSender(endpoint, pending.ActivityId, halibutRedisTransport, log, HalibutQueueNodeSendingPulses.Receiver, DelayBetweenHeartBeatsForRequestProcessor)); + disposables.AddAsyncDisposable(new NodeHeartBeatSender(endpoint, pending.ActivityId, halibutRedisTransport, log, HalibutQueueNodeSendingPulses.RequestProcessorNode, DelayBetweenHeartBeatsForRequestProcessor)); var watcher = new WatchForRequestCancellationOrSenderDisconnect(endpoint, pending.ActivityId, halibutRedisTransport, NodeIsOfflineHeartBeatTimeoutForRequestSender, log); disposables.AddAsyncDisposable(watcher); @@ -441,9 +441,9 @@ public async Task ApplyResponse(ResponseMessage response, Guid requestActivityId response = ResponseMessage.FromException(response, new HalibutClientException(RequestAbandonedMessage)); } } - var payload = await messageReaderWriter.PrepareResponse(response, cancellationToken); + var responseJson = await messageReaderWriter.PrepareResponse(response, cancellationToken); log.Write(EventType.MessageExchange, "Sending response message for request {0}", requestActivityId); - await PollAndSubscribeToResponse.TrySendMessage(halibutRedisTransport, endpoint, requestActivityId, payload, TTLOfResponseMessage, log); + await PollAndSubscribeToResponse.SendResponse(halibutRedisTransport, endpoint, requestActivityId, responseJson, TTLOfResponseMessage, log); log.Write(EventType.MessageExchange, "Successfully applied response for request {0}", requestActivityId); } catch (Exception ex) From 38e95e526d539a677bc92d3a5fa26d45888f1726 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Mon, 18 Aug 2025 14:50:56 +1000 Subject: [PATCH 078/137] Cleanup --- .../RedisPendingRequestQueueBuilder.cs | 4 +- .../Halibut.Tests/ManyPollingTentacleTests.cs | 2 + .../Queue/Redis/NodeHeartBeatSenderFixture.cs | 1 + .../WatchForRedisLosingAllItsDataFixture.cs | 8 +-- .../{ => RedisHelpers}/RedisFacadeFixture.cs | 3 +- .../RedisFacadeWhenRedisGoesDownAwayTests.cs | 4 +- .../Redis/RedisPendingRequestQueueFixture.cs | 42 +++++++------- ...leDataLossWatchForRedisLosingAllItsData.cs | 1 + .../HalibutRedisTransportWithVirtuals.cs | 5 -- .../MessageReaderWriterExtensionsMethods.cs | 33 +++++------ ...LosingDataWatchForRedisLosingAllItsData.cs | 1 + .../Queue/Redis/Utils/RedisFacadeBuilder.cs | 1 + ...IMessageSerialiserAndDataStreamStorage.cs} | 13 ++++- .../MessageSerialiserAndDataStreamStorage.cs} | 7 ++- .../IWatchForRedisLosingAllItsData.cs | 3 +- .../WatchForRedisLosingAllItsData.cs | 43 +++++++++----- .../HalibutRedisTransport.cs | 4 +- .../IHalibutRedisTransport.cs | 12 +++- .../Redis/{ => RedisHelpers}/RedisFacade.cs | 57 +++++++++---------- .../Queue/Redis/RedisPendingRequestQueue.cs | 16 +++--- .../Redis/RedisPendingRequestQueueFactory.cs | 5 +- 21 files changed, 149 insertions(+), 116 deletions(-) rename source/Halibut.Tests/Queue/Redis/{ => RedisDataLoseDetection}/WatchForRedisLosingAllItsDataFixture.cs (96%) rename source/Halibut.Tests/Queue/Redis/{ => RedisHelpers}/RedisFacadeFixture.cs (99%) rename source/Halibut.Tests/Queue/Redis/{ => RedisHelpers}/RedisFacadeWhenRedisGoesDownAwayTests.cs (99%) rename source/Halibut/Queue/Redis/{IMessageReaderWriter.cs => MessageStorage/IMessageSerialiserAndDataStreamStorage.cs} (50%) rename source/Halibut/Queue/Redis/{MessageReaderWriter.cs => MessageStorage/MessageSerialiserAndDataStreamStorage.cs} (85%) rename source/Halibut/Queue/Redis/{ => RedisDataLoseDetection}/IWatchForRedisLosingAllItsData.cs (83%) rename source/Halibut/Queue/Redis/{ => RedisDataLoseDetection}/WatchForRedisLosingAllItsData.cs (75%) rename source/Halibut/Queue/Redis/{ => RedisHelpers}/HalibutRedisTransport.cs (99%) rename source/Halibut/Queue/Redis/{ => RedisHelpers}/IHalibutRedisTransport.cs (97%) rename source/Halibut/Queue/Redis/{ => RedisHelpers}/RedisFacade.cs (87%) diff --git a/source/Halibut.Tests/Builders/RedisPendingRequestQueueBuilder.cs b/source/Halibut.Tests/Builders/RedisPendingRequestQueueBuilder.cs index a3eeb311e..e91609634 100644 --- a/source/Halibut.Tests/Builders/RedisPendingRequestQueueBuilder.cs +++ b/source/Halibut.Tests/Builders/RedisPendingRequestQueueBuilder.cs @@ -4,6 +4,8 @@ using Halibut.Logging; using Halibut.Queue.QueuedDataStreams; using Halibut.Queue.Redis; +using Halibut.Queue.Redis.MessageStorage; +using Halibut.Queue.Redis.RedisHelpers; using Halibut.Tests.Queue; using Halibut.Tests.Queue.Redis.Utils; using Halibut.Tests.Support.Logging; @@ -56,7 +58,7 @@ public QueueHolder Build() var redisTransport = new HalibutRedisTransport(redisFacade); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); - var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); + var messageReaderWriter = new MessageSerialiserAndDataStreamStorage(messageSerializer, dataStreamStore); var request = new RequestMessageBuilder("poll://test-endpoint").Build(); diff --git a/source/Halibut.Tests/ManyPollingTentacleTests.cs b/source/Halibut.Tests/ManyPollingTentacleTests.cs index 602700119..d8584811f 100644 --- a/source/Halibut.Tests/ManyPollingTentacleTests.cs +++ b/source/Halibut.Tests/ManyPollingTentacleTests.cs @@ -12,6 +12,8 @@ using Halibut.Logging; using Halibut.Queue.QueuedDataStreams; using Halibut.Queue.Redis; +using Halibut.Queue.Redis.RedisDataLoseDetection; +using Halibut.Queue.Redis.RedisHelpers; using Halibut.ServiceModel; using Halibut.Tests.Queue.Redis.Utils; using Halibut.Tests.Support; diff --git a/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs b/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs index 28375104e..12383e2ae 100644 --- a/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs @@ -7,6 +7,7 @@ using FluentAssertions; using Halibut.Logging; using Halibut.Queue.Redis; +using Halibut.Queue.Redis.RedisHelpers; using Halibut.Tests.Builders; using Halibut.Tests.Queue.Redis.Utils; using Halibut.Tests.Support; diff --git a/source/Halibut.Tests/Queue/Redis/WatchForRedisLosingAllItsDataFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisDataLoseDetection/WatchForRedisLosingAllItsDataFixture.cs similarity index 96% rename from source/Halibut.Tests/Queue/Redis/WatchForRedisLosingAllItsDataFixture.cs rename to source/Halibut.Tests/Queue/Redis/RedisDataLoseDetection/WatchForRedisLosingAllItsDataFixture.cs index d95058d31..9fdbdeff1 100644 --- a/source/Halibut.Tests/Queue/Redis/WatchForRedisLosingAllItsDataFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisDataLoseDetection/WatchForRedisLosingAllItsDataFixture.cs @@ -2,16 +2,12 @@ using System; using System.Threading.Tasks; using FluentAssertions; -using Halibut.Logging; -using Halibut.Queue.Redis; +using Halibut.Queue.Redis.RedisDataLoseDetection; using Halibut.Tests.Queue.Redis.Utils; using Halibut.Tests.Support; -using Halibut.Tests.Support.Logging; -using Halibut.Tests.TestSetup.Redis; using NUnit.Framework; -using Octopus.TestPortForwarder; -namespace Halibut.Tests.Queue.Redis +namespace Halibut.Tests.Queue.Redis.RedisDataLoseDetection { [RedisTest] public class WatchForRedisLosingAllItsDataFixture : BaseTest diff --git a/source/Halibut.Tests/Queue/Redis/RedisFacadeFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisHelpers/RedisFacadeFixture.cs similarity index 99% rename from source/Halibut.Tests/Queue/Redis/RedisFacadeFixture.cs rename to source/Halibut.Tests/Queue/Redis/RedisHelpers/RedisFacadeFixture.cs index fcadd70ea..67e2dda7c 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisFacadeFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisHelpers/RedisFacadeFixture.cs @@ -5,13 +5,14 @@ using FluentAssertions; using Halibut.Logging; using Halibut.Queue.Redis; +using Halibut.Queue.Redis.RedisHelpers; using Halibut.Tests.Queue.Redis.Utils; using Halibut.Tests.Support; using Halibut.Tests.Support.Logging; using Halibut.Util.AsyncEx; using NUnit.Framework; -namespace Halibut.Tests.Queue.Redis +namespace Halibut.Tests.Queue.Redis.RedisHelpers { [RedisTest] public class RedisFacadeFixture : BaseTest diff --git a/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs b/source/Halibut.Tests/Queue/Redis/RedisHelpers/RedisFacadeWhenRedisGoesDownAwayTests.cs similarity index 99% rename from source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs rename to source/Halibut.Tests/Queue/Redis/RedisHelpers/RedisFacadeWhenRedisGoesDownAwayTests.cs index 7c75c317b..ad211ba4c 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisFacadeWhenRedisGoesDownAwayTests.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisHelpers/RedisFacadeWhenRedisGoesDownAwayTests.cs @@ -6,16 +6,16 @@ using FluentAssertions; using Halibut.Logging; using Halibut.Queue.Redis; +using Halibut.Queue.Redis.RedisHelpers; using Halibut.Tests.Queue.Redis.Utils; using Halibut.Tests.Support; using Halibut.Tests.Support.Logging; using Halibut.Tests.TestSetup.Redis; using Halibut.Util; using NUnit.Framework; -using Octopus.TestPortForwarder; using Try = Halibut.Util.Try; -namespace Halibut.Tests.Queue.Redis +namespace Halibut.Tests.Queue.Redis.RedisHelpers { [RedisTest] [SuppressMessage("Usage", "VSTHRD003:Avoid awaiting foreign Tasks")] diff --git a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs index 2292e0f76..fdb719845 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs @@ -10,6 +10,8 @@ using Halibut.Queue.QueuedDataStreams; using Halibut.Queue.Redis; using Halibut.Queue.Redis.Exceptions; +using Halibut.Queue.Redis.MessageStorage; +using Halibut.Queue.Redis.RedisHelpers; using Halibut.ServiceModel; using Halibut.Tests.Builders; using Halibut.Tests.Queue.Redis.Utils; @@ -44,7 +46,7 @@ public async Task DequeueAsync_ShouldReturnRequestFromRedis() var redisTransport = new HalibutRedisTransport(redisFacade); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); - var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); + var messageReaderWriter = new MessageSerialiserAndDataStreamStorage(messageSerializer, dataStreamStore); var request = new RequestMessageBuilder("poll://test-endpoint").Build(); @@ -73,7 +75,7 @@ public async Task WhenThePickupTimeoutExpires_AnErrorsReturned_AndTheRequestCanN var redisTransport = new HalibutRedisTransport(redisFacade); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); - var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); + var messageReaderWriter = new MessageSerialiserAndDataStreamStorage(messageSerializer, dataStreamStore); var request = new RequestMessageBuilder("poll://test-endpoint") .WithServiceEndpoint(b => b.WithPollingRequestQueueTimeout(TimeSpan.FromMilliseconds(100))) @@ -102,7 +104,7 @@ public async Task FullSendAndReceiveShouldWork() var redisTransport = new HalibutRedisTransport(redisFacade); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); - var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); + var messageReaderWriter = new MessageSerialiserAndDataStreamStorage(messageSerializer, dataStreamStore); var request = new RequestMessageBuilder("poll://test-endpoint").Build(); @@ -137,7 +139,7 @@ public async Task WhenReadingTheResponseFromTheQueueFails_TheQueueAndWaitTaskRet var redisTransport = new HalibutRedisTransport(redisFacade); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); - var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore) + var messageReaderWriter = new MessageSerialiserAndDataStreamStorage(messageSerializer, dataStreamStore) .ThrowsOnReadResponse(() => new OperationCanceledException()); var request = new RequestMessageBuilder("poll://test-endpoint").Build(); @@ -171,7 +173,7 @@ public async Task WhenEnteringTheQueue_AndRedisIsUnavailable_ARetryableException var redisTransport = new HalibutRedisTransport(redisFacade); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); - var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); + var messageReaderWriter = new MessageSerialiserAndDataStreamStorage(messageSerializer, dataStreamStore); var request = new RequestMessageBuilder("poll://test-endpoint").Build(); var queue = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); @@ -206,7 +208,7 @@ public async Task WhenEnteringTheQueue_AndRedisIsUnavailableAndDataLoseOccurs_AR var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); - var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); + var messageReaderWriter = new MessageSerialiserAndDataStreamStorage(messageSerializer, dataStreamStore); var request = new RequestMessageBuilder("poll://test-endpoint").Build(); @@ -230,7 +232,7 @@ public async Task WhenPreparingRequestFails_ARetryableExceptionIsThrown() var redisTransport = new HalibutRedisTransport(redisFacade); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); - var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore) + var messageReaderWriter = new MessageSerialiserAndDataStreamStorage(messageSerializer, dataStreamStore) .ThrowsOnPrepareRequest(() => new OperationCanceledException()); var request = new RequestMessageBuilder("poll://test-endpoint").Build(); @@ -252,7 +254,7 @@ public async Task WhenDataLostIsDetected_InFlightRequestShouldBeAbandoned_AndARe var redisTransport = new HalibutRedisTransport(redisFacade); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); - var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); + var messageReaderWriter = new MessageSerialiserAndDataStreamStorage(messageSerializer, dataStreamStore); var request = new RequestMessageBuilder("poll://test-endpoint").Build(); @@ -296,7 +298,7 @@ public async Task OnceARequestIsComplete_NoInflightDisposableShouldExist() var redisTransport = new HalibutRedisTransport(redisFacade); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); - var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); + var messageReaderWriter = new MessageSerialiserAndDataStreamStorage(messageSerializer, dataStreamStore); var request = new RequestMessageBuilder("poll://test-endpoint").Build(); @@ -329,7 +331,7 @@ public async Task OnceARequestIsComplete_NoSenderHeartBeatsShouldBeSent() var redisTransport = new HalibutRedisTransport(redisFacade); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); - var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); + var messageReaderWriter = new MessageSerialiserAndDataStreamStorage(messageSerializer, dataStreamStore); var request = new RequestMessageBuilder("poll://test-endpoint").Build(); @@ -373,7 +375,7 @@ public async Task OnceARequestIsComplete_NoReceiverHeartBeatsShouldBeSent() var redisTransport = new HalibutRedisTransport(redisFacade); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); - var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); + var messageReaderWriter = new MessageSerialiserAndDataStreamStorage(messageSerializer, dataStreamStore); var request = new RequestMessageBuilder("poll://test-endpoint").Build(); @@ -418,7 +420,7 @@ public async Task FullSendAndReceiveWithDataStreamShouldWork() var redisTransport = new HalibutRedisTransport(redisFacade); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); - var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); + var messageReaderWriter = new MessageSerialiserAndDataStreamStorage(messageSerializer, dataStreamStore); var request = new RequestMessageBuilder("poll://test-endpoint").Build(); request.Params = new[] { new ComplexObjectMultipleDataStreams(DataStream.FromString("hello"), DataStream.FromString("world")) }; @@ -461,7 +463,7 @@ public async Task WhenTheReceiversConnectionToRedisIsInterruptedAndRestoredBefor var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); - var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); + var messageReaderWriter = new MessageSerialiserAndDataStreamStorage(messageSerializer, dataStreamStore); var request = new RequestMessageBuilder("poll://test-endpoint").Build(); @@ -507,7 +509,7 @@ public async Task WhenTheReceiverDoesntCollectWorkImmediately_TheRequestCanSitOn var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); - var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); + var messageReaderWriter = new MessageSerialiserAndDataStreamStorage(messageSerializer, dataStreamStore); var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(redisFacade), messageReaderWriter, new HalibutTimeoutsAndLimits()); // We are testing that we don't expect heart beats before the request is collected. @@ -539,7 +541,7 @@ public async Task WhenTheSendersConnectionToRedisIsBrieflyInterruptedWhileSendin var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); - var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); + var messageReaderWriter = new MessageSerialiserAndDataStreamStorage(messageSerializer, dataStreamStore); var request = new RequestMessageBuilder("poll://test-endpoint").Build(); @@ -576,7 +578,7 @@ public async Task WhenTheReceiverDequeuesWorkAndThenDisconnectsFromRedisForEver_ var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); - var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); + var messageReaderWriter = new MessageSerialiserAndDataStreamStorage(messageSerializer, dataStreamStore); var halibutTimeoutAndLimits = new HalibutTimeoutsAndLimits(); halibutTimeoutAndLimits.PollingRequestQueueTimeout = TimeSpan.FromDays(1); @@ -629,7 +631,7 @@ public async Task WhenTheReceiverDequeuesWork_AndTheSenderDisconnects_AndNeverRe var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); - var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); + var messageReaderWriter = new MessageSerialiserAndDataStreamStorage(messageSerializer, dataStreamStore); var halibutTimeoutAndLimits = new HalibutTimeoutsAndLimits(); halibutTimeoutAndLimits.PollingRequestQueueTimeout = TimeSpan.FromDays(1); @@ -676,7 +678,7 @@ public async Task WhenTheSenderBrieflyDisconnectsFromRedisRightWhenTheReceiverSe var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); - var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); + var messageReaderWriter = new MessageSerialiserAndDataStreamStorage(messageSerializer, dataStreamStore); await using var stableConnection = RedisFacadeBuilder.CreateRedisFacade(prefix: guid); using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); @@ -719,7 +721,7 @@ public async Task WhenTheRequestReceiverDetectsRedisDataLose_AndTheRequestSender var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); - var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); + var messageReaderWriter = new MessageSerialiserAndDataStreamStorage(messageSerializer, dataStreamStore); await using var stableConnection = RedisFacadeBuilder.CreateRedisFacade(prefix: guid); @@ -804,7 +806,7 @@ public async Task CancellingARequestShouldResultInTheDequeuedResponseTokenBeingC var redisTransport = new HalibutRedisTransport(redisFacade); var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); var messageSerializer = new QueueMessageSerializerBuilder().Build(); - var messageReaderWriter = new MessageReaderWriter(messageSerializer, dataStreamStore); + var messageReaderWriter = new MessageSerialiserAndDataStreamStorage(messageSerializer, dataStreamStore); var request = new RequestMessageBuilder("poll://test-endpoint").Build(); request.Params = new[] { new ComplexObjectMultipleDataStreams(DataStream.FromString("hello"), DataStream.FromString("world")) }; diff --git a/source/Halibut.Tests/Queue/Redis/Utils/CancellableDataLossWatchForRedisLosingAllItsData.cs b/source/Halibut.Tests/Queue/Redis/Utils/CancellableDataLossWatchForRedisLosingAllItsData.cs index aa231fb91..e8a5135ea 100644 --- a/source/Halibut.Tests/Queue/Redis/Utils/CancellableDataLossWatchForRedisLosingAllItsData.cs +++ b/source/Halibut.Tests/Queue/Redis/Utils/CancellableDataLossWatchForRedisLosingAllItsData.cs @@ -2,6 +2,7 @@ using System.Threading; using System.Threading.Tasks; using Halibut.Queue.Redis; +using Halibut.Queue.Redis.RedisDataLoseDetection; using Halibut.Util; using Try = Halibut.Tests.Support.Try; diff --git a/source/Halibut.Tests/Queue/Redis/Utils/HalibutRedisTransportWithVirtuals.cs b/source/Halibut.Tests/Queue/Redis/Utils/HalibutRedisTransportWithVirtuals.cs index 7c18f4832..9c4c1d395 100644 --- a/source/Halibut.Tests/Queue/Redis/Utils/HalibutRedisTransportWithVirtuals.cs +++ b/source/Halibut.Tests/Queue/Redis/Utils/HalibutRedisTransportWithVirtuals.cs @@ -62,11 +62,6 @@ public Task PublishCancellation(Uri endpoint, Guid requestId, CancellationToken return halibutRedisTransport.PublishCancellation(endpoint, requestId, cancellationToken); } - public string RequestCancelledMarkerKey(Uri endpoint, Guid requestId) - { - return halibutRedisTransport.RequestCancelledMarkerKey(endpoint, requestId); - } - public Task MarkRequestAsCancelled(Uri endpoint, Guid requestId, TimeSpan ttl, CancellationToken cancellationToken) { return halibutRedisTransport.MarkRequestAsCancelled(endpoint, requestId, ttl, cancellationToken); diff --git a/source/Halibut.Tests/Queue/Redis/Utils/MessageReaderWriterExtensionsMethods.cs b/source/Halibut.Tests/Queue/Redis/Utils/MessageReaderWriterExtensionsMethods.cs index fb5700201..db70897e6 100644 --- a/source/Halibut.Tests/Queue/Redis/Utils/MessageReaderWriterExtensionsMethods.cs +++ b/source/Halibut.Tests/Queue/Redis/Utils/MessageReaderWriterExtensionsMethods.cs @@ -2,58 +2,59 @@ using System.Threading; using System.Threading.Tasks; using Halibut.Queue.Redis; +using Halibut.Queue.Redis.MessageStorage; using Halibut.Transport.Protocol; namespace Halibut.Tests.Queue.Redis.Utils { public static class MessageReaderWriterExtensionsMethods { - public static IMessageReaderWriter ThrowsOnReadResponse(this IMessageReaderWriter messageReaderWriter, Func exceptionFactory) + public static IMessageSerialiserAndDataStreamStorage ThrowsOnReadResponse(this IMessageSerialiserAndDataStreamStorage messageSerialiserAndDataStreamStorage, Func exceptionFactory) { - return new MessageReaderWriterThatThrowsWhenReadingResponse(messageReaderWriter, exceptionFactory); + return new MessageSerialiserAndDataStreamStorageThatThrowsWhenReadingResponse(messageSerialiserAndDataStreamStorage, exceptionFactory); } - public static IMessageReaderWriter ThrowsOnPrepareRequest(this IMessageReaderWriter messageReaderWriter, Func exception) + public static IMessageSerialiserAndDataStreamStorage ThrowsOnPrepareRequest(this IMessageSerialiserAndDataStreamStorage messageSerialiserAndDataStreamStorage, Func exception) { - return new MessageReaderWriterThatThrowsOnPrepareRequest(messageReaderWriter, exception); + return new MessageSerialiserAndDataStreamStorageThatThrowsOnPrepareRequest(messageSerialiserAndDataStreamStorage, exception); } } - class MessageReaderWriterWithVirtualMethods : IMessageReaderWriter + class MessageSerialiserAndDataStreamStorageWithVirtualMethods : IMessageSerialiserAndDataStreamStorage { - readonly IMessageReaderWriter messageReaderWriter; + readonly IMessageSerialiserAndDataStreamStorage messageSerialiserAndDataStreamStorage; - public MessageReaderWriterWithVirtualMethods(IMessageReaderWriter messageReaderWriter) + public MessageSerialiserAndDataStreamStorageWithVirtualMethods(IMessageSerialiserAndDataStreamStorage messageSerialiserAndDataStreamStorage) { - this.messageReaderWriter = messageReaderWriter; + this.messageSerialiserAndDataStreamStorage = messageSerialiserAndDataStreamStorage; } public virtual Task PrepareRequest(RequestMessage request, CancellationToken cancellationToken) { - return messageReaderWriter.PrepareRequest(request, cancellationToken); + return messageSerialiserAndDataStreamStorage.PrepareRequest(request, cancellationToken); } public virtual Task ReadRequest(string jsonRequest, CancellationToken cancellationToken) { - return messageReaderWriter.ReadRequest(jsonRequest, cancellationToken); + return messageSerialiserAndDataStreamStorage.ReadRequest(jsonRequest, cancellationToken); } public virtual Task PrepareResponse(ResponseMessage response, CancellationToken cancellationToken) { - return messageReaderWriter.PrepareResponse(response, cancellationToken); + return messageSerialiserAndDataStreamStorage.PrepareResponse(response, cancellationToken); } public virtual Task ReadResponse(string jsonResponse, CancellationToken cancellationToken) { - return messageReaderWriter.ReadResponse(jsonResponse, cancellationToken); + return messageSerialiserAndDataStreamStorage.ReadResponse(jsonResponse, cancellationToken); } } - class MessageReaderWriterThatThrowsWhenReadingResponse : MessageReaderWriterWithVirtualMethods + class MessageSerialiserAndDataStreamStorageThatThrowsWhenReadingResponse : MessageSerialiserAndDataStreamStorageWithVirtualMethods { readonly Func exception; - public MessageReaderWriterThatThrowsWhenReadingResponse(IMessageReaderWriter messageReaderWriter, Func exception) : base(messageReaderWriter) + public MessageSerialiserAndDataStreamStorageThatThrowsWhenReadingResponse(IMessageSerialiserAndDataStreamStorage messageSerialiserAndDataStreamStorage, Func exception) : base(messageSerialiserAndDataStreamStorage) { this.exception = exception; } @@ -64,11 +65,11 @@ public override Task ReadResponse(string jsonResponse, Cancella } } - class MessageReaderWriterThatThrowsOnPrepareRequest : MessageReaderWriterWithVirtualMethods + class MessageSerialiserAndDataStreamStorageThatThrowsOnPrepareRequest : MessageSerialiserAndDataStreamStorageWithVirtualMethods { readonly Func exception; - public MessageReaderWriterThatThrowsOnPrepareRequest(IMessageReaderWriter messageReaderWriter, Func exception) : base(messageReaderWriter) + public MessageSerialiserAndDataStreamStorageThatThrowsOnPrepareRequest(IMessageSerialiserAndDataStreamStorage messageSerialiserAndDataStreamStorage, Func exception) : base(messageSerialiserAndDataStreamStorage) { this.exception = exception; } diff --git a/source/Halibut.Tests/Queue/Redis/Utils/NeverLosingDataWatchForRedisLosingAllItsData.cs b/source/Halibut.Tests/Queue/Redis/Utils/NeverLosingDataWatchForRedisLosingAllItsData.cs index b27e5c284..b376838a6 100644 --- a/source/Halibut.Tests/Queue/Redis/Utils/NeverLosingDataWatchForRedisLosingAllItsData.cs +++ b/source/Halibut.Tests/Queue/Redis/Utils/NeverLosingDataWatchForRedisLosingAllItsData.cs @@ -4,6 +4,7 @@ using System.Threading; using System.Threading.Tasks; using Halibut.Queue.Redis; +using Halibut.Queue.Redis.RedisDataLoseDetection; namespace Halibut.Tests.Queue.Redis.Utils { diff --git a/source/Halibut.Tests/Queue/Redis/Utils/RedisFacadeBuilder.cs b/source/Halibut.Tests/Queue/Redis/Utils/RedisFacadeBuilder.cs index ea022b749..68ace900c 100644 --- a/source/Halibut.Tests/Queue/Redis/Utils/RedisFacadeBuilder.cs +++ b/source/Halibut.Tests/Queue/Redis/Utils/RedisFacadeBuilder.cs @@ -1,6 +1,7 @@ using System; using Halibut.Logging; using Halibut.Queue.Redis; +using Halibut.Queue.Redis.RedisHelpers; using Halibut.Tests.Support.Logging; using Halibut.Tests.TestSetup.Redis; using Octopus.TestPortForwarder; diff --git a/source/Halibut/Queue/Redis/IMessageReaderWriter.cs b/source/Halibut/Queue/Redis/MessageStorage/IMessageSerialiserAndDataStreamStorage.cs similarity index 50% rename from source/Halibut/Queue/Redis/IMessageReaderWriter.cs rename to source/Halibut/Queue/Redis/MessageStorage/IMessageSerialiserAndDataStreamStorage.cs index f35b0583b..160267af2 100644 --- a/source/Halibut/Queue/Redis/IMessageReaderWriter.cs +++ b/source/Halibut/Queue/Redis/MessageStorage/IMessageSerialiserAndDataStreamStorage.cs @@ -1,10 +1,19 @@ +using System; using System.Threading; using System.Threading.Tasks; using Halibut.Transport.Protocol; -namespace Halibut.Queue.Redis +namespace Halibut.Queue.Redis.MessageStorage { - public interface IMessageReaderWriter + /// + /// Deals with preparing the request/response messages for storage in the + /// Redis Queue and helps with reading from the queue. + /// + /// This takes care of serialising the message into something that can be stored in + /// Redis, and calls IStoreDataStreamsForDistributedQueues for storage/retrievable + /// of DataStreams. + /// + public interface IMessageSerialiserAndDataStreamStorage { Task PrepareRequest(RequestMessage request, CancellationToken cancellationToken); Task ReadRequest(string jsonRequest, CancellationToken cancellationToken); diff --git a/source/Halibut/Queue/Redis/MessageReaderWriter.cs b/source/Halibut/Queue/Redis/MessageStorage/MessageSerialiserAndDataStreamStorage.cs similarity index 85% rename from source/Halibut/Queue/Redis/MessageReaderWriter.cs rename to source/Halibut/Queue/Redis/MessageStorage/MessageSerialiserAndDataStreamStorage.cs index ff6e8fda5..c7ceab190 100644 --- a/source/Halibut/Queue/Redis/MessageReaderWriter.cs +++ b/source/Halibut/Queue/Redis/MessageStorage/MessageSerialiserAndDataStreamStorage.cs @@ -1,16 +1,17 @@ +using System; using System.Threading; using System.Threading.Tasks; using Halibut.Queue.QueuedDataStreams; using Halibut.Transport.Protocol; -namespace Halibut.Queue.Redis +namespace Halibut.Queue.Redis.MessageStorage { - public class MessageReaderWriter : IMessageReaderWriter + public class MessageSerialiserAndDataStreamStorage : IMessageSerialiserAndDataStreamStorage { readonly QueueMessageSerializer queueMessageSerializer; readonly IStoreDataStreamsForDistributedQueues storeDataStreamsForDistributedQueues; - public MessageReaderWriter(QueueMessageSerializer queueMessageSerializer, IStoreDataStreamsForDistributedQueues storeDataStreamsForDistributedQueues) + public MessageSerialiserAndDataStreamStorage(QueueMessageSerializer queueMessageSerializer, IStoreDataStreamsForDistributedQueues storeDataStreamsForDistributedQueues) { this.queueMessageSerializer = queueMessageSerializer; this.storeDataStreamsForDistributedQueues = storeDataStreamsForDistributedQueues; diff --git a/source/Halibut/Queue/Redis/IWatchForRedisLosingAllItsData.cs b/source/Halibut/Queue/Redis/RedisDataLoseDetection/IWatchForRedisLosingAllItsData.cs similarity index 83% rename from source/Halibut/Queue/Redis/IWatchForRedisLosingAllItsData.cs rename to source/Halibut/Queue/Redis/RedisDataLoseDetection/IWatchForRedisLosingAllItsData.cs index 1a0d7f1f2..2cc0e66e4 100644 --- a/source/Halibut/Queue/Redis/IWatchForRedisLosingAllItsData.cs +++ b/source/Halibut/Queue/Redis/RedisDataLoseDetection/IWatchForRedisLosingAllItsData.cs @@ -2,11 +2,12 @@ using System.Threading; using System.Threading.Tasks; -namespace Halibut.Queue.Redis +namespace Halibut.Queue.Redis.RedisDataLoseDetection { public interface IWatchForRedisLosingAllItsData : IAsyncDisposable { /// + /// Returns a Cancellation token which is triggered when data lose occurs. /// Will cause the caller to wait until we are connected to redis and so can detect datalose. /// /// Time to wait for this to reach a state where it can detect datalose diff --git a/source/Halibut/Queue/Redis/WatchForRedisLosingAllItsData.cs b/source/Halibut/Queue/Redis/RedisDataLoseDetection/WatchForRedisLosingAllItsData.cs similarity index 75% rename from source/Halibut/Queue/Redis/WatchForRedisLosingAllItsData.cs rename to source/Halibut/Queue/Redis/RedisDataLoseDetection/WatchForRedisLosingAllItsData.cs index e5d1bcb51..30406c5b6 100644 --- a/source/Halibut/Queue/Redis/WatchForRedisLosingAllItsData.cs +++ b/source/Halibut/Queue/Redis/RedisDataLoseDetection/WatchForRedisLosingAllItsData.cs @@ -3,27 +3,43 @@ using System.Threading; using System.Threading.Tasks; using Halibut.Diagnostics; +using Halibut.Queue.Redis.RedisHelpers; using Halibut.Util; -namespace Halibut.Queue.Redis +namespace Halibut.Queue.Redis.RedisDataLoseDetection { public class WatchForRedisLosingAllItsData : IWatchForRedisLosingAllItsData { - RedisFacade redisFacade; + readonly RedisFacade redisFacade; readonly ILog log; - internal TimeSpan SetupDelay { get;} - internal TimeSpan WatchInterval { get; } - internal TimeSpan KeyTTL { get; } + /// + /// If we are yet to contact redis to watch it for data lose, this is the delay + /// between errors used when retrying to connect to redis. + /// + internal TimeSpan SetupErrorBackoffDelay { get;} + + /// + /// The amount of time between checks to check if redis has had data lose. + /// + internal TimeSpan DataLoseCheckInterval { get; } + + /// + /// The TTL of the key used for data lose detection. The TTL is reset + /// each time we check for data lose. This exists so that the data is + /// eventually removed from redis. + /// + internal TimeSpan DataLostKeyTtl { get; } + CancelOnDisposeCancellationToken cts = new(); public WatchForRedisLosingAllItsData(RedisFacade redisFacade, ILog log, TimeSpan? setupDelay = null, TimeSpan? watchInterval = null, TimeSpan? keyTTL = null) { this.redisFacade = redisFacade; this.log = log; - this.SetupDelay = setupDelay ?? TimeSpan.FromSeconds(1); - this.WatchInterval = watchInterval ?? TimeSpan.FromSeconds(60); - this.KeyTTL = keyTTL ?? TimeSpan.FromMinutes(60); + this.SetupErrorBackoffDelay = setupDelay ?? TimeSpan.FromSeconds(1); + this.DataLoseCheckInterval = watchInterval ?? TimeSpan.FromSeconds(60); + this.DataLostKeyTtl = keyTTL ?? TimeSpan.FromHours(8); var _ = Task.Run(async () => await KeepWatchingForDataLose(cts.Token)); } @@ -42,7 +58,6 @@ public async Task GetTokenForDataLoseDetection(TimeSpan timeT return await taskCompletionSource.Task; } - // TODO: Check if tentacle needs this to be classified as exception that can be retried. await using var cts = new CancelOnDisposeCancellationToken(cancellationToken); cts.CancelAfter(timeToWait); return await taskCompletionSource.Task.WaitAsync(cts.Token); @@ -71,8 +86,8 @@ async Task WatchForDataLose(CancellationToken cancellationToken) { if (!hasSetKey) { - log.Write(EventType.Diagnostic, "Setting initial data loss monitoring key {0} with TTL {1} minutes", key, KeyTTL.TotalMinutes); - await redisFacade.SetString(key, guid, KeyTTL, cancellationToken); + log.Write(EventType.Diagnostic, "Setting initial data loss monitoring key {0} with TTL {1} minutes", key, DataLostKeyTtl.TotalMinutes); + await redisFacade.SetString(key, guid, DataLostKeyTtl, cancellationToken); taskCompletionSource.TrySetResult(cts.Token); hasSetKey = true; log.Write(EventType.Diagnostic, "Successfully set initial data loss monitoring key {0}, monitoring is now active", key); @@ -89,7 +104,7 @@ async Task WatchForDataLose(CancellationToken cancellationToken) return; } - await redisFacade.SetTtlForString(key, KeyTTL, cancellationToken); + await redisFacade.SetTtlForString(key, DataLostKeyTtl, cancellationToken); } } catch (Exception ex) @@ -99,8 +114,8 @@ async Task WatchForDataLose(CancellationToken cancellationToken) await Try.IgnoringError(async () => { - if (!hasSetKey) await Task.Delay(SetupDelay, cancellationToken); - else await Task.Delay(WatchInterval, cancellationToken); + if (!hasSetKey) await Task.Delay(SetupErrorBackoffDelay, cancellationToken); + else await Task.Delay(DataLoseCheckInterval, cancellationToken); }); } diff --git a/source/Halibut/Queue/Redis/HalibutRedisTransport.cs b/source/Halibut/Queue/Redis/RedisHelpers/HalibutRedisTransport.cs similarity index 99% rename from source/Halibut/Queue/Redis/HalibutRedisTransport.cs rename to source/Halibut/Queue/Redis/RedisHelpers/HalibutRedisTransport.cs index 0971adee3..d7584eddc 100644 --- a/source/Halibut/Queue/Redis/HalibutRedisTransport.cs +++ b/source/Halibut/Queue/Redis/RedisHelpers/HalibutRedisTransport.cs @@ -1,14 +1,12 @@ #if NET8_0_OR_GREATER using System; -using System.Diagnostics; using System.Threading; using System.Threading.Tasks; using Halibut.Util; -using Newtonsoft.Json; using StackExchange.Redis; -namespace Halibut.Queue.Redis +namespace Halibut.Queue.Redis.RedisHelpers { public class HalibutRedisTransport : IHalibutRedisTransport { diff --git a/source/Halibut/Queue/Redis/IHalibutRedisTransport.cs b/source/Halibut/Queue/Redis/RedisHelpers/IHalibutRedisTransport.cs similarity index 97% rename from source/Halibut/Queue/Redis/IHalibutRedisTransport.cs rename to source/Halibut/Queue/Redis/RedisHelpers/IHalibutRedisTransport.cs index 7ea6603f9..24a002b98 100644 --- a/source/Halibut/Queue/Redis/IHalibutRedisTransport.cs +++ b/source/Halibut/Queue/Redis/RedisHelpers/IHalibutRedisTransport.cs @@ -11,8 +11,12 @@ public interface IHalibutRedisTransport { Task SubscribeToRequestMessagePulseChannel(Uri endpoint, Action onRequestMessagePulse, CancellationToken cancellationToken); Task PulseRequestPushedToEndpoint(Uri endpoint, CancellationToken cancellationToken); + + Task PushRequestGuidOnToQueue(Uri endpoint, Guid guid, CancellationToken cancellationToken); Task TryPopNextRequestGuid(Uri endpoint, CancellationToken cancellationToken); + + Task PutRequest(Uri endpoint, Guid requestId, string requestMessage, TimeSpan requestPickupTimeout, CancellationToken cancellationToken); Task TryGetAndRemoveRequest(Uri endpoint, Guid requestId, CancellationToken cancellationToken); Task IsRequestStillOnQueue(Uri endpoint, Guid requestId, CancellationToken cancellationToken); @@ -20,12 +24,12 @@ public interface IHalibutRedisTransport Task SubscribeToRequestCancellation(Uri endpoint, Guid request, Func onRpcCancellation, CancellationToken cancellationToken); - Task PublishCancellation(Uri endpoint, Guid requestId, CancellationToken cancellationToken); - string RequestCancelledMarkerKey(Uri endpoint, Guid requestId); + Task MarkRequestAsCancelled(Uri endpoint, Guid requestId, TimeSpan ttl, CancellationToken cancellationToken); Task IsRequestMarkedAsCancelled(Uri endpoint, Guid requestId, CancellationToken cancellationToken); + Task SubscribeToNodeHeartBeatChannel( Uri endpoint, Guid request, @@ -35,11 +39,13 @@ Task SubscribeToNodeHeartBeatChannel( Task SendNodeHeartBeat(Uri endpoint, Guid requestId, HalibutQueueNodeSendingPulses nodeSendingPulsesType, CancellationToken cancellationToken); + Task SubscribeToResponseChannel(Uri endpoint, Guid identifier, Func onValueReceived, CancellationToken cancellationToken); - Task PublishThatResponseIsAvailable(Uri endpoint, Guid identifier, string value, CancellationToken cancellationToken); + + Task SetResponseMessage(Uri endpoint, Guid identifier, string responseMessage, TimeSpan ttl, CancellationToken cancellationToken); Task GetResponseMessage(Uri endpoint, Guid identifier, CancellationToken cancellationToken); Task DeleteResponseMessage(Uri endpoint, Guid identifier, CancellationToken cancellationToken); diff --git a/source/Halibut/Queue/Redis/RedisFacade.cs b/source/Halibut/Queue/Redis/RedisHelpers/RedisFacade.cs similarity index 87% rename from source/Halibut/Queue/Redis/RedisFacade.cs rename to source/Halibut/Queue/Redis/RedisHelpers/RedisFacade.cs index e43bb4a98..b38038a98 100644 --- a/source/Halibut/Queue/Redis/RedisFacade.cs +++ b/source/Halibut/Queue/Redis/RedisHelpers/RedisFacade.cs @@ -2,11 +2,11 @@ using System.Diagnostics; using System.Threading; using System.Threading.Tasks; +using Halibut.Diagnostics; using Halibut.Util; -using Halibut.Diagnostics; // Add logging support using StackExchange.Redis; -namespace Halibut.Queue.Redis +namespace Halibut.Queue.Redis.RedisHelpers { public class RedisFacade : IAsyncDisposable { @@ -16,12 +16,15 @@ public class RedisFacade : IAsyncDisposable internal TimeSpan MaxDurationToRetryFor = TimeSpan.FromSeconds(30); ConnectionMultiplexer Connection => connection.Value; - - string keyPrefix; + /// + /// All Keys will be prefixed with this, this allows for multiple halibuts to use + /// the same redis without interfering with each other. + /// + readonly string keyPrefix; - CancelOnDisposeCancellationToken cts; - CancellationToken facadeCancellationToken; + readonly CancelOnDisposeCancellationToken objectLifetimeCts; + readonly CancellationToken objectLifeTimeCancellationToken; public RedisFacade(string configuration, string keyPrefix, ILog log) : this(ConfigurationOptions.Parse(configuration), keyPrefix, log) { @@ -29,20 +32,18 @@ public RedisFacade(string configuration, string keyPrefix, ILog log) : this(Conf } public RedisFacade(ConfigurationOptions redisOptions, string keyPrefix, ILog log) { - this.keyPrefix = keyPrefix ?? "halibut"; - this.log = log; - this.cts = new CancelOnDisposeCancellationToken(); - this.facadeCancellationToken = cts.Token; + this.keyPrefix = keyPrefix; + this.log = log.ForContext(); + objectLifetimeCts = new CancelOnDisposeCancellationToken(); + objectLifeTimeCancellationToken = objectLifetimeCts.Token; - // aka have more goes at connecting. + // Tells the client to make multiple attempts to create the TCP connection to redis. redisOptions.AbortOnConnectFail = false; connection = new Lazy(() => { var multiplexer = ConnectionMultiplexer.Connect(redisOptions); - //redisOptions.ReconnectRetryPolicy = new LinearRetry() - // Subscribe to connection events multiplexer.ConnectionFailed += OnConnectionFailed; multiplexer.ConnectionRestored += OnConnectionRestored; @@ -51,31 +52,25 @@ public RedisFacade(ConfigurationOptions redisOptions, string keyPrefix, ILog log return multiplexer; }); } - - private void OnConnectionFailed(object? sender, ConnectionFailedEventArgs e) + + void OnConnectionFailed(object? sender, ConnectionFailedEventArgs e) { - var message = $"Redis connection failed - EndPoint: {e.EndPoint}, Failure: {e.FailureType}, Exception: {e.Exception?.Message}"; - log?.Write(EventType.Error, message); + log.Write(EventType.Error, "Redis connection failed - EndPoint: {0}, Failure: {1}, Exception: {2}", e.EndPoint, e.FailureType, e.Exception?.Message); } - private void OnErrorMessage(object? sender, RedisErrorEventArgs e) + void OnErrorMessage(object? sender, RedisErrorEventArgs e) { - var message = $"Redis error - EndPoint: {e.EndPoint}, Message: {e.Message}"; - log?.Write(EventType.Error, message); + log.Write(EventType.Error, "Redis error - EndPoint: {0}, Message: {1}", e.EndPoint, e.Message); } - - private void OnConnectionRestored(object? sender, ConnectionFailedEventArgs e) + + void OnConnectionRestored(object? sender, ConnectionFailedEventArgs e) { - var message = $"Redis connection restored - EndPoint: {e.EndPoint}"; - log?.Write(EventType.Diagnostic, message); + log.Write(EventType.Diagnostic, "Redis connection restored - EndPoint: {0}", e.EndPoint); } - /// - /// Executes an operation with retry logic. Retries for up to 12 seconds with 1-second intervals. - /// - private async Task ExecuteWithRetry(Func> operation, CancellationToken cancellationToken) + async Task ExecuteWithRetry(Func> operation, CancellationToken cancellationToken) { - await using var linkedTokenSource = new CancelOnDisposeCancellationToken(cancellationToken, facadeCancellationToken); + await using var linkedTokenSource = new CancelOnDisposeCancellationToken(cancellationToken, objectLifeTimeCancellationToken); var combinedToken = linkedTokenSource.Token; var retryDelay = TimeSpan.FromSeconds(1); @@ -102,7 +97,7 @@ private async Task ExecuteWithRetry(Func> operation, CancellationT /// private async Task ExecuteWithRetry(Func operation, CancellationToken cancellationToken) { - await using var linkedTokenSource = new CancelOnDisposeCancellationToken(cancellationToken, facadeCancellationToken); + await using var linkedTokenSource = new CancelOnDisposeCancellationToken(cancellationToken, objectLifeTimeCancellationToken); var combinedToken = linkedTokenSource.Token; var retryDelay = TimeSpan.FromSeconds(1); @@ -129,7 +124,7 @@ private async Task ExecuteWithRetry(Func operation, CancellationToken canc public async ValueTask DisposeAsync() { - await Try.IgnoringError(async () => await cts.DisposeAsync()); + await Try.IgnoringError(async () => await objectLifetimeCts.DisposeAsync()); if (connection.IsValueCreated) { diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index cb06e37b2..58ea540f9 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -6,6 +6,8 @@ using System.Threading.Tasks; using Halibut.Diagnostics; using Halibut.Queue.Redis.Exceptions; +using Halibut.Queue.Redis.MessageStorage; +using Halibut.Queue.Redis.RedisDataLoseDetection; using Halibut.ServiceModel; using Halibut.Transport.Protocol; using Halibut.Util; @@ -20,7 +22,7 @@ class RedisPendingRequestQueue : IPendingRequestQueue, IDisposable readonly ILog log; readonly IHalibutRedisTransport halibutRedisTransport; readonly HalibutTimeoutsAndLimits halibutTimeoutsAndLimits; - readonly IMessageReaderWriter messageReaderWriter; + readonly IMessageSerialiserAndDataStreamStorage messageSerialiserAndDataStreamStorage; readonly AsyncManualResetEvent hasItemsForEndpoint = new(); readonly CancelOnDisposeCancellationToken queueCts = new (); @@ -38,13 +40,13 @@ public RedisPendingRequestQueue( IWatchForRedisLosingAllItsData watchForRedisLosingAllItsData, ILog log, IHalibutRedisTransport halibutRedisTransport, - IMessageReaderWriter messageReaderWriter, + IMessageSerialiserAndDataStreamStorage messageSerialiserAndDataStreamStorage, HalibutTimeoutsAndLimits halibutTimeoutsAndLimits) { this.endpoint = endpoint; this.watchForRedisLosingAllItsData = watchForRedisLosingAllItsData; this.log = log.ForContext(); - this.messageReaderWriter = messageReaderWriter; + this.messageSerialiserAndDataStreamStorage = messageSerialiserAndDataStreamStorage; this.halibutRedisTransport = halibutRedisTransport; this.halibutTimeoutsAndLimits = halibutTimeoutsAndLimits; this.queueToken = queueCts.Token; @@ -107,7 +109,7 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can string payload; try { - payload = await messageReaderWriter.PrepareRequest(request, cancellationToken); + payload = await messageSerialiserAndDataStreamStorage.PrepareRequest(request, cancellationToken); } catch (Exception ex) { @@ -316,7 +318,7 @@ async Task TryClearRequestFromQueue(RequestMessage request, RedisPendingRe try { - var response = await messageReaderWriter.ReadResponse(responseJson, cancellationToken); + var response = await messageSerialiserAndDataStreamStorage.ReadResponse(responseJson, cancellationToken); log.Write(EventType.Diagnostic, "Successfully deserialized response for request {0}", activityId); return response; } @@ -441,7 +443,7 @@ public async Task ApplyResponse(ResponseMessage response, Guid requestActivityId response = ResponseMessage.FromException(response, new HalibutClientException(RequestAbandonedMessage)); } } - var responseJson = await messageReaderWriter.PrepareResponse(response, cancellationToken); + var responseJson = await messageSerialiserAndDataStreamStorage.PrepareResponse(response, cancellationToken); log.Write(EventType.MessageExchange, "Sending response message for request {0}", requestActivityId); await PollAndSubscribeToResponse.SendResponse(halibutRedisTransport, endpoint, requestActivityId, responseJson, TTLOfResponseMessage, log); log.Write(EventType.MessageExchange, "Successfully applied response for request {0}", requestActivityId); @@ -524,7 +526,7 @@ await Task.WhenAny( continue; } - var request = await messageReaderWriter.ReadRequest(jsonRequest, cancellationToken); + var request = await messageSerialiserAndDataStreamStorage.ReadRequest(jsonRequest, cancellationToken); log.Write(EventType.Diagnostic, "Successfully collected request {0} from queue for endpoint {1}", request.ActivityId, endpoint); return request; diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueueFactory.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueueFactory.cs index a03a592a3..09af4a9ee 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueueFactory.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueueFactory.cs @@ -4,6 +4,9 @@ using System.Threading.Tasks; using Halibut.Diagnostics; using Halibut.Queue.QueuedDataStreams; +using Halibut.Queue.Redis.MessageStorage; +using Halibut.Queue.Redis.RedisDataLoseDetection; +using Halibut.Queue.Redis.RedisHelpers; using Halibut.ServiceModel; namespace Halibut.Queue.Redis @@ -41,7 +44,7 @@ public IPendingRequestQueue CreateQueue(Uri endpoint) watchForRedisLosingAllItsData, logFactory.ForEndpoint(endpoint), halibutRedisTransport, - new MessageReaderWriter(queueMessageSerializer, dataStreamStorage), + new MessageSerialiserAndDataStreamStorage(queueMessageSerializer, dataStreamStorage), halibutTimeoutsAndLimits); } From 901cd7064e53eea636a41ade4488cb9049396d08 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Mon, 18 Aug 2025 15:01:17 +1000 Subject: [PATCH 079/137] Cleanup --- .../Queue/Redis/RedisHelpers/RedisFacade.cs | 64 +++++++++++-------- 1 file changed, 37 insertions(+), 27 deletions(-) diff --git a/source/Halibut/Queue/Redis/RedisHelpers/RedisFacade.cs b/source/Halibut/Queue/Redis/RedisHelpers/RedisFacade.cs index b38038a98..0adfbf331 100644 --- a/source/Halibut/Queue/Redis/RedisHelpers/RedisFacade.cs +++ b/source/Halibut/Queue/Redis/RedisHelpers/RedisFacade.cs @@ -13,6 +13,7 @@ public class RedisFacade : IAsyncDisposable readonly Lazy connection; readonly ILog log; // We can survive redis being unavailable for this amount of time. + // Generally redis will try for 5s, we add our own retries to try for longer. internal TimeSpan MaxDurationToRetryFor = TimeSpan.FromSeconds(30); ConnectionMultiplexer Connection => connection.Value; @@ -92,10 +93,7 @@ async Task ExecuteWithRetry(Func> operation, CancellationToken can } } - /// - /// Executes an operation with retry logic. Retries for up to 12 seconds with 1-second intervals. - /// - private async Task ExecuteWithRetry(Func operation, CancellationToken cancellationToken) + async Task ExecuteWithRetry(Func operation, CancellationToken cancellationToken) { await using var linkedTokenSource = new CancelOnDisposeCancellationToken(cancellationToken, objectLifeTimeCancellationToken); var combinedToken = linkedTokenSource.Token; @@ -200,7 +198,6 @@ public async Task SetInHash(string key, string field, string payload, TimeSpan t { key = "hash:" + keyPrefix + ":" + key; - // Retry each operation independently await ExecuteWithRetry(async () => { var database = Connection.GetDatabase(); @@ -210,9 +207,14 @@ await ExecuteWithRetry(async () => await SetTtlForKeyRaw(key, ttl, cancellationToken); } + string ToHashKey(string key) + { + return "hash:" + keyPrefix + ":" + key; + } + public async Task HashContainsKey(string key, string field, CancellationToken cancellationToken) { - key = "hash:" + keyPrefix + ":" + key; + key = ToHashKey(key); return await ExecuteWithRetry(async () => { var database = Connection.GetDatabase(); @@ -222,7 +224,7 @@ public async Task HashContainsKey(string key, string field, CancellationTo public async Task TryGetAndDeleteFromHash(string key, string field, CancellationToken cancellationToken) { - key = "hash:" + keyPrefix + ":" + key; + key = ToHashKey(key); // Retry each operation independently var value = await ExecuteWithRetry(async () => @@ -231,8 +233,11 @@ public async Task HashContainsKey(string key, string field, CancellationTo return await database.HashGetAsync(key, new RedisValue(field)); }, cancellationToken); - // TODO: If we retry this is not idempotent. - // TODO: This needs to be tested in RedisPendingRequestsQueueFixture + // Retry does make this non-idempotent, what can happen is the key is deleted on redis. + // But we do not get a response saying it is deleted. We try again and get told + // it is already deleted. + // In the Redis Queue this can result in no-body picking up the Request, and the + // request eventually timing out. var res = await ExecuteWithRetry(async () => { var database = Connection.GetDatabase(); @@ -244,12 +249,17 @@ public async Task HashContainsKey(string key, string field, CancellationTo // Someone else deleted this, so return nothing to make the get and delete appear to be atomic. return null; } - return (string?)value; + return value; + } + + string ToListKey(string key) + { + return "list:" + keyPrefix + ":" + key; } public async Task ListRightPushAsync(string key, string payload, TimeSpan ttlForAllInList, CancellationToken cancellationToken) { - key = "list:" + keyPrefix + ":" + key; + key = ToListKey(key); await ExecuteWithRetry(async () => { var database = Connection.GetDatabase(); @@ -260,7 +270,7 @@ await ExecuteWithRetry(async () => public async Task ListLeftPopAsync(string key, CancellationToken cancellationToken) { - key = "list:" + keyPrefix + ":" + key; + key = ToListKey(key); return await ExecuteWithRetry(async () => { var database = Connection.GetDatabase(); @@ -270,10 +280,15 @@ await ExecuteWithRetry(async () => return null; } - return (string?)value; + return value; }, cancellationToken); } + string ToStringKey(string key) + { + return "string:" + keyPrefix + ":" + key; + } + public async Task SetString(string key, string value, TimeSpan ttl, CancellationToken cancellationToken) { key = ToStringKey(key); @@ -286,26 +301,12 @@ await ExecuteWithRetry(async () => await SetTtlForKeyRaw(key, ttl, cancellationToken); } - string ToStringKey(string key) - { - return "string:" + keyPrefix + ":" + key; - } - public async Task SetTtlForString(string key, TimeSpan ttl, CancellationToken cancellationToken) { await SetTtlForKeyRaw(ToStringKey(key), ttl, cancellationToken); } - async Task SetTtlForKeyRaw(string key, TimeSpan ttl, CancellationToken cancellationToken) - { - await ExecuteWithRetry(async () => - { - var database = Connection.GetDatabase(); - await database.KeyExpireAsync(key, ttl); - }, cancellationToken); - } - public async Task GetString(string key, CancellationToken cancellationToken) { key = ToStringKey(key); @@ -325,5 +326,14 @@ public async Task DeleteString(string key, CancellationToken cancellationT return await database.KeyDeleteAsync(key); }, cancellationToken); } + + async Task SetTtlForKeyRaw(string key, TimeSpan ttl, CancellationToken cancellationToken) + { + await ExecuteWithRetry(async () => + { + var database = Connection.GetDatabase(); + await database.KeyExpireAsync(key, ttl); + }, cancellationToken); + } } } \ No newline at end of file From 374b395694e005e342a4079ef2ea4ff70247090f Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Mon, 18 Aug 2025 15:20:58 +1000 Subject: [PATCH 080/137] . --- .../Queue/Redis/NodeHeartBeatSenderFixture.cs | 7 +- .../Redis/RedisPendingRequestQueueFixture.cs | 1 + .../HalibutRedisTransportWithVirtuals.cs | 1 + .../HalibutQueueNodeSendingPulses.cs | 17 +++ .../NodeHeartBeat/NodeHeartBeatSender.cs | 80 +++++++++++ .../NodeHeartBeatWatcher.cs} | 133 +++++------------- .../Redis/NodeHeartBeat/NodeWatcherResult.cs | 12 ++ .../RedisHelpers/HalibutRedisTransport.cs | 1 + .../RedisHelpers/IHalibutRedisTransport.cs | 1 + .../Queue/Redis/RedisPendingRequestQueue.cs | 33 ++--- ...orRequestCancellationOrSenderDisconnect.cs | 5 +- source/Halibut/Util/TimeSpanHelper.cs | 12 ++ 12 files changed, 180 insertions(+), 123 deletions(-) create mode 100644 source/Halibut/Queue/Redis/NodeHeartBeat/HalibutQueueNodeSendingPulses.cs create mode 100644 source/Halibut/Queue/Redis/NodeHeartBeat/NodeHeartBeatSender.cs rename source/Halibut/Queue/Redis/{NodeHeartBeatSender.cs => NodeHeartBeat/NodeHeartBeatWatcher.cs} (53%) create mode 100644 source/Halibut/Queue/Redis/NodeHeartBeat/NodeWatcherResult.cs create mode 100644 source/Halibut/Util/TimeSpanHelper.cs diff --git a/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs b/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs index 12383e2ae..0cd9c3abb 100644 --- a/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs @@ -7,6 +7,7 @@ using FluentAssertions; using Halibut.Logging; using Halibut.Queue.Redis; +using Halibut.Queue.Redis.NodeHeartBeat; using Halibut.Queue.Redis.RedisHelpers; using Halibut.Tests.Builders; using Halibut.Tests.Queue.Redis.Utils; @@ -174,7 +175,7 @@ public async Task WaitUntilNodeProcessingRequestFlatLines_WhenHeartbeatsStop_Sho await pendingRequest.RequestHasBeenCollectedAndWillBeTransferred(); // Start the watcher - var watcherTask = NodeHeartBeatSender.WatchThatNodeProcessingTheRequestIsStillAlive( + var watcherTask = NodeHeartBeatWatcher.WatchThatNodeProcessingTheRequestIsStillAlive( endpoint, request, pendingRequest, @@ -216,7 +217,7 @@ public async Task NoIssueReturnedWhenNodeProcessingRequestIsNotSeenToGoOffline() // Start the watcher without marking request as collected using var cts = new CancellationTokenSource(); - var watcherTask = NodeHeartBeatSender.WatchThatNodeProcessingTheRequestIsStillAlive( + var watcherTask = NodeHeartBeatWatcher.WatchThatNodeProcessingTheRequestIsStillAlive( endpoint, request, pendingRequest, @@ -261,7 +262,7 @@ public async Task WaitUntilNodeProcessingRequestFlatLines_WhenConnectionInterrup await pendingRequest.RequestHasBeenCollectedAndWillBeTransferred(); // Start watcher with stable connection - var watcherTask = NodeHeartBeatSender.WatchThatNodeProcessingTheRequestIsStillAlive( + var watcherTask = NodeHeartBeatWatcher.WatchThatNodeProcessingTheRequestIsStillAlive( endpoint, request, pendingRequest, diff --git a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs index fdb719845..10c537da8 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs @@ -11,6 +11,7 @@ using Halibut.Queue.Redis; using Halibut.Queue.Redis.Exceptions; using Halibut.Queue.Redis.MessageStorage; +using Halibut.Queue.Redis.NodeHeartBeat; using Halibut.Queue.Redis.RedisHelpers; using Halibut.ServiceModel; using Halibut.Tests.Builders; diff --git a/source/Halibut.Tests/Queue/Redis/Utils/HalibutRedisTransportWithVirtuals.cs b/source/Halibut.Tests/Queue/Redis/Utils/HalibutRedisTransportWithVirtuals.cs index 9c4c1d395..162e55d79 100644 --- a/source/Halibut.Tests/Queue/Redis/Utils/HalibutRedisTransportWithVirtuals.cs +++ b/source/Halibut.Tests/Queue/Redis/Utils/HalibutRedisTransportWithVirtuals.cs @@ -4,6 +4,7 @@ using System.Threading; using System.Threading.Tasks; using Halibut.Queue.Redis; +using Halibut.Queue.Redis.NodeHeartBeat; using StackExchange.Redis; namespace Halibut.Tests.Queue.Redis.Utils diff --git a/source/Halibut/Queue/Redis/NodeHeartBeat/HalibutQueueNodeSendingPulses.cs b/source/Halibut/Queue/Redis/NodeHeartBeat/HalibutQueueNodeSendingPulses.cs new file mode 100644 index 000000000..9baffb7cd --- /dev/null +++ b/source/Halibut/Queue/Redis/NodeHeartBeat/HalibutQueueNodeSendingPulses.cs @@ -0,0 +1,17 @@ +#if NET8_0_OR_GREATER +using System; + +namespace Halibut.Queue.Redis.NodeHeartBeat +{ + public enum HalibutQueueNodeSendingPulses + { + // The node the RPC is executing on. + // The node that calls QueueAndWait + RequestSenderNode, + + // The node sending/receiving the Request to/from the service. + // The node that calls Dequeue and ApplyResponse. + RequestProcessorNode + } +} +#endif \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/NodeHeartBeat/NodeHeartBeatSender.cs b/source/Halibut/Queue/Redis/NodeHeartBeat/NodeHeartBeatSender.cs new file mode 100644 index 000000000..73c606880 --- /dev/null +++ b/source/Halibut/Queue/Redis/NodeHeartBeat/NodeHeartBeatSender.cs @@ -0,0 +1,80 @@ + +#if NET8_0_OR_GREATER +using System; +using System.Threading; +using System.Threading.Tasks; +using Halibut.Diagnostics; +using Halibut.Util; + +namespace Halibut.Queue.Redis.NodeHeartBeat +{ + public class NodeHeartBeatSender : IAsyncDisposable + { + readonly Uri endpoint; + readonly Guid requestActivityId; + readonly IHalibutRedisTransport halibutRedisTransport; + readonly CancelOnDisposeCancellationToken cts; + readonly ILog log; + readonly HalibutQueueNodeSendingPulses nodeSendingPulsesType; + + internal Task TaskSendingPulses; + public NodeHeartBeatSender( + Uri endpoint, + Guid requestActivityId, + IHalibutRedisTransport halibutRedisTransport, + ILog log, + HalibutQueueNodeSendingPulses nodeSendingPulsesType, + TimeSpan defaultDelayBetweenPulses) + { + this.endpoint = endpoint; + this.requestActivityId = requestActivityId; + this.halibutRedisTransport = halibutRedisTransport; + this.nodeSendingPulsesType = nodeSendingPulsesType; + cts = new CancelOnDisposeCancellationToken(); + this.log = log.ForContext(); + this.log.Write(EventType.Diagnostic, "Starting NodeHeartBeatSender for {0} node, request {1}, endpoint {2}", nodeSendingPulsesType, requestActivityId, endpoint); + TaskSendingPulses = Task.Run(() => SendPulsesWhileProcessingRequest(defaultDelayBetweenPulses, cts.Token)); + } + + async Task SendPulsesWhileProcessingRequest(TimeSpan defaultDelayBetweenPulses, CancellationToken cancellationToken) + { + log.Write(EventType.Diagnostic, "Starting heartbeat pulse loop for {0} node, request {1}", nodeSendingPulsesType, requestActivityId); + + TimeSpan delayBetweenPulse; + while (!cancellationToken.IsCancellationRequested) + { + try + { + await halibutRedisTransport.SendNodeHeartBeat(endpoint, requestActivityId, nodeSendingPulsesType, cancellationToken); + delayBetweenPulse = defaultDelayBetweenPulses; + log.Write(EventType.Diagnostic, "Successfully sent heartbeat for {0} node, request {1}, next pulse in {2} seconds", nodeSendingPulsesType, requestActivityId, delayBetweenPulse.TotalSeconds); + } + catch (Exception ex) + { + if(cancellationToken.IsCancellationRequested) + { + log.Write(EventType.Diagnostic, "Heartbeat pulse loop cancelled for {0} node, request {1}", nodeSendingPulsesType, requestActivityId); + return; + } + // Send pulses more frequently when we were unable to send a pulse. + delayBetweenPulse = defaultDelayBetweenPulses / 2; + log.WriteException(EventType.Diagnostic, "Failed to send heartbeat for {0} node, request {1}, switching to panic mode with {2} second intervals", ex, nodeSendingPulsesType, requestActivityId, delayBetweenPulse.TotalSeconds); + } + + await Try.IgnoringError(async () => await Task.Delay(delayBetweenPulse, cancellationToken)); + } + + log.Write(EventType.Diagnostic, "Heartbeat pulse loop ended for {0} node, request {1}", nodeSendingPulsesType, requestActivityId); + } + + public async ValueTask DisposeAsync() + { + log.Write(EventType.Diagnostic, "Disposing NodeHeartBeatSender for {0} node, request {1}", nodeSendingPulsesType, requestActivityId); + + await Try.IgnoringError(async () => await cts.DisposeAsync()); + + log.Write(EventType.Diagnostic, "NodeHeartBeatSender disposed for {0} node, request {1}", nodeSendingPulsesType, requestActivityId); + } + } +} +#endif \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs b/source/Halibut/Queue/Redis/NodeHeartBeat/NodeHeartBeatWatcher.cs similarity index 53% rename from source/Halibut/Queue/Redis/NodeHeartBeatSender.cs rename to source/Halibut/Queue/Redis/NodeHeartBeat/NodeHeartBeatWatcher.cs index 95470767c..c993f249a 100644 --- a/source/Halibut/Queue/Redis/NodeHeartBeatSender.cs +++ b/source/Halibut/Queue/Redis/NodeHeartBeat/NodeHeartBeatWatcher.cs @@ -1,91 +1,16 @@ - #if NET8_0_OR_GREATER using System; using System.Threading; using System.Threading.Tasks; using Halibut.Diagnostics; -using Halibut.ServiceModel; using Halibut.Transport.Protocol; using Halibut.Util; -namespace Halibut.Queue.Redis +namespace Halibut.Queue.Redis.NodeHeartBeat { - public enum HalibutQueueNodeSendingPulses - { - // The node the RPC is executing on. - // The node that calls QueueAndWait - RequestSenderNode, - - // The node sending/receiving the Request to/from the service. - // The node that calls Dequeue and ApplyResponse. - RequestProcessorNode - } - public class NodeHeartBeatSender : IAsyncDisposable + public class NodeHeartBeatWatcher { - public enum NodeProcessingRequestWatcherResult - { - NodeMayHaveDisconnected, - NoDisconnectSeen - } - - readonly Uri endpoint; - readonly Guid requestActivityId; - private readonly IHalibutRedisTransport halibutRedisTransport; - private readonly CancelOnDisposeCancellationToken cts; - private readonly ILog log; - private readonly HalibutQueueNodeSendingPulses nodeSendingPulsesType; - - internal Task TaskSendingPulses; - public NodeHeartBeatSender( - Uri endpoint, - Guid requestActivityId, - IHalibutRedisTransport halibutRedisTransport, - ILog log, - HalibutQueueNodeSendingPulses nodeSendingPulsesType, - TimeSpan defaultDelayBetweenPulses) - { - this.endpoint = endpoint; - this.requestActivityId = requestActivityId; - this.halibutRedisTransport = halibutRedisTransport; - this.nodeSendingPulsesType = nodeSendingPulsesType; - cts = new CancelOnDisposeCancellationToken(); - this.log = log.ForContext(); - this.log.Write(EventType.Diagnostic, "Starting NodeHeartBeatSender for {0} node, request {1}, endpoint {2}", nodeSendingPulsesType, requestActivityId, endpoint); - TaskSendingPulses = Task.Run(() => SendPulsesWhileProcessingRequest(defaultDelayBetweenPulses, cts.Token)); - } - - async Task SendPulsesWhileProcessingRequest(TimeSpan defaultDelayBetweenPulses, CancellationToken cancellationToken) - { - log.Write(EventType.Diagnostic, "Starting heartbeat pulse loop for {0} node, request {1}", nodeSendingPulsesType, requestActivityId); - - TimeSpan delayBetweenPulse; - while (!cancellationToken.IsCancellationRequested) - { - try - { - await halibutRedisTransport.SendNodeHeartBeat(endpoint, requestActivityId, nodeSendingPulsesType, cancellationToken); - delayBetweenPulse = defaultDelayBetweenPulses; - log.Write(EventType.Diagnostic, "Successfully sent heartbeat for {0} node, request {1}, next pulse in {2} seconds", nodeSendingPulsesType, requestActivityId, delayBetweenPulse.TotalSeconds); - } - catch (Exception ex) - { - if(cancellationToken.IsCancellationRequested) - { - log.Write(EventType.Diagnostic, "Heartbeat pulse loop cancelled for {0} node, request {1}", nodeSendingPulsesType, requestActivityId); - return; - } - // Panic send pulses. - delayBetweenPulse = defaultDelayBetweenPulses / 2; - log.WriteException(EventType.Diagnostic, "Failed to send heartbeat for {0} node, request {1}, switching to panic mode with {2} second intervals", ex, nodeSendingPulsesType, requestActivityId, delayBetweenPulse.TotalSeconds); - } - - await Try.IgnoringError(async () => await Task.Delay(delayBetweenPulse, cancellationToken)); - } - - log.Write(EventType.Diagnostic, "Heartbeat pulse loop ended for {0} node, request {1}", nodeSendingPulsesType, requestActivityId); - } - - public static async Task WatchThatNodeProcessingTheRequestIsStillAlive( + public static async Task WatchThatNodeProcessingTheRequestIsStillAlive( Uri endpoint, RequestMessage request, RedisPendingRequest redisPending, @@ -106,11 +31,15 @@ public static async Task WatchThatNodeProces } catch (Exception) when (cts.Token.IsCancellationRequested) { - return NodeProcessingRequestWatcherResult.NoDisconnectSeen; + return NodeWatcherResult.NoDisconnectSeen; + } + catch (Exception) + { + return NodeWatcherResult.NodeMayHaveDisconnected; } } - public static async Task WatchThatNodeWhichSentTheRequestIsStillAlive( + public static async Task WatchThatNodeWhichSentTheRequestIsStillAlive( Uri endpoint, Guid requestActivityId, IHalibutRedisTransport halibutRedisTransport, @@ -118,10 +47,21 @@ public static async Task WatchThatNodeWhichS TimeSpan maxTimeBetweenSenderHeartBeetsBeforeSenderIsAssumedToBeOffline, CancellationToken watchCancellationToken) { - return await WatchForPulsesFromNode(endpoint, requestActivityId, halibutRedisTransport, log, maxTimeBetweenSenderHeartBeetsBeforeSenderIsAssumedToBeOffline, HalibutQueueNodeSendingPulses.RequestSenderNode, watchCancellationToken); + try + { + return await WatchForPulsesFromNode(endpoint, requestActivityId, halibutRedisTransport, log, maxTimeBetweenSenderHeartBeetsBeforeSenderIsAssumedToBeOffline, HalibutQueueNodeSendingPulses.RequestSenderNode, watchCancellationToken); + } + catch (Exception) when (watchCancellationToken.IsCancellationRequested) + { + return NodeWatcherResult.NoDisconnectSeen; + } + catch (Exception) + { + return NodeWatcherResult.NodeMayHaveDisconnected; + } } - private static async Task WatchForPulsesFromNode( + static async Task WatchForPulsesFromNode( Uri endpoint, Guid requestActivityId, IHalibutRedisTransport halibutRedisTransport, @@ -137,6 +77,12 @@ private static async Task WatchForPulsesFrom try { + // Currently we will wait until the CT is cancelled to get a subscription, + // instead it would be better if we either + // - waited for maxTimeBetweenHeartBeetsBeforeNodeIsAssumedToBeOffline to get a subscription. + // - SubscribeToNodeHeartBeatChannel returned immediately even if it doesn't have a subscription, and instead it works + // in the background to get one unless the CT is triggered, or it is disposed. + // https://whimsical.com/subscribetonodeheartbeatchannel-should-timeout-while-waiting-to--NFWwmPkE7pTBdm2PRUC8Tf await using var subscription = await halibutRedisTransport.SubscribeToNodeHeartBeatChannel( endpoint, requestActivityId, @@ -155,18 +101,18 @@ private static async Task WatchForPulsesFrom if (timeSinceLastHeartBeat > maxTimeBetweenHeartBeetsBeforeNodeIsAssumedToBeOffline) { log.Write(EventType.Diagnostic, "{0} node appears disconnected, request {1}, last heartbeat was {2} seconds ago", watchingForPulsesFrom, requestActivityId, timeSinceLastHeartBeat.TotalSeconds); - return NodeProcessingRequestWatcherResult.NodeMayHaveDisconnected; + return NodeWatcherResult.NodeMayHaveDisconnected; } - var timeToWait = TimeSpan.FromSeconds(30); - var timeBeforeTimeoutPlusOneSecond = maxTimeBetweenHeartBeetsBeforeNodeIsAssumedToBeOffline - timeSinceLastHeartBeat + TimeSpan.FromSeconds(1); - if (timeBeforeTimeoutPlusOneSecond < timeToWait) timeToWait = timeBeforeTimeoutPlusOneSecond; + var timeToWait = TimeSpanHelper.Min( + TimeSpan.FromSeconds(30), + maxTimeBetweenHeartBeetsBeforeNodeIsAssumedToBeOffline - timeSinceLastHeartBeat + TimeSpan.FromSeconds(1)); await Try.IgnoringError(async () => await Task.Delay(timeToWait, watchCancellationToken)); } log.Write(EventType.Diagnostic, "{0} node watcher cancelled, request {1}", watchingForPulsesFrom, requestActivityId); - return NodeProcessingRequestWatcherResult.NoDisconnectSeen; + return NodeWatcherResult.NoDisconnectSeen; } catch (Exception ex) when (!watchCancellationToken.IsCancellationRequested) { @@ -174,7 +120,7 @@ private static async Task WatchForPulsesFrom throw; } } - + static async Task WaitForRequestToBeCollected(Uri endpoint, RequestMessage request, RedisPendingRequest redisPending, IHalibutRedisTransport halibutRedisTransport, TimeSpan timeBetweenCheckingIfRequestWasCollected, ILog log, CancellationToken cancellationToken) @@ -194,7 +140,7 @@ static async Task WaitForRequestToBeCollected(Uri endpoint, RequestMessage reque } - // So check ourselves if the request has been collected. + // Check ourselves if the request has been collected. var requestIsStillOnQueue = await halibutRedisTransport.IsRequestStillOnQueue(endpoint, request.ActivityId, cancellationToken); if(!requestIsStillOnQueue) { @@ -218,15 +164,6 @@ await Task.WhenAny( log.Write(EventType.Diagnostic, "Stopped waiting for request {0} to be collected (cancelled)", request.ActivityId); } - - public async ValueTask DisposeAsync() - { - log.Write(EventType.Diagnostic, "Disposing NodeHeartBeatSender for {0} node, request {1}", nodeSendingPulsesType, requestActivityId); - - await Try.IgnoringError(async () => await cts.DisposeAsync()); - - log.Write(EventType.Diagnostic, "NodeHeartBeatSender disposed for {0} node, request {1}", nodeSendingPulsesType, requestActivityId); - } } } #endif \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/NodeHeartBeat/NodeWatcherResult.cs b/source/Halibut/Queue/Redis/NodeHeartBeat/NodeWatcherResult.cs new file mode 100644 index 000000000..8915633ea --- /dev/null +++ b/source/Halibut/Queue/Redis/NodeHeartBeat/NodeWatcherResult.cs @@ -0,0 +1,12 @@ +#if NET8_0_OR_GREATER +using System; + +namespace Halibut.Queue.Redis.NodeHeartBeat +{ + public enum NodeWatcherResult + { + NodeMayHaveDisconnected, + NoDisconnectSeen + } +} +#endif \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/RedisHelpers/HalibutRedisTransport.cs b/source/Halibut/Queue/Redis/RedisHelpers/HalibutRedisTransport.cs index d7584eddc..90270fb08 100644 --- a/source/Halibut/Queue/Redis/RedisHelpers/HalibutRedisTransport.cs +++ b/source/Halibut/Queue/Redis/RedisHelpers/HalibutRedisTransport.cs @@ -3,6 +3,7 @@ using System; using System.Threading; using System.Threading.Tasks; +using Halibut.Queue.Redis.NodeHeartBeat; using Halibut.Util; using StackExchange.Redis; diff --git a/source/Halibut/Queue/Redis/RedisHelpers/IHalibutRedisTransport.cs b/source/Halibut/Queue/Redis/RedisHelpers/IHalibutRedisTransport.cs index 24a002b98..27466e725 100644 --- a/source/Halibut/Queue/Redis/RedisHelpers/IHalibutRedisTransport.cs +++ b/source/Halibut/Queue/Redis/RedisHelpers/IHalibutRedisTransport.cs @@ -3,6 +3,7 @@ using System; using System.Threading; using System.Threading.Tasks; +using Halibut.Queue.Redis.NodeHeartBeat; using StackExchange.Redis; namespace Halibut.Queue.Redis diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index 58ea540f9..39401ba8b 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -7,6 +7,7 @@ using Halibut.Diagnostics; using Halibut.Queue.Redis.Exceptions; using Halibut.Queue.Redis.MessageStorage; +using Halibut.Queue.Redis.NodeHeartBeat; using Halibut.Queue.Redis.RedisDataLoseDetection; using Halibut.ServiceModel; using Halibut.Transport.Protocol; @@ -179,7 +180,7 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can if (watchProcessingNodeStillHasHeartBeat.IsCompleted) { var watcherResult = await watchProcessingNodeStillHasHeartBeat; - if (watcherResult == NodeHeartBeatSender.NodeProcessingRequestWatcherResult.NodeMayHaveDisconnected) + if (watcherResult == NodeWatcherResult.NodeMayHaveDisconnected) { // Make a list ditch effort to check if a response exists now. if (await pollAndSubscribeToResponse.TryGetResponseFromRedis("Watcher", cancellationToken)) @@ -233,27 +234,19 @@ void InBackgroundSendCancellationIfRequestWasCancelled(RequestMessage request, R } } - async Task WatchProcessingNodeIsStillConnectedInBackground(RequestMessage request, RedisPendingRequest redisPending, CancellationToken cancellationToken) + async Task WatchProcessingNodeIsStillConnectedInBackground(RequestMessage request, RedisPendingRequest redisPending, CancellationToken cancellationToken) { await Task.Yield(); - try - { - return await NodeHeartBeatSender.WatchThatNodeProcessingTheRequestIsStillAlive( - endpoint, - request, - redisPending, - halibutRedisTransport, - TimeBetweenCheckingIfRequestWasCollected, - log, - RequestReceivingNodeIsOfflineHeartBeatTimeout, - cancellationToken); - } - catch (Exception ex) - { - log.WriteException(EventType.Error, "Error watching processing node for request {0}, endpoint {1}", ex, request.ActivityId, endpoint); - } - - return null; + + return await NodeHeartBeatWatcher.WatchThatNodeProcessingTheRequestIsStillAlive( + endpoint, + request, + redisPending, + halibutRedisTransport, + TimeBetweenCheckingIfRequestWasCollected, + log, + RequestReceivingNodeIsOfflineHeartBeatTimeout, + cancellationToken); } async Task TryClearRequestFromQueue(RequestMessage request, RedisPendingRequest redisPending) diff --git a/source/Halibut/Queue/Redis/WatchForRequestCancellationOrSenderDisconnect.cs b/source/Halibut/Queue/Redis/WatchForRequestCancellationOrSenderDisconnect.cs index 4ed7f9ee7..870ea1d3f 100644 --- a/source/Halibut/Queue/Redis/WatchForRequestCancellationOrSenderDisconnect.cs +++ b/source/Halibut/Queue/Redis/WatchForRequestCancellationOrSenderDisconnect.cs @@ -5,6 +5,7 @@ using System.Threading; using System.Threading.Tasks; using Halibut.Diagnostics; +using Halibut.Queue.Redis.NodeHeartBeat; using Halibut.Util; namespace Halibut.Queue.Redis @@ -55,9 +56,9 @@ async Task WatchThatNodeWhichSentTheRequestIsStillAlive(Uri endpoint, Guid reque var watchCancellationToken = keepWatchingCancellationToken.Token; try { - var res = await NodeHeartBeatSender + var res = await NodeHeartBeatWatcher .WatchThatNodeWhichSentTheRequestIsStillAlive(endpoint, requestActivityId, halibutRedisTransport, log, nodeOfflineTimeoutBetweenHeartBeatsFromSender, watchCancellationToken); - if (res == NodeHeartBeatSender.NodeProcessingRequestWatcherResult.NodeMayHaveDisconnected) + if (res == NodeWatcherResult.NodeMayHaveDisconnected) { await requestCancellationTokenSource.CancelAsync(); } diff --git a/source/Halibut/Util/TimeSpanHelper.cs b/source/Halibut/Util/TimeSpanHelper.cs new file mode 100644 index 000000000..c64922c10 --- /dev/null +++ b/source/Halibut/Util/TimeSpanHelper.cs @@ -0,0 +1,12 @@ +using System; + +namespace Halibut.Util +{ + public static class TimeSpanHelper + { + public static TimeSpan Min(TimeSpan t1, TimeSpan t2) + { + return t1 < t2 ? t1 : t2; + } + } +} \ No newline at end of file From d961a7f75b685ec42f41ff0c5a9eb79a05366d82 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Mon, 18 Aug 2025 15:22:54 +1000 Subject: [PATCH 081/137] . --- .../Queue/Redis/NodeHeartBeatSenderFixture.cs | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs b/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs index 0cd9c3abb..081df1915 100644 --- a/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs @@ -1,7 +1,6 @@ #if NET8_0_OR_GREATER using System; using System.Collections.Concurrent; -using System.Linq; using System.Threading; using System.Threading.Tasks; using FluentAssertions; @@ -16,7 +15,6 @@ using Halibut.Tests.TestSetup.Redis; using Nito.AsyncEx; using NUnit.Framework; -using Octopus.TestPortForwarder; namespace Halibut.Tests.Queue.Redis { @@ -193,7 +191,7 @@ public async Task WaitUntilNodeProcessingRequestFlatLines_WhenHeartbeatsStop_Sho // Assert var result = await watcherTask; - result.Should().Be(NodeHeartBeatSender.NodeProcessingRequestWatcherResult.NodeMayHaveDisconnected); + result.Should().Be(NodeWatcherResult.NodeMayHaveDisconnected); } [Test] @@ -231,7 +229,7 @@ public async Task NoIssueReturnedWhenNodeProcessingRequestIsNotSeenToGoOffline() // Assert var result = await watcherTask; - result.Should().Be(NodeHeartBeatSender.NodeProcessingRequestWatcherResult.NoDisconnectSeen); + result.Should().Be(NodeWatcherResult.NoDisconnectSeen); } [Test] @@ -280,7 +278,7 @@ public async Task WaitUntilNodeProcessingRequestFlatLines_WhenConnectionInterrup // Assert - Watcher should detect flatline var result = await watcherTask; - result.Should().Be(NodeHeartBeatSender.NodeProcessingRequestWatcherResult.NodeMayHaveDisconnected); + result.Should().Be(NodeWatcherResult.NodeMayHaveDisconnected); } [Test] From 6b20d3e5231a28bf316896d896e65a25f75bd06a Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Mon, 18 Aug 2025 15:36:38 +1000 Subject: [PATCH 082/137] . --- .../Halibut/Queue/QueueMessageSerializer.cs | 3 - .../Cancellation/RequestCancelledSender.cs | 49 ++++++++++++ .../WatchForRequestCancellation.cs | 42 +--------- .../Queue/Redis/RedisPendingRequest.cs | 3 +- .../Queue/Redis/RedisPendingRequestQueue.cs | 13 ++-- .../Redis/RedisPendingRequestQueueFactory.cs | 2 - .../PollAndSubscribeToResponse.cs | 77 ++++++------------- .../ResponseMessageSender.cs | 46 +++++++++++ ...orRequestCancellationOrSenderDisconnect.cs | 8 +- 9 files changed, 133 insertions(+), 110 deletions(-) create mode 100644 source/Halibut/Queue/Redis/Cancellation/RequestCancelledSender.cs rename source/Halibut/Queue/Redis/{ => Cancellation}/WatchForRequestCancellation.cs (65%) rename source/Halibut/Queue/Redis/{ => ResponseMessageTransfer}/PollAndSubscribeToResponse.cs (68%) create mode 100644 source/Halibut/Queue/Redis/ResponseMessageTransfer/ResponseMessageSender.cs diff --git a/source/Halibut/Queue/QueueMessageSerializer.cs b/source/Halibut/Queue/QueueMessageSerializer.cs index 75555f623..237f2b011 100644 --- a/source/Halibut/Queue/QueueMessageSerializer.cs +++ b/source/Halibut/Queue/QueueMessageSerializer.cs @@ -8,7 +8,6 @@ namespace Halibut.Queue { - // TODO make an interface public class QueueMessageSerializer { readonly Func createStreamCapturingSerializer; @@ -51,8 +50,6 @@ public QueueMessageSerializer(Func createStreamCa } } - - // By making this a generic type, each message specifies the exact type it sends/expects // And it is impossible to deserialize the wrong type - any mismatched type will refuse to deserialize class MessageEnvelope diff --git a/source/Halibut/Queue/Redis/Cancellation/RequestCancelledSender.cs b/source/Halibut/Queue/Redis/Cancellation/RequestCancelledSender.cs new file mode 100644 index 000000000..74aa0a2d1 --- /dev/null +++ b/source/Halibut/Queue/Redis/Cancellation/RequestCancelledSender.cs @@ -0,0 +1,49 @@ +using System; +using System.Threading.Tasks; +using Halibut.Diagnostics; +using Halibut.Transport.Protocol; +using Halibut.Util; + +#if NET8_0_OR_GREATER +namespace Halibut.Queue.Redis.Cancellation +{ + public class RequestCancelledSender + { + // How long the CancelRequestMarker will sit in redis before it times out. + // If it does timeout it won't matter since the request-sender will stop sending heart beats + // causing the request-processor to cancel the request anyway. + static TimeSpan CancelRequestMarkerTTL = TimeSpan.FromMinutes(15); + + public static async Task TrySendCancellation( + IHalibutRedisTransport halibutRedisTransport, + Uri endpoint, + RequestMessage request, + ILog log) + { + log.Write(EventType.Diagnostic, "Attempting to send cancellation for request - Endpoint: {0}, ActivityId: {1}", endpoint, request.ActivityId); + + await using var cts = new CancelOnDisposeCancellationToken(); + cts.CancelAfter(TimeSpan.FromMinutes(2)); // Best efforts. + + try + { + log.Write(EventType.Diagnostic, "Publishing cancellation notification - Endpoint: {0}, ActivityId: {1}", endpoint, request.ActivityId); + await halibutRedisTransport.PublishCancellation(endpoint, request.ActivityId, cts.Token); + + log.Write(EventType.Diagnostic, "Marking request as cancelled - Endpoint: {0}, ActivityId: {1}", endpoint, request.ActivityId); + await halibutRedisTransport.MarkRequestAsCancelled(endpoint, request.ActivityId, CancelRequestMarkerTTL, cts.Token); + + log.Write(EventType.Diagnostic, "Successfully sent cancellation for request - Endpoint: {0}, ActivityId: {1}", endpoint, request.ActivityId); + } + catch (OperationCanceledException ex) + { + log.Write(EventType.Error, "Cancellation send operation timed out after 2 minutes - Endpoint: {0}, ActivityId: {1}, Error: {2}", endpoint, request.ActivityId, ex.Message); + } + catch (Exception ex) + { + log.Write(EventType.Error, "Failed to send cancellation for request - Endpoint: {0}, ActivityId: {1}, Error: {2}", endpoint, request.ActivityId, ex.Message); + } + } + } +} +#endif \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs b/source/Halibut/Queue/Redis/Cancellation/WatchForRequestCancellation.cs similarity index 65% rename from source/Halibut/Queue/Redis/WatchForRequestCancellation.cs rename to source/Halibut/Queue/Redis/Cancellation/WatchForRequestCancellation.cs index 08e9c098b..3ed4849bc 100644 --- a/source/Halibut/Queue/Redis/WatchForRequestCancellation.cs +++ b/source/Halibut/Queue/Redis/Cancellation/WatchForRequestCancellation.cs @@ -7,46 +7,10 @@ using Halibut.Transport.Protocol; using Halibut.Util; -namespace Halibut.Queue.Redis +namespace Halibut.Queue.Redis.Cancellation { public class WatchForRequestCancellation : IAsyncDisposable { - public static async Task TrySendCancellation( - IHalibutRedisTransport halibutRedisTransport, - Uri endpoint, - RequestMessage request, - ILog log) - { - log.Write(EventType.Diagnostic, "Attempting to send cancellation for request - Endpoint: {0}, ActivityId: {1}", endpoint, request.ActivityId); - - await using var cts = new CancelOnDisposeCancellationToken(); - cts.CancelAfter(TimeSpan.FromMinutes(2)); // Best efforts. - - try - { - log.Write(EventType.Diagnostic, "Publishing cancellation notification - Endpoint: {0}, ActivityId: {1}", endpoint, request.ActivityId); - await halibutRedisTransport.PublishCancellation(endpoint, request.ActivityId, cts.Token); - - log.Write(EventType.Diagnostic, "Marking request as cancelled - Endpoint: {0}, ActivityId: {1}", endpoint, request.ActivityId); - await halibutRedisTransport.MarkRequestAsCancelled(endpoint, request.ActivityId, CancelRequestMarkerTTL, cts.Token); - - log.Write(EventType.Diagnostic, "Successfully sent cancellation for request - Endpoint: {0}, ActivityId: {1}", endpoint, request.ActivityId); - } - catch (OperationCanceledException ex) - { - log.Write(EventType.Error, "Cancellation send operation timed out after 2 minutes - Endpoint: {0}, ActivityId: {1}, Error: {2}", endpoint, request.ActivityId, ex.Message); - } - catch (Exception ex) - { - log.Write(EventType.Error, "Failed to send cancellation for request - Endpoint: {0}, ActivityId: {1}, Error: {2}", endpoint, request.ActivityId, ex.Message); - } - } - - // How long the CancelRequestMarker will sit in redis before it times out. - // If it does timeout it won't matter since the request-sender will stop sending heart beats - // causing the request-processor to cancel the request anyway. - static TimeSpan CancelRequestMarkerTTL = TimeSpan.FromMinutes(5); - readonly CancelOnDisposeCancellationToken requestCancelledCts = new(); public CancellationToken RequestCancelledCancellationToken => requestCancelledCts.Token; public bool SenderCancelledTheRequest { get; private set; } @@ -81,9 +45,7 @@ async Task WatchForCancellation(Uri endpoint, Guid requestActivityId, IHalibutRe log.Write(EventType.Diagnostic, "Starting polling loop for request cancellation - Endpoint: {0}, ActivityId: {1}", endpoint, requestActivityId); - // Also poll to see if the request is cancelled since we can miss - // the publication. - // TODO: reconsider if we need this since the heart beats should take care of this. + // Also poll to see if the request is cancelled since we can miss the publication. while (!token.IsCancellationRequested) { try diff --git a/source/Halibut/Queue/Redis/RedisPendingRequest.cs b/source/Halibut/Queue/Redis/RedisPendingRequest.cs index e94955382..33ecb79cc 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequest.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequest.cs @@ -45,7 +45,8 @@ public RedisPendingRequest(RequestMessage request, ILog log) /// This gives the user an opportunity to remove the pending request from shared places and optionally /// call BeginTransfer /// - /// + /// Should the cancellationToken be triggered, this allows for overriding + /// the reason the cancellation token was triggered. The returned error will be thrown. /// public async Task WaitUntilComplete(Func checkIfPendingRequestWasCollectedOrRemoveIt, Func overrideCancellationReason, diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index 39401ba8b..209cb941c 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -5,10 +5,12 @@ using System.Threading; using System.Threading.Tasks; using Halibut.Diagnostics; +using Halibut.Queue.Redis.Cancellation; using Halibut.Queue.Redis.Exceptions; using Halibut.Queue.Redis.MessageStorage; using Halibut.Queue.Redis.NodeHeartBeat; using Halibut.Queue.Redis.RedisDataLoseDetection; +using Halibut.Queue.Redis.ResponseMessageTransfer; using Halibut.ServiceModel; using Halibut.Transport.Protocol; using Halibut.Util; @@ -226,7 +228,7 @@ void InBackgroundSendCancellationIfRequestWasCancelled(RequestMessage request, R if (redisPending.PendingRequestCancellationToken.IsCancellationRequested) { log.Write(EventType.Diagnostic, "Request {0} was cancelled, sending cancellation to endpoint {1}", request.ActivityId, endpoint); - Task.Run(async () => await WatchForRequestCancellation.TrySendCancellation(halibutRedisTransport, endpoint, request, log)); + Task.Run(async () => await RequestCancelledSender.TrySendCancellation(halibutRedisTransport, endpoint, request, log)); } else { @@ -351,20 +353,19 @@ async Task TryClearRequestFromQueue(RequestMessage request, RedisPendingRe public async Task DequeueAsync(CancellationToken cancellationToken) { - // TODO: is it good or bad that redis exceptions will bubble out of here. - // I think it will kill the TCP connection, which will force re-connect (in perhaps a backoff function) + // Is it good or bad that redis exceptions will bubble out of here? + // It will kill the TCP connection, which will force re-connect (in perhaps a backoff function) // This could result in connecting to a node that is actually connected to redis. It could also // cause a cascade of failure from high load. var pending = await DequeueNextAsync(); if (pending == null) return null; - var disposables = new DisposableCollection(); try { // There is a chance the data loss occured after we got the data but before here. // In that case we will just time out because of the lack of heart beats. - var dataLossCT = await this.watchForRedisLosingAllItsData.GetTokenForDataLoseDetection(TimeSpan.FromSeconds(30), queueToken); + var dataLossCT = await watchForRedisLosingAllItsData.GetTokenForDataLoseDetection(TimeSpan.FromSeconds(30), queueToken); disposables.AddAsyncDisposable(new NodeHeartBeatSender(endpoint, pending.ActivityId, halibutRedisTransport, log, HalibutQueueNodeSendingPulses.RequestProcessorNode, DelayBetweenHeartBeatsForRequestProcessor)); var watcher = new WatchForRequestCancellationOrSenderDisconnect(endpoint, pending.ActivityId, halibutRedisTransport, NodeIsOfflineHeartBeatTimeoutForRequestSender, log); @@ -438,7 +439,7 @@ public async Task ApplyResponse(ResponseMessage response, Guid requestActivityId } var responseJson = await messageSerialiserAndDataStreamStorage.PrepareResponse(response, cancellationToken); log.Write(EventType.MessageExchange, "Sending response message for request {0}", requestActivityId); - await PollAndSubscribeToResponse.SendResponse(halibutRedisTransport, endpoint, requestActivityId, responseJson, TTLOfResponseMessage, log); + await ResponseMessageSender.SendResponse(halibutRedisTransport, endpoint, requestActivityId, responseJson, TTLOfResponseMessage, log); log.Write(EventType.MessageExchange, "Successfully applied response for request {0}", requestActivityId); } catch (Exception ex) diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueueFactory.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueueFactory.cs index 09af4a9ee..7e86b869e 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueueFactory.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueueFactory.cs @@ -36,8 +36,6 @@ public RedisPendingRequestQueueFactory( this.watchForRedisLosingAllItsData = watchForRedisLosingAllItsData; } - - public IPendingRequestQueue CreateQueue(Uri endpoint) { return new RedisPendingRequestQueue(endpoint, diff --git a/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs b/source/Halibut/Queue/Redis/ResponseMessageTransfer/PollAndSubscribeToResponse.cs similarity index 68% rename from source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs rename to source/Halibut/Queue/Redis/ResponseMessageTransfer/PollAndSubscribeToResponse.cs index cef5449f5..c77c996a3 100644 --- a/source/Halibut/Queue/Redis/PollAndSubscribeToResponse.cs +++ b/source/Halibut/Queue/Redis/ResponseMessageTransfer/PollAndSubscribeToResponse.cs @@ -7,56 +7,23 @@ using Halibut.Util; using Nito.AsyncEx; -namespace Halibut.Queue.Redis +namespace Halibut.Queue.Redis.ResponseMessageTransfer { public class PollAndSubscribeToResponse : IAsyncDisposable { - public static async Task SendResponse( - IHalibutRedisTransport halibutRedisTransport, - Uri endpoint, - Guid activityId, - string responseMessage, - TimeSpan ttl, - ILog log) - { - log.Write(EventType.Diagnostic, "Attempting to set response for - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); - - await using var cts = new CancelOnDisposeCancellationToken(); - // More than ten minutes to send the response to redis, seems sus. - cts.CancelAfter(TimeSpan.FromMinutes(10)); - - try - { - log.Write(EventType.Diagnostic, "Marking response as set - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); - await halibutRedisTransport.SetResponseMessage(endpoint, activityId, responseMessage, ttl, cts.Token); - - log.Write(EventType.Diagnostic, "Publishing response notification - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); - await halibutRedisTransport.PublishThatResponseIsAvailable(endpoint, activityId, responseMessage, cts.Token); - - log.Write(EventType.Diagnostic, "Successfully set response - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); - } - catch (OperationCanceledException ex) - { - log.Write(EventType.Error, "Set response operation timed out after 2 minutes - Endpoint: {0}, ActivityId: {1}, Error: {2}", endpoint, activityId, ex.Message); - } - catch (Exception ex) - { - log.Write(EventType.Error, "Failed to set response - Endpoint: {0}, ActivityId: {1}, Error: {2}", endpoint, activityId, ex.Message); - } - } - - readonly CancelOnDisposeCancellationToken watcherToken; - + readonly CancelOnDisposeCancellationToken objectLifeTimeCts; readonly ILog log; - readonly IHalibutRedisTransport halibutRedisTransport; readonly Uri endpoint; readonly Guid activityId; readonly LinearBackoffStrategy pollBackoffStrategy; - TaskCompletionSource ResponseJsonCompletionSource = new(); + readonly TaskCompletionSource responseJsonCompletionSource = new(); - public Task ResponseJson => ResponseJsonCompletionSource.Task; + /// + /// An awaitable task that returns when the response is available. + /// + public Task ResponseJson => responseJsonCompletionSource.Task; public PollAndSubscribeToResponse(Uri endpoint, Guid activityId, IHalibutRedisTransport halibutRedisTransport, ILog log) { @@ -72,19 +39,20 @@ public PollAndSubscribeToResponse(Uri endpoint, Guid activityId, IHalibutRedisTr ); this.log.Write(EventType.Diagnostic, "Starting to watch for response - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); - watcherToken = new CancelOnDisposeCancellationToken(); - var token = watcherToken.Token; - watcherToken.AwaitTasksBeforeCTSDispose(Task.Run(async () => await WaitForResponse(token))); + objectLifeTimeCts = new CancelOnDisposeCancellationToken(); + var token = objectLifeTimeCts.Token; + objectLifeTimeCts.AwaitTasksBeforeCTSDispose(Task.Run(async () => await WaitForResponse(token))); } - readonly SemaphoreSlim trySetResultSemaphore = new SemaphoreSlim(1, 1); - async Task WaitForResponse(CancellationToken token) { try { log.Write(EventType.Diagnostic, "Subscribing to response notifications - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); + // This could wait forever to subscribe to redis if redis is offline. We need some way of limiting how long we take + // to subscribe. + // https://whimsical.com/subscribetonodeheartbeatchannel-should-timeout-while-waiting-to--NFWwmPkE7pTBdm2PRUC8Tf await using var _ = await halibutRedisTransport.SubscribeToResponseChannel(endpoint, activityId, async _ => { @@ -96,8 +64,7 @@ async Task WaitForResponse(CancellationToken token) log.Write(EventType.Diagnostic, "Starting polling loop for response - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); - // Also poll to see if the value is set since we can miss - // the publication. + // Also poll to see if the value is set since we can miss the publication. while (!token.IsCancellationRequested) { try @@ -129,6 +96,8 @@ async Task WaitForResponse(CancellationToken token) } } } + + readonly SemaphoreSlim trySetResultSemaphore = new(1, 1); /// /// Makes an attempt to get the response from redis. @@ -140,7 +109,7 @@ public async Task TryGetResponseFromRedis(string detectedBy, CancellationT { using var l = await trySetResultSemaphore.LockAsync(token); - if (ResponseJsonCompletionSource.Task.IsCompleted) return true; + if (responseJsonCompletionSource.Task.IsCompleted) return true; var responseJson = await halibutRedisTransport.GetResponseMessage(endpoint, activityId, token); @@ -150,8 +119,8 @@ public async Task TryGetResponseFromRedis(string detectedBy, CancellationT await DeleteResponseFromRedis(detectedBy, token); - TrySetResponse(responseJson, token); - await Try.IgnoringError(async () => await watcherToken.CancelAsync()); + TrySetResponse(responseJson); + await Try.IgnoringError(async () => await objectLifeTimeCts.CancelAsync()); log.Write(EventType.Diagnostic, "Cancelling polling loop for response - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); return true; } @@ -171,11 +140,11 @@ async Task DeleteResponseFromRedis(string detectedBy, CancellationToken token) } } - void TrySetResponse(string value, CancellationToken cancellationToken) + void TrySetResponse(string value) { try { - ResponseJsonCompletionSource.TrySetResult(value); + responseJsonCompletionSource.TrySetResult(value); } catch (Exception ex) { @@ -187,10 +156,10 @@ public async ValueTask DisposeAsync() { log.Write(EventType.Diagnostic, "Disposing GenericWatcher for response - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); - await Try.IgnoringError(async () => await watcherToken.CancelAsync()); + await Try.IgnoringError(async () => await objectLifeTimeCts.CancelAsync()); // If the message task is not yet complete, then mark it as cancelled - Try.IgnoringError(() => ResponseJsonCompletionSource.TrySetCanceled()); + Try.IgnoringError(() => responseJsonCompletionSource.TrySetCanceled()); log.Write(EventType.Diagnostic, "Disposed GenericWatcher for response - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); } diff --git a/source/Halibut/Queue/Redis/ResponseMessageTransfer/ResponseMessageSender.cs b/source/Halibut/Queue/Redis/ResponseMessageTransfer/ResponseMessageSender.cs new file mode 100644 index 000000000..6b82c8897 --- /dev/null +++ b/source/Halibut/Queue/Redis/ResponseMessageTransfer/ResponseMessageSender.cs @@ -0,0 +1,46 @@ +#if NET8_0_OR_GREATER +using System; +using System.Threading.Tasks; +using Halibut.Diagnostics; +using Halibut.Util; + +namespace Halibut.Queue.Redis.ResponseMessageTransfer +{ + public class ResponseMessageSender + { + public static async Task SendResponse( + IHalibutRedisTransport halibutRedisTransport, + Uri endpoint, + Guid activityId, + string responseMessage, + TimeSpan ttl, + ILog log) + { + log.Write(EventType.Diagnostic, "Attempting to set response for - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); + + await using var cts = new CancelOnDisposeCancellationToken(); + // More than ten minutes to send the response to redis, seems sus. + cts.CancelAfter(TimeSpan.FromMinutes(10)); + + try + { + log.Write(EventType.Diagnostic, "Marking response as set - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); + await halibutRedisTransport.SetResponseMessage(endpoint, activityId, responseMessage, ttl, cts.Token); + + log.Write(EventType.Diagnostic, "Publishing response notification - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); + await halibutRedisTransport.PublishThatResponseIsAvailable(endpoint, activityId, responseMessage, cts.Token); + + log.Write(EventType.Diagnostic, "Successfully set response - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); + } + catch (OperationCanceledException ex) + { + log.Write(EventType.Error, "Set response operation timed out after 2 minutes - Endpoint: {0}, ActivityId: {1}, Error: {2}", endpoint, activityId, ex.Message); + } + catch (Exception ex) + { + log.Write(EventType.Error, "Failed to set response - Endpoint: {0}, ActivityId: {1}, Error: {2}", endpoint, activityId, ex.Message); + } + } + } +} +#endif \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/WatchForRequestCancellationOrSenderDisconnect.cs b/source/Halibut/Queue/Redis/WatchForRequestCancellationOrSenderDisconnect.cs index 870ea1d3f..1a249437a 100644 --- a/source/Halibut/Queue/Redis/WatchForRequestCancellationOrSenderDisconnect.cs +++ b/source/Halibut/Queue/Redis/WatchForRequestCancellationOrSenderDisconnect.cs @@ -1,10 +1,10 @@ #if NET8_0_OR_GREATER using System; -using System.Security.Policy; using System.Threading; using System.Threading.Tasks; using Halibut.Diagnostics; +using Halibut.Queue.Redis.Cancellation; using Halibut.Queue.Redis.NodeHeartBeat; using Halibut.Util; @@ -17,10 +17,10 @@ public class WatchForRequestCancellationOrSenderDisconnect : IAsyncDisposable public CancellationToken RequestProcessingCancellationToken { get; } readonly CancelOnDisposeCancellationToken keepWatchingCancellationToken; - - DisposableCollection disposableCollection = new(); - WatchForRequestCancellation watchForRequestCancellation; + readonly DisposableCollection disposableCollection = new(); + + readonly WatchForRequestCancellation watchForRequestCancellation; public bool SenderCancelledTheRequest => watchForRequestCancellation.SenderCancelledTheRequest; public WatchForRequestCancellationOrSenderDisconnect( From ec493fb75e74a15cbac77e79ce40611afe2a071d Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Mon, 18 Aug 2025 15:38:25 +1000 Subject: [PATCH 083/137] . --- source/Halibut/Queue/QueueMessageSerializer.cs | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/source/Halibut/Queue/QueueMessageSerializer.cs b/source/Halibut/Queue/QueueMessageSerializer.cs index 237f2b011..2cb3bf1ec 100644 --- a/source/Halibut/Queue/QueueMessageSerializer.cs +++ b/source/Halibut/Queue/QueueMessageSerializer.cs @@ -8,6 +8,11 @@ namespace Halibut.Queue { + /// + /// Uses the same JSON serializer used by Halibit to send messages over the wire to + /// serialise messages for the queue. Note that the queue serialises to JSON rather + /// than BSON which is what is sent over the wire. + /// public class QueueMessageSerializer { readonly Func createStreamCapturingSerializer; From 3b2df41488bb0a8bb20c4e6fa775fb5ce67dfa29 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Mon, 18 Aug 2025 15:45:25 +1000 Subject: [PATCH 084/137] Merged in retryable --- ...onReturnedByHalibutProxyExtensionMethod.cs | 59 ++++++++++--------- 1 file changed, 32 insertions(+), 27 deletions(-) diff --git a/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs b/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs index cdc8d9dae..32d119afd 100644 --- a/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs +++ b/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs @@ -15,48 +15,53 @@ public static class ExceptionReturnedByHalibutProxyExtensionMethod public static HalibutRetryableErrorType IsRetryableError(this Exception exception) { var halibutNetworkExceptionType = IsNetworkError(exception); - switch (halibutNetworkExceptionType) - { - case HalibutNetworkExceptionType.IsNetworkError: - return HalibutRetryableErrorType.IsRetryable; - case HalibutNetworkExceptionType.UnknownError: - return HalibutRetryableErrorType.UnknownError; - case HalibutNetworkExceptionType.NotANetworkError: - return HalibutRetryableErrorType.NotRetryable; - default: - throw new ArgumentOutOfRangeException(); - } + + // All network errors can be retried. + if (halibutNetworkExceptionType == HalibutNetworkExceptionType.IsNetworkError) return HalibutRetryableErrorType.IsRetryable; + + if(IsRedisRetryableError(exception)) return HalibutRetryableErrorType.IsRetryable; + + if(halibutNetworkExceptionType == HalibutNetworkExceptionType.NotANetworkError) return HalibutRetryableErrorType.NotRetryable; + + return HalibutRetryableErrorType.UnknownError; } - - /// - /// Classifies the exception thrown from a halibut proxy as a network error or not. - /// In some cases it is not possible to tell if the exception is a network error. - /// - /// - /// The exception thrown from a Halibut proxy object/param> - /// - public static HalibutNetworkExceptionType IsNetworkError(this Exception exception) + + static bool IsRedisRetryableError(Exception exception) { - // TODO: This should be in is retryable but for now it needs to be here to work with tentacle client. if (exception is RedisDataLoseHalibutClientException || exception is RedisQueueShutdownClientException || exception is CouldNotGetDataLoseTokenInTimeHalibutClientException || exception is ErrorWhilePreparingRequestForQueueHalibutClientException || exception is ErrorOccuredWhenInsertingDataIntoRedisHalibutPendingRequestQueue) { - return HalibutNetworkExceptionType.IsNetworkError; + return true; } if (exception is HalibutClientException) { - if (exception.Message.Contains("The request was abandoned, possibly because the node processing the request shutdown or redis lost all of its data.")) return HalibutNetworkExceptionType.IsNetworkError; - if (exception.Message.Contains("The node processing the request did not send a heartbeat for long enough, and so the node is now assumed to be offline.")) return HalibutNetworkExceptionType.IsNetworkError; - if (exception.Message.Contains("Error occured when reading data from the queue")) return HalibutNetworkExceptionType.IsNetworkError; - if(exception.Message.Contains("error occured when preparing request for queue")) return HalibutNetworkExceptionType.IsNetworkError; + if (exception.Message.Contains("The request was abandoned, possibly because the node processing the request shutdown or redis lost all of its data.")) return true; + if (exception.Message.Contains("The node processing the request did not send a heartbeat for long enough, and so the node is now assumed to be offline.")) return true; + if (exception.Message.Contains("Error occured when reading data from the queue")) return true; + if(exception.Message.Contains("error occured when preparing request for queue")) return true; } - // TODO end + if (exception is HalibutClientException && exception.InnerException != null) + { + return IsRedisRetryableError(exception.InnerException); + } + return false; + } + + /// + /// Classifies the exception thrown from a halibut proxy as a network error or not. + /// In some cases it is not possible to tell if the exception is a network error. + /// + /// + /// The exception thrown from a Halibut proxy object/param> + /// + public static HalibutNetworkExceptionType IsNetworkError(this Exception exception) + { if (exception is NoMatchingServiceOrMethodHalibutClientException) { return HalibutNetworkExceptionType.NotANetworkError; From dadb074190f962a78221cc76396693953fb5ddb4 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Tue, 19 Aug 2025 10:08:02 +1000 Subject: [PATCH 085/137] . --- .../Support/TestAttributes/AllQueuesTestCasesAttribute.cs | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/source/Halibut.Tests/Support/TestAttributes/AllQueuesTestCasesAttribute.cs b/source/Halibut.Tests/Support/TestAttributes/AllQueuesTestCasesAttribute.cs index e5c75cc85..58d9d9257 100644 --- a/source/Halibut.Tests/Support/TestAttributes/AllQueuesTestCasesAttribute.cs +++ b/source/Halibut.Tests/Support/TestAttributes/AllQueuesTestCasesAttribute.cs @@ -6,6 +6,7 @@ using Halibut.Tests.Builders; using Halibut.Tests.Support.BackwardsCompatibility; using Halibut.Tests.Support.TestCases; +using Halibut.Tests.TestSetup.Redis; using NUnit.Framework; namespace Halibut.Tests.Support.TestAttributes @@ -26,7 +27,10 @@ public static IEnumerable GetEnumerator() { var factories = new List(); #if NET8_0_OR_GREATER - factories.Add(new PendingRequestQueueTestCase("Redis", () => new RedisPendingRequestQueueBuilder())); + if (EnsureRedisIsAvailableSetupFixture.WillRunRedisTests) + { + factories.Add(new PendingRequestQueueTestCase("Redis", () => new RedisPendingRequestQueueBuilder())); + } #endif factories.Add(new PendingRequestQueueTestCase("InMemory", () => new PendingRequestQueueBuilder())); From 20cf74e2657d7b62d5c456f3eec9a689a644c17c Mon Sep 17 00:00:00 2001 From: Rhys Parry Date: Tue, 19 Aug 2025 11:40:28 +1000 Subject: [PATCH 086/137] Don't re-send response --- .../Queue/Redis/Utils/HalibutRedisTransportWithVirtuals.cs | 4 ++-- .../Halibut/Queue/Redis/RedisHelpers/HalibutRedisTransport.cs | 4 ++-- .../Queue/Redis/RedisHelpers/IHalibutRedisTransport.cs | 2 +- .../Redis/ResponseMessageTransfer/ResponseMessageSender.cs | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/source/Halibut.Tests/Queue/Redis/Utils/HalibutRedisTransportWithVirtuals.cs b/source/Halibut.Tests/Queue/Redis/Utils/HalibutRedisTransportWithVirtuals.cs index 162e55d79..3830b970f 100644 --- a/source/Halibut.Tests/Queue/Redis/Utils/HalibutRedisTransportWithVirtuals.cs +++ b/source/Halibut.Tests/Queue/Redis/Utils/HalibutRedisTransportWithVirtuals.cs @@ -88,9 +88,9 @@ public Task SubscribeToResponseChannel(Uri endpoint, Guid iden return halibutRedisTransport.SubscribeToResponseChannel(endpoint, identifier, onValueReceived, cancellationToken); } - public Task PublishThatResponseIsAvailable(Uri endpoint, Guid identifier, string value, CancellationToken cancellationToken) + public Task PublishThatResponseIsAvailable(Uri endpoint, Guid identifier, CancellationToken cancellationToken) { - return halibutRedisTransport.PublishThatResponseIsAvailable(endpoint, identifier, value, cancellationToken); + return halibutRedisTransport.PublishThatResponseIsAvailable(endpoint, identifier, cancellationToken); } public Task SetResponseMessage(Uri endpoint, Guid identifier, string responseMessage, TimeSpan ttl, CancellationToken cancellationToken) diff --git a/source/Halibut/Queue/Redis/RedisHelpers/HalibutRedisTransport.cs b/source/Halibut/Queue/Redis/RedisHelpers/HalibutRedisTransport.cs index 90270fb08..c5327d736 100644 --- a/source/Halibut/Queue/Redis/RedisHelpers/HalibutRedisTransport.cs +++ b/source/Halibut/Queue/Redis/RedisHelpers/HalibutRedisTransport.cs @@ -237,10 +237,10 @@ public async Task SubscribeToResponseChannel(Uri endpoint, Gui }, cancellationToken); } - public async Task PublishThatResponseIsAvailable(Uri endpoint, Guid identifier, string value, CancellationToken cancellationToken) + public async Task PublishThatResponseIsAvailable(Uri endpoint, Guid identifier, CancellationToken cancellationToken) { var channelName = ResponseChannelName(endpoint, identifier); - await facade.PublishToChannel(channelName, value, cancellationToken); + await facade.PublishToChannel(channelName, "{}", cancellationToken); } // Response diff --git a/source/Halibut/Queue/Redis/RedisHelpers/IHalibutRedisTransport.cs b/source/Halibut/Queue/Redis/RedisHelpers/IHalibutRedisTransport.cs index 27466e725..80f0642aa 100644 --- a/source/Halibut/Queue/Redis/RedisHelpers/IHalibutRedisTransport.cs +++ b/source/Halibut/Queue/Redis/RedisHelpers/IHalibutRedisTransport.cs @@ -44,7 +44,7 @@ Task SubscribeToNodeHeartBeatChannel( Task SubscribeToResponseChannel(Uri endpoint, Guid identifier, Func onValueReceived, CancellationToken cancellationToken); - Task PublishThatResponseIsAvailable(Uri endpoint, Guid identifier, string value, CancellationToken cancellationToken); + Task PublishThatResponseIsAvailable(Uri endpoint, Guid identifier, CancellationToken cancellationToken); Task SetResponseMessage(Uri endpoint, Guid identifier, string responseMessage, TimeSpan ttl, CancellationToken cancellationToken); diff --git a/source/Halibut/Queue/Redis/ResponseMessageTransfer/ResponseMessageSender.cs b/source/Halibut/Queue/Redis/ResponseMessageTransfer/ResponseMessageSender.cs index 6b82c8897..40cfb6985 100644 --- a/source/Halibut/Queue/Redis/ResponseMessageTransfer/ResponseMessageSender.cs +++ b/source/Halibut/Queue/Redis/ResponseMessageTransfer/ResponseMessageSender.cs @@ -28,7 +28,7 @@ public static async Task SendResponse( await halibutRedisTransport.SetResponseMessage(endpoint, activityId, responseMessage, ttl, cts.Token); log.Write(EventType.Diagnostic, "Publishing response notification - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); - await halibutRedisTransport.PublishThatResponseIsAvailable(endpoint, activityId, responseMessage, cts.Token); + await halibutRedisTransport.PublishThatResponseIsAvailable(endpoint, activityId, cts.Token); log.Write(EventType.Diagnostic, "Successfully set response - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); } From d9096ce9a2abbbf24022c409af417fc47fcd7e05 Mon Sep 17 00:00:00 2001 From: Rhys Parry Date: Tue, 19 Aug 2025 12:40:18 +1000 Subject: [PATCH 087/137] Fix typo in Redis queue doc --- docs/RedisQueue.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/RedisQueue.md b/docs/RedisQueue.md index b992fbb1d..2d690ab72 100644 --- a/docs/RedisQueue.md +++ b/docs/RedisQueue.md @@ -5,7 +5,7 @@ a cluster of multiple clients need to send commands to polling services which co clients. For example if we have two clients ClientA and ClientB and the Service connects to B, yet A wants -to execute an RPC. Currently that wont work as the request will end up in the in memory queue for ClientA +to execute an RPC. Currently that won't work as the request will end up in the in memory queue for ClientA but it needs to be accessible to ClientB. The Redis queue solves this, as the request is placed into Redis allowing ClientB to access the request and From 377d690462c5cfc960abf34979001b1f835f7fff Mon Sep 17 00:00:00 2001 From: Rhys Parry Date: Tue, 19 Aug 2025 12:48:19 +1000 Subject: [PATCH 088/137] fmt ExceptionReturnedByHalibutProxyExtensionMethod --- .../ExceptionReturnedByHalibutProxyExtensionMethod.cs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs b/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs index 32d119afd..8499c48e6 100644 --- a/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs +++ b/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs @@ -19,9 +19,9 @@ public static HalibutRetryableErrorType IsRetryableError(this Exception exceptio // All network errors can be retried. if (halibutNetworkExceptionType == HalibutNetworkExceptionType.IsNetworkError) return HalibutRetryableErrorType.IsRetryable; - if(IsRedisRetryableError(exception)) return HalibutRetryableErrorType.IsRetryable; + if (IsRedisRetryableError(exception)) return HalibutRetryableErrorType.IsRetryable; - if(halibutNetworkExceptionType == HalibutNetworkExceptionType.NotANetworkError) return HalibutRetryableErrorType.NotRetryable; + if (halibutNetworkExceptionType == HalibutNetworkExceptionType.NotANetworkError) return HalibutRetryableErrorType.NotRetryable; return HalibutRetryableErrorType.UnknownError; } @@ -42,7 +42,7 @@ static bool IsRedisRetryableError(Exception exception) if (exception.Message.Contains("The request was abandoned, possibly because the node processing the request shutdown or redis lost all of its data.")) return true; if (exception.Message.Contains("The node processing the request did not send a heartbeat for long enough, and so the node is now assumed to be offline.")) return true; if (exception.Message.Contains("Error occured when reading data from the queue")) return true; - if(exception.Message.Contains("error occured when preparing request for queue")) return true; + if (exception.Message.Contains("error occured when preparing request for queue")) return true; } if (exception is HalibutClientException && exception.InnerException != null) From 197924fdde33f86546cb57cfd9bf7efda28122ee Mon Sep 17 00:00:00 2001 From: Rhys Parry Date: Tue, 19 Aug 2025 13:14:41 +1000 Subject: [PATCH 089/137] format Halibut.Util --- source/Halibut/Util/AwaitAllAndIgnoreException.cs | 2 +- source/Halibut/Util/CancelOnDisposeCancellationToken.cs | 8 ++++---- source/Halibut/Util/LinearBackoffStrategy.cs | 6 +++++- source/Halibut/Util/StringExtensionMethods.cs | 6 +++++- 4 files changed, 15 insertions(+), 7 deletions(-) diff --git a/source/Halibut/Util/AwaitAllAndIgnoreException.cs b/source/Halibut/Util/AwaitAllAndIgnoreException.cs index 6891c5329..576072059 100644 --- a/source/Halibut/Util/AwaitAllAndIgnoreException.cs +++ b/source/Halibut/Util/AwaitAllAndIgnoreException.cs @@ -13,7 +13,7 @@ public void AddTasks(params Task[] tasksToAdd) foreach (var task in tasksToAdd) { #pragma warning disable VSTHRD003 - tasks.Add(Halibut.Util.Try.IgnoringError(async () => await task)); + tasks.Add(Try.IgnoringError(async () => await task)); #pragma warning restore VSTHRD003 } } diff --git a/source/Halibut/Util/CancelOnDisposeCancellationToken.cs b/source/Halibut/Util/CancelOnDisposeCancellationToken.cs index 0e3ef0372..0681cc0dc 100644 --- a/source/Halibut/Util/CancelOnDisposeCancellationToken.cs +++ b/source/Halibut/Util/CancelOnDisposeCancellationToken.cs @@ -2,8 +2,6 @@ using System; using System.Threading; using System.Threading.Tasks; -using Halibut.Queue.Redis; -using Halibut.Transport.Protocol; namespace Halibut.Util { @@ -19,7 +17,7 @@ public sealed class CancelOnDisposeCancellationToken : IAsyncDisposable readonly AwaitAllAndIgnoreException awaitAllAndIgnoreException = new(); public CancelOnDisposeCancellationToken(params CancellationToken[] token) - : this(CancellationTokenSource.CreateLinkedTokenSource(token)) + : this(CancellationTokenSource.CreateLinkedTokenSource(token)) { } public CancelOnDisposeCancellationToken() : this(new CancellationTokenSource()) @@ -37,7 +35,9 @@ private CancelOnDisposeCancellationToken(CancellationTokenSource cancellationTok public async ValueTask DisposeAsync() { if (disposed) + { return; + } disposed = true; @@ -69,4 +69,4 @@ public void AwaitTasksBeforeCTSDispose(params Task[] tasksUsingToken) awaitAllAndIgnoreException.AddTasks(tasksUsingToken); } } -} \ No newline at end of file +} \ No newline at end of file diff --git a/source/Halibut/Util/LinearBackoffStrategy.cs b/source/Halibut/Util/LinearBackoffStrategy.cs index 9ca036d0c..9f6cfc774 100644 --- a/source/Halibut/Util/LinearBackoffStrategy.cs +++ b/source/Halibut/Util/LinearBackoffStrategy.cs @@ -66,10 +66,12 @@ public virtual void Success() public virtual TimeSpan GetSleepPeriod() { if (attemptCount <= 0) + { return TimeSpan.Zero; + } var delay = InitialDelay + TimeSpan.FromTicks((attemptCount - 1) * Increment.Ticks); - + // Cap at maximum delay return delay > MaximumDelay ? MaximumDelay : delay; } @@ -80,7 +82,9 @@ public virtual TimeSpan GetSleepPeriod() public TimeSpan CalculateDelayForAttempt(int attemptNumber) { if (attemptNumber <= 0) + { return TimeSpan.Zero; + } var delay = InitialDelay + TimeSpan.FromTicks((attemptNumber - 1) * Increment.Ticks); diff --git a/source/Halibut/Util/StringExtensionMethods.cs b/source/Halibut/Util/StringExtensionMethods.cs index ed6785a53..71f3fc01a 100644 --- a/source/Halibut/Util/StringExtensionMethods.cs +++ b/source/Halibut/Util/StringExtensionMethods.cs @@ -8,7 +8,11 @@ public static class StringExtensionMethods [return: NotNullIfNotNull("str")] public static Guid? ToGuid(this string? str) { - if (str == null) return null; + if (str == null) + { + return null; + } + return Guid.Parse(str); } } From 32bec0ee55d341205936fb8b021ea90910a4dbb8 Mon Sep 17 00:00:00 2001 From: Rhys Parry Date: Tue, 19 Aug 2025 13:16:42 +1000 Subject: [PATCH 090/137] Cleanup trailing whitespace --- source/Halibut.Tests/Builders/PendingRequestQueueBuilder.cs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/Halibut.Tests/Builders/PendingRequestQueueBuilder.cs b/source/Halibut.Tests/Builders/PendingRequestQueueBuilder.cs index 3b0065745..6dd3e5a67 100644 --- a/source/Halibut.Tests/Builders/PendingRequestQueueBuilder.cs +++ b/source/Halibut.Tests/Builders/PendingRequestQueueBuilder.cs @@ -48,7 +48,7 @@ public class QueueHolder : IAsyncDisposable { public IPendingRequestQueue PendingRequestQueue { get; } public DisposableCollection DisposableCollection { get; } - + public QueueHolder(IPendingRequestQueue pendingRequestQueue, DisposableCollection disposableCollection) { this.PendingRequestQueue = pendingRequestQueue; From 2c08d3415308ea46f5af62683682c5c54c4e1e29 Mon Sep 17 00:00:00 2001 From: Rhys Parry Date: Tue, 19 Aug 2025 15:04:32 +1000 Subject: [PATCH 091/137] format HalibutRuntimeBuilder --- source/Halibut/HalibutRuntimeBuilder.cs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/source/Halibut/HalibutRuntimeBuilder.cs b/source/Halibut/HalibutRuntimeBuilder.cs index 83f2fd8aa..ce50749d2 100644 --- a/source/Halibut/HalibutRuntimeBuilder.cs +++ b/source/Halibut/HalibutRuntimeBuilder.cs @@ -161,9 +161,9 @@ public HalibutRuntime Build() configureMessageSerializerBuilder?.Invoke(builder); var messageSerializer = builder.WithTypeRegistry(typeRegistry).Build(); - var queueMessageSerializer = new QueueMessageSerializer(messageSerializer.CreateStreamCapturingSerializer); - var queueFactory = this.queueFactoryFactory?.Invoke(queueMessageSerializer)??new PendingRequestQueueFactoryAsync(halibutTimeoutsAndLimits, logFactory); + var queueFactory = this.queueFactoryFactory?.Invoke(queueMessageSerializer) + ?? new PendingRequestQueueFactoryAsync(halibutTimeoutsAndLimits, logFactory); var streamFactory = this.streamFactory ?? new StreamFactory(); var connectionsObserver = this.connectionsObserver ?? NoOpConnectionsObserver.Instance; From b87cf60d53e0874be5f82f8e76cddb8454cddf34 Mon Sep 17 00:00:00 2001 From: Rhys Parry Date: Tue, 19 Aug 2025 15:35:52 +1000 Subject: [PATCH 092/137] Minor nits in QueueMessageSerializer --- .../Halibut/Queue/QueueMessageSerializer.cs | 19 ++++++++----------- 1 file changed, 8 insertions(+), 11 deletions(-) diff --git a/source/Halibut/Queue/QueueMessageSerializer.cs b/source/Halibut/Queue/QueueMessageSerializer.cs index 2cb3bf1ec..52f544e21 100644 --- a/source/Halibut/Queue/QueueMessageSerializer.cs +++ b/source/Halibut/Queue/QueueMessageSerializer.cs @@ -9,7 +9,7 @@ namespace Halibut.Queue { /// - /// Uses the same JSON serializer used by Halibit to send messages over the wire to + /// Uses the same JSON serializer used by Halibut to send messages over the wire to /// serialise messages for the queue. Note that the queue serialises to JSON rather /// than BSON which is what is sent over the wire. /// @@ -41,18 +41,15 @@ public QueueMessageSerializer(Func createStreamCa public (T Message, IReadOnlyList DataStreams) ReadMessage(string json) { using var reader = new JsonTextReader(new StringReader(json)); - { - - var streamCapturingSerializer = createStreamCapturingSerializer(); - var result = streamCapturingSerializer.Serializer.Deserialize>(reader); + var streamCapturingSerializer = createStreamCapturingSerializer(); + var result = streamCapturingSerializer.Serializer.Deserialize>(reader); - if (result == null) - { - throw new Exception("messageEnvelope is null"); - } - - return (result.Message, streamCapturingSerializer.DataStreams); + if (result == null) + { + throw new Exception("messageEnvelope is null"); } + + return (result.Message, streamCapturingSerializer.DataStreams); } // By making this a generic type, each message specifies the exact type it sends/expects From 4158de2b1dc2df41ac1cd0aac33a319e48c29f5e Mon Sep 17 00:00:00 2001 From: Rhys Parry Date: Tue, 19 Aug 2025 15:56:05 +1000 Subject: [PATCH 093/137] Fix nits for RedisPendingRequest --- .../Queue/Redis/RedisPendingRequest.cs | 54 +++++++++++-------- 1 file changed, 31 insertions(+), 23 deletions(-) diff --git a/source/Halibut/Queue/Redis/RedisPendingRequest.cs b/source/Halibut/Queue/Redis/RedisPendingRequest.cs index 33ecb79cc..b5fc9988b 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequest.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequest.cs @@ -54,13 +54,11 @@ public async Task WaitUntilComplete(Func checkIfPendingRequestWasCollected { log.Write(EventType.MessageExchange, "Request {0} was queued", request); - var pendingRequestPickupTimeout = Try.IgnoringError(async () => await Task.Delay(request.Destination.PollingRequestQueueTimeout, cancellationToken)); var responseWaiterTask = responseWaiter.WaitAsync(cancellationToken); await Task.WhenAny(pendingRequestPickupTimeout, responseWaiterTask); - // Response has been returned so just say we are done. if (responseWaiter.IsSet) { @@ -68,10 +66,13 @@ public async Task WaitUntilComplete(Func checkIfPendingRequestWasCollected return; } - if (!requestCollected.IsSet) await checkIfPendingRequestWasCollectedOrRemoveIt(); + if (!requestCollected.IsSet) + { + await checkIfPendingRequestWasCollectedOrRemoveIt(); + } - using (await transferLock.LockAsync(CancellationToken.None)) { - + using (await transferLock.LockAsync(CancellationToken.None)) + { if (responseWaiter.IsSet) { log.Write(EventType.MessageExchange, "Request {0} was collected by the polling endpoint", request); @@ -100,8 +101,9 @@ public async Task WaitUntilComplete(Func checkIfPendingRequestWasCollected operationCanceledException = new OperationCanceledException($"Request {request} was collected by the polling endpoint, will try to cancel the request"); } - throw requestCollected.IsSet ? new TransferringRequestCancelledException(operationCanceledException) : new ConnectingRequestCancelledException(operationCanceledException); - + throw requestCollected.IsSet + ? new TransferringRequestCancelledException(operationCanceledException) + : new ConnectingRequestCancelledException(operationCanceledException); } if (!requestCollected.IsSet) @@ -112,15 +114,14 @@ public async Task WaitUntilComplete(Func checkIfPendingRequestWasCollected log.Write(EventType.MessageExchange, "Request {0} timed out before it could be collected by the polling endpoint", request); SetResponseNoLock(ResponseMessage.FromException( - request, - new TimeoutException($"A request was sent to a polling endpoint, but the polling endpoint did not collect the request within the allowed time ({request.Destination.PollingRequestQueueTimeout}), so the request timed out."), - ConnectionState.Connecting), - false); + request, + new TimeoutException($"A request was sent to a polling endpoint, but the polling endpoint did not collect the request within the allowed time ({request.Destination.PollingRequestQueueTimeout}), so the request timed out."), + ConnectionState.Connecting), + requestWasCollected: false); return; } } - // The request has been collected so now wait patiently for a response log.Write(EventType.MessageExchange, "Request {0} was eventually collected by the polling endpoint", request); try @@ -143,10 +144,10 @@ public async Task WaitUntilComplete(Func checkIfPendingRequestWasCollected log.Write(EventType.MessageExchange, "Request {0} was cancelled before a response was received" + overrideCancellationReason()??"", request); SetResponseNoLock(ResponseMessage.FromException( - request, - new TimeoutException($"A request was sent to a polling endpoint, the polling endpoint collected it but the request was cancelled before the polling endpoint responded." + overrideCancellationReason()??""), - ConnectionState.Connecting), - false); + request, + new TimeoutException($"A request was sent to a polling endpoint, the polling endpoint collected it but the request was cancelled before the polling endpoint responded." + overrideCancellationReason()??""), + ConnectionState.Connecting), + requestWasCollected: false); await Try.IgnoringError(async () => await pendingRequestCancellationTokenSource.CancelAsync()); } } @@ -156,15 +157,14 @@ public async Task WaitUntilComplete(Func checkIfPendingRequestWasCollected // This should never happen. log.Write(EventType.MessageExchange, "Request {0} had an internal error, unexpectedly stopped waiting for the response.", request); await SetResponseAsync(ResponseMessage.FromException( - request, - new PendingRequestQueueInternalException($"Request {request.Id} had an internal error, unexpectedly stopped waiting for the response.")), - false); + request, + new PendingRequestQueueInternalException($"Request {request.Id} had an internal error, unexpectedly stopped waiting for the response.")), + requestWasCollected: false); } } public static OperationCanceledException CreateExceptionForRequestWasCancelledBeforeCollected(RequestMessage request, ILog log) { - log.Write(EventType.MessageExchange, "Request {0} was cancelled before it could be collected by the polling endpoint", request); return new OperationCanceledException($"Request {request} was cancelled before it could be collected by the polling endpoint"); } @@ -202,7 +202,7 @@ public async Task RequestHasBeenCollectedAndWillBeTransferred() public async Task SetResponse(ResponseMessage response) { // If someone is calling this then we know for sure they collected the request - return await SetResponseAsync(response, true); + return await SetResponseAsync(response, requestWasCollected: true); } async Task SetResponseAsync(ResponseMessage response, bool requestWasCollected) @@ -215,10 +215,18 @@ async Task SetResponseAsync(ResponseMessage response, bool requ ResponseMessage SetResponseNoLock(ResponseMessage response, bool requestWasCollected) { - if(this.response != null) return this.response; + if (this.response != null) + { + return this.response; + } + this.response = response; responseWaiter.Set(); - if(requestWasCollected) requestCollected.Set(); // Also the request has been collected, if we have a response. + if (requestWasCollected) + { + requestCollected.Set(); // Also the request has been collected, if we have a response. + } + return this.response; } From 10c1108506320b70c59096ba046b199540b634fb Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Tue, 19 Aug 2025 15:58:11 +1000 Subject: [PATCH 094/137] More doco --- docs/RedisQueue.md | 72 ++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 72 insertions(+) diff --git a/docs/RedisQueue.md b/docs/RedisQueue.md index b992fbb1d..76e26cd05 100644 --- a/docs/RedisQueue.md +++ b/docs/RedisQueue.md @@ -23,3 +23,75 @@ Note that Redis is configured to have no backup, everything must be in memory. T ## TODO design. +### Context: Pending Request Queue. + +Halibut turns an RPC call into a RequestMessage which is placed into the Pending Request Queue. This is done by calling: `ResponseMessage QueueAndWait(RequestMessage)`. Which is a blocking call that queues the RequestMessage and waits for the ResponseMessage before returning. + +Polling service, e.g, Tentacle, call into the `Dequeue` method of the queue to get the next `RequestMessage` to processing. It then responds by calling `ApplyResponse(ResponseMessage)`, doing so results in `QueueAndWait()` returning the ResponseMessage. This in turn results in the RPC call completing. + +The Redis Pending Request Queue solves the problem where we have multiple clients, that wish to execute RPC calls to a single Polling Service that is connected to exactly one client. For example Client A makes an RPC call, but the service is connected to Client B. The Redis Pending Request Queue is what moves the `RequestMessage` from Client A to Client B to be sent to the service. + +### Context: Redis + +First we need to understand just a little about Redis and how we are using redis: + - Redis may have data lose. + - Pub/Sub does not have guaranteed delivery, we can miss publication. + - Pub/Sub channels are not pets in Redis, they can be created simply by "subscribing" and are "deleted" when there are no subscribers to that channel. + - Redis is connected to via the network, which can be flaky we will make retries to Redis when we can. + +## High Level design. + +Setup: + - Client A is executing the RPC call + - Client B has the Polling service connected to it. + +At a high level steps the Redis Queue goes through to execute an RPC are: + + 1. Client B subscribes to the unique "RequestMessage Pulse Channel", as the client service is connected to it. The channel is keyed by the polling client id e.g. "poll://123" + 2. Client A executes an RPC and so Calls QueueAndWait with a RequestMessage. Each RequestMessage has a unique `GUID`. + 2.1 Client A subscribes to the `ResponseMessage channel` keyed by `GUID` to be notified when a response is available. + 3. Client A serialises the message and places the message into a hash in Redis keyed by the RequestMessage `Guid`. + 4. Client A Adds the `GUID` to the polling clients unique Redis list (aka queue). The key is the polling client id e.g. "poll://123". + 5. Client A pulses the polling clients unique "RequestMessage Pulse Channel", to alert to it that it has work to do. + 6. Client B receives the Pulse message and tries to dequeue a `GUID` from the polling clients unique Redis list (aka queue). + 7. Client B now has the `GUID` of the request and so atomically gets and deletes the RequestMessage from the Redis Hash using that guid. + 8. Client B sends the request to the tentacle, waits for the response, and calls `ApplyResponse()` with the ResponseMessage. + 9. Client B writes the `ResponseMessage` to redis in a hash using the `GUID` as the key. + 10. Client B Pulses the `ResponseMessage channel` keyed by the RequestMessage `GUID`, that a Response is available. + 11. Client A receives a pulse on the `ResponseMessage channel` and so knows a Response is available, it reads the response from Redis and returns from the `QueueAndWait()` method. + + +## Pub/Sub and Poll. + +Since Pub/Sub does not have guaranteed delivery in Redis, in any place that we do Pub/Sub we must also have a form of polling. For example: + - When Dequeuing work not only are we subscribed but when `Dequeue()` is called we also check for work on the queue anyway. (Note that Dequeue() returns every 30s if there is no work, and thus we have polling.) + - When waiting for a Response, we are not only subscribed to the response channel we also poll to see if the Response has been sent back. + +## Dealing with nodes that disappear mid request. + +Either node could go offline at any time, including during execution of an RPC. For example: + - The node executing the RPC could go offline, when the node with the Service connected is sending the Request to the Service. + - The node sending the Request to the Service could go offline. + +To handle this case in a way that allows for large file transfers aka request that take a long time, we have a concept of "heart beats". + +When executing an RPC both nodes involved will send heart beats to a unique channel keyed by the request ID AND the nodes role in the RPC. For example: +- The node executing RPC will pulse heart beats to a channel with a key such as `NodeSendingRequest:GUID` +- The node sending the request to the service will pulse heart beats to a channel with a key such as: `NodeReceivingRequest:GUID` + +Now each node can watch for heart beats from the other node, when heart beats stop being sent they can assume it is offline and cancel/abandon the request. + +## Dealing with Redis losing its data. + +Since redis can lose data at anytime the queue is able to detect data lose and cancel any inflight requests when data lose occurs. + +## Message serialisation + +Message serialisation is provided by re-using the serialiser halibut uses for transferring requests/responses over the wire. + + +### DataStream + +DataStreams are not stored in the queue, instead an implementation of `IStoreDataStreamsForDistributedQueues` must be provided. It will be called with the DataStreams that are to be stored, and will be called again with the "husks" of a DataStream that needs to be re-hydrated. DataStreams have unique GUIDs which make it easier to find the data for re-hydration. + +Sub classing DataStream is a useful technique for avoiding the storage of DataStream data when it is trivial to read the data from some known places. For example a DataStream might be subclassed to hold the file location on disk that should be read when sending the data for a data stream. The halibut serialiser has been updated to work with sub classes of DataStream, in that it will ignore the sub class and send just the DataStream across the wire. This makes it safe to sub class DataStream for efficient storage and have that work with both listening and polling clients. From 13f3c526ac33e78833bb91aefcc345eab94aacdd Mon Sep 17 00:00:00 2001 From: Rhys Parry Date: Tue, 19 Aug 2025 16:01:21 +1000 Subject: [PATCH 095/137] Fix typo in GetTokenForDataLossDetection --- .../WatchForRedisLosingAllItsDataFixture.cs | 12 ++++++------ ...ncellableDataLossWatchForRedisLosingAllItsData.cs | 2 +- .../NeverLosingDataWatchForRedisLosingAllItsData.cs | 2 +- .../IWatchForRedisLosingAllItsData.cs | 10 +++++----- .../WatchForRedisLosingAllItsData.cs | 2 +- .../Halibut/Queue/Redis/RedisPendingRequestQueue.cs | 4 ++-- 6 files changed, 16 insertions(+), 16 deletions(-) diff --git a/source/Halibut.Tests/Queue/Redis/RedisDataLoseDetection/WatchForRedisLosingAllItsDataFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisDataLoseDetection/WatchForRedisLosingAllItsDataFixture.cs index 9fdbdeff1..d645afa01 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisDataLoseDetection/WatchForRedisLosingAllItsDataFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisDataLoseDetection/WatchForRedisLosingAllItsDataFixture.cs @@ -21,7 +21,7 @@ public async Task WhenTheConnectionToRedisCanNotBeCreated_WhenAskingForALostData await using var watcher = new WatchForRedisLosingAllItsData(redisFacade, HalibutLog, watchInterval:TimeSpan.FromSeconds(1)); - await AssertException.Throws(watcher.GetTokenForDataLoseDetection(TimeSpan.FromSeconds(1), CancellationToken)); + await AssertException.Throws(watcher.GetTokenForDataLossDetection(TimeSpan.FromSeconds(1), CancellationToken)); } [Test] @@ -39,7 +39,7 @@ public async Task WhenTheConnectionToRedisIsInitiallyDown_WhenAskingForALostData }); - await watcher.GetTokenForDataLoseDetection(TimeSpan.FromSeconds(20), CancellationToken); + await watcher.GetTokenForDataLossDetection(TimeSpan.FromSeconds(20), CancellationToken); } [Test] @@ -51,7 +51,7 @@ public async Task WatchForARealRedisLosingAllOfItsData_TimesOutWhenWaitingForCTW await using var watcher = new WatchForRedisLosingAllItsData(redisFacade, HalibutLog, watchInterval:TimeSpan.FromSeconds(1)); - await AssertException.Throws(watcher.GetTokenForDataLoseDetection(TimeSpan.FromSeconds(1), CancellationToken)); + await AssertException.Throws(watcher.GetTokenForDataLossDetection(TimeSpan.FromSeconds(1), CancellationToken)); } [Test] @@ -59,7 +59,7 @@ public async Task WhenRedisRunsForLongerThanTheKeyTTL_NoDataLoseShouldBeDetected { await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); await using var watcher = new WatchForRedisLosingAllItsData(redisFacade, HalibutLog, watchInterval:TimeSpan.FromMilliseconds(100), keyTTL: TimeSpan.FromSeconds(2)); - var watcherCt = await watcher.GetTokenForDataLoseDetection(TimeSpan.FromSeconds(20), CancellationToken); + var watcherCt = await watcher.GetTokenForDataLossDetection(TimeSpan.FromSeconds(20), CancellationToken); await Task.Delay(TimeSpan.FromSeconds(4)); watcherCt.IsCancellationRequested.Should().BeFalse(); @@ -85,7 +85,7 @@ public async Task WhenRedisLosesAllOfIts_TheWatcherShouldDetectTheDataLose() await using var watcher = new WatchForRedisLosingAllItsData(redisFacade, HalibutLog, watchInterval:TimeSpan.FromSeconds(1)); Logger.Information("Getting initial cancellation token for data loss detection (20 second timeout)"); - var watcherCT = await watcher.GetTokenForDataLoseDetection(TimeSpan.FromSeconds(20), CancellationToken); + var watcherCT = await watcher.GetTokenForDataLossDetection(TimeSpan.FromSeconds(20), CancellationToken); Logger.Information("Initial cancellation token obtained, IsCancellationRequested: {IsCancellationRequested}", watcherCT.IsCancellationRequested); watcherCT.IsCancellationRequested.Should().BeFalse(); @@ -106,7 +106,7 @@ public async Task WhenRedisLosesAllOfIts_TheWatcherShouldDetectTheDataLose() watcherCT.IsCancellationRequested.Should().BeTrue("Should have detected the data loss"); Logger.Information("Getting new cancellation token to verify recovery"); - var nextToken = await watcher.GetTokenForDataLoseDetection(TimeSpan.FromSeconds(20), CancellationToken); + var nextToken = await watcher.GetTokenForDataLossDetection(TimeSpan.FromSeconds(20), CancellationToken); nextToken.IsCancellationRequested.Should().BeFalse("The new token should have no data loss"); } diff --git a/source/Halibut.Tests/Queue/Redis/Utils/CancellableDataLossWatchForRedisLosingAllItsData.cs b/source/Halibut.Tests/Queue/Redis/Utils/CancellableDataLossWatchForRedisLosingAllItsData.cs index e8a5135ea..b8e57756b 100644 --- a/source/Halibut.Tests/Queue/Redis/Utils/CancellableDataLossWatchForRedisLosingAllItsData.cs +++ b/source/Halibut.Tests/Queue/Redis/Utils/CancellableDataLossWatchForRedisLosingAllItsData.cs @@ -31,7 +31,7 @@ public async ValueTask DisposeAsync() await Try.CatchingError(async () => await cancellationToken.DisposeAsync()); } - public async Task GetTokenForDataLoseDetection(TimeSpan timeToWait, CancellationToken cancellationToken) + public async Task GetTokenForDataLossDetection(TimeSpan timeToWait, CancellationToken cancellationToken) { #pragma warning disable VSTHRD003 return await TaskCompletionSource.Task; diff --git a/source/Halibut.Tests/Queue/Redis/Utils/NeverLosingDataWatchForRedisLosingAllItsData.cs b/source/Halibut.Tests/Queue/Redis/Utils/NeverLosingDataWatchForRedisLosingAllItsData.cs index b376838a6..51f139d14 100644 --- a/source/Halibut.Tests/Queue/Redis/Utils/NeverLosingDataWatchForRedisLosingAllItsData.cs +++ b/source/Halibut.Tests/Queue/Redis/Utils/NeverLosingDataWatchForRedisLosingAllItsData.cs @@ -14,7 +14,7 @@ namespace Halibut.Tests.Queue.Redis.Utils /// public class NeverLosingDataWatchForRedisLosingAllItsData : IWatchForRedisLosingAllItsData { - public Task GetTokenForDataLoseDetection(TimeSpan timeToWait, CancellationToken cancellationToken) + public Task GetTokenForDataLossDetection(TimeSpan timeToWait, CancellationToken cancellationToken) { return Task.FromResult(CancellationToken.None); } diff --git a/source/Halibut/Queue/Redis/RedisDataLoseDetection/IWatchForRedisLosingAllItsData.cs b/source/Halibut/Queue/Redis/RedisDataLoseDetection/IWatchForRedisLosingAllItsData.cs index 2cc0e66e4..2d873abcc 100644 --- a/source/Halibut/Queue/Redis/RedisDataLoseDetection/IWatchForRedisLosingAllItsData.cs +++ b/source/Halibut/Queue/Redis/RedisDataLoseDetection/IWatchForRedisLosingAllItsData.cs @@ -7,12 +7,12 @@ namespace Halibut.Queue.Redis.RedisDataLoseDetection public interface IWatchForRedisLosingAllItsData : IAsyncDisposable { /// - /// Returns a Cancellation token which is triggered when data lose occurs. - /// Will cause the caller to wait until we are connected to redis and so can detect datalose. + /// Returns a Cancellation token which is triggered when data loss occurs. + /// Will cause the caller to wait until we are connected to redis and so can detect data loss. /// - /// Time to wait for this to reach a state where it can detect datalose + /// Time to wait for this to reach a state where it can detect data loss /// - /// A cancellation token which is triggered when data lose occurs. - Task GetTokenForDataLoseDetection(TimeSpan timeToWait, CancellationToken cancellationToken); + /// A cancellation token which is triggered when data loss occurs. + Task GetTokenForDataLossDetection(TimeSpan timeToWait, CancellationToken cancellationToken); } } \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/RedisDataLoseDetection/WatchForRedisLosingAllItsData.cs b/source/Halibut/Queue/Redis/RedisDataLoseDetection/WatchForRedisLosingAllItsData.cs index 30406c5b6..b09b46641 100644 --- a/source/Halibut/Queue/Redis/RedisDataLoseDetection/WatchForRedisLosingAllItsData.cs +++ b/source/Halibut/Queue/Redis/RedisDataLoseDetection/WatchForRedisLosingAllItsData.cs @@ -51,7 +51,7 @@ public WatchForRedisLosingAllItsData(RedisFacade redisFacade, ILog log, TimeSpan /// Time to wait for this to reach a state where it can detect datalose /// /// A cancellation token which is triggered when data lose occurs. - public async Task GetTokenForDataLoseDetection(TimeSpan timeToWait, CancellationToken cancellationToken) + public async Task GetTokenForDataLossDetection(TimeSpan timeToWait, CancellationToken cancellationToken) { if (taskCompletionSource.Task.IsCompleted) { diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index 209cb941c..67946310d 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -74,7 +74,7 @@ private async Task DataLossCancellationToken(CancellationToke { // TODO this must throw something that can be retried. await using var cts = new CancelOnDisposeCancellationToken(queueCts.Token, cancellationToken ?? CancellationToken.None); - return await watchForRedisLosingAllItsData.GetTokenForDataLoseDetection(TimeSpan.FromSeconds(30), cts.Token); + return await watchForRedisLosingAllItsData.GetTokenForDataLossDetection(TimeSpan.FromSeconds(30), cts.Token); } public async Task QueueAndWaitAsync(RequestMessage request, CancellationToken requestCancellationToken) @@ -365,7 +365,7 @@ async Task TryClearRequestFromQueue(RequestMessage request, RedisPendingRe { // There is a chance the data loss occured after we got the data but before here. // In that case we will just time out because of the lack of heart beats. - var dataLossCT = await watchForRedisLosingAllItsData.GetTokenForDataLoseDetection(TimeSpan.FromSeconds(30), queueToken); + var dataLossCT = await watchForRedisLosingAllItsData.GetTokenForDataLossDetection(TimeSpan.FromSeconds(30), queueToken); disposables.AddAsyncDisposable(new NodeHeartBeatSender(endpoint, pending.ActivityId, halibutRedisTransport, log, HalibutQueueNodeSendingPulses.RequestProcessorNode, DelayBetweenHeartBeatsForRequestProcessor)); var watcher = new WatchForRequestCancellationOrSenderDisconnect(endpoint, pending.ActivityId, halibutRedisTransport, NodeIsOfflineHeartBeatTimeoutForRequestSender, log); From 33502f3f17bfcbf7230fb9958febd8c539676ab3 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Tue, 19 Aug 2025 16:50:06 +1000 Subject: [PATCH 096/137] test cleanup --- .../WatchForRedisLosingAllItsDataFixture.cs | 25 +--- .../Redis/RedisHelpers/RedisFacadeFixture.cs | 47 ++++++- .../RedisFacadeWhenRedisGoesDownAwayTests.cs | 133 +++++++----------- 3 files changed, 99 insertions(+), 106 deletions(-) diff --git a/source/Halibut.Tests/Queue/Redis/RedisDataLoseDetection/WatchForRedisLosingAllItsDataFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisDataLoseDetection/WatchForRedisLosingAllItsDataFixture.cs index 9fdbdeff1..52cc40727 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisDataLoseDetection/WatchForRedisLosingAllItsDataFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisDataLoseDetection/WatchForRedisLosingAllItsDataFixture.cs @@ -25,7 +25,7 @@ public async Task WhenTheConnectionToRedisCanNotBeCreated_WhenAskingForALostData } [Test] - public async Task WhenTheConnectionToRedisIsInitiallyDown_WhenAskingForALostDataCancellationToken_AndTheConnectionToRedisReturns_TheCTIsReturned() + public async Task WhenTheConnectionToRedisIsInitiallyDown_WhenAskingForALostDataCancellationToken_AndTheConnectionToRedisReturns_TheCancellationTokenIsReturned() { using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); portForwarder.EnterKillNewAndExistingConnectionsMode(); @@ -41,21 +41,9 @@ public async Task WhenTheConnectionToRedisIsInitiallyDown_WhenAskingForALostData await watcher.GetTokenForDataLoseDetection(TimeSpan.FromSeconds(20), CancellationToken); } - - [Test] - public async Task WatchForARealRedisLosingAllOfItsData_TimesOutWhenWaitingForCTWhenNoConnectionToRedisCanBeEstablished() - { - using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); - portForwarder.EnterKillNewAndExistingConnectionsMode(); - await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder, null); - await using var watcher = new WatchForRedisLosingAllItsData(redisFacade, HalibutLog, watchInterval:TimeSpan.FromSeconds(1)); - - - await AssertException.Throws(watcher.GetTokenForDataLoseDetection(TimeSpan.FromSeconds(1), CancellationToken)); - } [Test] - public async Task WhenRedisRunsForLongerThanTheKeyTTL_NoDataLoseShouldBeDetected() + public async Task WhenTheWatcherWatchesRedisForMoreThanTheKeyTTL_NoDataLoseShouldBeDetected() { await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); await using var watcher = new WatchForRedisLosingAllItsData(redisFacade, HalibutLog, watchInterval:TimeSpan.FromMilliseconds(100), keyTTL: TimeSpan.FromSeconds(2)); @@ -66,14 +54,11 @@ public async Task WhenRedisRunsForLongerThanTheKeyTTL_NoDataLoseShouldBeDetected } [Test] - public async Task WhenRedisLosesAllOfIts_TheWatcherShouldDetectTheDataLose() + public async Task WhenWatchingRedisForDataLose_AndRedisLosesAllDaya_DataLoseIsDetected() { - Logger.Information("Starting WatchForARealRedisLosingAllOfItsData_E2E_Test"); - // Arrange - Create Redis container using the builder Logger.Information("Creating Redis container"); - await using var container = new RedisContainerBuilder() - .Build(); + await using var container = new RedisContainerBuilder().Build(); Logger.Information("Starting Redis container"); await container.StartAsync(); @@ -82,7 +67,7 @@ public async Task WhenRedisLosesAllOfIts_TheWatcherShouldDetectTheDataLose() // Create RedisFacade connected to the containerized Redis await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(host: "localhost", container.RedisPort); - await using var watcher = new WatchForRedisLosingAllItsData(redisFacade, HalibutLog, watchInterval:TimeSpan.FromSeconds(1)); + await using var watcher = new WatchForRedisLosingAllItsData(redisFacade, HalibutLog, watchInterval: TimeSpan.FromSeconds(1)); Logger.Information("Getting initial cancellation token for data loss detection (20 second timeout)"); var watcherCT = await watcher.GetTokenForDataLoseDetection(TimeSpan.FromSeconds(20), CancellationToken); diff --git a/source/Halibut.Tests/Queue/Redis/RedisHelpers/RedisFacadeFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisHelpers/RedisFacadeFixture.cs index 67e2dda7c..108ec76fb 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisHelpers/RedisFacadeFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisHelpers/RedisFacadeFixture.cs @@ -1,6 +1,7 @@ #if NET8_0_OR_GREATER using System; using System.Collections.Generic; +using System.Linq; using System.Threading.Tasks; using FluentAssertions; using Halibut.Logging; @@ -10,6 +11,7 @@ using Halibut.Tests.Support; using Halibut.Tests.Support.Logging; using Halibut.Util.AsyncEx; +using Nito.AsyncEx; using NUnit.Framework; namespace Halibut.Tests.Queue.Redis.RedisHelpers @@ -17,7 +19,6 @@ namespace Halibut.Tests.Queue.Redis.RedisHelpers [RedisTest] public class RedisFacadeFixture : BaseTest { - [Test] public async Task SetString_AndGetString_ShouldStoreAndRetrieveValue() { @@ -545,6 +546,48 @@ await ShouldEventually.Eventually(async () => stillExists2.Should().Be(value2); } + [Test] + public async Task TryGetAndDeleteFromHash_WithConcurrentCalls_ShouldReturnValueToExactlyOneCall() + { + // Arrange + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); + var key = Guid.NewGuid().ToString(); + var field = "test-field"; + var payload = "test-payload"; + const int concurrentCallCount = 20; + + // Set a value in the hash + await redisFacade.SetInHash(key, field, payload, TimeSpan.FromMinutes(1), CancellationToken); + + var countDownLatch = new AsyncCountdownEvent(concurrentCallCount); + + // Act - Make multiple concurrent calls to TryGetAndDeleteFromHash + var concurrentTasks = new Task[concurrentCallCount]; + for (int i = 0; i < concurrentCallCount; i++) + { + concurrentTasks[i] = Task.Run(async () => + { + countDownLatch.Signal(); + await countDownLatch.WaitAsync(); + return await redisFacade.TryGetAndDeleteFromHash(key, field, CancellationToken); + }); + } + + var results = await Task.WhenAll(concurrentTasks); + + // Assert - Exactly one call should get the payload, all others should get null + var nonNullResults = results.Where(result => result != null).ToArray(); + var nullResults = results.Where(result => result == null).ToArray(); + + nonNullResults.Should().HaveCount(1, "exactly one concurrent call should retrieve the value"); + nonNullResults[0].Should().Be(payload, "the successful call should return the correct payload"); + nullResults.Should().HaveCount(concurrentCallCount - 1, "all other concurrent calls should return null"); + + // Verify the hash key no longer exists + var existsAfterConcurrentCalls = await redisFacade.HashContainsKey(key, field, CancellationToken); + existsAfterConcurrentCalls.Should().BeFalse("the hash key should be deleted after the successful TryGetAndDeleteFromHash call"); + } + [Test] public async Task DisposeAsync_ShouldCleanupResourcesAndNotThrow() { @@ -573,8 +616,6 @@ public async Task DisposeAsync_CalledMultipleTimes_ShouldNotThrow() Func secondDisposeAction = async () => await redisFacade.DisposeAsync(); await secondDisposeAction.Should().NotThrowAsync(); } - - } } #endif \ No newline at end of file diff --git a/source/Halibut.Tests/Queue/Redis/RedisHelpers/RedisFacadeWhenRedisGoesDownAwayTests.cs b/source/Halibut.Tests/Queue/Redis/RedisHelpers/RedisFacadeWhenRedisGoesDownAwayTests.cs index ad211ba4c..de157973a 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisHelpers/RedisFacadeWhenRedisGoesDownAwayTests.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisHelpers/RedisFacadeWhenRedisGoesDownAwayTests.cs @@ -1,11 +1,9 @@ #if NET8_0_OR_GREATER using System; using System.Collections.Concurrent; -using System.Diagnostics.CodeAnalysis; using System.Threading.Tasks; using FluentAssertions; using Halibut.Logging; -using Halibut.Queue.Redis; using Halibut.Queue.Redis.RedisHelpers; using Halibut.Tests.Queue.Redis.Utils; using Halibut.Tests.Support; @@ -18,21 +16,14 @@ namespace Halibut.Tests.Queue.Redis.RedisHelpers { [RedisTest] - [SuppressMessage("Usage", "VSTHRD003:Avoid awaiting foreign Tasks")] public class RedisFacadeWhenRedisGoesDownAwayTests : BaseTest { - static RedisFacade CreateRedisFacade(int? port = 0, Guid? guid = null) - { - port = port == 0 ? RedisTestHost.Port() : port; - return new RedisFacade("localhost:" + port, (guid ?? Guid.NewGuid()).ToString(), new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix("")); - } - [Test] - public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_SometimeLaterOnWeCanDoBasicCalls() + public async Task WhenTheEstablishedConnectionToRedisBrieflyGoesDown_WeCanEventuallyInteractWithRedisAgain() { using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); - await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder); await redisFacade.SetString("foo", "bar", TimeSpan.FromMinutes(1), CancellationToken); @@ -41,40 +32,25 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe portForwarder.EnterKillNewAndExistingConnectionsMode(); portForwarder.ReturnToNormalMode(); - // After a short delay it does seem to work again. - await Task.Delay(1000); - await redisFacade.GetString("foo", CancellationToken); } [Test] - public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelyDoBasicCalls() + public async Task WhenTheEstablishedConnectionToRedisBrieflyGoesDown_WeCanPublishToAChannel() { using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); - await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); - - await redisFacade.SetString("foo", "bar", TimeSpan.FromMinutes(1), CancellationToken); - - (await redisFacade.GetString("foo", CancellationToken)).Should().Be("bar"); - - portForwarder.EnterKillNewAndExistingConnectionsMode(); - portForwarder.ReturnToNormalMode(); - - // No delay here - - await redisFacade.GetString("foo", CancellationToken); - } - - [Test] - public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelyPublishToChannel() - { - using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); - - await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); + var guid = Guid.NewGuid(); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder, guid); + + await using var redisFacadeReliable = RedisFacadeBuilder.CreateRedisFacade(prefix: guid); + // TODO subscribe to test-channel using redisFacadeReliable, and record messages received. + // then at the end of the test check that we got the published message. + // Establish connection first await redisFacade.SetString("connection", "established", TimeSpan.FromMinutes(1), CancellationToken); + portForwarder.EnterKillNewAndExistingConnectionsMode(); portForwarder.ReturnToNormalMode(); @@ -84,142 +60,137 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenRe } [Test] - public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelySetInHash() + public async Task WhenTheEstablishedConnectionToRedisBrieflyGoesDown_WeCanImmediatelySetInHash() { using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); - await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder); // Establish connection first await redisFacade.SetString("connection", "established", TimeSpan.FromMinutes(1), CancellationToken); portForwarder.EnterKillNewAndExistingConnectionsMode(); portForwarder.ReturnToNormalMode(); - - // No delay here - should retry and succeed + await redisFacade.SetInHash("test-hash", "test-field", "test-value", TimeSpan.FromMinutes(1), CancellationToken); + // TODO: check we can get the value from the hash. } [Test] - public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelyTryGetAndDeleteFromHash() + public async Task WhenTheEstablishedConnectionToRedisBrieflyGoesDown_WeCanImmediatelyTryGetAndDeleteFromHash() { using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); - await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder); // Establish connection and set up test data await redisFacade.SetInHash("test-hash", "test-field", "test-value", TimeSpan.FromMinutes(1), CancellationToken); portForwarder.EnterKillNewAndExistingConnectionsMode(); portForwarder.ReturnToNormalMode(); - - // No delay here - should retry and succeed + var result = await redisFacade.TryGetAndDeleteFromHash("test-hash", "test-field", CancellationToken); result.Should().Be("test-value"); } [Test] - public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelyListRightPush() + public async Task WhenTheEstablishedConnectionToRedisBrieflyGoesDown_WeCanImmediatelyListRightPush() { using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); - await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder); // Establish connection first await redisFacade.SetString("connection", "established", TimeSpan.FromMinutes(1), CancellationToken); portForwarder.EnterKillNewAndExistingConnectionsMode(); portForwarder.ReturnToNormalMode(); - - // No delay here - should retry and succeed + await redisFacade.ListRightPushAsync("test-list", "test-item", TimeSpan.FromMinutes(1), CancellationToken); + // TODO: check that we can pop test-list. } [Test] - public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelyListLeftPop() + public async Task WhenTheEstablishedConnectionToRedisBrieflyGoesDown_WeCanImmediatelyListLeftPop() { using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); - await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder); // Establish connection and set up test data await redisFacade.ListRightPushAsync("test-list", "test-item", TimeSpan.FromMinutes(1), CancellationToken); portForwarder.EnterKillNewAndExistingConnectionsMode(); portForwarder.ReturnToNormalMode(); - - // No delay here - should retry and succeed + var result = await redisFacade.ListLeftPopAsync("test-list", CancellationToken); result.Should().Be("test-item"); } [Test] - public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelySetString() + public async Task WhenTheEstablishedConnectionToRedisBrieflyGoesDown_WeCanImmediatelySetString() { using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); - await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder); // Establish connection first await redisFacade.SetString("connection", "established", TimeSpan.FromMinutes(1), CancellationToken); portForwarder.EnterKillNewAndExistingConnectionsMode(); portForwarder.ReturnToNormalMode(); - - // No delay here - should retry and succeed + await redisFacade.SetString("test-key", "test-value", TimeSpan.FromMinutes(1), CancellationToken); + // TODO: assert we can read back the string here. } [Test] - public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelyGetString() + public async Task WhenTheEstablishedConnectionToRedisBrieflyGoesDown_WeCanImmediatelyGetString() { using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); - await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder); // Establish connection and set up test data await redisFacade.SetString("test-key", "test-value", TimeSpan.FromMinutes(1), CancellationToken); portForwarder.EnterKillNewAndExistingConnectionsMode(); portForwarder.ReturnToNormalMode(); - - // No delay here - should retry and succeed + var result = await redisFacade.GetString("test-key", CancellationToken); result.Should().Be("test-value"); } [Test] - public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndThenReConnected_WeCanImmediatelyHashContainsKey() + public async Task WhenTheEstablishedConnectionToRedisBrieflyGoesDown_WeCanImmediatelyHashContainsKey() { using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); - await using var redisFacade = CreateRedisFacade(portForwarder.ListeningPort); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder); // Establish connection and set up test data await redisFacade.SetInHash("test-hash", "test-field", "test-value", TimeSpan.FromMinutes(1), CancellationToken); portForwarder.EnterKillNewAndExistingConnectionsMode(); portForwarder.ReturnToNormalMode(); - - // No delay here - should retry and succeed + var exists = await redisFacade.HashContainsKey("test-hash", "test-field", CancellationToken); exists.Should().BeTrue(); } [Test] - public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndWeTryToSubscribe_WhenTheConnectionIsRestored_WeCanReceiveMessages() + public async Task WhenTheConnectionToRedisHasBeenEstablished_AndIsLaterTerminated_AndThenWeTryToSubscribe_WhenTheConnectionIsRestored_WeCanReceiveMessages() { using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); - var redisLogCreator = new TestContextLogCreator("Redis", LogLevel.Trace); var guid = Guid.NewGuid(); await using var redisViaPortForwarder = RedisFacadeBuilder.CreateRedisFacade(portForwarder: portForwarder, prefix: guid); await using var redisStableConnection = RedisFacadeBuilder.CreateRedisFacade(prefix: guid); - await redisViaPortForwarder.SetString("Establish connection", "before we subscribe", TimeSpan.FromMinutes(1), CancellationToken); + await redisStableConnection.PublishToChannel("bob", "establishing connection to redis", CancellationToken); portForwarder.EnterKillNewAndExistingConnectionsMode(); var msgs = new ConcurrentBag(); - var subscribeToChannelTask = redisViaPortForwarder.SubscribeToChannel("bob", async message => + using var subscribeToChannelTask = redisViaPortForwarder.SubscribeToChannel("bob", async message => { await Task.CompletedTask; msgs.Add(message.Message!); @@ -229,7 +200,7 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndWeTryT await Task.Delay(2000); await redisStableConnection.PublishToChannel("bob", "MISSED", CancellationToken); - // Just in case the subscriber reconnects faster than the publish call. + // Just in case the subscriber reconnects faster than redis publishes the MISSED message. await Task.Delay(2000); portForwarder.ReturnToNormalMode(); @@ -242,7 +213,7 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndWeTryT await Task.Delay(1000); } - msgs.Should().Contain("RECONNECT", "Since the subscriber should eventually connect up"); + msgs.Should().Contain("RECONNECT", "Since the subscriber should eventually connect back up"); msgs.Should().NotContain("MISSED", "Since this was sent when the subscriber could not have been connected. " + "If this is seen maybe the test itself has a bug."); } @@ -251,7 +222,6 @@ public async Task WhenTheConnectionHasBeenEstablishedAndThenTerminated_AndWeTryT public async Task WhenTheConnectionIsNeverEstablished_AndWeTryToSubscribe_WhenTheConnectionIsRestored_WeCanReceiveMessages() { using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); - var redisLogCreator = new TestContextLogCreator("Redis", LogLevel.Trace); var guid = Guid.NewGuid(); await using var redisViaPortForwarder = RedisFacadeBuilder.CreateRedisFacade(portForwarder: portForwarder, prefix: guid); await using var redisStableConnection = RedisFacadeBuilder.CreateRedisFacade(prefix: guid); @@ -259,14 +229,12 @@ public async Task WhenTheConnectionIsNeverEstablished_AndWeTryToSubscribe_WhenTh portForwarder.EnterKillNewAndExistingConnectionsMode(); var msgs = new ConcurrentBag(); - var subscribeToChannelTask = redisViaPortForwarder.SubscribeToChannel("bob", async message => + using var subscribeToChannelTask = redisViaPortForwarder.SubscribeToChannel("bob", async message => { await Task.CompletedTask; msgs.Add(message.Message!); }, CancellationToken); - await using var _ = new FuncAsyncDisposable(() => Try.IgnoringError(async () => await (await subscribeToChannelTask).DisposeAsync())); - // Give everything enough time to have a crack at trying to subscribe to messages. await Task.Delay(2000); await redisStableConnection.PublishToChannel("bob", "MISSED", CancellationToken); @@ -289,17 +257,14 @@ public async Task WhenTheConnectionIsNeverEstablished_AndWeTryToSubscribe_WhenTh "If this is seen maybe the test itself has a bug."); } - [WindowsTest] + [Test] public async Task WhenSubscribedAndTheConnectionGoesDown_WhenTheConnectionIsRestored_MessagesCanEventuallyBeSentToTheSubscriberAgain() { using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); - var redisLogCreator = new TestContextLogCreator("Redis", LogLevel.Trace); - - var guid = Guid.NewGuid().ToString(); - await using var redisViaPortForwarder = new RedisFacade("localhost:" + portForwarder.ListeningPort, guid, redisLogCreator.CreateNewForPrefix("Unstable")); - - await using var redisStableConnection = new RedisFacade("localhost:" + RedisTestHost.Port(), guid, redisLogCreator.CreateNewForPrefix("Stable")); + var guid = Guid.NewGuid(); + await using var redisViaPortForwarder = RedisFacadeBuilder.CreateRedisFacade(portForwarder: portForwarder, prefix: guid); + await using var redisStableConnection = RedisFacadeBuilder.CreateRedisFacade(prefix: guid); var msgs = new ConcurrentBag(); await using var channel = await redisViaPortForwarder.SubscribeToChannel("bob", async message => @@ -311,18 +276,20 @@ public async Task WhenSubscribedAndTheConnectionGoesDown_WhenTheConnectionIsRest // Check both sides can publish. await redisViaPortForwarder.PublishToChannel("bob", "hello unstable", CancellationToken); await redisStableConnection.PublishToChannel("bob", "hello stable", CancellationToken); - await Task.Delay(1000); // TODO better - msgs.Should().BeEquivalentTo("hello unstable", "hello stable"); + + await ShouldEventually.Eventually(() => msgs.Should().BeEquivalentTo("hello unstable", "hello stable"), TimeSpan.FromSeconds(10), CancellationToken); + portForwarder.EnterKillNewAndExistingConnectionsMode(); // The stable connection should still be able to publish to redis. // But the subscriber on the unstable connection will not got the message. await redisStableConnection.PublishToChannel("bob", "MISSED", CancellationToken); - await Task.Delay(1111); + await Task.Delay(1111); // Delay for some amount of time for redis to publish MISSED this won't be received since the connection is down. portForwarder.ReturnToNormalMode(); while (msgs.Count <= 2) { + CancellationToken.ThrowIfCancellationRequested(); Logger.Information("Trying again"); await redisStableConnection.PublishToChannel("bob", "RECONNECT", CancellationToken); await Task.Delay(1000); From c0fb6337adc3a526ef0a9a31635cf9d8cd799ca9 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Tue, 19 Aug 2025 16:55:25 +1000 Subject: [PATCH 097/137] test cleanup --- .../RedisFacadeWhenRedisGoesDownAwayTests.cs | 35 ++++++++++++------- 1 file changed, 23 insertions(+), 12 deletions(-) diff --git a/source/Halibut.Tests/Queue/Redis/RedisHelpers/RedisFacadeWhenRedisGoesDownAwayTests.cs b/source/Halibut.Tests/Queue/Redis/RedisHelpers/RedisFacadeWhenRedisGoesDownAwayTests.cs index de157973a..84f59be21 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisHelpers/RedisFacadeWhenRedisGoesDownAwayTests.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisHelpers/RedisFacadeWhenRedisGoesDownAwayTests.cs @@ -3,15 +3,9 @@ using System.Collections.Concurrent; using System.Threading.Tasks; using FluentAssertions; -using Halibut.Logging; -using Halibut.Queue.Redis.RedisHelpers; using Halibut.Tests.Queue.Redis.Utils; using Halibut.Tests.Support; -using Halibut.Tests.Support.Logging; -using Halibut.Tests.TestSetup.Redis; -using Halibut.Util; using NUnit.Framework; -using Try = Halibut.Util.Try; namespace Halibut.Tests.Queue.Redis.RedisHelpers { @@ -45,8 +39,12 @@ public async Task WhenTheEstablishedConnectionToRedisBrieflyGoesDown_WeCanPublis await using var redisFacadeReliable = RedisFacadeBuilder.CreateRedisFacade(prefix: guid); - // TODO subscribe to test-channel using redisFacadeReliable, and record messages received. - // then at the end of the test check that we got the published message. + var receivedMessages = new ConcurrentBag(); + await using var subscription = await redisFacadeReliable.SubscribeToChannel("test-channel", async message => + { + await Task.CompletedTask; + receivedMessages.Add(message.Message!); + }, CancellationToken); // Establish connection first await redisFacade.SetString("connection", "established", TimeSpan.FromMinutes(1), CancellationToken); @@ -55,8 +53,11 @@ public async Task WhenTheEstablishedConnectionToRedisBrieflyGoesDown_WeCanPublis portForwarder.EnterKillNewAndExistingConnectionsMode(); portForwarder.ReturnToNormalMode(); - // No delay here - should retry and succeed + // Assert await redisFacade.PublishToChannel("test-channel", "test-message", CancellationToken); + + // Check that publish actually happened. + await ShouldEventually.Eventually(() => receivedMessages.Should().Contain("test-message"), TimeSpan.FromSeconds(10), CancellationToken); } [Test] @@ -72,8 +73,12 @@ public async Task WhenTheEstablishedConnectionToRedisBrieflyGoesDown_WeCanImmedi portForwarder.EnterKillNewAndExistingConnectionsMode(); portForwarder.ReturnToNormalMode(); + // Assert await redisFacade.SetInHash("test-hash", "test-field", "test-value", TimeSpan.FromMinutes(1), CancellationToken); - // TODO: check we can get the value from the hash. + + // Check that the value was set. + var retrievedValue = await redisFacade.TryGetAndDeleteFromHash("test-hash", "test-field", CancellationToken); + retrievedValue.Should().Be("test-value"); } [Test] @@ -107,7 +112,10 @@ public async Task WhenTheEstablishedConnectionToRedisBrieflyGoesDown_WeCanImmedi portForwarder.ReturnToNormalMode(); await redisFacade.ListRightPushAsync("test-list", "test-item", TimeSpan.FromMinutes(1), CancellationToken); - // TODO: check that we can pop test-list. + + // Check we actually added something to the queue. + var poppedValue = await redisFacade.ListLeftPopAsync("test-list", CancellationToken); + poppedValue.Should().Be("test-item"); } [Test] @@ -141,7 +149,10 @@ public async Task WhenTheEstablishedConnectionToRedisBrieflyGoesDown_WeCanImmedi portForwarder.ReturnToNormalMode(); await redisFacade.SetString("test-key", "test-value", TimeSpan.FromMinutes(1), CancellationToken); - // TODO: assert we can read back the string here. + + // Verify we can read back the string + var retrievedValue = await redisFacade.GetString("test-key", CancellationToken); + retrievedValue.Should().Be("test-value"); } [Test] From c4c48feee77425c291bf40efef3ea5813a14a22a Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Wed, 20 Aug 2025 07:28:43 +1000 Subject: [PATCH 098/137] . --- .../NodeHeartBeatSenderFixture.cs | 204 ++++++------------ 1 file changed, 71 insertions(+), 133 deletions(-) rename source/Halibut.Tests/Queue/Redis/{ => NodeHeartBeat}/NodeHeartBeatSenderFixture.cs (69%) diff --git a/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs b/source/Halibut.Tests/Queue/Redis/NodeHeartBeat/NodeHeartBeatSenderFixture.cs similarity index 69% rename from source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs rename to source/Halibut.Tests/Queue/Redis/NodeHeartBeat/NodeHeartBeatSenderFixture.cs index 081df1915..2fb224301 100644 --- a/source/Halibut.Tests/Queue/Redis/NodeHeartBeatSenderFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/NodeHeartBeat/NodeHeartBeatSenderFixture.cs @@ -16,12 +16,11 @@ using Nito.AsyncEx; using NUnit.Framework; -namespace Halibut.Tests.Queue.Redis +namespace Halibut.Tests.Queue.Redis.NodeHeartBeat { [RedisTest] public class NodeHeartBeatSenderFixture : BaseTest { - // TODO: ai tests need review [Test] public async Task WhenCreated_ShouldStartSendingHeartbeats() { @@ -45,16 +44,16 @@ public async Task WhenCreated_ShouldStartSendingHeartbeats() // Act await using var heartBeatSender = new NodeHeartBeatSender(endpoint, requestActivityId, redisTransport, log, HalibutQueueNodeSendingPulses.RequestProcessorNode, defaultDelayBetweenPulses: TimeSpan.FromSeconds(1)); - // Wait for a few heartbeats + // Wait for a heart beat. await Task.WhenAny(Task.Delay(TimeSpan.FromSeconds(20), CancellationToken), anyHeartBeatReceived.WaitAsync()); // Assert anyHeartBeatReceived.IsSet.Should().BeTrue("Should have received at least one heartbeat"); } - // Not sure this is a good test - //[Test] - public async Task WhenRedisConnectionIsInterrupted_ShouldSwitchToPanicMode() + + [Test] + public async Task WhenHeartBeatsAreBeingSent_AndTheConnectionToRedisIsBrieflyDown_HeatBeatsShouldBeSentAgain() { // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid()); @@ -69,6 +68,7 @@ public async Task WhenRedisConnectionIsInterrupted_ShouldSwitchToPanicMode() var redisTransport = new HalibutRedisTransport(unstableRedisFacade); var heartbeatsReceived = new ConcurrentBag(); + var heartBeatReceivedEvent = new AsyncManualResetEvent(false); // Subscribe with stable connection to monitor heartbeats await using var subscription = await new HalibutRedisTransport(stableRedisFacade) @@ -76,6 +76,7 @@ public async Task WhenRedisConnectionIsInterrupted_ShouldSwitchToPanicMode() endpoint, requestActivityId, HalibutQueueNodeSendingPulses.RequestProcessorNode, async () => { await Task.CompletedTask; + heartBeatReceivedEvent.Set(); heartbeatsReceived.Add(DateTimeOffset.Now); }, CancellationToken); @@ -83,27 +84,21 @@ public async Task WhenRedisConnectionIsInterrupted_ShouldSwitchToPanicMode() await using var heartBeatSender = new NodeHeartBeatSender(endpoint, requestActivityId, redisTransport, log, HalibutQueueNodeSendingPulses.RequestProcessorNode, TimeSpan.FromSeconds(1)); // Wait for initial heartbeat - await Task.Delay(TimeSpan.FromSeconds(2), CancellationToken); - var initialHeartbeatCount = heartbeatsReceived.Count; + await heartBeatReceivedEvent.WaitAsync(CancellationToken); // Interrupt connection portForwarder.EnterKillNewAndExistingConnectionsMode(); - // Wait during the outage - await Task.Delay(TimeSpan.FromSeconds(10), CancellationToken); - var heartbeatsDuringOutage = heartbeatsReceived.Count - initialHeartbeatCount; + // Outage is 10s + await Task.Delay(TimeSpan.FromSeconds(4), CancellationToken); + heartBeatReceivedEvent.Reset(); // Restore connection portForwarder.ReturnToNormalMode(); - - // Wait for recovery - await Task.Delay(TimeSpan.FromSeconds(15), CancellationToken); - var heartbeatsAfterRecovery = heartbeatsReceived.Count - initialHeartbeatCount - heartbeatsDuringOutage; // Assert - initialHeartbeatCount.Should().BeGreaterThan(0, "Should have received initial heartbeats"); - heartbeatsDuringOutage.Should().Be(0, "Should not receive heartbeats during network outage"); - heartbeatsAfterRecovery.Should().BeGreaterThan(0, "Should resume sending heartbeats after recovery"); + await Task.WhenAny(Task.Delay(TimeSpan.FromSeconds(10)), heartBeatReceivedEvent.WaitAsync(CancellationToken)); + heartBeatReceivedEvent.IsSet.Should().BeTrue("Heart beats should be sent again after the interruption."); } [Test] @@ -136,17 +131,18 @@ public async Task WhenDisposed_ShouldStopSendingHeartbeats() // Dispose the sender await heartBeatSender.DisposeAsync(); - await heartBeatSender.TaskSendingPulses; - + await Task.WhenAny(Task.Delay(TimeSpan.FromSeconds(4)), heartBeatSender.TaskSendingPulses); anyHeartBeatReceived.Reset(); await Task.WhenAny(Task.Delay(TimeSpan.FromSeconds(5), CancellationToken), anyHeartBeatReceived.WaitAsync()); + // Assert anyHeartBeatReceived.IsSet.Should().BeFalse(); + heartBeatSender.TaskSendingPulses.IsCompleted.Should().BeTrue(); } [Test] - public async Task WaitUntilNodeProcessingRequestFlatLines_WhenHeartbeatsStop_ShouldReturnProcessingNodeIsLikelyDisconnected() + public async Task WhenWatchingTheNodeProcessingTheRequestIsStillAlive_AndHeartbeatsStopBeingSent_ShouldReturnProcessingNodeIsLikelyDisconnected() { // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid()); @@ -167,7 +163,13 @@ public async Task WaitUntilNodeProcessingRequestFlatLines_WhenHeartbeatsStop_Sho var pendingRequest = new RedisPendingRequest(request, log); // Start heartbeat sender - await using var heartBeatSender = new NodeHeartBeatSender(endpoint, requestActivityId, unstableRedisTransport, log, HalibutQueueNodeSendingPulses.RequestProcessorNode, TimeSpan.FromSeconds(1)); + await using var heartBeatSender = new NodeHeartBeatSender( + endpoint, + requestActivityId, + unstableRedisTransport, + log, + HalibutQueueNodeSendingPulses.RequestProcessorNode, + defaultDelayBetweenPulses: TimeSpan.FromMilliseconds(200)); // Mark request as collected so watcher proceeds to monitoring phase await pendingRequest.RequestHasBeenCollectedAndWillBeTransferred(); @@ -180,60 +182,25 @@ public async Task WaitUntilNodeProcessingRequestFlatLines_WhenHeartbeatsStop_Sho stableRedisTransport, TimeSpan.FromSeconds(1), log, - TimeSpan.FromSeconds(10), // Short timeout for test + TimeSpan.FromSeconds(5), // Short timeout for test CancellationToken); // Wait for initial heartbeats to establish baseline await Task.Delay(TimeSpan.FromSeconds(3), CancellationToken); + watcherTask.IsCompleted.Should().BeFalse(); // Act - Kill the connection to stop heartbeats portForwarder.EnterKillNewAndExistingConnectionsMode(); // Assert + await Task.WhenAny(Task.Delay(TimeSpan.FromSeconds(10)), watcherTask); + watcherTask.IsCompleted.Should().BeTrue("Since it should have detected no heart beats have been sent for some time."); var result = await watcherTask; result.Should().Be(NodeWatcherResult.NodeMayHaveDisconnected); } [Test] - public async Task NoIssueReturnedWhenNodeProcessingRequestIsNotSeenToGoOffline() - { - // Arrange - var endpoint = new Uri("poll://" + Guid.NewGuid()); - var requestActivityId = Guid.NewGuid(); - var log = new TestContextLogCreator("NodeHeartBeat", LogLevel.Trace).CreateNewForPrefix(""); - await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); - var redisTransport = new HalibutRedisTransport(redisFacade); - - var request = new RequestMessageBuilder(endpoint.ToString()) - .WithActivityId(requestActivityId) - .Build(); - var pendingRequest = new RedisPendingRequest(request, log); - await pendingRequest.RequestHasBeenCollectedAndWillBeTransferred(); - - // Start heartbeat sender - await using var heartBeatSender = new NodeHeartBeatSender(endpoint, requestActivityId, redisTransport, log, HalibutQueueNodeSendingPulses.RequestProcessorNode, TimeSpan.FromSeconds(1)); - - // Start the watcher without marking request as collected - using var cts = new CancellationTokenSource(); - var watcherTask = NodeHeartBeatWatcher.WatchThatNodeProcessingTheRequestIsStillAlive( - endpoint, - request, - pendingRequest, - redisTransport, - TimeSpan.FromSeconds(1), - log, - TimeSpan.FromMinutes(5), - cts.Token); - - await cts.CancelAsync(); - - // Assert - var result = await watcherTask; - result.Should().Be(NodeWatcherResult.NoDisconnectSeen); - } - - [Test] - public async Task WaitUntilNodeProcessingRequestFlatLines_WhenConnectionInterruptedDuringMonitoring_ShouldStillDetectFlatline() + public async Task WhenWatchingTheNodeProcessingTheRequestIsStillAlive_AndTheWatchersConnectionToRedisGoesDown_ShouldReturnProcessingNodeIsLikelyDisconnected() { // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid()); @@ -253,36 +220,45 @@ public async Task WaitUntilNodeProcessingRequestFlatLines_WhenConnectionInterrup .Build(); var pendingRequest = new RedisPendingRequest(request, log); - // Start heartbeat sender with unstable connection - await using var heartBeatSender = new NodeHeartBeatSender(endpoint, requestActivityId, unstableRedisTransport, log, HalibutQueueNodeSendingPulses.RequestProcessorNode, TimeSpan.FromSeconds(1)); + // Start heartbeat sender + await using var heartBeatSender = new NodeHeartBeatSender( + endpoint, + requestActivityId, + stableRedisTransport, + log, + HalibutQueueNodeSendingPulses.RequestProcessorNode, + defaultDelayBetweenPulses: TimeSpan.FromMilliseconds(200)); - // Mark request as collected + // Mark request as collected so watcher proceeds to monitoring phase await pendingRequest.RequestHasBeenCollectedAndWillBeTransferred(); - // Start watcher with stable connection + // Start the watcher var watcherTask = NodeHeartBeatWatcher.WatchThatNodeProcessingTheRequestIsStillAlive( endpoint, request, pendingRequest, - stableRedisTransport, + unstableRedisTransport, TimeSpan.FromSeconds(1), log, - TimeSpan.FromSeconds(15), // Short timeout for test + TimeSpan.FromSeconds(5), // Short timeout for test CancellationToken); - // Wait for initial heartbeats + // Wait for initial heartbeats to establish baseline await Task.Delay(TimeSpan.FromSeconds(3), CancellationToken); + watcherTask.IsCompleted.Should().BeFalse(); - // Act - Interrupt the heartbeat sender's connection + // Act - Kill the connection to stop heartbeats portForwarder.EnterKillNewAndExistingConnectionsMode(); - // Assert - Watcher should detect flatline + // Assert + await Task.WhenAny(Task.Delay(TimeSpan.FromSeconds(10)), watcherTask); + watcherTask.IsCompleted.Should().BeTrue("Since it should have detected no heart beats have been sent for some time."); var result = await watcherTask; result.Should().Be(NodeWatcherResult.NodeMayHaveDisconnected); } - - [Test] - public async Task WhenMultipleHeartBeatSendersForSameRequest_OnlyOneSetOfHeartbeatsShouldBeReceived() + + [Test] + public async Task WhenWatchingTheNodeProcessingTheRequestIsStillAlive_AndTheConnectionIsSuperStableAndWeStopWatching_WatcherShouldReturnNodeStayedConnected() { // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid()); @@ -291,71 +267,33 @@ public async Task WhenMultipleHeartBeatSendersForSameRequest_OnlyOneSetOfHeartbe await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); var redisTransport = new HalibutRedisTransport(redisFacade); - var heartbeatsReceived = new ConcurrentBag(); + var request = new RequestMessageBuilder(endpoint.ToString()) + .WithActivityId(requestActivityId) + .Build(); + var pendingRequest = new RedisPendingRequest(request, log); + await pendingRequest.RequestHasBeenCollectedAndWillBeTransferred(); - await using var subscription = await redisTransport.SubscribeToNodeHeartBeatChannel( - endpoint, requestActivityId, HalibutQueueNodeSendingPulses.RequestProcessorNode, async () => - { - await Task.CompletedTask; - heartbeatsReceived.Add(DateTimeOffset.Now); - }, CancellationToken); - - // Act - Create multiple senders for the same request - await using var heartBeatSender1 = new NodeHeartBeatSender(endpoint, requestActivityId, redisTransport, log, HalibutQueueNodeSendingPulses.RequestProcessorNode, TimeSpan.FromSeconds(1)); - await using var heartBeatSender2 = new NodeHeartBeatSender(endpoint, requestActivityId, redisTransport, log, HalibutQueueNodeSendingPulses.RequestProcessorNode, TimeSpan.FromSeconds(1)); + await using var heartBeatSender = new NodeHeartBeatSender(endpoint, requestActivityId, redisTransport, log, HalibutQueueNodeSendingPulses.RequestProcessorNode, TimeSpan.FromSeconds(1)); - // Wait for heartbeats - await Task.Delay(TimeSpan.FromSeconds(10), CancellationToken); - - // Assert - heartbeatsReceived.Should().NotBeEmpty("Should have received heartbeats"); - // Note: We can't easily assert the exact count since both senders are publishing, - // but we can verify the system handles multiple senders gracefully - } + using var watcherCts = new CancellationTokenSource(); + var watcherTask = NodeHeartBeatWatcher.WatchThatNodeProcessingTheRequestIsStillAlive( + endpoint, + request, + pendingRequest, + redisTransport, + timeBetweenCheckingIfRequestWasCollected: TimeSpan.FromSeconds(1), + log, + maxTimeBetweenHeartBeetsBeforeProcessingNodeIsAssumedToBeOffline: TimeSpan.FromMinutes(1), + watcherCts.Token); - [Test] - public async Task WhenHeartBeatSenderConnectionRecovery_ShouldResumeNormalHeartbeatInterval() - { - // Arrange - var endpoint = new Uri("poll://" + Guid.NewGuid()); - var requestActivityId = Guid.NewGuid(); - var log = new TestContextLogCreator("NodeHeartBeat", LogLevel.Trace).CreateNewForPrefix(""); - var guid = Guid.NewGuid(); - - using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); - await using var unstableRedisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder, guid); - await using var stableRedisFacade = RedisFacadeBuilder.CreateRedisFacade(prefix: guid); - - var unstableRedisTransport = new HalibutRedisTransport(unstableRedisFacade); - var stableRedisTransport = new HalibutRedisTransport(stableRedisFacade); - - var heartbeatTimestamps = new ConcurrentBag(); + await Task.Delay(100); - var heartBeatsReceived = new AsyncManualResetEvent(false); - - await using var subscription = await stableRedisTransport.SubscribeToNodeHeartBeatChannel( - endpoint, requestActivityId, HalibutQueueNodeSendingPulses.RequestProcessorNode, async () => - { - await Task.CompletedTask; - heartbeatTimestamps.Add(DateTimeOffset.Now); - heartBeatsReceived.Set(); - }, CancellationToken); - // Act - await using var heartBeatSender = new NodeHeartBeatSender(endpoint, requestActivityId, unstableRedisTransport, log, HalibutQueueNodeSendingPulses.RequestProcessorNode, TimeSpan.FromSeconds(1)); - - // Wait for initial heartbeats (normal 15s interval) - await heartBeatsReceived.WaitAsync(CancellationToken); - - // Interrupt connection to trigger panic mode (7s interval) - portForwarder.EnterKillNewAndExistingConnectionsMode(); - await Task.Delay(TimeSpan.FromSeconds(2), CancellationToken); - heartBeatsReceived.Reset(); + await watcherCts.CancelAsync(); - // Restore connection - portForwarder.ReturnToNormalMode(); - - await heartBeatsReceived.WaitAsync(CancellationToken); + // Assert + var result = await watcherTask; + result.Should().Be(NodeWatcherResult.NoDisconnectSeen); } [Test] From 2f902451e78c7da7d08c344949fd1c264a2deee3 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Wed, 20 Aug 2025 07:39:47 +1000 Subject: [PATCH 099/137] CancelOnDispose doco --- .../Util/CancelOnDisposeCancellationToken.cs | 26 +++++++++++++++++-- 1 file changed, 24 insertions(+), 2 deletions(-) diff --git a/source/Halibut/Util/CancelOnDisposeCancellationToken.cs b/source/Halibut/Util/CancelOnDisposeCancellationToken.cs index 0681cc0dc..3bcd04f22 100644 --- a/source/Halibut/Util/CancelOnDisposeCancellationToken.cs +++ b/source/Halibut/Util/CancelOnDisposeCancellationToken.cs @@ -7,7 +7,24 @@ namespace Halibut.Util { /// - /// An async disposable wrapper for CancellationTokenSource that safely cancels and DOES NOT dispose it. + /// Helps with safely working with CancellationTokenSources. + /// + /// CancellationTokens and CancellationTokenSources can be tricky to work with since: + /// - Asking for a token from a disposed CTS throws which is often surprising. + /// - Disposal of a CTS does not cancel the token. + /// - Even if the CTS is cancelled then dispose, race conditions exists where some + /// tasks using the cancelled token DO NOT GET CANCELLED e.g. Task.Delay(); + /// + /// To help with some of those this class: + /// - Gets a copy of the Token from the CTS, before it is disposed. So asking + /// for the token never throws. + /// - Always cancels the CTS before disposing of it, so anything with the token + /// general (except in dotnet race condition cases) gets cancelled. + /// - Supports awaiting tasks that are using the CTS's Token in dispose. Specifically + /// when disposed this class will cancel the CTS, then await those tasks given to it + /// (ignoring errors) and only then disposing the CTS. This avoids the bugs/race + /// conditions in Dotnet. + /// /// public sealed class CancelOnDisposeCancellationToken : IAsyncDisposable { @@ -24,7 +41,7 @@ public CancelOnDisposeCancellationToken() : this(new CancellationTokenSource()) { } - private CancelOnDisposeCancellationToken(CancellationTokenSource cancellationTokenSource) + CancelOnDisposeCancellationToken(CancellationTokenSource cancellationTokenSource) { this.cancellationTokenSource = cancellationTokenSource; Token = cancellationTokenSource.Token; @@ -64,6 +81,11 @@ public void CancelAfter(TimeSpan timeSpan) cancellationTokenSource.CancelAfter(timeSpan); } + /// + /// Tasks supplied here will be awaited on in the dispose method after + /// the Token is cancelled and before the token is disposed. + /// + /// public void AwaitTasksBeforeCTSDispose(params Task[] tasksUsingToken) { awaitAllAndIgnoreException.AddTasks(tasksUsingToken); From 5a2715b4aecce2e0693e682d908aeb6f9b78160c Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Wed, 20 Aug 2025 07:40:36 +1000 Subject: [PATCH 100/137] . --- .../Redis/{ => Utils}/TestRedisPendingRequestQueueFactory.cs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) rename source/Halibut.Tests/Queue/Redis/{ => Utils}/TestRedisPendingRequestQueueFactory.cs (96%) diff --git a/source/Halibut.Tests/Queue/Redis/TestRedisPendingRequestQueueFactory.cs b/source/Halibut.Tests/Queue/Redis/Utils/TestRedisPendingRequestQueueFactory.cs similarity index 96% rename from source/Halibut.Tests/Queue/Redis/TestRedisPendingRequestQueueFactory.cs rename to source/Halibut.Tests/Queue/Redis/Utils/TestRedisPendingRequestQueueFactory.cs index d1d040e2d..21ffce8e2 100644 --- a/source/Halibut.Tests/Queue/Redis/TestRedisPendingRequestQueueFactory.cs +++ b/source/Halibut.Tests/Queue/Redis/Utils/TestRedisPendingRequestQueueFactory.cs @@ -4,7 +4,7 @@ using Halibut.Queue.Redis; using Halibut.ServiceModel; -namespace Halibut.Tests.Queue.Redis +namespace Halibut.Tests.Queue.Redis.Utils { public class TestRedisPendingRequestQueueFactory : IPendingRequestQueueFactory { From 6e88938416b5dda150f90ea48c8bf21604845f5c Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Wed, 20 Aug 2025 08:02:40 +1000 Subject: [PATCH 101/137] cleanup --- .../Redis/RedisPendingRequestQueueFixture.cs | 78 ++++++++----------- 1 file changed, 33 insertions(+), 45 deletions(-) diff --git a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs index 10c537da8..f200f1f14 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs @@ -7,7 +7,6 @@ using Halibut.Exceptions; using Halibut.Logging; using Halibut.Queue; -using Halibut.Queue.QueuedDataStreams; using Halibut.Queue.Redis; using Halibut.Queue.Redis.Exceptions; using Halibut.Queue.Redis.MessageStorage; @@ -21,7 +20,6 @@ using Halibut.Tests.Support.TestAttributes; using Halibut.Tests.Support.TestCases; using Halibut.Tests.TestServices.Async; -using Halibut.Tests.TestSetup.Redis; using Halibut.Tests.Util; using Halibut.TestUtils.Contracts; using Halibut.Transport.Protocol; @@ -29,7 +27,6 @@ using NSubstitute; using NSubstitute.Extensions; using NUnit.Framework; -using Octopus.TestPortForwarder; using ILog = Halibut.Diagnostics.ILog; namespace Halibut.Tests.Queue.Redis @@ -42,16 +39,11 @@ public async Task DequeueAsync_ShouldReturnRequestFromRedis() { // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid()); - var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); var redisTransport = new HalibutRedisTransport(redisFacade); - var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); - var messageSerializer = new QueueMessageSerializerBuilder().Build(); - var messageReaderWriter = new MessageSerialiserAndDataStreamStorage(messageSerializer, dataStreamStore); - var request = new RequestMessageBuilder("poll://test-endpoint").Build(); - var sut = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + var sut = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), HalibutLog, redisTransport, CreateMessageSerialiserAndDataStreamStorage(), new HalibutTimeoutsAndLimits()); await sut.WaitUntilQueueIsSubscribedToReceiveMessages(); var task = sut.QueueAndWaitAsync(request, CancellationToken.None); @@ -67,31 +59,26 @@ public async Task DequeueAsync_ShouldReturnRequestFromRedis() } [Test] - public async Task WhenThePickupTimeoutExpires_AnErrorsReturned_AndTheRequestCanNotBeCollected() + public async Task WhenThePickupTimeoutExpires_AnErrorsIsReturnedAndTheRequestCanNotBeCollected() { // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid()); - var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); var redisTransport = new HalibutRedisTransport(redisFacade); - var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); - var messageSerializer = new QueueMessageSerializerBuilder().Build(); - var messageReaderWriter = new MessageSerialiserAndDataStreamStorage(messageSerializer, dataStreamStore); var request = new RequestMessageBuilder("poll://test-endpoint") .WithServiceEndpoint(b => b.WithPollingRequestQueueTimeout(TimeSpan.FromMilliseconds(100))) .Build(); - var sut = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + var sut = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), HalibutLog, redisTransport, CreateMessageSerialiserAndDataStreamStorage(), new HalibutTimeoutsAndLimits()); await sut.WaitUntilQueueIsSubscribedToReceiveMessages(); - var response = await sut.QueueAndWaitAsync(request, CancellationToken.None); - - response.Error!.Message.Should().Contain("A request was sent to a polling endpoint, but the polling endpoint did not collect the request within the allowed time"); - // Act + var response = await sut.QueueAndWaitAsync(request, CancellationToken.None); var result = await sut.DequeueAsync(CancellationToken); + // Assert + response.Error!.Message.Should().Contain("A request was sent to a polling endpoint, but the polling endpoint did not collect the request within the allowed time"); result.Should().BeNull(); } @@ -100,19 +87,16 @@ public async Task FullSendAndReceiveShouldWork() { // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid()); - var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); var redisTransport = new HalibutRedisTransport(redisFacade); - var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); - var messageSerializer = new QueueMessageSerializerBuilder().Build(); - var messageReaderWriter = new MessageSerialiserAndDataStreamStorage(messageSerializer, dataStreamStore); var request = new RequestMessageBuilder("poll://test-endpoint").Build(); - var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); - var node2Receiver = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), HalibutLog, redisTransport, CreateMessageSerialiserAndDataStreamStorage(), new HalibutTimeoutsAndLimits()); + var node2Receiver = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), HalibutLog, redisTransport, CreateMessageSerialiserAndDataStreamStorage(), new HalibutTimeoutsAndLimits()); await node2Receiver.WaitUntilQueueIsSubscribedToReceiveMessages(); + // Act var queueAndWaitAsync = node1Sender.QueueAndWaitAsync(request, CancellationToken.None); var requestMessageWithCancellationToken = await node2Receiver.DequeueAsync(CancellationToken); @@ -127,6 +111,7 @@ public async Task FullSendAndReceiveShouldWork() var responseMessage = await queueAndWaitAsync; + // Assert responseMessage.Result.Should().Be("Yay"); } @@ -135,19 +120,17 @@ public async Task WhenReadingTheResponseFromTheQueueFails_TheQueueAndWaitTaskRet { // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid()); - var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); var redisTransport = new HalibutRedisTransport(redisFacade); - var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); - var messageSerializer = new QueueMessageSerializerBuilder().Build(); - var messageReaderWriter = new MessageSerialiserAndDataStreamStorage(messageSerializer, dataStreamStore) + var messageReaderWriter = CreateMessageSerialiserAndDataStreamStorage() .ThrowsOnReadResponse(() => new OperationCanceledException()); var request = new RequestMessageBuilder("poll://test-endpoint").Build(); - var queue = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + var queue = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), HalibutLog, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); await queue.WaitUntilQueueIsSubscribedToReceiveMessages(); + // Act var queueAndWaitAsync = queue.QueueAndWaitAsync(request, CancellationToken.None); var requestMessageWithCancellationToken = await queue.DequeueAsync(CancellationToken); @@ -155,9 +138,10 @@ await queue.ApplyResponse(ResponseMessage.FromResult(requestMessageWithCancellat requestMessageWithCancellationToken.RequestMessage.ActivityId); var responseMessage = await queueAndWaitAsync; + + // Assert responseMessage.Error.Should().NotBeNull(); - - CreateExceptionFromResponse(responseMessage, log).IsRetryableError().Should().Be(HalibutRetryableErrorType.IsRetryable); + CreateExceptionFromResponse(responseMessage, HalibutLog).IsRetryableError().Should().Be(HalibutRetryableErrorType.IsRetryable); } [Test] @@ -165,23 +149,21 @@ public async Task WhenEnteringTheQueue_AndRedisIsUnavailable_ARetryableException { // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid()); - var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); - await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder, null); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder); redisFacade.MaxDurationToRetryFor = TimeSpan.FromSeconds(1); var redisTransport = new HalibutRedisTransport(redisFacade); - var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); - var messageSerializer = new QueueMessageSerializerBuilder().Build(); - var messageReaderWriter = new MessageSerialiserAndDataStreamStorage(messageSerializer, dataStreamStore); var request = new RequestMessageBuilder("poll://test-endpoint").Build(); - var queue = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + var queue = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), HalibutLog, redisTransport, CreateMessageSerialiserAndDataStreamStorage(), new HalibutTimeoutsAndLimits()); portForwarder.EnterKillNewAndExistingConnectionsMode(); - // Act Assert + // Act var exception = await AssertThrowsAny.Exception(async () => await queue.QueueAndWaitAsync(request, CancellationToken.None)); + + // Assert exception.IsRetryableError().Should().Be(HalibutRetryableErrorType.IsRetryable); exception.Message.Should().Contain("ailed since an error occured inserting the data into the queue"); } @@ -207,16 +189,14 @@ public async Task WhenEnteringTheQueue_AndRedisIsUnavailableAndDataLoseOccurs_AR throw new OperationCanceledException(); }); - var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); - var messageSerializer = new QueueMessageSerializerBuilder().Build(); - var messageReaderWriter = new MessageSerialiserAndDataStreamStorage(messageSerializer, dataStreamStore); - var request = new RequestMessageBuilder("poll://test-endpoint").Build(); - var queue = new RedisPendingRequestQueue(endpoint, redisDataLoseDetector, log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + var queue = new RedisPendingRequestQueue(endpoint, redisDataLoseDetector, log, redisTransport, CreateMessageSerialiserAndDataStreamStorage(), new HalibutTimeoutsAndLimits()); - // Act Assert + // Act var exception = await AssertThrowsAny.Exception(async () => await queue.QueueAndWaitAsync(request, CancellationToken.None)); + + // Assert exception.IsRetryableError().Should().Be(HalibutRetryableErrorType.IsRetryable); exception.Message.Should().Contain("was cancelled because we detected that redis lost all of its data."); } @@ -854,6 +834,14 @@ StreamCapturingJsonSerializer StreamCapturingSerializer() return new QueueMessageSerializer(StreamCapturingSerializer); } } + + static MessageSerialiserAndDataStreamStorage CreateMessageSerialiserAndDataStreamStorage() + { + var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); + var messageSerializer = new QueueMessageSerializerBuilder().Build(); + var messageReaderWriter = new MessageSerialiserAndDataStreamStorage(messageSerializer, dataStreamStore); + return messageReaderWriter; + } } } #endif \ No newline at end of file From 039ec904836872040763dc44357cbb775009d201 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Wed, 20 Aug 2025 08:04:11 +1000 Subject: [PATCH 102/137] cleanup --- .../Queue/Redis/RedisPendingRequestQueueFixture.cs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs index f200f1f14..74b5acee6 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs @@ -173,7 +173,6 @@ public async Task WhenEnteringTheQueue_AndRedisIsUnavailableAndDataLoseOccurs_AR { // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid()); - var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder, null); @@ -191,7 +190,7 @@ public async Task WhenEnteringTheQueue_AndRedisIsUnavailableAndDataLoseOccurs_AR var request = new RequestMessageBuilder("poll://test-endpoint").Build(); - var queue = new RedisPendingRequestQueue(endpoint, redisDataLoseDetector, log, redisTransport, CreateMessageSerialiserAndDataStreamStorage(), new HalibutTimeoutsAndLimits()); + var queue = new RedisPendingRequestQueue(endpoint, redisDataLoseDetector, HalibutLog, redisTransport, CreateMessageSerialiserAndDataStreamStorage(), new HalibutTimeoutsAndLimits()); // Act var exception = await AssertThrowsAny.Exception(async () => await queue.QueueAndWaitAsync(request, CancellationToken.None)); From 4e0db6c0efa078352c0f0ab3c1cee7dfabc55179 Mon Sep 17 00:00:00 2001 From: Rhys Parry Date: Wed, 20 Aug 2025 10:21:45 +1000 Subject: [PATCH 103/137] Rename request to requestId --- .../Redis/Utils/HalibutRedisTransportWithVirtuals.cs | 8 ++++---- .../Queue/Redis/RedisHelpers/HalibutRedisTransport.cs | 10 +++++----- .../Queue/Redis/RedisHelpers/IHalibutRedisTransport.cs | 10 +++++++--- 3 files changed, 16 insertions(+), 12 deletions(-) diff --git a/source/Halibut.Tests/Queue/Redis/Utils/HalibutRedisTransportWithVirtuals.cs b/source/Halibut.Tests/Queue/Redis/Utils/HalibutRedisTransportWithVirtuals.cs index 3830b970f..2cd5b1f6d 100644 --- a/source/Halibut.Tests/Queue/Redis/Utils/HalibutRedisTransportWithVirtuals.cs +++ b/source/Halibut.Tests/Queue/Redis/Utils/HalibutRedisTransportWithVirtuals.cs @@ -53,9 +53,9 @@ public Task IsRequestStillOnQueue(Uri endpoint, Guid requestId, Cancellati return halibutRedisTransport.IsRequestStillOnQueue(endpoint, requestId, cancellationToken); } - public Task SubscribeToRequestCancellation(Uri endpoint, Guid request, Func onRpcCancellation, CancellationToken cancellationToken) + public Task SubscribeToRequestCancellation(Uri endpoint, Guid requestId, Func onRpcCancellation, CancellationToken cancellationToken) { - return halibutRedisTransport.SubscribeToRequestCancellation(endpoint, request, onRpcCancellation, cancellationToken); + return halibutRedisTransport.SubscribeToRequestCancellation(endpoint, requestId, onRpcCancellation, cancellationToken); } public Task PublishCancellation(Uri endpoint, Guid requestId, CancellationToken cancellationToken) @@ -73,9 +73,9 @@ public Task IsRequestMarkedAsCancelled(Uri endpoint, Guid requestId, Cance return halibutRedisTransport.IsRequestMarkedAsCancelled(endpoint, requestId, cancellationToken); } - public Task SubscribeToNodeHeartBeatChannel(Uri endpoint, Guid request, HalibutQueueNodeSendingPulses nodeSendingPulsesType, Func onHeartBeat, CancellationToken cancellationToken) + public Task SubscribeToNodeHeartBeatChannel(Uri endpoint, Guid requestId, HalibutQueueNodeSendingPulses nodeSendingPulsesType, Func onHeartBeat, CancellationToken cancellationToken) { - return halibutRedisTransport.SubscribeToNodeHeartBeatChannel(endpoint, request, nodeSendingPulsesType, onHeartBeat, cancellationToken); + return halibutRedisTransport.SubscribeToNodeHeartBeatChannel(endpoint, requestId, nodeSendingPulsesType, onHeartBeat, cancellationToken); } public Task SendNodeHeartBeat(Uri endpoint, Guid requestId, HalibutQueueNodeSendingPulses nodeSendingPulsesType, CancellationToken cancellationToken) diff --git a/source/Halibut/Queue/Redis/RedisHelpers/HalibutRedisTransport.cs b/source/Halibut/Queue/Redis/RedisHelpers/HalibutRedisTransport.cs index c5327d736..e53083639 100644 --- a/source/Halibut/Queue/Redis/RedisHelpers/HalibutRedisTransport.cs +++ b/source/Halibut/Queue/Redis/RedisHelpers/HalibutRedisTransport.cs @@ -137,13 +137,13 @@ static string RequestCancelledChannelName(Uri endpoint, Guid requestId) /// /// /// - /// + /// /// Called when the RPC has been cancelled. /// /// - public async Task SubscribeToRequestCancellation(Uri endpoint, Guid request, Func onRpcCancellation, CancellationToken cancellationToken) + public async Task SubscribeToRequestCancellation(Uri endpoint, Guid requestId, Func onRpcCancellation, CancellationToken cancellationToken) { - var channelName = RequestCancelledChannelName(endpoint, request); + var channelName = RequestCancelledChannelName(endpoint, requestId); return await facade.SubscribeToChannel(channelName, async foo => { string? response = foo.Message; @@ -197,12 +197,12 @@ static string NodeHeartBeatChannel(Uri endpoint, Guid requestId, HalibutQueueNod public async Task SubscribeToNodeHeartBeatChannel( Uri endpoint, - Guid request, + Guid requestId, HalibutQueueNodeSendingPulses nodeSendingPulsesType, Func onHeartBeat, CancellationToken cancellationToken) { - var channelName = NodeHeartBeatChannel(endpoint, request, nodeSendingPulsesType); + var channelName = NodeHeartBeatChannel(endpoint, requestId, nodeSendingPulsesType); return await facade.SubscribeToChannel(channelName, async foo => { string? response = foo.Message; diff --git a/source/Halibut/Queue/Redis/RedisHelpers/IHalibutRedisTransport.cs b/source/Halibut/Queue/Redis/RedisHelpers/IHalibutRedisTransport.cs index 80f0642aa..24f14f263 100644 --- a/source/Halibut/Queue/Redis/RedisHelpers/IHalibutRedisTransport.cs +++ b/source/Halibut/Queue/Redis/RedisHelpers/IHalibutRedisTransport.cs @@ -22,7 +22,9 @@ public interface IHalibutRedisTransport Task TryGetAndRemoveRequest(Uri endpoint, Guid requestId, CancellationToken cancellationToken); Task IsRequestStillOnQueue(Uri endpoint, Guid requestId, CancellationToken cancellationToken); - Task SubscribeToRequestCancellation(Uri endpoint, Guid request, + Task SubscribeToRequestCancellation( + Uri endpoint, + Guid requestId, Func onRpcCancellation, CancellationToken cancellationToken); Task PublishCancellation(Uri endpoint, Guid requestId, CancellationToken cancellationToken); @@ -33,7 +35,7 @@ Task SubscribeToRequestCancellation(Uri endpoint, Guid request Task SubscribeToNodeHeartBeatChannel( Uri endpoint, - Guid request, + Guid requestId, HalibutQueueNodeSendingPulses nodeSendingPulsesType, Func onHeartBeat, CancellationToken cancellationToken); @@ -41,7 +43,9 @@ Task SubscribeToNodeHeartBeatChannel( Task SendNodeHeartBeat(Uri endpoint, Guid requestId, HalibutQueueNodeSendingPulses nodeSendingPulsesType, CancellationToken cancellationToken); - Task SubscribeToResponseChannel(Uri endpoint, Guid identifier, + Task SubscribeToResponseChannel( + Uri endpoint, + Guid identifier, Func onValueReceived, CancellationToken cancellationToken); Task PublishThatResponseIsAvailable(Uri endpoint, Guid identifier, CancellationToken cancellationToken); From 0c7aa1732183e34a0e7033a10aab540bdedd68ac Mon Sep 17 00:00:00 2001 From: Rhys Parry Date: Wed, 20 Aug 2025 10:45:20 +1000 Subject: [PATCH 104/137] Consolidate key generation in RedisFacade --- .../Queue/Redis/RedisHelpers/RedisFacade.cs | 61 ++++++++++--------- 1 file changed, 33 insertions(+), 28 deletions(-) diff --git a/source/Halibut/Queue/Redis/RedisHelpers/RedisFacade.cs b/source/Halibut/Queue/Redis/RedisHelpers/RedisFacade.cs index 0adfbf331..9279f83bd 100644 --- a/source/Halibut/Queue/Redis/RedisHelpers/RedisFacade.cs +++ b/source/Halibut/Queue/Redis/RedisHelpers/RedisFacade.cs @@ -145,7 +145,7 @@ public async ValueTask DisposeAsync() public async Task SubscribeToChannel(string channelName, Func onMessage, CancellationToken cancellationToken) { - channelName = "channel:" + keyPrefix + ":" + channelName; + channelName = ToPrefixedChannelName(channelName); while (true) { cancellationToken.ThrowIfCancellationRequested(); @@ -183,10 +183,15 @@ public async Task SubscribeToChannel(string channelName, Func< } } } + + string ToPrefixedChannelName(string channelName) + { + return "channel:" + keyPrefix + ":" + channelName; + } public async Task PublishToChannel(string channelName, string payload, CancellationToken cancellationToken) { - channelName = "channel:" + keyPrefix + ":" + channelName; + channelName = ToPrefixedChannelName(channelName); await ExecuteWithRetry(async () => { var subscriber = Connection.GetSubscriber(); @@ -196,41 +201,41 @@ await ExecuteWithRetry(async () => public async Task SetInHash(string key, string field, string payload, TimeSpan ttl, CancellationToken cancellationToken) { - key = "hash:" + keyPrefix + ":" + key; + var hashKey = ToHashKey(key); await ExecuteWithRetry(async () => { var database = Connection.GetDatabase(); - await database.HashSetAsync(key, new RedisValue(field), new RedisValue(payload)); + await database.HashSetAsync(hashKey, new RedisValue(field), new RedisValue(payload)); }, cancellationToken); - await SetTtlForKeyRaw(key, ttl, cancellationToken); + await SetTtlForKeyRaw(hashKey, ttl, cancellationToken); } - string ToHashKey(string key) + RedisKey ToHashKey(string key) { return "hash:" + keyPrefix + ":" + key; } public async Task HashContainsKey(string key, string field, CancellationToken cancellationToken) { - key = ToHashKey(key); + var hashKey = ToHashKey(key); return await ExecuteWithRetry(async () => { var database = Connection.GetDatabase(); - return await database.HashExistsAsync(key, new RedisValue(field)); + return await database.HashExistsAsync(hashKey, new RedisValue(field)); }, cancellationToken); } public async Task TryGetAndDeleteFromHash(string key, string field, CancellationToken cancellationToken) { - key = ToHashKey(key); - + var hashKey = ToHashKey(key); + // Retry each operation independently var value = await ExecuteWithRetry(async () => { var database = Connection.GetDatabase(); - return await database.HashGetAsync(key, new RedisValue(field)); + return await database.HashGetAsync(hashKey, new RedisValue(field)); }, cancellationToken); // Retry does make this non-idempotent, what can happen is the key is deleted on redis. @@ -241,7 +246,7 @@ public async Task HashContainsKey(string key, string field, CancellationTo var res = await ExecuteWithRetry(async () => { var database = Connection.GetDatabase(); - return await database.KeyDeleteAsync(key); + return await database.KeyDeleteAsync(hashKey); }, cancellationToken); if (!res) @@ -252,29 +257,30 @@ public async Task HashContainsKey(string key, string field, CancellationTo return value; } - string ToListKey(string key) + RedisKey ToListKey(string key) { return "list:" + keyPrefix + ":" + key; } public async Task ListRightPushAsync(string key, string payload, TimeSpan ttlForAllInList, CancellationToken cancellationToken) { - key = ToListKey(key); + var listKey = ToListKey(key); await ExecuteWithRetry(async () => { var database = Connection.GetDatabase(); - await database.ListRightPushAsync(key, payload); + await database.ListRightPushAsync(listKey, payload); }, cancellationToken); - await SetTtlForKeyRaw(key, ttlForAllInList, cancellationToken); } + await SetTtlForKeyRaw(listKey, ttlForAllInList, cancellationToken); + } public async Task ListLeftPopAsync(string key, CancellationToken cancellationToken) { - key = ToListKey(key); + var listKey = ToListKey(key); return await ExecuteWithRetry(async () => { var database = Connection.GetDatabase(); - var value = await database.ListLeftPopAsync(key); + var value = await database.ListLeftPopAsync(listKey); if (value.IsNull) { return null; @@ -284,50 +290,49 @@ await ExecuteWithRetry(async () => }, cancellationToken); } - string ToStringKey(string key) + RedisKey ToStringKey(string key) { return "string:" + keyPrefix + ":" + key; } public async Task SetString(string key, string value, TimeSpan ttl, CancellationToken cancellationToken) { - key = ToStringKey(key); + var stringKey = ToStringKey(key); await ExecuteWithRetry(async () => { var database = Connection.GetDatabase(); - await database.StringSetAsync(key, value); + await database.StringSetAsync(stringKey, value); }, cancellationToken); - await SetTtlForKeyRaw(key, ttl, cancellationToken); + await SetTtlForKeyRaw(stringKey, ttl, cancellationToken); } public async Task SetTtlForString(string key, TimeSpan ttl, CancellationToken cancellationToken) { await SetTtlForKeyRaw(ToStringKey(key), ttl, cancellationToken); - } public async Task GetString(string key, CancellationToken cancellationToken) { - key = ToStringKey(key); + var stringKey = ToStringKey(key); return await ExecuteWithRetry(async () => { var database = Connection.GetDatabase(); - return await database.StringGetAsync(key); + return await database.StringGetAsync(stringKey); }, cancellationToken); } public async Task DeleteString(string key, CancellationToken cancellationToken) { - key = ToStringKey(key); + var stringKey = ToStringKey(key); return await ExecuteWithRetry(async () => { var database = Connection.GetDatabase(); - return await database.KeyDeleteAsync(key); + return await database.KeyDeleteAsync(stringKey); }, cancellationToken); } - async Task SetTtlForKeyRaw(string key, TimeSpan ttl, CancellationToken cancellationToken) + async Task SetTtlForKeyRaw(RedisKey key, TimeSpan ttl, CancellationToken cancellationToken) { await ExecuteWithRetry(async () => { From 2f1443331ad79d251a80fbf77c9e8ac6b532a3ee Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Wed, 20 Aug 2025 10:49:56 +1000 Subject: [PATCH 105/137] test cleanup --- .../RedisPendingRequestQueueBuilder.cs | 2 +- .../Redis/RedisPendingRequestQueueFixture.cs | 334 ++++++++---------- ...ngAllItsData.cs => RedisNeverLosesData.cs} | 2 +- 3 files changed, 154 insertions(+), 184 deletions(-) rename source/Halibut.Tests/Queue/Redis/Utils/{NeverLosingDataWatchForRedisLosingAllItsData.cs => RedisNeverLosesData.cs} (88%) diff --git a/source/Halibut.Tests/Builders/RedisPendingRequestQueueBuilder.cs b/source/Halibut.Tests/Builders/RedisPendingRequestQueueBuilder.cs index e91609634..2a04b86b8 100644 --- a/source/Halibut.Tests/Builders/RedisPendingRequestQueueBuilder.cs +++ b/source/Halibut.Tests/Builders/RedisPendingRequestQueueBuilder.cs @@ -62,7 +62,7 @@ public QueueHolder Build() var request = new RequestMessageBuilder("poll://test-endpoint").Build(); - var queue = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, halibutTimeoutsAndLimits); + var queue = new RedisPendingRequestQueue(endpoint, new RedisNeverLosesData(), log, redisTransport, messageReaderWriter, halibutTimeoutsAndLimits); #pragma warning disable VSTHRD002 queue.WaitUntilQueueIsSubscribedToReceiveMessages().GetAwaiter().GetResult(); diff --git a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs index 74b5acee6..c4a6fec7b 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs @@ -43,7 +43,7 @@ public async Task DequeueAsync_ShouldReturnRequestFromRedis() var redisTransport = new HalibutRedisTransport(redisFacade); var request = new RequestMessageBuilder("poll://test-endpoint").Build(); - var sut = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), HalibutLog, redisTransport, CreateMessageSerialiserAndDataStreamStorage(), new HalibutTimeoutsAndLimits()); + var sut = new RedisPendingRequestQueue(endpoint, new RedisNeverLosesData(), HalibutLog, redisTransport, CreateMessageSerialiserAndDataStreamStorage(), new HalibutTimeoutsAndLimits()); await sut.WaitUntilQueueIsSubscribedToReceiveMessages(); var task = sut.QueueAndWaitAsync(request, CancellationToken.None); @@ -70,7 +70,7 @@ public async Task WhenThePickupTimeoutExpires_AnErrorsIsReturnedAndTheRequestCan .WithServiceEndpoint(b => b.WithPollingRequestQueueTimeout(TimeSpan.FromMilliseconds(100))) .Build(); - var sut = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), HalibutLog, redisTransport, CreateMessageSerialiserAndDataStreamStorage(), new HalibutTimeoutsAndLimits()); + var sut = new RedisPendingRequestQueue(endpoint, new RedisNeverLosesData(), HalibutLog, redisTransport, CreateMessageSerialiserAndDataStreamStorage(), new HalibutTimeoutsAndLimits()); await sut.WaitUntilQueueIsSubscribedToReceiveMessages(); // Act @@ -92,8 +92,8 @@ public async Task FullSendAndReceiveShouldWork() var request = new RequestMessageBuilder("poll://test-endpoint").Build(); - var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), HalibutLog, redisTransport, CreateMessageSerialiserAndDataStreamStorage(), new HalibutTimeoutsAndLimits()); - var node2Receiver = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), HalibutLog, redisTransport, CreateMessageSerialiserAndDataStreamStorage(), new HalibutTimeoutsAndLimits()); + var node1Sender = new RedisPendingRequestQueue(endpoint, new RedisNeverLosesData(), HalibutLog, redisTransport, CreateMessageSerialiserAndDataStreamStorage(), new HalibutTimeoutsAndLimits()); + var node2Receiver = new RedisPendingRequestQueue(endpoint, new RedisNeverLosesData(), HalibutLog, redisTransport, CreateMessageSerialiserAndDataStreamStorage(), new HalibutTimeoutsAndLimits()); await node2Receiver.WaitUntilQueueIsSubscribedToReceiveMessages(); // Act @@ -114,6 +114,42 @@ public async Task FullSendAndReceiveShouldWork() // Assert responseMessage.Result.Should().Be("Yay"); } + + [Test] + public async Task FullSendAndReceiveWithDataStreamShouldWork() + { + // Arrange + var endpoint = new Uri("poll://" + Guid.NewGuid()); + await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); + var redisTransport = new HalibutRedisTransport(redisFacade); + var messageReaderWriter = CreateMessageSerialiserAndDataStreamStorage(); + + var request = new RequestMessageBuilder("poll://test-endpoint").Build(); + request.Params = new[] { new ComplexObjectMultipleDataStreams(DataStream.FromString("hello"), DataStream.FromString("world")) }; + + var node1Sender = new RedisPendingRequestQueue(endpoint, new RedisNeverLosesData(), HalibutLog, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + var node2Receiver = new RedisPendingRequestQueue(endpoint, new RedisNeverLosesData(), HalibutLog, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + await node2Receiver.WaitUntilQueueIsSubscribedToReceiveMessages(); + + var queueAndWaitAsync = node1Sender.QueueAndWaitAsync(request, CancellationToken.None); + + var requestMessageWithCancellationToken = await node2Receiver.DequeueAsync(CancellationToken); + + var objWithDataStreams = (ComplexObjectMultipleDataStreams)requestMessageWithCancellationToken!.RequestMessage.Params[0]; + (await objWithDataStreams.Payload1!.ReadAsString(CancellationToken)).Should().Be("hello"); + (await objWithDataStreams.Payload2!.ReadAsString(CancellationToken)).Should().Be("world"); + + var response = ResponseMessage.FromResult(requestMessageWithCancellationToken.RequestMessage, + new ComplexObjectMultipleDataStreams(DataStream.FromString("good"), DataStream.FromString("bye"))); + + await node2Receiver.ApplyResponse(response, requestMessageWithCancellationToken.RequestMessage.ActivityId); + + var responseMessage = await queueAndWaitAsync; + + var returnObject = (ComplexObjectMultipleDataStreams)responseMessage.Result!; + (await returnObject.Payload1!.ReadAsString(CancellationToken)).Should().Be("good"); + (await returnObject.Payload2!.ReadAsString(CancellationToken)).Should().Be("bye"); + } [Test] public async Task WhenReadingTheResponseFromTheQueueFails_TheQueueAndWaitTaskReturnsAnUnknownError() @@ -127,7 +163,7 @@ public async Task WhenReadingTheResponseFromTheQueueFails_TheQueueAndWaitTaskRet var request = new RequestMessageBuilder("poll://test-endpoint").Build(); - var queue = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), HalibutLog, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + var queue = new RedisPendingRequestQueue(endpoint, new RedisNeverLosesData(), HalibutLog, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); await queue.WaitUntilQueueIsSubscribedToReceiveMessages(); // Act @@ -157,7 +193,7 @@ public async Task WhenEnteringTheQueue_AndRedisIsUnavailable_ARetryableException var redisTransport = new HalibutRedisTransport(redisFacade); var request = new RequestMessageBuilder("poll://test-endpoint").Build(); - var queue = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), HalibutLog, redisTransport, CreateMessageSerialiserAndDataStreamStorage(), new HalibutTimeoutsAndLimits()); + var queue = new RedisPendingRequestQueue(endpoint, new RedisNeverLosesData(), HalibutLog, redisTransport, CreateMessageSerialiserAndDataStreamStorage(), new HalibutTimeoutsAndLimits()); portForwarder.EnterKillNewAndExistingConnectionsMode(); // Act @@ -199,24 +235,66 @@ public async Task WhenEnteringTheQueue_AndRedisIsUnavailableAndDataLoseOccurs_AR exception.IsRetryableError().Should().Be(HalibutRetryableErrorType.IsRetryable); exception.Message.Should().Contain("was cancelled because we detected that redis lost all of its data."); } + + [Test] + public async Task WhenTheRequestReceiverNodeDetectsRedisDataLose_AndTheRequestSenderDoesNotYetDetectDataLose_TheRequestSenderNodeReturnsARetryableResponse() + { + // Arrange + var endpoint = new Uri("poll://" + Guid.NewGuid()); + var guid = Guid.NewGuid(); + + var messageReaderWriter = CreateMessageSerialiserAndDataStreamStorage(); + + await using var stableConnection = RedisFacadeBuilder.CreateRedisFacade(prefix: guid); + + var redisDataLoseDetectorOnReceiver = new CancellableDataLossWatchForRedisLosingAllItsData(); + var node1Sender = new RedisPendingRequestQueue(endpoint, new RedisNeverLosesData(), HalibutLog, new HalibutRedisTransport(stableConnection), messageReaderWriter, new HalibutTimeoutsAndLimits()); + var node2Receiver = new RedisPendingRequestQueue(endpoint, redisDataLoseDetectorOnReceiver, HalibutLog, new HalibutRedisTransport(stableConnection), messageReaderWriter, new HalibutTimeoutsAndLimits()); + await node2Receiver.WaitUntilQueueIsSubscribedToReceiveMessages(); + + var request = new RequestMessageBuilder("poll://test-endpoint").Build(); + var queueAndWaitTask = node1Sender.QueueAndWaitAsync(request, CancellationToken.None); + + var dequeuedRequest = await node2Receiver.DequeueAsync(CancellationToken); + + // Act + await redisDataLoseDetectorOnReceiver.DataLossHasOccured(); + + var responseToSendBack = CreateNonRetryableErrorResponse(dequeuedRequest); + + await node2Receiver.ApplyResponse(responseToSendBack, dequeuedRequest!.RequestMessage.ActivityId); + + var response = await queueAndWaitTask; + response.Error.Should().NotBeNull(); + + // Assert + CreateExceptionFromResponse(response, HalibutLog) + .IsRetryableError().Should().Be(HalibutRetryableErrorType.IsRetryable); + } + + ResponseMessage CreateNonRetryableErrorResponse(RequestMessageWithCancellationToken? dequeuedRequest) + { + var responseThatWouldNotBeRetried = ResponseMessage.FromException(dequeuedRequest!.RequestMessage, new NoMatchingServiceOrMethodHalibutClientException("")); + CreateExceptionFromResponse(responseThatWouldNotBeRetried, HalibutLog) + .IsRetryableError().Should().Be(HalibutRetryableErrorType.NotRetryable); + return responseThatWouldNotBeRetried; + } [Test] public async Task WhenPreparingRequestFails_ARetryableExceptionIsThrown() { // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid()); - var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); var redisTransport = new HalibutRedisTransport(redisFacade); - var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); - var messageSerializer = new QueueMessageSerializerBuilder().Build(); - var messageReaderWriter = new MessageSerialiserAndDataStreamStorage(messageSerializer, dataStreamStore) + + var messageReaderWriter = CreateMessageSerialiserAndDataStreamStorage() .ThrowsOnPrepareRequest(() => new OperationCanceledException()); var request = new RequestMessageBuilder("poll://test-endpoint").Build(); - var queue = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + var queue = new RedisPendingRequestQueue(endpoint, new RedisNeverLosesData(), HalibutLog, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); // Act Assert var exception = await AssertThrowsAny.Exception(async () => await queue.QueueAndWaitAsync(request, CancellationToken.None)); @@ -229,19 +307,16 @@ public async Task WhenDataLostIsDetected_InFlightRequestShouldBeAbandoned_AndARe { // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid()); - var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); var redisTransport = new HalibutRedisTransport(redisFacade); - var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); - var messageSerializer = new QueueMessageSerializerBuilder().Build(); - var messageReaderWriter = new MessageSerialiserAndDataStreamStorage(messageSerializer, dataStreamStore); + var messageReaderWriter = CreateMessageSerialiserAndDataStreamStorage(); var request = new RequestMessageBuilder("poll://test-endpoint").Build(); await using var dataLossWatcher = new CancellableDataLossWatchForRedisLosingAllItsData(); - var node1Sender = new RedisPendingRequestQueue(endpoint, dataLossWatcher, log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); - var node2Receiver = new RedisPendingRequestQueue(endpoint, dataLossWatcher, log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + var node1Sender = new RedisPendingRequestQueue(endpoint, dataLossWatcher, HalibutLog, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + var node2Receiver = new RedisPendingRequestQueue(endpoint, dataLossWatcher, HalibutLog, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); await node2Receiver.WaitUntilQueueIsSubscribedToReceiveMessages(); var queueAndWaitAsync = node1Sender.QueueAndWaitAsync(request, CancellationToken.None); @@ -256,8 +331,7 @@ public async Task WhenDataLostIsDetected_InFlightRequestShouldBeAbandoned_AndARe // Assert requestMessageWithCancellationToken!.CancellationToken.IsCancellationRequested.Should().BeTrue("The receiver of the data should just give up processing"); - // We don't want to just await queueAndWaitAsync, since timeouts will kick in resulting in a response no matter what. - // We cant to see that it quickly returns. + // Verify that queueAndWaitAsync quickly returns with an error when data lose has occured. await Task.WhenAny(Task.Delay(5000), queueAndWaitAsync); queueAndWaitAsync.IsCompleted.Should().BeTrue("As soon as data loss is detected the queueAndWait should return."); @@ -273,16 +347,13 @@ public async Task OnceARequestIsComplete_NoInflightDisposableShouldExist() { // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid()); - var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); var redisTransport = new HalibutRedisTransport(redisFacade); - var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); - var messageSerializer = new QueueMessageSerializerBuilder().Build(); - var messageReaderWriter = new MessageSerialiserAndDataStreamStorage(messageSerializer, dataStreamStore); + var messageReaderWriter = CreateMessageSerialiserAndDataStreamStorage(); var request = new RequestMessageBuilder("poll://test-endpoint").Build(); - var queue = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + var queue = new RedisPendingRequestQueue(endpoint, new RedisNeverLosesData(), HalibutLog, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); await queue.WaitUntilQueueIsSubscribedToReceiveMessages(); // Act @@ -302,20 +373,16 @@ public async Task OnceARequestIsComplete_NoInflightDisposableShouldExist() } [Test] - public async Task OnceARequestIsComplete_NoSenderHeartBeatsShouldBeSent() + public async Task OnceARequestIsComplete_NoRequestSenderNodeHeartBeatsShouldBeSent() { // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid()); - var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); var redisTransport = new HalibutRedisTransport(redisFacade); - var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); - var messageSerializer = new QueueMessageSerializerBuilder().Build(); - var messageReaderWriter = new MessageSerialiserAndDataStreamStorage(messageSerializer, dataStreamStore); var request = new RequestMessageBuilder("poll://test-endpoint").Build(); - var queue = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + var queue = new RedisPendingRequestQueue(endpoint, new RedisNeverLosesData(), HalibutLog, redisTransport, CreateMessageSerialiserAndDataStreamStorage(), new HalibutTimeoutsAndLimits()); await queue.WaitUntilQueueIsSubscribedToReceiveMessages(); queue.DelayBetweenHeartBeatsForRequestSender = TimeSpan.FromSeconds(1); @@ -346,20 +413,16 @@ public async Task OnceARequestIsComplete_NoSenderHeartBeatsShouldBeSent() } [Test] - public async Task OnceARequestIsComplete_NoReceiverHeartBeatsShouldBeSent() + public async Task OnceARequestIsComplete_NoRequestProcessorNodeHeartBeatsShouldBeSent() { // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid()); - var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); var redisTransport = new HalibutRedisTransport(redisFacade); - var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); - var messageSerializer = new QueueMessageSerializerBuilder().Build(); - var messageReaderWriter = new MessageSerialiserAndDataStreamStorage(messageSerializer, dataStreamStore); var request = new RequestMessageBuilder("poll://test-endpoint").Build(); - var queue = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + var queue = new RedisPendingRequestQueue(endpoint, new RedisNeverLosesData(), HalibutLog, redisTransport, CreateMessageSerialiserAndDataStreamStorage(), new HalibutTimeoutsAndLimits()); await queue.WaitUntilQueueIsSubscribedToReceiveMessages(); queue.DelayBetweenHeartBeatsForRequestProcessor = TimeSpan.FromSeconds(1); @@ -390,70 +453,27 @@ public async Task OnceARequestIsComplete_NoReceiverHeartBeatsShouldBeSent() } [Test] - public async Task FullSendAndReceiveWithDataStreamShouldWork() + public async Task WhenTheRequestProcessorNodeConnectionToRedisIsInterrupted_AndRestoredBeforeWorkIsPublished_TheReceiverShouldBeAbleToCollectThatWorkQuickly() { // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid()); - var senderLog = new TestContextLogCreator("QueueSender", LogLevel.Trace).CreateNewForPrefix(""); - var receiverLog = new TestContextLogCreator("ReceiverLog", LogLevel.Trace).CreateNewForPrefix(""); - await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); - var redisTransport = new HalibutRedisTransport(redisFacade); - var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); - var messageSerializer = new QueueMessageSerializerBuilder().Build(); - var messageReaderWriter = new MessageSerialiserAndDataStreamStorage(messageSerializer, dataStreamStore); - - var request = new RequestMessageBuilder("poll://test-endpoint").Build(); - request.Params = new[] { new ComplexObjectMultipleDataStreams(DataStream.FromString("hello"), DataStream.FromString("world")) }; - - var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), senderLog, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); - var node2Receiver = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), receiverLog, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); - await node2Receiver.WaitUntilQueueIsSubscribedToReceiveMessages(); - - var queueAndWaitAsync = node1Sender.QueueAndWaitAsync(request, CancellationToken.None); - - var requestMessageWithCancellationToken = await node2Receiver.DequeueAsync(CancellationToken); - - var objWithDataStreams = (ComplexObjectMultipleDataStreams)requestMessageWithCancellationToken!.RequestMessage.Params[0]; - (await objWithDataStreams.Payload1!.ReadAsString(CancellationToken)).Should().Be("hello"); - (await objWithDataStreams.Payload2!.ReadAsString(CancellationToken)).Should().Be("world"); - - var response = ResponseMessage.FromResult(requestMessageWithCancellationToken.RequestMessage, - new ComplexObjectMultipleDataStreams(DataStream.FromString("good"), DataStream.FromString("bye"))); - - await node2Receiver.ApplyResponse(response, requestMessageWithCancellationToken.RequestMessage.ActivityId); - - var responseMessage = await queueAndWaitAsync; - - var returnObject = (ComplexObjectMultipleDataStreams)responseMessage.Result!; - (await returnObject.Payload1!.ReadAsString(CancellationToken)).Should().Be("good"); - (await returnObject.Payload2!.ReadAsString(CancellationToken)).Should().Be("bye"); - } - - [Test] - public async Task WhenTheReceiversConnectionToRedisIsInterruptedAndRestoredBeforeWorkIsPublished_TheReceiverShouldBeAbleToCollectThatWorkQuickly() - { - // Arrange - var endpoint = new Uri("poll://" + Guid.NewGuid()); - var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); var guid = Guid.NewGuid(); await using var redisFacadeSender = RedisFacadeBuilder.CreateRedisFacade(prefix: guid); using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); - await using var redisFacadeReceiver = RedisFacadeBuilder.CreateRedisFacade(portForwarder, guid); - - var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); - var messageSerializer = new QueueMessageSerializerBuilder().Build(); - var messageReaderWriter = new MessageSerialiserAndDataStreamStorage(messageSerializer, dataStreamStore); + await using var unstableRedisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder, guid); + + var messageReaderWriter = CreateMessageSerialiserAndDataStreamStorage(); var request = new RequestMessageBuilder("poll://test-endpoint").Build(); - var halibutTimeoutAndLimits = new HalibutTimeoutsAndLimits(); - halibutTimeoutAndLimits.PollingQueueWaitTimeout = TimeSpan.FromDays(1); // We should not need to rely on the timeout working for very short disconnects. + var highDequeueTimoueHalibutLimits = new HalibutTimeoutsAndLimits(); + highDequeueTimoueHalibutLimits.PollingQueueWaitTimeout = TimeSpan.FromDays(1); // We should not need to rely on the timeout working for very short disconnects. - var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(redisFacadeSender), messageReaderWriter, halibutTimeoutAndLimits); - var node2Receiver = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(redisFacadeReceiver), messageReaderWriter, halibutTimeoutAndLimits); - await node2Receiver.WaitUntilQueueIsSubscribedToReceiveMessages(); - var dequeueTask = node2Receiver.DequeueAsync(CancellationToken); + var requestSenderQueue = new RedisPendingRequestQueue(endpoint, new RedisNeverLosesData(), HalibutLog, new HalibutRedisTransport(redisFacadeSender), messageReaderWriter, highDequeueTimoueHalibutLimits); + var requestProcessQueueWithUnstableConnection = new RedisPendingRequestQueue(endpoint, new RedisNeverLosesData(), HalibutLog, new HalibutRedisTransport(unstableRedisFacade), messageReaderWriter, highDequeueTimoueHalibutLimits); + await requestProcessQueueWithUnstableConnection.WaitUntilQueueIsSubscribedToReceiveMessages(); + var dequeueTask = requestProcessQueueWithUnstableConnection.DequeueAsync(CancellationToken); await Task.Delay(5000, CancellationToken); // Allow some time for the receiver to subscribe to work. dequeueTask.IsCompleted.Should().BeFalse("Dequeue should not have "); @@ -464,10 +484,13 @@ public async Task WhenTheReceiversConnectionToRedisIsInterruptedAndRestoredBefor portForwarder.ReturnToNormalMode(); // The network outage gets all fixed up :D Logger.Information("Network restored!"); - // The receiver should be able to get itself back into a state where it can collect messages quickly, within this time. + // The receiver should be able to get itself back into a state where, + // new RequestMessages that are published are quickly collected. + // However first we allow some time for the subscriptions to re-connect to redis, + // we don't know how long that will take so give it what feels like too much time. await Task.Delay(TimeSpan.FromSeconds(30), CancellationToken); - var queueAndWaitAsync = node1Sender.QueueAndWaitAsync(request, CancellationToken.None); + var queueAndWaitAsync = requestSenderQueue.QueueAndWaitAsync(request, CancellationToken.None); // Surely it will be done in 25s, it should take less than 1s. await Task.WhenAny(Task.Delay(TimeSpan.FromSeconds(20), CancellationToken), dequeueTask); @@ -479,19 +502,22 @@ public async Task WhenTheReceiversConnectionToRedisIsInterruptedAndRestoredBefor requestReceived!.RequestMessage.ActivityId.Should().Be(request.ActivityId); } + /// + /// We want to check that the queue doesn't do something like: + /// - place work on the queue + /// - not receive a heart beat from the RequestProcessorNode, because the request is not yet collected. + /// - timeout because we did not receive that heart beat. + /// [Test] public async Task WhenTheReceiverDoesntCollectWorkImmediately_TheRequestCanSitOnTheQueueForSometime_AndBeOnTheQueueLongerThanTheHeartBeatTimeout() { // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid()); - var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); - var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); - var messageSerializer = new QueueMessageSerializerBuilder().Build(); - var messageReaderWriter = new MessageSerialiserAndDataStreamStorage(messageSerializer, dataStreamStore); + var messageReaderWriter = CreateMessageSerialiserAndDataStreamStorage(); - var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(redisFacade), messageReaderWriter, new HalibutTimeoutsAndLimits()); + var node1Sender = new RedisPendingRequestQueue(endpoint, new RedisNeverLosesData(), HalibutLog, new HalibutRedisTransport(redisFacade), messageReaderWriter, new HalibutTimeoutsAndLimits()); // We are testing that we don't expect heart beats before the request is collected. node1Sender.RequestReceivingNodeIsOfflineHeartBeatTimeout = TimeSpan.FromSeconds(1); await node1Sender.WaitUntilQueueIsSubscribedToReceiveMessages(); @@ -508,28 +534,22 @@ public async Task WhenTheReceiverDoesntCollectWorkImmediately_TheRequestCanSitOn } [Test] - public async Task WhenTheSendersConnectionToRedisIsBrieflyInterruptedWhileSendingWork_TheWorkIsStillSent() + public async Task WhenTheSendersConnectionToRedisIsBrieflyInterruptedWhileSendingTheRequestMessageToRedis_TheWorkIsStillSent() { // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid()); - var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); var guid = Guid.NewGuid(); await using var redisFacadeReceiver = RedisFacadeBuilder.CreateRedisFacade(prefix: guid); using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); await using var redisFacadeSender = RedisFacadeBuilder.CreateRedisFacade(portForwarder, guid); - - var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); - var messageSerializer = new QueueMessageSerializerBuilder().Build(); - var messageReaderWriter = new MessageSerialiserAndDataStreamStorage(messageSerializer, dataStreamStore); + + var messageReaderWriter = CreateMessageSerialiserAndDataStreamStorage(); var request = new RequestMessageBuilder("poll://test-endpoint").Build(); - var halibutTimeoutAndLimits = new HalibutTimeoutsAndLimits(); - halibutTimeoutAndLimits.PollingQueueWaitTimeout = TimeSpan.FromDays(1); // We should not need to rely on the timeout working for very short disconnects. - - var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(redisFacadeSender), messageReaderWriter, halibutTimeoutAndLimits); - var node2Receiver = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(redisFacadeSender), messageReaderWriter, halibutTimeoutAndLimits); + var node1Sender = new RedisPendingRequestQueue(endpoint, new RedisNeverLosesData(), HalibutLog, new HalibutRedisTransport(redisFacadeSender), messageReaderWriter, new HalibutTimeoutsAndLimits()); + var node2Receiver = new RedisPendingRequestQueue(endpoint, new RedisNeverLosesData(), HalibutLog, new HalibutRedisTransport(redisFacadeSender), messageReaderWriter, new HalibutTimeoutsAndLimits()); await node2Receiver.WaitUntilQueueIsSubscribedToReceiveMessages(); portForwarder.EnterKillNewAndExistingConnectionsMode(); @@ -549,16 +569,13 @@ public async Task WhenTheSendersConnectionToRedisIsBrieflyInterruptedWhileSendin } [Test] - public async Task WhenTheReceiverDequeuesWorkAndThenDisconnectsFromRedisForEver_TheSenderEventuallyTimesOut() + public async Task WhenTheRequestProcessorNodeDequeuesWork_AndThenDisconnectsFromRedisForEver_TheRequestSenderNodeEventuallyTimesOut() { // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid()); - var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); var guid = Guid.NewGuid(); - - var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); - var messageSerializer = new QueueMessageSerializerBuilder().Build(); - var messageReaderWriter = new MessageSerialiserAndDataStreamStorage(messageSerializer, dataStreamStore); + + var messageReaderWriter = CreateMessageSerialiserAndDataStreamStorage(); var halibutTimeoutAndLimits = new HalibutTimeoutsAndLimits(); halibutTimeoutAndLimits.PollingRequestQueueTimeout = TimeSpan.FromDays(1); @@ -569,8 +586,8 @@ public async Task WhenTheReceiverDequeuesWorkAndThenDisconnectsFromRedisForEver_ using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); await using var unstableRedisConnection = RedisFacadeBuilder.CreateRedisFacade(portForwarder, guid); - var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(stableRedisConnection), messageReaderWriter, halibutTimeoutAndLimits); - var node2Receiver = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(unstableRedisConnection), messageReaderWriter, halibutTimeoutAndLimits); + var node1Sender = new RedisPendingRequestQueue(endpoint, new RedisNeverLosesData(), HalibutLog, new HalibutRedisTransport(stableRedisConnection), messageReaderWriter, halibutTimeoutAndLimits); + var node2Receiver = new RedisPendingRequestQueue(endpoint, new RedisNeverLosesData(), HalibutLog, new HalibutRedisTransport(unstableRedisConnection), messageReaderWriter, halibutTimeoutAndLimits); await node2Receiver.WaitUntilQueueIsSubscribedToReceiveMessages(); // Lower this to complete the test sooner. @@ -579,6 +596,7 @@ public async Task WhenTheReceiverDequeuesWorkAndThenDisconnectsFromRedisForEver_ node1Sender.RequestReceivingNodeIsOfflineHeartBeatTimeout = TimeSpan.FromSeconds(10); node2Receiver.RequestReceivingNodeIsOfflineHeartBeatTimeout = TimeSpan.FromSeconds(10); + // Act var request = new RequestMessageBuilder("poll://test-endpoint").Build(); // Setting this low shows we don't timeout because the request was not picked up in time. @@ -590,6 +608,7 @@ public async Task WhenTheReceiverDequeuesWorkAndThenDisconnectsFromRedisForEver_ // Now disconnect the receiver from redis. portForwarder.EnterKillNewAndExistingConnectionsMode(); + // Assert await Task.WhenAny(Task.Delay(TimeSpan.FromSeconds(20), CancellationToken), queueAndWaitTask); queueAndWaitTask.IsCompleted.Should().BeTrue(); @@ -598,20 +617,17 @@ public async Task WhenTheReceiverDequeuesWorkAndThenDisconnectsFromRedisForEver_ response.Error.Should().NotBeNull(); response.Error!.Message.Should().Contain("The node processing the request did not send a heartbeat for long enough, and so the node is now assumed to be offline."); - CreateExceptionFromResponse(response, log).IsRetryableError().Should().Be(HalibutRetryableErrorType.IsRetryable); + CreateExceptionFromResponse(response, HalibutLog).IsRetryableError().Should().Be(HalibutRetryableErrorType.IsRetryable); } [Test] - public async Task WhenTheReceiverDequeuesWork_AndTheSenderDisconnects_AndNeverReconnects_TheDequeuedWorkIsEventuallyCancelled() + public async Task WhenTheRequestProcessorNodeDequeuesWork_AndTheRequestSenderNodeDisconnects_AndNeverReconnects_TheDequeuedWorkIsEventuallyCancelled() { // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid()); - var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); var guid = Guid.NewGuid(); - - var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); - var messageSerializer = new QueueMessageSerializerBuilder().Build(); - var messageReaderWriter = new MessageSerialiserAndDataStreamStorage(messageSerializer, dataStreamStore); + + var messageReaderWriter = CreateMessageSerialiserAndDataStreamStorage(); var halibutTimeoutAndLimits = new HalibutTimeoutsAndLimits(); halibutTimeoutAndLimits.PollingRequestQueueTimeout = TimeSpan.FromDays(1); @@ -622,20 +638,20 @@ public async Task WhenTheReceiverDequeuesWork_AndTheSenderDisconnects_AndNeverRe using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); await using var unstableRedisConnection = RedisFacadeBuilder.CreateRedisFacade(portForwarder, guid); - var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(unstableRedisConnection), messageReaderWriter, halibutTimeoutAndLimits); - var node2Receiver = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(stableRedisConnection), messageReaderWriter, halibutTimeoutAndLimits); + var node1Sender = new RedisPendingRequestQueue(endpoint, new RedisNeverLosesData(), HalibutLog, new HalibutRedisTransport(unstableRedisConnection), messageReaderWriter, halibutTimeoutAndLimits); + var node2Receiver = new RedisPendingRequestQueue(endpoint, new RedisNeverLosesData(), HalibutLog, new HalibutRedisTransport(stableRedisConnection), messageReaderWriter, halibutTimeoutAndLimits); await node2Receiver.WaitUntilQueueIsSubscribedToReceiveMessages(); node1Sender.DelayBetweenHeartBeatsForRequestSender = TimeSpan.FromSeconds(1); node2Receiver.DelayBetweenHeartBeatsForRequestSender = TimeSpan.FromSeconds(1); - node1Sender.NodeIsOfflineHeartBeatTimeoutForRequestSender = TimeSpan.FromSeconds(15); - node2Receiver.NodeIsOfflineHeartBeatTimeoutForRequestSender = TimeSpan.FromSeconds(15); + node1Sender.NodeIsOfflineHeartBeatTimeoutForRequestSender = TimeSpan.FromSeconds(10); + node2Receiver.NodeIsOfflineHeartBeatTimeoutForRequestSender = TimeSpan.FromSeconds(10); node1Sender.TimeBetweenCheckingIfRequestWasCollected = TimeSpan.FromSeconds(1); node2Receiver.TimeBetweenCheckingIfRequestWasCollected = TimeSpan.FromSeconds(1); var request = new RequestMessageBuilder("poll://test-endpoint").Build(); - var queueAndWaitTask = node1Sender.QueueAndWaitAsync(request, CancellationToken.None); + var queueAndWaitTask = node1Sender.QueueAndWaitAsync(request, CancellationToken); var dequeuedRequest = await node2Receiver.DequeueAsync(CancellationToken); dequeuedRequest!.CancellationToken.IsCancellationRequested.Should().BeFalse(); @@ -643,35 +659,30 @@ public async Task WhenTheReceiverDequeuesWork_AndTheSenderDisconnects_AndNeverRe // Now disconnect the sender from redis. portForwarder.EnterKillNewAndExistingConnectionsMode(); - await Task.WhenAny(Task.Delay(TimeSpan.FromMinutes(2), dequeuedRequest.CancellationToken)); + await Task.WhenAny(Task.Delay(TimeSpan.FromSeconds(35), dequeuedRequest.CancellationToken)); dequeuedRequest.CancellationToken.IsCancellationRequested.Should().BeTrue(); } [Test] - public async Task WhenTheSenderBrieflyDisconnectsFromRedisRightWhenTheReceiverSendsTheResponseBack_TheSenderStillGetsTheResponse() + public async Task WhenTheRequestSenderNodeBrieflyDisconnectsFromRedis_AtExactlyTheTimeWhenTheRequestReceiverNodeSendsTheResponseBack_TheRequestSenderNodeStillGetsTheResponse() { // Arrange var endpoint = new Uri("poll://" + Guid.NewGuid()); - var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); var guid = Guid.NewGuid(); - - var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); - var messageSerializer = new QueueMessageSerializerBuilder().Build(); - var messageReaderWriter = new MessageSerialiserAndDataStreamStorage(messageSerializer, dataStreamStore); + + var messageReaderWriter = CreateMessageSerialiserAndDataStreamStorage(); await using var stableConnection = RedisFacadeBuilder.CreateRedisFacade(prefix: guid); using var portForwarder = PortForwardingToRedisBuilder.ForwardingToRedis(Logger); await using var unreliableConnection = RedisFacadeBuilder.CreateRedisFacade(portForwarder, guid); - var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(unreliableConnection), messageReaderWriter, new HalibutTimeoutsAndLimits()); - var node2Receiver = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(stableConnection), messageReaderWriter, new HalibutTimeoutsAndLimits()); + var node1Sender = new RedisPendingRequestQueue(endpoint, new RedisNeverLosesData(), HalibutLog, new HalibutRedisTransport(unreliableConnection), messageReaderWriter, new HalibutTimeoutsAndLimits()); + var node2Receiver = new RedisPendingRequestQueue(endpoint, new RedisNeverLosesData(), HalibutLog, new HalibutRedisTransport(stableConnection), messageReaderWriter, new HalibutTimeoutsAndLimits()); await node2Receiver.WaitUntilQueueIsSubscribedToReceiveMessages(); var request = new RequestMessageBuilder("poll://test-endpoint").Build(); - // TODO: This only needs to be set because we do not detect the work was collected as soon as it has been collected. - request.Destination.PollingRequestQueueTimeout = TimeSpan.FromDays(1); - var queueAndWaitTask = node1Sender.QueueAndWaitAsync(request, CancellationToken.None); + var queueAndWaitTask = node1Sender.QueueAndWaitAsync(request, CancellationToken); var dequeuedRequest = await node2Receiver.DequeueAsync(CancellationToken); @@ -691,47 +702,6 @@ public async Task WhenTheSenderBrieflyDisconnectsFromRedisRightWhenTheReceiverSe response.Result.Should().Be("Yay"); } - [Test] - public async Task WhenTheRequestReceiverDetectsRedisDataLose_AndTheRequestSenderDoesNotYetDetectDataLose_TheSenderReceivesARetryableResponse() - { - // Arrange - var endpoint = new Uri("poll://" + Guid.NewGuid()); - var log = new TestContextLogCreator("Redis", LogLevel.Trace).CreateNewForPrefix(""); - var guid = Guid.NewGuid(); - - var dataStreamStore = new InMemoryStoreDataStreamsForDistributedQueues(); - var messageSerializer = new QueueMessageSerializerBuilder().Build(); - var messageReaderWriter = new MessageSerialiserAndDataStreamStorage(messageSerializer, dataStreamStore); - - await using var stableConnection = RedisFacadeBuilder.CreateRedisFacade(prefix: guid); - - var redisDataLoseDetectorOnReceiver = new CancellableDataLossWatchForRedisLosingAllItsData(); - var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, new HalibutRedisTransport(stableConnection), messageReaderWriter, new HalibutTimeoutsAndLimits()); - var node2Receiver = new RedisPendingRequestQueue(endpoint, redisDataLoseDetectorOnReceiver, log, new HalibutRedisTransport(stableConnection), messageReaderWriter, new HalibutTimeoutsAndLimits()); - await node2Receiver.WaitUntilQueueIsSubscribedToReceiveMessages(); - - var request = new RequestMessageBuilder("poll://test-endpoint").Build(); - var queueAndWaitTask = node1Sender.QueueAndWaitAsync(request, CancellationToken.None); - - var dequeuedRequest = await node2Receiver.DequeueAsync(CancellationToken); - - // Act - await redisDataLoseDetectorOnReceiver.DataLossHasOccured(); - - var responseThatWouldNotBeRetried = ResponseMessage.FromException(dequeuedRequest!.RequestMessage, new NoMatchingServiceOrMethodHalibutClientException("")); - CreateExceptionFromResponse(responseThatWouldNotBeRetried, log) - .IsRetryableError().Should().Be(HalibutRetryableErrorType.NotRetryable); - - await node2Receiver.ApplyResponse(ResponseMessage.FromResult(dequeuedRequest!.RequestMessage, "Yay"), dequeuedRequest!.RequestMessage.ActivityId); - - var response = await queueAndWaitTask; - response.Error.Should().NotBeNull(); - - // Assert - CreateExceptionFromResponse(response, log) - .IsRetryableError().Should().Be(HalibutRetryableErrorType.IsRetryable); - } - static Exception CreateExceptionFromResponse(ResponseMessage responseThatWouldNotBeRetried, ILog log) { try @@ -762,7 +732,7 @@ public async Task WhenUsingTheRedisQueue_ASimpleEchoServiceCanBeCalled(ClientAnd new RedisPendingRequestQueueFactory( queueMessageSerializer, dataStreamStore, - new NeverLosingDataWatchForRedisLosingAllItsData(), + new RedisNeverLosesData(), redisTransport, new HalibutTimeoutsAndLimits(), logFactory) @@ -791,7 +761,7 @@ public async Task CancellingARequestShouldResultInTheDequeuedResponseTokenBeingC var request = new RequestMessageBuilder("poll://test-endpoint").Build(); request.Params = new[] { new ComplexObjectMultipleDataStreams(DataStream.FromString("hello"), DataStream.FromString("world")) }; - var node1Sender = new RedisPendingRequestQueue(endpoint, new NeverLosingDataWatchForRedisLosingAllItsData(), log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); + var node1Sender = new RedisPendingRequestQueue(endpoint, new RedisNeverLosesData(), log, redisTransport, messageReaderWriter, new HalibutTimeoutsAndLimits()); await node1Sender.WaitUntilQueueIsSubscribedToReceiveMessages(); using var cts = new CancellationTokenSource(); diff --git a/source/Halibut.Tests/Queue/Redis/Utils/NeverLosingDataWatchForRedisLosingAllItsData.cs b/source/Halibut.Tests/Queue/Redis/Utils/RedisNeverLosesData.cs similarity index 88% rename from source/Halibut.Tests/Queue/Redis/Utils/NeverLosingDataWatchForRedisLosingAllItsData.cs rename to source/Halibut.Tests/Queue/Redis/Utils/RedisNeverLosesData.cs index 51f139d14..b373c62cd 100644 --- a/source/Halibut.Tests/Queue/Redis/Utils/NeverLosingDataWatchForRedisLosingAllItsData.cs +++ b/source/Halibut.Tests/Queue/Redis/Utils/RedisNeverLosesData.cs @@ -12,7 +12,7 @@ namespace Halibut.Tests.Queue.Redis.Utils /// Test implementation of IWatchForRedisLosingAllItsData that returns CancellationToken.None /// to indicate no data loss detection is active during testing. /// - public class NeverLosingDataWatchForRedisLosingAllItsData : IWatchForRedisLosingAllItsData + public class RedisNeverLosesData : IWatchForRedisLosingAllItsData { public Task GetTokenForDataLossDetection(TimeSpan timeToWait, CancellationToken cancellationToken) { From cb7a3b5e8985996c164db669ccc6b0dede7c5855 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Wed, 20 Aug 2025 11:04:25 +1000 Subject: [PATCH 106/137] Tests for CancelOnDisposeCts --- .../Queue/QueueMessageSerializerFixture.cs | 1 - ...CancelOnDisposeCancellationTokenFixture.cs | 239 ++++++++++++++++++ 2 files changed, 239 insertions(+), 1 deletion(-) create mode 100644 source/Halibut.Tests/Util/CancelOnDisposeCancellationTokenFixture.cs diff --git a/source/Halibut.Tests/Queue/QueueMessageSerializerFixture.cs b/source/Halibut.Tests/Queue/QueueMessageSerializerFixture.cs index aab0e0229..facef06e3 100644 --- a/source/Halibut.Tests/Queue/QueueMessageSerializerFixture.cs +++ b/source/Halibut.Tests/Queue/QueueMessageSerializerFixture.cs @@ -1,7 +1,6 @@ #if NET8_0_OR_GREATER using System; using System.IO; -using System.Linq; using System.Threading; using System.Threading.Tasks; using FluentAssertions; diff --git a/source/Halibut.Tests/Util/CancelOnDisposeCancellationTokenFixture.cs b/source/Halibut.Tests/Util/CancelOnDisposeCancellationTokenFixture.cs new file mode 100644 index 000000000..68fae676e --- /dev/null +++ b/source/Halibut.Tests/Util/CancelOnDisposeCancellationTokenFixture.cs @@ -0,0 +1,239 @@ +using System; +using System.Threading; +using System.Threading.Tasks; +using FluentAssertions; +using Halibut.Util; +using Nito.AsyncEx; +using NUnit.Framework; + +namespace Halibut.Tests.Util +{ + public class CancelOnDisposeCancellationTokenFixture : BaseTest + { + [Test] + public async Task Constructor_NoParameters_ShouldCreateValidToken() + { + // Arrange & Act + await using var cancellationToken = new CancelOnDisposeCancellationToken(); + + // Assert + cancellationToken.Token.Should().NotBeNull(); + cancellationToken.Token.IsCancellationRequested.Should().BeFalse(); + } + + [Test] + public async Task Constructor_WithSingleToken_ShouldCreateLinkedToken() + { + // Arrange + using var parentCts = new CancellationTokenSource(); + var parentToken = parentCts.Token; + + // Act + await using var cancellationToken = new CancelOnDisposeCancellationToken(parentToken); + + // Assert + cancellationToken.Token.Should().NotBeNull(); + cancellationToken.Token.IsCancellationRequested.Should().BeFalse(); + + // When parent is cancelled, child should also be cancelled + await parentCts.CancelAsync(); + cancellationToken.Token.IsCancellationRequested.Should().BeTrue(); + } + + [Test] + public async Task Constructor_WithMultipleTokens_ShouldCreateLinkedToken() + { + // Arrange + using var parentCts1 = new CancellationTokenSource(); + using var parentCts2 = new CancellationTokenSource(); + var parentToken1 = parentCts1.Token; + var parentToken2 = parentCts2.Token; + + // Act + await using var cancellationToken = new CancelOnDisposeCancellationToken(parentToken1, parentToken2); + + // Assert + cancellationToken.Token.Should().NotBeNull(); + cancellationToken.Token.IsCancellationRequested.Should().BeFalse(); + + // When any parent is cancelled, child should also be cancelled + await parentCts1.CancelAsync(); + cancellationToken.Token.IsCancellationRequested.Should().BeTrue(); + } + + [Test] + public async Task Token_PropertyAccess_ShouldNotThrowAfterDisposal() + { + // Arrange + var cancellationToken = new CancelOnDisposeCancellationToken(); + var token = cancellationToken.Token; + + // Act + await cancellationToken.DisposeAsync(); + + // Assert - accessing Token property should not throw + var tokenAfterDispose = cancellationToken.Token; + tokenAfterDispose.Should().Be(token); // Should be the same token instance + } + + [Test] + public async Task CancelAsync_ShouldCancelToken() + { + // Arrange + await using var cancellationToken = new CancelOnDisposeCancellationToken(); + + // Act + await cancellationToken.CancelAsync(); + + // Assert + cancellationToken.Token.IsCancellationRequested.Should().BeTrue(); + } + + [Test] + public async Task CancelAfter_ShouldCancelTokenAfterTimeout() + { + // Arrange + await using var cancellationToken = new CancelOnDisposeCancellationToken(); + + // Act + cancellationToken.CancelAfter(TimeSpan.FromMilliseconds(200)); + + // Assert - token should not be cancelled immediately + cancellationToken.Token.IsCancellationRequested.Should().BeFalse(); + + // Wait for timeout + Thread.Sleep(500); + cancellationToken.Token.IsCancellationRequested.Should().BeTrue(); + } + + [Test] + public async Task AwaitTasksBeforeCTSDispose_ShouldWaitForTasksOnDispose() + { + // Arrange + var taskCompleted = false; + var cancellationToken = new CancelOnDisposeCancellationToken(); + + var manualResetEvent = new AsyncManualResetEvent(); + // Act + cancellationToken.AwaitTasksBeforeCTSDispose(manualResetEvent.WaitAsync(CancellationToken)); + + // Start disposal (don't await yet) + var disposeTask = cancellationToken.DisposeAsync(); + + disposeTask.IsCompleted.Should().BeFalse(); + + manualResetEvent.Set(); + await Task.WhenAny(Task.Delay(TimeSpan.FromSeconds(1)), Task.Run(async () => await disposeTask)); + await disposeTask; + + // Assert + taskCompleted.Should().BeTrue(); + } + + [Test] + public async Task AwaitTasksBeforeCTSDispose_ShouldHandleTaskExceptions() + { + // Arrange + var cancellationToken = new CancelOnDisposeCancellationToken(); + + var faultyTask = Task.Run(async () => + { + await Task.CompletedTask; + throw new InvalidOperationException("Test exception"); + }); + + // Act + cancellationToken.AwaitTasksBeforeCTSDispose(faultyTask); + + // Assert - dispose should not throw even though the task throws + await cancellationToken.DisposeAsync(); + + // Task should be faulted + faultyTask.IsFaulted.Should().BeTrue(); + } + + [Test] + public async Task DisposeAsync_ShouldCancelTokenBeforeDispose() + { + // Arrange + var cancellationToken = new CancelOnDisposeCancellationToken(); + var token = cancellationToken.Token; + + // Act + await cancellationToken.DisposeAsync(); + + // Assert + token.IsCancellationRequested.Should().BeTrue(); + } + + [Test] + public async Task DisposeAsync_CalledMultipleTimes_ShouldNotThrow() + { + // Arrange + var cancellationToken = new CancelOnDisposeCancellationToken(); + + // Act & Assert - multiple dispose calls should not throw + await cancellationToken.DisposeAsync(); + await cancellationToken.DisposeAsync(); + await cancellationToken.DisposeAsync(); + } + + [Test] + public async Task DisposeAsync_WithTasksUsingToken_ShouldWaitForCancellation() + { + // Arrange + var taskCancelled = false; + var cancellationToken = new CancelOnDisposeCancellationToken(); + + var taskUsingToken = Task.Run(async () => + { + try + { + await Task.Delay(1000, cancellationToken.Token); + } + catch (OperationCanceledException) + { + taskCancelled = true; + } + }); + + cancellationToken.AwaitTasksBeforeCTSDispose(taskUsingToken); + + // Act + await cancellationToken.DisposeAsync(); + + // Assert + taskCancelled.Should().BeTrue(); + } + + [Test] + public async Task DisposeAsync_WithMultipleTasks_ShouldWaitForAllTasks() + { + // Arrange + var task1Completed = false; + var task2Completed = false; + var cancellationToken = new CancelOnDisposeCancellationToken(); + + var task1 = Task.Run(async () => + { + await Task.CompletedTask; + task1Completed = true; + }); + + var task2 = Task.Run(async () => + { + await Task.CompletedTask; + task2Completed = true; + }); + + cancellationToken.AwaitTasksBeforeCTSDispose(task1, task2); + + // Act + await cancellationToken.DisposeAsync(); + + // Assert + task1Completed.Should().BeTrue(); + task2Completed.Should().BeTrue(); + } + } +} From 986cd9b5d46562837e8641f55b2584ed458f0de1 Mon Sep 17 00:00:00 2001 From: Rhys Parry Date: Wed, 20 Aug 2025 11:06:52 +1000 Subject: [PATCH 107/137] Cleanup nits in HalibutRedisTransport --- .../Redis/RedisHelpers/HalibutRedisTransport.cs | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/source/Halibut/Queue/Redis/RedisHelpers/HalibutRedisTransport.cs b/source/Halibut/Queue/Redis/RedisHelpers/HalibutRedisTransport.cs index e53083639..9825620f3 100644 --- a/source/Halibut/Queue/Redis/RedisHelpers/HalibutRedisTransport.cs +++ b/source/Halibut/Queue/Redis/RedisHelpers/HalibutRedisTransport.cs @@ -72,7 +72,7 @@ public async Task PushRequestGuidOnToQueue(Uri endpoint, Guid guid, Cancellation // Pending Request Message // Stores the Pending Request Message for collection by the service. // Note that the service will first need to TryPopNextRequestGuid to be able to - // fins the RequestMessage. + // find the RequestMessage. static string RequestMessageKey(Uri endpoint, Guid requestId) { @@ -158,7 +158,7 @@ public async Task PublishCancellation(Uri endpoint, Guid requestId, Cancellation } // Request cancellation - // Since pub/sub does not have guaranteed deliver, cancellation can also + // Since pub/sub does not have guaranteed delivery, cancellation can also // be detected by the RequestCancelledMarker. The node processing the request // will poll for the existence of the RequestCancelledMarker, and if found // it knows the RPC has been cancelled. @@ -187,7 +187,7 @@ public async Task IsRequestMarkedAsCancelled(Uri endpoint, Guid requestId, // Another channel for the `RequestProcessorNode` where the node that is sending the // request to the service (e.g. Tentacle) is publishing heart beats, for the duration // of processing the request. - // Both nodes are able to monitor to the heart beat channel of the other node to detect + // Both nodes are able to monitor the heart beat channel of the other node to detect // if the other node has gone offline. static string NodeHeartBeatChannel(Uri endpoint, Guid requestId, HalibutQueueNodeSendingPulses nodeSendingPulsesType) @@ -220,12 +220,14 @@ public async Task SendNodeHeartBeat(Uri endpoint, Guid requestId, HalibutQueueNo // The node processing the request `RequestProcessorNode` will publish to this channel // once the Response is available. - string ResponseChannelName(Uri endpoint, Guid identifier) + static string ResponseChannelName(Uri endpoint, Guid identifier) { return $"{Namespace}::ResponseAvailableChannel::{endpoint}::{identifier}"; } - public async Task SubscribeToResponseChannel(Uri endpoint, Guid identifier, + public async Task SubscribeToResponseChannel( + Uri endpoint, + Guid identifier, Func onValueReceived, CancellationToken cancellationToken) { @@ -246,7 +248,7 @@ public async Task PublishThatResponseIsAvailable(Uri endpoint, Guid identifier, // Response // This is where the Response is placed in Redis. - string ResponseMessageKey(Uri endpoint, Guid identifier) + static string ResponseMessageKey(Uri endpoint, Guid identifier) { return $"{Namespace}::Response::{endpoint}::{identifier}"; } From ce3fd6d6ff1a53cc0ab849faddb0031f4df0fca4 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Wed, 20 Aug 2025 11:08:17 +1000 Subject: [PATCH 108/137] . --- .../Queue/QueueMessageSerializerFixture.cs | 25 +++++++++++++------ 1 file changed, 17 insertions(+), 8 deletions(-) diff --git a/source/Halibut.Tests/Queue/QueueMessageSerializerFixture.cs b/source/Halibut.Tests/Queue/QueueMessageSerializerFixture.cs index facef06e3..3d6eccdca 100644 --- a/source/Halibut.Tests/Queue/QueueMessageSerializerFixture.cs +++ b/source/Halibut.Tests/Queue/QueueMessageSerializerFixture.cs @@ -14,11 +14,10 @@ namespace Halibut.Tests.Queue public class QueueMessageSerializerFixture : BaseTest { [Test] - public void SerializeAndDeserializeMessage_ShouldRoundTrip() + public void SerializeAndDeserializeSimpleStringMessage_ShouldRoundTrip() { // Arrange - var sut = new QueueMessageSerializerBuilder() - .Build(); + var sut = new QueueMessageSerializerBuilder().Build(); const string testMessage = "Hello, Queue!"; @@ -33,11 +32,10 @@ public void SerializeAndDeserializeMessage_ShouldRoundTrip() } [Test] - public void SerializeAndDeserializeMessage_ShouldRoundTrip_RequestMessage() + public void SerializeAndDeserializeRequestMessage_ShouldRoundTrip_RequestMessage() { // Arrange - var sut = new QueueMessageSerializerBuilder() - .Build(); + var sut = new QueueMessageSerializerBuilder().Build(); var request = new RequestMessage() { @@ -60,7 +58,7 @@ public void SerializeAndDeserializeMessage_ShouldRoundTrip_RequestMessage() } [Test] - public void SerializeAndDeserializeMessageWithDataStream_ShouldRoundTrip_RequestMessage() + public void SerializeAndDeserializeRequestMessageWithDataStream_ShouldRoundTrip_RequestMessage() { var typeRegistry = new TypeRegistry(); typeRegistry.Register(typeof(IHaveTypeWithDataStreamsService)); @@ -94,7 +92,18 @@ public void SerializeAndDeserializeMessageWithDataStream_ShouldRoundTrip_Request var (deserializedMessage, deserializedDataStreams) = sut.ReadMessage(json); // Assert - //deserializedMessage.Should().BeEquivalentTo(request); + // Manually check each field of the deserializedMessage matches the request + deserializedMessage.Id.Should().Be(request.Id); + deserializedMessage.ActivityId.Should().Be(request.ActivityId); + deserializedMessage.Destination.BaseUri.Should().Be(request.Destination.BaseUri); + deserializedMessage.ServiceName.Should().Be(request.ServiceName); + deserializedMessage.MethodName.Should().Be(request.MethodName); + + // Check Params array structure (DataStreams are replaced with placeholders during serialization) + deserializedMessage.Params.Should().HaveCount(request.Params.Length); + deserializedMessage.Params[0].Should().Be(request.Params[0]); // First param is a simple string + // Note: Params[1] and Params[2] contain DataStreams which get replaced during serialization + deserializedDataStreams.Count.Should().Be(2); } From 8c6928d9393a751f275222419446a8b0cd9fd737 Mon Sep 17 00:00:00 2001 From: Rhys Parry Date: Wed, 20 Aug 2025 11:08:58 +1000 Subject: [PATCH 109/137] Renamed DataLoss exception --- .../ExceptionReturnedByHalibutProxyExtensionMethod.cs | 2 +- ...> CouldNotGetDataLossTokenInTimeHalibutClientException.cs} | 4 ++-- source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) rename source/Halibut/Queue/Redis/Exceptions/{CouldNotGetDataLoseTokenInTimeHalibutClientException.cs => CouldNotGetDataLossTokenInTimeHalibutClientException.cs} (58%) diff --git a/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs b/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs index 8499c48e6..67937d566 100644 --- a/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs +++ b/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs @@ -30,7 +30,7 @@ static bool IsRedisRetryableError(Exception exception) { if (exception is RedisDataLoseHalibutClientException || exception is RedisQueueShutdownClientException - || exception is CouldNotGetDataLoseTokenInTimeHalibutClientException + || exception is CouldNotGetDataLossTokenInTimeHalibutClientException || exception is ErrorWhilePreparingRequestForQueueHalibutClientException || exception is ErrorOccuredWhenInsertingDataIntoRedisHalibutPendingRequestQueue) { diff --git a/source/Halibut/Queue/Redis/Exceptions/CouldNotGetDataLoseTokenInTimeHalibutClientException.cs b/source/Halibut/Queue/Redis/Exceptions/CouldNotGetDataLossTokenInTimeHalibutClientException.cs similarity index 58% rename from source/Halibut/Queue/Redis/Exceptions/CouldNotGetDataLoseTokenInTimeHalibutClientException.cs rename to source/Halibut/Queue/Redis/Exceptions/CouldNotGetDataLossTokenInTimeHalibutClientException.cs index e60e52ed7..540f9c205 100644 --- a/source/Halibut/Queue/Redis/Exceptions/CouldNotGetDataLoseTokenInTimeHalibutClientException.cs +++ b/source/Halibut/Queue/Redis/Exceptions/CouldNotGetDataLossTokenInTimeHalibutClientException.cs @@ -2,9 +2,9 @@ namespace Halibut.Queue.Redis.Exceptions { - public class CouldNotGetDataLoseTokenInTimeHalibutClientException : HalibutClientException + public class CouldNotGetDataLossTokenInTimeHalibutClientException : HalibutClientException { - public CouldNotGetDataLoseTokenInTimeHalibutClientException(string message, Exception inner) : base(message, inner) + public CouldNotGetDataLossTokenInTimeHalibutClientException(string message, Exception inner) : base(message, inner) { } } diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index 67946310d..9699d6832 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -87,7 +87,7 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can catch (Exception ex) { if (requestCancellationToken.IsCancellationRequested) throw RedisPendingRequest.CreateExceptionForRequestWasCancelledBeforeCollected(request, log); - throw new CouldNotGetDataLoseTokenInTimeHalibutClientException("Unable to reconnect to redis to get data loss detection CT", ex); + throw new CouldNotGetDataLossTokenInTimeHalibutClientException("Unable to reconnect to redis to get data loss detection CT", ex); } Exception? CancellationReason() From 6729cc7ed0533e4a599dbeae2f8fc63dbc1348c0 Mon Sep 17 00:00:00 2001 From: Rhys Parry Date: Wed, 20 Aug 2025 11:13:51 +1000 Subject: [PATCH 110/137] Rename other data loss exception --- .../Queue/Redis/RedisPendingRequestQueueFixture.cs | 2 +- .../ExceptionReturnedByHalibutProxyExtensionMethod.cs | 2 +- .../Exceptions/RedisDataLoseHalibutClientException.cs | 11 ----------- .../Exceptions/RedisDataLossHalibutClientException.cs | 11 +++++++++++ .../Halibut/Queue/Redis/RedisPendingRequestQueue.cs | 2 +- 5 files changed, 14 insertions(+), 14 deletions(-) delete mode 100644 source/Halibut/Queue/Redis/Exceptions/RedisDataLoseHalibutClientException.cs create mode 100644 source/Halibut/Queue/Redis/Exceptions/RedisDataLossHalibutClientException.cs diff --git a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs index c4a6fec7b..ecda7d9fa 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs @@ -339,7 +339,7 @@ public async Task WhenDataLostIsDetected_InFlightRequestShouldBeAbandoned_AndARe // Sigh it can go down either of these paths! var e = await AssertException.Throws(queueAndWaitAsync); e.And.IsRetryableError().Should().Be(HalibutRetryableErrorType.IsRetryable); - e.And.Should().BeOfType(); + e.And.Should().BeOfType(); } [Test] diff --git a/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs b/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs index 67937d566..ca4785818 100644 --- a/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs +++ b/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs @@ -28,7 +28,7 @@ public static HalibutRetryableErrorType IsRetryableError(this Exception exceptio static bool IsRedisRetryableError(Exception exception) { - if (exception is RedisDataLoseHalibutClientException + if (exception is RedisDataLossHalibutClientException || exception is RedisQueueShutdownClientException || exception is CouldNotGetDataLossTokenInTimeHalibutClientException || exception is ErrorWhilePreparingRequestForQueueHalibutClientException diff --git a/source/Halibut/Queue/Redis/Exceptions/RedisDataLoseHalibutClientException.cs b/source/Halibut/Queue/Redis/Exceptions/RedisDataLoseHalibutClientException.cs deleted file mode 100644 index e4bbaa10e..000000000 --- a/source/Halibut/Queue/Redis/Exceptions/RedisDataLoseHalibutClientException.cs +++ /dev/null @@ -1,11 +0,0 @@ -using System; - -namespace Halibut.Queue.Redis.Exceptions -{ - public class RedisDataLoseHalibutClientException : HalibutClientException - { - public RedisDataLoseHalibutClientException(string message) : base(message) - { - } - } -} \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/Exceptions/RedisDataLossHalibutClientException.cs b/source/Halibut/Queue/Redis/Exceptions/RedisDataLossHalibutClientException.cs new file mode 100644 index 000000000..e5f70e229 --- /dev/null +++ b/source/Halibut/Queue/Redis/Exceptions/RedisDataLossHalibutClientException.cs @@ -0,0 +1,11 @@ +using System; + +namespace Halibut.Queue.Redis.Exceptions +{ + public class RedisDataLossHalibutClientException : HalibutClientException + { + public RedisDataLossHalibutClientException(string message) : base(message) + { + } + } +} \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index 9699d6832..52da3818c 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -92,7 +92,7 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can Exception? CancellationReason() { - if (dataLoseCt.IsCancellationRequested) return new RedisDataLoseHalibutClientException($"Request {request.ActivityId} was cancelled because we detected that redis lost all of its data."); + if (dataLoseCt.IsCancellationRequested) return new RedisDataLossHalibutClientException($"Request {request.ActivityId} was cancelled because we detected that redis lost all of its data."); if (queueToken.IsCancellationRequested) return new RedisQueueShutdownClientException($"Request {request.ActivityId} was cancelled because the queue is shutting down."); return null; } From cbc5ab8857ab73773db70b67128091d818db4c1c Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Wed, 20 Aug 2025 11:16:09 +1000 Subject: [PATCH 111/137] Re arrange redis queue --- .../Redis/RedisPendingRequestQueueFixture.cs | 22 ++--- .../Queue/Redis/RedisPendingRequestQueue.cs | 97 ++++++++----------- 2 files changed, 54 insertions(+), 65 deletions(-) diff --git a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs index c4a6fec7b..1a6dc3aa8 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs @@ -384,7 +384,7 @@ public async Task OnceARequestIsComplete_NoRequestSenderNodeHeartBeatsShouldBeSe var queue = new RedisPendingRequestQueue(endpoint, new RedisNeverLosesData(), HalibutLog, redisTransport, CreateMessageSerialiserAndDataStreamStorage(), new HalibutTimeoutsAndLimits()); await queue.WaitUntilQueueIsSubscribedToReceiveMessages(); - queue.DelayBetweenHeartBeatsForRequestSender = TimeSpan.FromSeconds(1); + queue.RequestSenderNodeHeartBeatRate = TimeSpan.FromSeconds(1); // Act var queueAndWaitAsync = queue.QueueAndWaitAsync(request, CancellationToken.None); @@ -424,7 +424,7 @@ public async Task OnceARequestIsComplete_NoRequestProcessorNodeHeartBeatsShouldB var queue = new RedisPendingRequestQueue(endpoint, new RedisNeverLosesData(), HalibutLog, redisTransport, CreateMessageSerialiserAndDataStreamStorage(), new HalibutTimeoutsAndLimits()); await queue.WaitUntilQueueIsSubscribedToReceiveMessages(); - queue.DelayBetweenHeartBeatsForRequestProcessor = TimeSpan.FromSeconds(1); + queue.RequestReceiverNodeHeartBeatRate = TimeSpan.FromSeconds(1); // Act var queueAndWaitAsync = queue.QueueAndWaitAsync(request, CancellationToken.None); @@ -519,7 +519,7 @@ public async Task WhenTheReceiverDoesntCollectWorkImmediately_TheRequestCanSitOn var node1Sender = new RedisPendingRequestQueue(endpoint, new RedisNeverLosesData(), HalibutLog, new HalibutRedisTransport(redisFacade), messageReaderWriter, new HalibutTimeoutsAndLimits()); // We are testing that we don't expect heart beats before the request is collected. - node1Sender.RequestReceivingNodeIsOfflineHeartBeatTimeout = TimeSpan.FromSeconds(1); + node1Sender.RequestReceiverNodeHeartBeatTimeout = TimeSpan.FromSeconds(1); await node1Sender.WaitUntilQueueIsSubscribedToReceiveMessages(); var request = new RequestMessageBuilder("poll://test-endpoint").Build(); @@ -591,10 +591,10 @@ public async Task WhenTheRequestProcessorNodeDequeuesWork_AndThenDisconnectsFrom await node2Receiver.WaitUntilQueueIsSubscribedToReceiveMessages(); // Lower this to complete the test sooner. - node1Sender.DelayBetweenHeartBeatsForRequestProcessor = TimeSpan.FromSeconds(1); - node2Receiver.DelayBetweenHeartBeatsForRequestProcessor = TimeSpan.FromSeconds(1); - node1Sender.RequestReceivingNodeIsOfflineHeartBeatTimeout = TimeSpan.FromSeconds(10); - node2Receiver.RequestReceivingNodeIsOfflineHeartBeatTimeout = TimeSpan.FromSeconds(10); + node1Sender.RequestReceiverNodeHeartBeatRate = TimeSpan.FromSeconds(1); + node2Receiver.RequestReceiverNodeHeartBeatRate = TimeSpan.FromSeconds(1); + node1Sender.RequestReceiverNodeHeartBeatTimeout = TimeSpan.FromSeconds(10); + node2Receiver.RequestReceiverNodeHeartBeatTimeout = TimeSpan.FromSeconds(10); // Act var request = new RequestMessageBuilder("poll://test-endpoint").Build(); @@ -642,10 +642,10 @@ public async Task WhenTheRequestProcessorNodeDequeuesWork_AndTheRequestSenderNod var node2Receiver = new RedisPendingRequestQueue(endpoint, new RedisNeverLosesData(), HalibutLog, new HalibutRedisTransport(stableRedisConnection), messageReaderWriter, halibutTimeoutAndLimits); await node2Receiver.WaitUntilQueueIsSubscribedToReceiveMessages(); - node1Sender.DelayBetweenHeartBeatsForRequestSender = TimeSpan.FromSeconds(1); - node2Receiver.DelayBetweenHeartBeatsForRequestSender = TimeSpan.FromSeconds(1); - node1Sender.NodeIsOfflineHeartBeatTimeoutForRequestSender = TimeSpan.FromSeconds(10); - node2Receiver.NodeIsOfflineHeartBeatTimeoutForRequestSender = TimeSpan.FromSeconds(10); + node1Sender.RequestSenderNodeHeartBeatRate = TimeSpan.FromSeconds(1); + node2Receiver.RequestSenderNodeHeartBeatRate = TimeSpan.FromSeconds(1); + node1Sender.RequestSenderNodeHeartBeatTimeout = TimeSpan.FromSeconds(10); + node2Receiver.RequestSenderNodeHeartBeatTimeout = TimeSpan.FromSeconds(10); node1Sender.TimeBetweenCheckingIfRequestWasCollected = TimeSpan.FromSeconds(1); node2Receiver.TimeBetweenCheckingIfRequestWasCollected = TimeSpan.FromSeconds(1); diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index 67946310d..c659cbf26 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -31,12 +31,39 @@ class RedisPendingRequestQueue : IPendingRequestQueue, IDisposable readonly CancelOnDisposeCancellationToken queueCts = new (); internal ConcurrentDictionary DisposablesForInFlightRequests = new(); - // TODO: this needs to be used in all public methods. readonly CancellationToken queueToken; + // Used for testing. int numberOfInFlightRequestsThatHaveReachedTheStageOfBeingReadyForCollection = 0; - Task PulseChannelSubDisposer { get; } + Task RequestMessageAvailablePulseChannelSubscriberDisposer { get; } + + public bool IsEmpty => Count == 0; + public int Count => numberOfInFlightRequestsThatHaveReachedTheStageOfBeingReadyForCollection; + + // The timespan is more generous for the sender going offline, since if it does go offline, + // under some cases the request completing is advantageous. That node needs to + // re-do the entire RPC for idempotent RPCs this might mean that the task required is already done. + internal TimeSpan RequestSenderNodeHeartBeatTimeout { get; set; } = TimeSpan.FromSeconds(90); + + // How often the Request Sender sends a heart beat. + internal TimeSpan RequestSenderNodeHeartBeatRate { get; set; } = TimeSpan.FromSeconds(15); + + /// + /// The amount of time since the last heart beat from the node sending the request to Tentacle + /// before the node is assumed to be offline. + /// + /// Setting this too high means things above the RPC might not have time to retry. + /// + public TimeSpan RequestReceiverNodeHeartBeatTimeout { get; set; } = TimeSpan.FromSeconds(60); + + // How often the Request Receiver node sends a heart beat. + internal TimeSpan RequestReceiverNodeHeartBeatRate { get; set; } = TimeSpan.FromSeconds(15); + + // How long the response message can live in redis. + internal TimeSpan TTLOfResponseMessage { get; set; } = TimeSpan.FromMinutes(20); + + internal TimeSpan TimeBetweenCheckingIfRequestWasCollected { get; set; } = TimeSpan.FromSeconds(30); public RedisPendingRequestQueue( Uri endpoint, @@ -54,25 +81,14 @@ public RedisPendingRequestQueue( this.halibutTimeoutsAndLimits = halibutTimeoutsAndLimits; this.queueToken = queueCts.Token; - // TODO: can we unsub if no tentacle is asking for a work for an extended period of time? - // and also NOT sub if the queue is being created to send work. - // The advice is many channels with few subscribers is better than a single channel with many subscribers. - // If we end up with too many channels, we could shared the channels based on modulo of the hash of the endpoint, - // which means we might have only 1000 channels and num_tentacles/1000 subscribers to each channel. For 300K tentacles. - PulseChannelSubDisposer = Task.Run(async () => await this.halibutRedisTransport.SubscribeToRequestMessagePulseChannel(endpoint, _ => hasItemsForEndpoint.Set(), queueToken)); + // Ideally we would only subscribe subscribers are using this queue. + RequestMessageAvailablePulseChannelSubscriberDisposer = Task.Run(async () => await this.halibutRedisTransport.SubscribeToRequestMessagePulseChannel(endpoint, _ => hasItemsForEndpoint.Set(), queueToken)); } - internal async Task WaitUntilQueueIsSubscribedToReceiveMessages() => await PulseChannelSubDisposer; - - public async ValueTask DisposeAsync() - { - await Try.IgnoringError(async () => await queueCts.DisposeAsync()); - await Try.IgnoringError(async () => await (await PulseChannelSubDisposer).DisposeAsync()); - } + internal async Task WaitUntilQueueIsSubscribedToReceiveMessages() => await RequestMessageAvailablePulseChannelSubscriberDisposer; - private async Task DataLossCancellationToken(CancellationToken? cancellationToken) + async Task DataLossCancellationToken(CancellationToken? cancellationToken) { - // TODO this must throw something that can be retried. await using var cts = new CancelOnDisposeCancellationToken(queueCts.Token, cancellationToken ?? CancellationToken.None); return await watchForRedisLosingAllItsData.GetTokenForDataLossDetection(TimeSpan.FromSeconds(30), cts.Token); } @@ -127,7 +143,7 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can var tryClearRequestFromQueueAtMostOnce = new AsyncLazy(async () => await TryClearRequestFromQueue(request, pending)); try { - await using var senderPulse = new NodeHeartBeatSender(endpoint, request.ActivityId, halibutRedisTransport, log, HalibutQueueNodeSendingPulses.RequestSenderNode, DelayBetweenHeartBeatsForRequestSender); + await using var senderPulse = new NodeHeartBeatSender(endpoint, request.ActivityId, halibutRedisTransport, log, HalibutQueueNodeSendingPulses.RequestSenderNode, RequestSenderNodeHeartBeatRate); // Make the request available before we tell people it is available. try { @@ -218,8 +234,6 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can } })); } - - } @@ -247,7 +261,7 @@ void InBackgroundSendCancellationIfRequestWasCancelled(RequestMessage request, R halibutRedisTransport, TimeBetweenCheckingIfRequestWasCollected, log, - RequestReceivingNodeIsOfflineHeartBeatTimeout, + RequestReceiverNodeHeartBeatTimeout, cancellationToken); } @@ -286,10 +300,6 @@ async Task TryClearRequestFromQueue(RequestMessage request, RedisPendingRe } return false; } - - - - const string ResponseMessageSubscriptionName = "ResponseMessage"; async Task WaitForResponse( PollAndSubscribeToResponse pollAndSubscribeToResponse, @@ -324,32 +334,6 @@ async Task TryClearRequestFromQueue(RequestMessage request, RedisPendingRe } } - - public bool IsEmpty => Count == 0; - public int Count => numberOfInFlightRequestsThatHaveReachedTheStageOfBeingReadyForCollection; - - // The timespan is more generous for the sender going offline, since if it does go offline, - // since under some cases the request completing is advantageous. That node needs to - // re-do the entire RPC for idempotent RPCs this might mean that the task required is already done. - internal TimeSpan NodeIsOfflineHeartBeatTimeoutForRequestSender { get; set; } = TimeSpan.FromSeconds(90); - - internal TimeSpan DelayBetweenHeartBeatsForRequestSender { get; set; } = TimeSpan.FromSeconds(15); - - /// - /// The amount of time since the last heart beat from the node sending the request to Tentacle - /// before the node is assumed to be offline. - /// - /// Setting this too high means things above the RPC might not have time to retry. - /// - public TimeSpan RequestReceivingNodeIsOfflineHeartBeatTimeout { get; set; } = TimeSpan.FromSeconds(60); - - internal TimeSpan DelayBetweenHeartBeatsForRequestProcessor { get; set; } = TimeSpan.FromSeconds(15); - - internal TimeSpan TTLOfResponseMessage { get; set; } = TimeSpan.FromMinutes(20); - - internal TimeSpan TimeBetweenCheckingIfRequestWasCollected { get; set; } = TimeSpan.FromSeconds(30); - - public async Task DequeueAsync(CancellationToken cancellationToken) { @@ -367,8 +351,8 @@ async Task TryClearRequestFromQueue(RequestMessage request, RedisPendingRe // In that case we will just time out because of the lack of heart beats. var dataLossCT = await watchForRedisLosingAllItsData.GetTokenForDataLossDetection(TimeSpan.FromSeconds(30), queueToken); - disposables.AddAsyncDisposable(new NodeHeartBeatSender(endpoint, pending.ActivityId, halibutRedisTransport, log, HalibutQueueNodeSendingPulses.RequestProcessorNode, DelayBetweenHeartBeatsForRequestProcessor)); - var watcher = new WatchForRequestCancellationOrSenderDisconnect(endpoint, pending.ActivityId, halibutRedisTransport, NodeIsOfflineHeartBeatTimeoutForRequestSender, log); + disposables.AddAsyncDisposable(new NodeHeartBeatSender(endpoint, pending.ActivityId, halibutRedisTransport, log, HalibutQueueNodeSendingPulses.RequestProcessorNode, RequestReceiverNodeHeartBeatRate)); + var watcher = new WatchForRequestCancellationOrSenderDisconnect(endpoint, pending.ActivityId, halibutRedisTransport, RequestSenderNodeHeartBeatTimeout, log); disposables.AddAsyncDisposable(watcher); var cts = new CancelOnDisposeCancellationToken(watcher.RequestProcessingCancellationToken, dataLossCT); @@ -405,6 +389,7 @@ public async ValueTask DisposeAsync() } public const string RequestAbandonedMessage = "The request was abandoned, possibly because the node processing the request shutdown or redis lost all of its data."; + public async Task ApplyResponse(ResponseMessage response, Guid requestActivityId) { log.Write(EventType.MessageExchange, "Applying response for request {0}", requestActivityId); @@ -527,7 +512,11 @@ await Task.WhenAny( } } - + public async ValueTask DisposeAsync() + { + await Try.IgnoringError(async () => await queueCts.DisposeAsync()); + await Try.IgnoringError(async () => await (await RequestMessageAvailablePulseChannelSubscriberDisposer).DisposeAsync()); + } public void Dispose() { From 76efa2e71694606dcc97226d1bd855deff823c96 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Wed, 20 Aug 2025 11:18:18 +1000 Subject: [PATCH 112/137] clean redis PRQ builder --- .../Builders/RedisPendingRequestQueueBuilder.cs | 5 ----- .../Queue/Redis/Utils/TestRedisPendingRequestQueueFactory.cs | 2 -- 2 files changed, 7 deletions(-) diff --git a/source/Halibut.Tests/Builders/RedisPendingRequestQueueBuilder.cs b/source/Halibut.Tests/Builders/RedisPendingRequestQueueBuilder.cs index 2a04b86b8..f2aa00e0b 100644 --- a/source/Halibut.Tests/Builders/RedisPendingRequestQueueBuilder.cs +++ b/source/Halibut.Tests/Builders/RedisPendingRequestQueueBuilder.cs @@ -2,7 +2,6 @@ #if NET8_0_OR_GREATER using System; using Halibut.Logging; -using Halibut.Queue.QueuedDataStreams; using Halibut.Queue.Redis; using Halibut.Queue.Redis.MessageStorage; using Halibut.Queue.Redis.RedisHelpers; @@ -60,13 +59,9 @@ public QueueHolder Build() var messageSerializer = new QueueMessageSerializerBuilder().Build(); var messageReaderWriter = new MessageSerialiserAndDataStreamStorage(messageSerializer, dataStreamStore); - var request = new RequestMessageBuilder("poll://test-endpoint").Build(); - var queue = new RedisPendingRequestQueue(endpoint, new RedisNeverLosesData(), log, redisTransport, messageReaderWriter, halibutTimeoutsAndLimits); -#pragma warning disable VSTHRD002 queue.WaitUntilQueueIsSubscribedToReceiveMessages().GetAwaiter().GetResult(); -#pragma warning restore VSTHRD002 return new QueueHolder(queue, disposableCollection); } diff --git a/source/Halibut.Tests/Queue/Redis/Utils/TestRedisPendingRequestQueueFactory.cs b/source/Halibut.Tests/Queue/Redis/Utils/TestRedisPendingRequestQueueFactory.cs index 21ffce8e2..ee7f0e56e 100644 --- a/source/Halibut.Tests/Queue/Redis/Utils/TestRedisPendingRequestQueueFactory.cs +++ b/source/Halibut.Tests/Queue/Redis/Utils/TestRedisPendingRequestQueueFactory.cs @@ -18,9 +18,7 @@ public TestRedisPendingRequestQueueFactory(RedisPendingRequestQueueFactory redis public IPendingRequestQueue CreateQueue(Uri endpoint) { var queue = (RedisPendingRequestQueue) redisPendingRequestQueueFactory.CreateQueue(endpoint); -#pragma warning disable VSTHRD002 queue.WaitUntilQueueIsSubscribedToReceiveMessages().GetAwaiter().GetResult(); -#pragma warning restore VSTHRD002 return queue; } } From 145904ca3241393706c334eb93c888fdad30255c Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Wed, 20 Aug 2025 11:26:54 +1000 Subject: [PATCH 113/137] clean up test --- .../Builders/ServiceEndPointBuilder.cs | 3 +- .../Halibut.Tests/ManyPollingTentacleTests.cs | 46 +++++++++++-------- .../AllQueuesTestCasesAttribute.cs | 10 +++- 3 files changed, 36 insertions(+), 23 deletions(-) diff --git a/source/Halibut.Tests/Builders/ServiceEndPointBuilder.cs b/source/Halibut.Tests/Builders/ServiceEndPointBuilder.cs index bfee5f055..e79c0323c 100644 --- a/source/Halibut.Tests/Builders/ServiceEndPointBuilder.cs +++ b/source/Halibut.Tests/Builders/ServiceEndPointBuilder.cs @@ -1,5 +1,4 @@ - -using Halibut.Transport.Protocol; +using Halibut.Transport.Protocol; using System; using Halibut.Diagnostics; diff --git a/source/Halibut.Tests/ManyPollingTentacleTests.cs b/source/Halibut.Tests/ManyPollingTentacleTests.cs index d8584811f..bdbc26a5d 100644 --- a/source/Halibut.Tests/ManyPollingTentacleTests.cs +++ b/source/Halibut.Tests/ManyPollingTentacleTests.cs @@ -22,6 +22,7 @@ using Halibut.Tests.TestServices; using Halibut.Tests.TestServices.Async; using Halibut.TestUtils.Contracts; +using Halibut.Util; using NUnit.Framework; using DisposableCollection = Halibut.Util.DisposableCollection; @@ -31,15 +32,24 @@ namespace Halibut.Tests [RedisTest] public class ManyPollingTentacleTests : BaseTest { + /// + /// Fuzz test, to check under load the queue still works. + /// + /// + /// [Test] [AllQueuesTestCases] [NonParallelizable] - public async Task ManyRequestToPollingTentacles_Works_AndDoesNotUseTooManyResources(PendingRequestQueueTestCase queueTestCase) + public async Task WhenMakingManyConcurrentRequestsToManyServices_AllRequestsCompleteSuccessfully_And(PendingRequestQueueTestCase queueTestCase) { + var numberOfPollingServices = 100; + int concurrency = 20; + int numberOfCallsToMake = Math.Min(numberOfPollingServices, 20); + var logFactory = new CachingLogFactory(new TestContextLogCreator("", LogLevel.Trace)); var services = GetDelegateServiceFactory(); await using var disposables = new DisposableCollection(); - var isRedis = queueTestCase.ToString().ToLower().Contains("redis"); + var isRedis = queueTestCase.Name == PendingRequestQueueTestCase.RedisTestCaseName; var log = new TestContextLogCreator("Redis", LogLevel.Fatal); await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(); await using (var octopus = new HalibutRuntimeBuilder() @@ -67,21 +77,20 @@ public async Task ManyRequestToPollingTentacles_Works_AndDoesNotUseTooManyResour { var listenPort = octopus.Listen(); octopus.Trust(Certificates.TentacleListening.Thumbprint); - - var _ = Task.Run(async () => + + var watchSubscriberCountCts = new CancelOnDisposeCancellationToken(CancellationToken); + watchSubscriberCountCts.AwaitTasksBeforeCTSDispose(Task.Run(async () => { - while (!CancellationToken.IsCancellationRequested) + while (!watchSubscriberCountCts.Token.IsCancellationRequested) { - GC.Collect(); Logger.Information("Total subscribers: {TotalSubs}", redisFacade.TotalSubscribers); - await Task.Delay(10000); + await Task.Delay(1000); } - }); + })); var serviceEndpoint = new ServiceEndPoint(new Uri("https://localhost:" + listenPort), Certificates.Octopus.Thumbprint, new HalibutTimeoutsAndLimitsForTestsBuilder().Build()); - - var pollEndpoints = Enumerable.Range(0, 100).Select(i => new Uri("poll://" + i + "Bob")).ToArray(); + var pollEndpoints = Enumerable.Range(0, numberOfPollingServices).Select(i => new Uri("poll://" + i + "Bob")).ToArray(); foreach (var pollEndpoint in pollEndpoints) { @@ -98,11 +107,9 @@ public async Task ManyRequestToPollingTentacles_Works_AndDoesNotUseTooManyResour .ToList(); var tasks = new List(); - - int concurrency = 20; - int limit = 20; - int total = concurrency * Math.Min(clients.Count, limit); - int callsMade = 0; + + int expectedTotalNumberOfCallsToBeMade = concurrency * numberOfCallsToMake; + int actualCountOfCallsMade = 0; var totalSw = Stopwatch.StartNew(); for (int i = 0; i < concurrency; i++) @@ -111,15 +118,15 @@ public async Task ManyRequestToPollingTentacles_Works_AndDoesNotUseTooManyResour { var shuffle = clients.ToArray(); Random.Shared.Shuffle(shuffle); - shuffle = shuffle.Take(limit).ToArray(); + shuffle = shuffle.Take(numberOfCallsToMake).ToArray(); foreach (var client in shuffle) { await client.SayHelloAsync("World"); - var v = Interlocked.Increment(ref callsMade); + var v = Interlocked.Increment(ref actualCountOfCallsMade); if (v % 5000 == 0) { var timePerCall = totalSw.ElapsedMilliseconds / v; - Logger.Information("Done: {CallsMade} / {Total} avg: {A}", v, total, timePerCall); + Logger.Information("Done: {CallsMade} / {Total} avg: {A}", v, expectedTotalNumberOfCallsToBeMade, timePerCall); } } @@ -132,13 +139,14 @@ public async Task ManyRequestToPollingTentacles_Works_AndDoesNotUseTooManyResour Logger.Information("Time was {T}", totalSw.ElapsedMilliseconds); - callsMade.Should().Be(total); + actualCountOfCallsMade.Should().Be(expectedTotalNumberOfCallsToBeMade); if(isRedis) { redisFacade.TotalSubscribers.Should().Be(pollEndpoints.Length); } + // Check for exceptions. foreach (var task in tasks) { await task; diff --git a/source/Halibut.Tests/Support/TestAttributes/AllQueuesTestCasesAttribute.cs b/source/Halibut.Tests/Support/TestAttributes/AllQueuesTestCasesAttribute.cs index 58d9d9257..924e2c2ef 100644 --- a/source/Halibut.Tests/Support/TestAttributes/AllQueuesTestCasesAttribute.cs +++ b/source/Halibut.Tests/Support/TestAttributes/AllQueuesTestCasesAttribute.cs @@ -29,10 +29,11 @@ public static IEnumerable GetEnumerator() #if NET8_0_OR_GREATER if (EnsureRedisIsAvailableSetupFixture.WillRunRedisTests) { - factories.Add(new PendingRequestQueueTestCase("Redis", () => new RedisPendingRequestQueueBuilder())); + + factories.Add(new PendingRequestQueueTestCase(PendingRequestQueueTestCase.RedisTestCaseName, () => new RedisPendingRequestQueueBuilder())); } #endif - factories.Add(new PendingRequestQueueTestCase("InMemory", () => new PendingRequestQueueBuilder())); + factories.Add(new PendingRequestQueueTestCase(PendingRequestQueueTestCase.InMemoryTestCaseName, () => new PendingRequestQueueBuilder())); return factories; } @@ -41,6 +42,11 @@ public static IEnumerable GetEnumerator() public class PendingRequestQueueTestCase { + + public static string RedisTestCaseName = "Redis"; + + public static string InMemoryTestCaseName = "InMemory"; + public readonly string Name; private Func BuilderBuilder { get; } From 3778dfd2aaab2ebfbb95f9df0d7bf5d7069e6aa5 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Wed, 20 Aug 2025 11:32:09 +1000 Subject: [PATCH 114/137] . --- .../Halibut.Tests/ManyPollingTentacleTests.cs | 17 +++++++---------- ...ingTentacleDequeuesRequestsInOrderFixture.cs | 1 - source/Halibut/DataStream.cs | 3 ++- 3 files changed, 9 insertions(+), 12 deletions(-) diff --git a/source/Halibut.Tests/ManyPollingTentacleTests.cs b/source/Halibut.Tests/ManyPollingTentacleTests.cs index bdbc26a5d..9e69ed52b 100644 --- a/source/Halibut.Tests/ManyPollingTentacleTests.cs +++ b/source/Halibut.Tests/ManyPollingTentacleTests.cs @@ -165,29 +165,26 @@ static DelegateServiceFactory GetDelegateServiceFactory() public class AsyncEchoServiceWithDelay : IAsyncEchoService { - public async Task LongRunningOperationAsync(CancellationToken cancellationToken) + public Task LongRunningOperationAsync(CancellationToken cancellationToken) { - await Task.Delay(10000, cancellationToken); - return 16; + throw new NotImplementedException(); } public async Task SayHelloAsync(string name, CancellationToken cancellationToken) { - await Task.Delay(0); + await Task.Delay(10, cancellationToken); return name + "..."; } - public async Task CrashAsync(CancellationToken cancellationToken) + public Task CrashAsync(CancellationToken cancellationToken) { - await Task.CompletedTask; - throw new DivideByZeroException(); + throw new NotImplementedException(); } - public async Task CountBytesAsync(DataStream dataStream, CancellationToken cancellationToken) + public Task CountBytesAsync(DataStream dataStream, CancellationToken cancellationToken) { - await Task.CompletedTask; - throw new Exception(); + throw new NotImplementedException(); } } } diff --git a/source/Halibut.Tests/PollingTentacleDequeuesRequestsInOrderFixture.cs b/source/Halibut.Tests/PollingTentacleDequeuesRequestsInOrderFixture.cs index ee7686dc0..b7a07a819 100644 --- a/source/Halibut.Tests/PollingTentacleDequeuesRequestsInOrderFixture.cs +++ b/source/Halibut.Tests/PollingTentacleDequeuesRequestsInOrderFixture.cs @@ -30,7 +30,6 @@ public async Task QueuedUpRequestsShouldBeDequeuedInOrder(ClientAndServiceTestCa { return new FuncPendingRequestQueueFactory(uri => { - // TODO: Test the new queue here. pendingRequestQueue = new PendingRequestQueueBuilder() .WithLog(logFactory.ForEndpoint(uri)) .WithPollingQueueWaitTimeout(TimeSpan.FromSeconds(1)) diff --git a/source/Halibut/DataStream.cs b/source/Halibut/DataStream.cs index 591ac23ef..51906c597 100644 --- a/source/Halibut/DataStream.cs +++ b/source/Halibut/DataStream.cs @@ -190,11 +190,12 @@ void IDataStreamInternal.Received(IDataStreamReceiver attachedReceiver) } /// - /// Be carefull + /// Used to re-hydrate deserialised data streams, which won't have a writer set. /// /// public void SetWriterAsync(Func writerAsync) { + if(this.writerAsync != null) throw new InvalidOperationException("Cannot set writer more than once."); this.writerAsync = writerAsync; } } From 4784065cd604e0f8c4a165740d28b02e7f86dfed Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Wed, 20 Aug 2025 11:34:23 +1000 Subject: [PATCH 115/137] . --- source/Halibut/Transport/Protocol/HalibutContractResolver.cs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/source/Halibut/Transport/Protocol/HalibutContractResolver.cs b/source/Halibut/Transport/Protocol/HalibutContractResolver.cs index 75aec0cbe..71cafcd25 100644 --- a/source/Halibut/Transport/Protocol/HalibutContractResolver.cs +++ b/source/Halibut/Transport/Protocol/HalibutContractResolver.cs @@ -12,6 +12,8 @@ public class HalibutContractResolver : DefaultContractResolver public override JsonContract ResolveContract(Type type) { + // Halibut supports sub classing of DataStream, over the wire we will send only the + // DataStream itself. if (typeof(DataStream).IsAssignableFrom(type)) { var contract = base.ResolveContract(typeof(DataStream)); From 11e6d186a76f57b9c31b784132ee557bfc24183e Mon Sep 17 00:00:00 2001 From: Rhys Parry Date: Wed, 20 Aug 2025 11:40:32 +1000 Subject: [PATCH 116/137] Nit cleanup of NodeHeartBeatWatcher --- .../Redis/NodeHeartBeat/NodeHeartBeatWatcher.cs | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/source/Halibut/Queue/Redis/NodeHeartBeat/NodeHeartBeatWatcher.cs b/source/Halibut/Queue/Redis/NodeHeartBeat/NodeHeartBeatWatcher.cs index c993f249a..a09033626 100644 --- a/source/Halibut/Queue/Redis/NodeHeartBeat/NodeHeartBeatWatcher.cs +++ b/source/Halibut/Queue/Redis/NodeHeartBeat/NodeHeartBeatWatcher.cs @@ -96,7 +96,6 @@ static async Task WatchForPulsesFromNode( while (!watchCancellationToken.IsCancellationRequested) { - var timeSinceLastHeartBeat = DateTimeOffset.Now - lastHeartBeat.Value; if (timeSinceLastHeartBeat > maxTimeBetweenHeartBeetsBeforeNodeIsAssumedToBeOffline) { @@ -121,9 +120,14 @@ static async Task WatchForPulsesFromNode( } } - static async Task WaitForRequestToBeCollected(Uri endpoint, RequestMessage request, RedisPendingRequest redisPending, IHalibutRedisTransport halibutRedisTransport, + static async Task WaitForRequestToBeCollected( + Uri endpoint, + RequestMessage request, + RedisPendingRequest redisPending, + IHalibutRedisTransport halibutRedisTransport, TimeSpan timeBetweenCheckingIfRequestWasCollected, - ILog log, CancellationToken cancellationToken) + ILog log, + CancellationToken cancellationToken) { log = log.ForContext(); log.Write(EventType.Diagnostic, "Waiting for request {0} to be collected from queue", request.ActivityId); @@ -133,16 +137,15 @@ static async Task WaitForRequestToBeCollected(Uri endpoint, RequestMessage reque try { // Has something else determined the request was collected? - if(redisPending.HasRequestBeenMarkedAsCollected) + if (redisPending.HasRequestBeenMarkedAsCollected) { log.Write(EventType.Diagnostic, "Request {0} has been marked as collected", request.ActivityId); return; } - // Check ourselves if the request has been collected. var requestIsStillOnQueue = await halibutRedisTransport.IsRequestStillOnQueue(endpoint, request.ActivityId, cancellationToken); - if(!requestIsStillOnQueue) + if (!requestIsStillOnQueue) { log.Write(EventType.Diagnostic, "Request {0} is no longer on queue", request.ActivityId); await redisPending.RequestHasBeenCollectedAndWillBeTransferred(); From 3643712231042934c27ed6fed454cc6da34e2863 Mon Sep 17 00:00:00 2001 From: Rhys Parry Date: Wed, 20 Aug 2025 11:42:26 +1000 Subject: [PATCH 117/137] Rename data loss detection namespace --- source/Halibut.Tests/ManyPollingTentacleTests.cs | 2 +- .../WatchForRedisLosingAllItsDataFixture.cs | 2 +- .../Utils/CancellableDataLossWatchForRedisLosingAllItsData.cs | 2 +- source/Halibut.Tests/Queue/Redis/Utils/RedisNeverLosesData.cs | 2 +- .../IWatchForRedisLosingAllItsData.cs | 2 +- .../WatchForRedisLosingAllItsData.cs | 2 +- source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs | 2 +- source/Halibut/Queue/Redis/RedisPendingRequestQueueFactory.cs | 2 +- 8 files changed, 8 insertions(+), 8 deletions(-) rename source/Halibut/Queue/Redis/{RedisDataLoseDetection => RedisDataLossDetection}/IWatchForRedisLosingAllItsData.cs (93%) rename source/Halibut/Queue/Redis/{RedisDataLoseDetection => RedisDataLossDetection}/WatchForRedisLosingAllItsData.cs (99%) diff --git a/source/Halibut.Tests/ManyPollingTentacleTests.cs b/source/Halibut.Tests/ManyPollingTentacleTests.cs index d8584811f..d9dfb1b21 100644 --- a/source/Halibut.Tests/ManyPollingTentacleTests.cs +++ b/source/Halibut.Tests/ManyPollingTentacleTests.cs @@ -12,7 +12,7 @@ using Halibut.Logging; using Halibut.Queue.QueuedDataStreams; using Halibut.Queue.Redis; -using Halibut.Queue.Redis.RedisDataLoseDetection; +using Halibut.Queue.Redis.RedisDataLossDetection; using Halibut.Queue.Redis.RedisHelpers; using Halibut.ServiceModel; using Halibut.Tests.Queue.Redis.Utils; diff --git a/source/Halibut.Tests/Queue/Redis/RedisDataLoseDetection/WatchForRedisLosingAllItsDataFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisDataLoseDetection/WatchForRedisLosingAllItsDataFixture.cs index 29dd52b51..011cfb64e 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisDataLoseDetection/WatchForRedisLosingAllItsDataFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisDataLoseDetection/WatchForRedisLosingAllItsDataFixture.cs @@ -2,7 +2,7 @@ using System; using System.Threading.Tasks; using FluentAssertions; -using Halibut.Queue.Redis.RedisDataLoseDetection; +using Halibut.Queue.Redis.RedisDataLossDetection; using Halibut.Tests.Queue.Redis.Utils; using Halibut.Tests.Support; using NUnit.Framework; diff --git a/source/Halibut.Tests/Queue/Redis/Utils/CancellableDataLossWatchForRedisLosingAllItsData.cs b/source/Halibut.Tests/Queue/Redis/Utils/CancellableDataLossWatchForRedisLosingAllItsData.cs index b8e57756b..1aa9f8f02 100644 --- a/source/Halibut.Tests/Queue/Redis/Utils/CancellableDataLossWatchForRedisLosingAllItsData.cs +++ b/source/Halibut.Tests/Queue/Redis/Utils/CancellableDataLossWatchForRedisLosingAllItsData.cs @@ -2,7 +2,7 @@ using System.Threading; using System.Threading.Tasks; using Halibut.Queue.Redis; -using Halibut.Queue.Redis.RedisDataLoseDetection; +using Halibut.Queue.Redis.RedisDataLossDetection; using Halibut.Util; using Try = Halibut.Tests.Support.Try; diff --git a/source/Halibut.Tests/Queue/Redis/Utils/RedisNeverLosesData.cs b/source/Halibut.Tests/Queue/Redis/Utils/RedisNeverLosesData.cs index b373c62cd..d8e16eb68 100644 --- a/source/Halibut.Tests/Queue/Redis/Utils/RedisNeverLosesData.cs +++ b/source/Halibut.Tests/Queue/Redis/Utils/RedisNeverLosesData.cs @@ -4,7 +4,7 @@ using System.Threading; using System.Threading.Tasks; using Halibut.Queue.Redis; -using Halibut.Queue.Redis.RedisDataLoseDetection; +using Halibut.Queue.Redis.RedisDataLossDetection; namespace Halibut.Tests.Queue.Redis.Utils { diff --git a/source/Halibut/Queue/Redis/RedisDataLoseDetection/IWatchForRedisLosingAllItsData.cs b/source/Halibut/Queue/Redis/RedisDataLossDetection/IWatchForRedisLosingAllItsData.cs similarity index 93% rename from source/Halibut/Queue/Redis/RedisDataLoseDetection/IWatchForRedisLosingAllItsData.cs rename to source/Halibut/Queue/Redis/RedisDataLossDetection/IWatchForRedisLosingAllItsData.cs index 2d873abcc..d45d49bf6 100644 --- a/source/Halibut/Queue/Redis/RedisDataLoseDetection/IWatchForRedisLosingAllItsData.cs +++ b/source/Halibut/Queue/Redis/RedisDataLossDetection/IWatchForRedisLosingAllItsData.cs @@ -2,7 +2,7 @@ using System.Threading; using System.Threading.Tasks; -namespace Halibut.Queue.Redis.RedisDataLoseDetection +namespace Halibut.Queue.Redis.RedisDataLossDetection { public interface IWatchForRedisLosingAllItsData : IAsyncDisposable { diff --git a/source/Halibut/Queue/Redis/RedisDataLoseDetection/WatchForRedisLosingAllItsData.cs b/source/Halibut/Queue/Redis/RedisDataLossDetection/WatchForRedisLosingAllItsData.cs similarity index 99% rename from source/Halibut/Queue/Redis/RedisDataLoseDetection/WatchForRedisLosingAllItsData.cs rename to source/Halibut/Queue/Redis/RedisDataLossDetection/WatchForRedisLosingAllItsData.cs index b09b46641..4d9ecead2 100644 --- a/source/Halibut/Queue/Redis/RedisDataLoseDetection/WatchForRedisLosingAllItsData.cs +++ b/source/Halibut/Queue/Redis/RedisDataLossDetection/WatchForRedisLosingAllItsData.cs @@ -6,7 +6,7 @@ using Halibut.Queue.Redis.RedisHelpers; using Halibut.Util; -namespace Halibut.Queue.Redis.RedisDataLoseDetection +namespace Halibut.Queue.Redis.RedisDataLossDetection { public class WatchForRedisLosingAllItsData : IWatchForRedisLosingAllItsData { diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index c13a376b8..3974e39a8 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -9,7 +9,7 @@ using Halibut.Queue.Redis.Exceptions; using Halibut.Queue.Redis.MessageStorage; using Halibut.Queue.Redis.NodeHeartBeat; -using Halibut.Queue.Redis.RedisDataLoseDetection; +using Halibut.Queue.Redis.RedisDataLossDetection; using Halibut.Queue.Redis.ResponseMessageTransfer; using Halibut.ServiceModel; using Halibut.Transport.Protocol; diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueueFactory.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueueFactory.cs index 7e86b869e..4d50bc983 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueueFactory.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueueFactory.cs @@ -5,7 +5,7 @@ using Halibut.Diagnostics; using Halibut.Queue.QueuedDataStreams; using Halibut.Queue.Redis.MessageStorage; -using Halibut.Queue.Redis.RedisDataLoseDetection; +using Halibut.Queue.Redis.RedisDataLossDetection; using Halibut.Queue.Redis.RedisHelpers; using Halibut.ServiceModel; From 22a8c6d9736250e11bc8909aa0de631abbebdfd9 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Wed, 20 Aug 2025 12:20:45 +1000 Subject: [PATCH 118/137] . --- .../Queue/Redis/NodeHeartBeat/NodeHeartBeatSenderFixture.cs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/source/Halibut.Tests/Queue/Redis/NodeHeartBeat/NodeHeartBeatSenderFixture.cs b/source/Halibut.Tests/Queue/Redis/NodeHeartBeat/NodeHeartBeatSenderFixture.cs index 2fb224301..3c5ec5057 100644 --- a/source/Halibut.Tests/Queue/Redis/NodeHeartBeat/NodeHeartBeatSenderFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/NodeHeartBeat/NodeHeartBeatSenderFixture.cs @@ -238,9 +238,9 @@ public async Task WhenWatchingTheNodeProcessingTheRequestIsStillAlive_AndTheWatc request, pendingRequest, unstableRedisTransport, - TimeSpan.FromSeconds(1), + timeBetweenCheckingIfRequestWasCollected: TimeSpan.FromSeconds(1), log, - TimeSpan.FromSeconds(5), // Short timeout for test + maxTimeBetweenHeartBeetsBeforeProcessingNodeIsAssumedToBeOffline: TimeSpan.FromSeconds(5), // Short timeout for test CancellationToken); // Wait for initial heartbeats to establish baseline @@ -251,7 +251,7 @@ public async Task WhenWatchingTheNodeProcessingTheRequestIsStillAlive_AndTheWatc portForwarder.EnterKillNewAndExistingConnectionsMode(); // Assert - await Task.WhenAny(Task.Delay(TimeSpan.FromSeconds(10)), watcherTask); + await Task.WhenAny(Task.Delay(TimeSpan.FromSeconds(20)), watcherTask); watcherTask.IsCompleted.Should().BeTrue("Since it should have detected no heart beats have been sent for some time."); var result = await watcherTask; result.Should().Be(NodeWatcherResult.NodeMayHaveDisconnected); From 4d5d7f22b4a131c16c0116694ee98870af17a72c Mon Sep 17 00:00:00 2001 From: Rhys Parry Date: Wed, 20 Aug 2025 13:04:34 +1000 Subject: [PATCH 119/137] Comment on QueueMessageSerializer origins --- source/Halibut/Queue/QueueMessageSerializer.cs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/source/Halibut/Queue/QueueMessageSerializer.cs b/source/Halibut/Queue/QueueMessageSerializer.cs index 52f544e21..12421fdb9 100644 --- a/source/Halibut/Queue/QueueMessageSerializer.cs +++ b/source/Halibut/Queue/QueueMessageSerializer.cs @@ -12,6 +12,8 @@ namespace Halibut.Queue /// Uses the same JSON serializer used by Halibut to send messages over the wire to /// serialise messages for the queue. Note that the queue serialises to JSON rather /// than BSON which is what is sent over the wire. + /// + /// Based on battle-tested MessageSerializer, any quirks may be inherited from there. /// public class QueueMessageSerializer { From 9cbad98f6648dcf0cf9e2a44ce892d38194af120 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Wed, 20 Aug 2025 13:34:38 +1000 Subject: [PATCH 120/137] Fix compilation on net48 --- source/Halibut.Tests/Halibut.Tests.csproj | 2 +- ...CancellationTokenSourceExtensionMethods.cs | 30 +++++++++++++++++++ 2 files changed, 31 insertions(+), 1 deletion(-) create mode 100644 source/Halibut.Tests/Util/CancellationTokenSourceExtensionMethods.cs diff --git a/source/Halibut.Tests/Halibut.Tests.csproj b/source/Halibut.Tests/Halibut.Tests.csproj index 9eb552ceb..602280ad8 100644 --- a/source/Halibut.Tests/Halibut.Tests.csproj +++ b/source/Halibut.Tests/Halibut.Tests.csproj @@ -7,7 +7,7 @@ true false 9.0 - VSTHRD002,VSTHRD003 + VSTHRD002,VSTHRD003;VSTHRD103 enable true diff --git a/source/Halibut.Tests/Util/CancellationTokenSourceExtensionMethods.cs b/source/Halibut.Tests/Util/CancellationTokenSourceExtensionMethods.cs new file mode 100644 index 000000000..27140b547 --- /dev/null +++ b/source/Halibut.Tests/Util/CancellationTokenSourceExtensionMethods.cs @@ -0,0 +1,30 @@ +// Copyright 2012-2013 Octopus Deploy Pty. Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +using System.Threading; +using System.Threading.Tasks; + +namespace Halibut.Tests.Util +{ + public static class CancellationTokenSourceExtensionMethods + { +#if NETFRAMEWORK + public static async Task CancelAsync(this CancellationTokenSource cts) + { + await Task.CompletedTask; + cts.Cancel(); + } +#endif + } +} \ No newline at end of file From f50beaf70ee2c072852c5ddb34503eb663bf3bdb Mon Sep 17 00:00:00 2001 From: Rhys Parry Date: Wed, 20 Aug 2025 14:03:19 +1000 Subject: [PATCH 121/137] Remove redundant overrideCancellationReason calls --- source/Halibut/Queue/Redis/RedisPendingRequest.cs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/source/Halibut/Queue/Redis/RedisPendingRequest.cs b/source/Halibut/Queue/Redis/RedisPendingRequest.cs index b5fc9988b..ff3cf2b27 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequest.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequest.cs @@ -142,10 +142,10 @@ public async Task WaitUntilComplete(Func checkIfPendingRequestWasCollected throw cancellationException; } - log.Write(EventType.MessageExchange, "Request {0} was cancelled before a response was received" + overrideCancellationReason()??"", request); + log.Write(EventType.MessageExchange, "Request {0} was cancelled before a response was received", request); SetResponseNoLock(ResponseMessage.FromException( request, - new TimeoutException($"A request was sent to a polling endpoint, the polling endpoint collected it but the request was cancelled before the polling endpoint responded." + overrideCancellationReason()??""), + new TimeoutException("A request was sent to a polling endpoint, the polling endpoint collected it but the request was cancelled before the polling endpoint responded."), ConnectionState.Connecting), requestWasCollected: false); await Try.IgnoringError(async () => await pendingRequestCancellationTokenSource.CancelAsync()); From 78f20cbdc17275baa19b6ca82935f23c1fda61b1 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Wed, 20 Aug 2025 14:08:34 +1000 Subject: [PATCH 122/137] . --- .../Util/CancelOnDisposeCancellationTokenFixture.cs | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/source/Halibut.Tests/Util/CancelOnDisposeCancellationTokenFixture.cs b/source/Halibut.Tests/Util/CancelOnDisposeCancellationTokenFixture.cs index 68fae676e..5b719c197 100644 --- a/source/Halibut.Tests/Util/CancelOnDisposeCancellationTokenFixture.cs +++ b/source/Halibut.Tests/Util/CancelOnDisposeCancellationTokenFixture.cs @@ -110,7 +110,6 @@ public async Task CancelAfter_ShouldCancelTokenAfterTimeout() public async Task AwaitTasksBeforeCTSDispose_ShouldWaitForTasksOnDispose() { // Arrange - var taskCompleted = false; var cancellationToken = new CancelOnDisposeCancellationToken(); var manualResetEvent = new AsyncManualResetEvent(); @@ -124,10 +123,9 @@ public async Task AwaitTasksBeforeCTSDispose_ShouldWaitForTasksOnDispose() manualResetEvent.Set(); await Task.WhenAny(Task.Delay(TimeSpan.FromSeconds(1)), Task.Run(async () => await disposeTask)); - await disposeTask; // Assert - taskCompleted.Should().BeTrue(); + disposeTask.IsCompleted.Should().BeTrue(); } [Test] From 9a7fe20e7ea67610fb4b5759b4f65fb6c469cff5 Mon Sep 17 00:00:00 2001 From: Rhys Parry Date: Wed, 20 Aug 2025 14:59:38 +1000 Subject: [PATCH 123/137] More data loss renames --- .../WatchForRedisLosingAllItsData.cs | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/source/Halibut/Queue/Redis/RedisDataLossDetection/WatchForRedisLosingAllItsData.cs b/source/Halibut/Queue/Redis/RedisDataLossDetection/WatchForRedisLosingAllItsData.cs index 4d9ecead2..4b2c12361 100644 --- a/source/Halibut/Queue/Redis/RedisDataLossDetection/WatchForRedisLosingAllItsData.cs +++ b/source/Halibut/Queue/Redis/RedisDataLossDetection/WatchForRedisLosingAllItsData.cs @@ -22,7 +22,7 @@ public class WatchForRedisLosingAllItsData : IWatchForRedisLosingAllItsData /// /// The amount of time between checks to check if redis has had data lose. /// - internal TimeSpan DataLoseCheckInterval { get; } + internal TimeSpan DataLossCheckInterval { get; } /// /// The TTL of the key used for data lose detection. The TTL is reset @@ -38,9 +38,9 @@ public WatchForRedisLosingAllItsData(RedisFacade redisFacade, ILog log, TimeSpan this.redisFacade = redisFacade; this.log = log; this.SetupErrorBackoffDelay = setupDelay ?? TimeSpan.FromSeconds(1); - this.DataLoseCheckInterval = watchInterval ?? TimeSpan.FromSeconds(60); + this.DataLossCheckInterval = watchInterval ?? TimeSpan.FromSeconds(60); this.DataLostKeyTtl = keyTTL ?? TimeSpan.FromHours(8); - var _ = Task.Run(async () => await KeepWatchingForDataLose(cts.Token)); + var _ = Task.Run(async () => await KeepWatchingForDataLoss(cts.Token)); } private TaskCompletionSource taskCompletionSource = new TaskCompletionSource(); @@ -63,15 +63,15 @@ public async Task GetTokenForDataLossDetection(TimeSpan timeT return await taskCompletionSource.Task.WaitAsync(cts.Token); } - private async Task KeepWatchingForDataLose(CancellationToken cancellationToken) + private async Task KeepWatchingForDataLoss(CancellationToken cancellationToken) { while (!cancellationToken.IsCancellationRequested) { - await Try.IgnoringError(async () => await WatchForDataLose(cancellationToken)); + await Try.IgnoringError(async () => await WatchForDataLoss(cancellationToken)); } } - async Task WatchForDataLose(CancellationToken cancellationToken) + async Task WatchForDataLoss(CancellationToken cancellationToken) { string guid = Guid.NewGuid().ToString(); var key = "WatchForDataLose::" + guid; @@ -115,7 +115,7 @@ async Task WatchForDataLose(CancellationToken cancellationToken) await Try.IgnoringError(async () => { if (!hasSetKey) await Task.Delay(SetupErrorBackoffDelay, cancellationToken); - else await Task.Delay(DataLoseCheckInterval, cancellationToken); + else await Task.Delay(DataLossCheckInterval, cancellationToken); }); } From 9e2b086f23b6675eae385b8fa52488a9bccc1685 Mon Sep 17 00:00:00 2001 From: Rhys Parry Date: Wed, 20 Aug 2025 15:32:40 +1000 Subject: [PATCH 124/137] Fix nits in RedisPendingRequestQueue --- .../Queue/Redis/RedisPendingRequestQueue.cs | 27 +++++++++---------- 1 file changed, 13 insertions(+), 14 deletions(-) diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index 3974e39a8..f9c1dc1e9 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -329,10 +329,9 @@ async Task TryClearRequestFromQueue(RequestMessage request, RedisPendingRe } catch (Exception ex) { - log.Write(EventType.Error, "Error deserializeing response for request {0}", activityId); + log.Write(EventType.Error, "Error deserializing response for request {0}", activityId); return ResponseMessage.FromException(requestMessage, new Exception("Error occured when reading data from the queue", ex)); } - } public async Task DequeueAsync(CancellationToken cancellationToken) @@ -373,13 +372,13 @@ public class WatcherAndDisposables : IAsyncDisposable { readonly DisposableCollection disposableCollection; public CancellationToken RequestCancelledForAnyReasonCancellationToken { get; } - public WatchForRequestCancellationOrSenderDisconnect watcher { get; } + public WatchForRequestCancellationOrSenderDisconnect Watcher { get; } public WatcherAndDisposables(DisposableCollection disposableCollection, CancellationToken requestCancelledForAnyReasonCancellationToken, WatchForRequestCancellationOrSenderDisconnect watcher) { this.disposableCollection = disposableCollection; this.RequestCancelledForAnyReasonCancellationToken = requestCancelledForAnyReasonCancellationToken; - this.watcher = watcher; + this.Watcher = watcher; } public async ValueTask DisposeAsync() @@ -416,7 +415,7 @@ public async Task ApplyResponse(ResponseMessage response, Guid requestActivityId if (watcherAndDisposables != null && watcherAndDisposables.RequestCancelledForAnyReasonCancellationToken.IsCancellationRequested) { // TODO: test - if (!watcherAndDisposables.watcher.SenderCancelledTheRequest) + if (!watcherAndDisposables.Watcher.SenderCancelledTheRequest) { log.Write(EventType.Diagnostic, "Response for request {0}, has been overridden with an abandon message as the request was abandoned", requestActivityId); response = ResponseMessage.FromException(response, new HalibutClientException(RequestAbandonedMessage)); @@ -434,26 +433,26 @@ public async Task ApplyResponse(ResponseMessage response, Guid requestActivityId } finally { - log.Write(EventType.Diagnostic, "Disposing in-flight request resources for request {0}", requestActivityId); - if (watcherAndDisposables != null) - { - await watcherAndDisposables.DisposeAsync(); - } + log.Write(EventType.Diagnostic, "Disposing in-flight request resources for request {0}", requestActivityId); + if (watcherAndDisposables != null) + { + await watcherAndDisposables.DisposeAsync(); + } } - } async Task DequeueNextAsync() { - await using var cts = new CancelOnDisposeCancellationToken(queueToken); try { hasItemsForEndpoint.Reset(); var first = await TryRemoveNextItemFromQueue(cts.Token); - if (first != null) return first; - + if (first != null) + { + return first; + } await Task.WhenAny( hasItemsForEndpoint.WaitAsync(cts.Token), From fd1412ece12292882229740c1389a56901854d27 Mon Sep 17 00:00:00 2001 From: Rhys Parry Date: Thu, 21 Aug 2025 09:37:32 +1000 Subject: [PATCH 125/137] fix formatting in MessageReaderWriterExtensionMethods --- .../Queue/Redis/Utils/MessageReaderWriterExtensionsMethods.cs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/Halibut.Tests/Queue/Redis/Utils/MessageReaderWriterExtensionsMethods.cs b/source/Halibut.Tests/Queue/Redis/Utils/MessageReaderWriterExtensionsMethods.cs index db70897e6..126313b2d 100644 --- a/source/Halibut.Tests/Queue/Redis/Utils/MessageReaderWriterExtensionsMethods.cs +++ b/source/Halibut.Tests/Queue/Redis/Utils/MessageReaderWriterExtensionsMethods.cs @@ -18,7 +18,7 @@ public static IMessageSerialiserAndDataStreamStorage ThrowsOnPrepareRequest(this { return new MessageSerialiserAndDataStreamStorageThatThrowsOnPrepareRequest(messageSerialiserAndDataStreamStorage, exception); } -} + } class MessageSerialiserAndDataStreamStorageWithVirtualMethods : IMessageSerialiserAndDataStreamStorage { From 80cb4551bee6b311a67743cf685d32290923d2ea Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Thu, 21 Aug 2025 10:41:39 +1000 Subject: [PATCH 126/137] . --- ...onReturnedByHalibutProxyExtensionMethod.cs | 4 +++ source/Halibut/HalibutRuntime.cs | 3 ++- .../Halibut/Queue/QueueMessageSerializer.cs | 3 +++ .../WatchForRedisLosingAllItsData.cs | 2 +- .../Queue/Redis/RedisPendingRequestQueue.cs | 16 ++++++------ .../Util/AwaitAllAndIgnoreException.cs | 26 ------------------- .../Util/CancelOnDisposeCancellationToken.cs | 12 ++++++--- 7 files changed, 27 insertions(+), 39 deletions(-) delete mode 100644 source/Halibut/Util/AwaitAllAndIgnoreException.cs diff --git a/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs b/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs index ca4785818..6747ae368 100644 --- a/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs +++ b/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs @@ -39,6 +39,10 @@ static bool IsRedisRetryableError(Exception exception) if (exception is HalibutClientException) { + // Sometimes the error occurs NOT on the node executing the RPC, e.g. the Node talking to tentacle. + // In that case we need to look at error messages, since we won't have the original exception type. + // We will also need to check error messages any time Error Responses are raised rather than a raw exception + // bubbling out of the QueueAndWait method. if (exception.Message.Contains("The request was abandoned, possibly because the node processing the request shutdown or redis lost all of its data.")) return true; if (exception.Message.Contains("The node processing the request did not send a heartbeat for long enough, and so the node is now assumed to be offline.")) return true; if (exception.Message.Contains("Error occured when reading data from the queue")) return true; diff --git a/source/Halibut/HalibutRuntime.cs b/source/Halibut/HalibutRuntime.cs index 12b98eadb..93d005c90 100644 --- a/source/Halibut/HalibutRuntime.cs +++ b/source/Halibut/HalibutRuntime.cs @@ -91,7 +91,8 @@ IPendingRequestQueue GetQueue(Uri target) var queue = queues.GetOrAdd(target, u => createdQueue = queueFactory.CreateQueue(target)); if (createdQueue != null && !ReferenceEquals(createdQueue, queue)) { - createdQueue.DisposeAsync(); + // We created a queue that won't be used, dispose of it in the background. + Task.Run(() => Try.IgnoringError(() => createdQueue.DisposeAsync())); } return queue; diff --git a/source/Halibut/Queue/QueueMessageSerializer.cs b/source/Halibut/Queue/QueueMessageSerializer.cs index 12421fdb9..8b55594bd 100644 --- a/source/Halibut/Queue/QueueMessageSerializer.cs +++ b/source/Halibut/Queue/QueueMessageSerializer.cs @@ -54,6 +54,9 @@ public QueueMessageSerializer(Func createStreamCa return (result.Message, streamCapturingSerializer.DataStreams); } + // This class is copied from `MessageSerializer`, since this class tries to + // use what it can from that battle tested class. That class had this envelope, + // and so shall we, since we don't want to learn the hard way why it was like that :D. // By making this a generic type, each message specifies the exact type it sends/expects // And it is impossible to deserialize the wrong type - any mismatched type will refuse to deserialize class MessageEnvelope diff --git a/source/Halibut/Queue/Redis/RedisDataLossDetection/WatchForRedisLosingAllItsData.cs b/source/Halibut/Queue/Redis/RedisDataLossDetection/WatchForRedisLosingAllItsData.cs index 4b2c12361..85755c3f5 100644 --- a/source/Halibut/Queue/Redis/RedisDataLossDetection/WatchForRedisLosingAllItsData.cs +++ b/source/Halibut/Queue/Redis/RedisDataLossDetection/WatchForRedisLosingAllItsData.cs @@ -74,7 +74,7 @@ private async Task KeepWatchingForDataLoss(CancellationToken cancellationToken) async Task WatchForDataLoss(CancellationToken cancellationToken) { string guid = Guid.NewGuid().ToString(); - var key = "WatchForDataLose::" + guid; + var key = "WatchForDataLoss::" + guid; var hasSetKey = false; log.Write(EventType.Diagnostic, "Starting Redis data loss monitoring with key {0}", key); diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index f9c1dc1e9..838eb822c 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -95,10 +95,10 @@ async Task DataLossCancellationToken(CancellationToken? cance public async Task QueueAndWaitAsync(RequestMessage request, CancellationToken requestCancellationToken) { - CancellationToken dataLoseCt; + CancellationToken dataLossCt; try { - dataLoseCt = await DataLossCancellationToken(requestCancellationToken); + dataLossCt = await DataLossCancellationToken(requestCancellationToken); } catch (Exception ex) { @@ -108,7 +108,7 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can Exception? CancellationReason() { - if (dataLoseCt.IsCancellationRequested) return new RedisDataLossHalibutClientException($"Request {request.ActivityId} was cancelled because we detected that redis lost all of its data."); + if (dataLossCt.IsCancellationRequested) return new RedisDataLossHalibutClientException($"Request {request.ActivityId} was cancelled because we detected that redis lost all of its data."); if (queueToken.IsCancellationRequested) return new RedisQueueShutdownClientException($"Request {request.ActivityId} was cancelled because the queue is shutting down."); return null; } @@ -120,7 +120,7 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can } - await using var cts = new CancelOnDisposeCancellationToken(queueCts.Token, requestCancellationToken, dataLoseCt); + await using var cts = new CancelOnDisposeCancellationToken(queueCts.Token, requestCancellationToken, dataLossCt); var cancellationToken = cts.Token; using var pending = new RedisPendingRequest(request, log); @@ -140,7 +140,7 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can // Start listening for a response to the request, we don't want to miss the response. await using var pollAndSubscribeToResponse = new PollAndSubscribeToResponse(endpoint, request.ActivityId, halibutRedisTransport, log); - var tryClearRequestFromQueueAtMostOnce = new AsyncLazy(async () => await TryClearRequestFromQueue(request, pending)); + var tryClearRequestFromQueueAtMostOnce = new AsyncLazy(async () => await TryClearRequestFromQueue(pending)); try { await using var senderPulse = new NodeHeartBeatSender(endpoint, request.ActivityId, halibutRedisTransport, log, HalibutQueueNodeSendingPulses.RequestSenderNode, RequestSenderNodeHeartBeatRate); @@ -265,8 +265,9 @@ void InBackgroundSendCancellationIfRequestWasCancelled(RequestMessage request, R cancellationToken); } - async Task TryClearRequestFromQueue(RequestMessage request, RedisPendingRequest redisPending) - { + async Task TryClearRequestFromQueue(RedisPendingRequest redisPending) + { + var request = redisPending.Request; log.Write(EventType.Diagnostic, "Attempting to clear request {0} from queue for endpoint {1}", request.ActivityId, endpoint); // The time the message is allowed to sit on the queue for has elapsed. @@ -414,7 +415,6 @@ public async Task ApplyResponse(ResponseMessage response, Guid requestActivityId if (watcherAndDisposables != null && watcherAndDisposables.RequestCancelledForAnyReasonCancellationToken.IsCancellationRequested) { - // TODO: test if (!watcherAndDisposables.Watcher.SenderCancelledTheRequest) { log.Write(EventType.Diagnostic, "Response for request {0}, has been overridden with an abandon message as the request was abandoned", requestActivityId); diff --git a/source/Halibut/Util/AwaitAllAndIgnoreException.cs b/source/Halibut/Util/AwaitAllAndIgnoreException.cs deleted file mode 100644 index 576072059..000000000 --- a/source/Halibut/Util/AwaitAllAndIgnoreException.cs +++ /dev/null @@ -1,26 +0,0 @@ -using System; -using System.Collections.Generic; -using System.Threading.Tasks; - -namespace Halibut.Util -{ - public class AwaitAllAndIgnoreException : IAsyncDisposable - { - List tasks = new List(); - - public void AddTasks(params Task[] tasksToAdd) - { - foreach (var task in tasksToAdd) - { -#pragma warning disable VSTHRD003 - tasks.Add(Try.IgnoringError(async () => await task)); -#pragma warning restore VSTHRD003 - } - } - - public async ValueTask DisposeAsync() - { - await Task.WhenAll(tasks); - } - } -} \ No newline at end of file diff --git a/source/Halibut/Util/CancelOnDisposeCancellationToken.cs b/source/Halibut/Util/CancelOnDisposeCancellationToken.cs index 3bcd04f22..1b4d4c637 100644 --- a/source/Halibut/Util/CancelOnDisposeCancellationToken.cs +++ b/source/Halibut/Util/CancelOnDisposeCancellationToken.cs @@ -1,5 +1,8 @@ #nullable enable using System; +using System.Collections.Concurrent; +using System.Collections.Generic; +using System.Linq; using System.Threading; using System.Threading.Tasks; @@ -31,7 +34,7 @@ public sealed class CancelOnDisposeCancellationToken : IAsyncDisposable readonly CancellationTokenSource cancellationTokenSource; bool disposed; - readonly AwaitAllAndIgnoreException awaitAllAndIgnoreException = new(); + readonly ConcurrentBag tasks = new(); public CancelOnDisposeCancellationToken(params CancellationToken[] token) : this(CancellationTokenSource.CreateLinkedTokenSource(token)) @@ -61,7 +64,7 @@ public async ValueTask DisposeAsync() await Try.IgnoringError(async () => await CancelAsync()); // Wait for any tasks that are using the token, before disposal - await Try.IgnoringError(async () => await awaitAllAndIgnoreException.DisposeAsync()); + await Task.WhenAll(tasks.Select(t => Try.IgnoringError(() => t))); Try.IgnoringError(() => cancellationTokenSource.Dispose()); } @@ -88,7 +91,10 @@ public void CancelAfter(TimeSpan timeSpan) /// public void AwaitTasksBeforeCTSDispose(params Task[] tasksUsingToken) { - awaitAllAndIgnoreException.AddTasks(tasksUsingToken); + foreach (var task in tasksUsingToken) + { + tasks.Add(task); + } } } } \ No newline at end of file From 14ae8810a059be3cd403b2cf7e74e4309df1c41c Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Thu, 21 Aug 2025 10:42:25 +1000 Subject: [PATCH 127/137] . --- source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index 838eb822c..b6cfe367b 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -81,7 +81,7 @@ public RedisPendingRequestQueue( this.halibutTimeoutsAndLimits = halibutTimeoutsAndLimits; this.queueToken = queueCts.Token; - // Ideally we would only subscribe subscribers are using this queue. + // Ideally we would only subscribe subscribers which are using this queue. RequestMessageAvailablePulseChannelSubscriberDisposer = Task.Run(async () => await this.halibutRedisTransport.SubscribeToRequestMessagePulseChannel(endpoint, _ => hasItemsForEndpoint.Set(), queueToken)); } From 8f1f96acba6dacd6cf895b085d7b922f06a1ce5d Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Thu, 21 Aug 2025 10:43:31 +1000 Subject: [PATCH 128/137] . --- .../ExceptionReturnedByHalibutProxyExtensionMethod.cs | 2 +- ...oRedisHalibutPendingRequestQueueHalibutClientException.cs} | 4 ++-- source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) rename source/Halibut/Queue/Redis/Exceptions/{ErrorOccuredWhenInsertingDataIntoRedisHalibutPendingRequestQueue.cs => ErrorOccuredWhenInsertingDataIntoRedisHalibutPendingRequestQueueHalibutClientException.cs} (57%) diff --git a/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs b/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs index 6747ae368..011eefa37 100644 --- a/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs +++ b/source/Halibut/Diagnostics/ExceptionReturnedByHalibutProxyExtensionMethod.cs @@ -32,7 +32,7 @@ static bool IsRedisRetryableError(Exception exception) || exception is RedisQueueShutdownClientException || exception is CouldNotGetDataLossTokenInTimeHalibutClientException || exception is ErrorWhilePreparingRequestForQueueHalibutClientException - || exception is ErrorOccuredWhenInsertingDataIntoRedisHalibutPendingRequestQueue) + || exception is ErrorOccuredWhenInsertingDataIntoRedisHalibutPendingRequestQueueHalibutClientException) { return true; } diff --git a/source/Halibut/Queue/Redis/Exceptions/ErrorOccuredWhenInsertingDataIntoRedisHalibutPendingRequestQueue.cs b/source/Halibut/Queue/Redis/Exceptions/ErrorOccuredWhenInsertingDataIntoRedisHalibutPendingRequestQueueHalibutClientException.cs similarity index 57% rename from source/Halibut/Queue/Redis/Exceptions/ErrorOccuredWhenInsertingDataIntoRedisHalibutPendingRequestQueue.cs rename to source/Halibut/Queue/Redis/Exceptions/ErrorOccuredWhenInsertingDataIntoRedisHalibutPendingRequestQueueHalibutClientException.cs index 511bda9e3..e06aff864 100644 --- a/source/Halibut/Queue/Redis/Exceptions/ErrorOccuredWhenInsertingDataIntoRedisHalibutPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/Exceptions/ErrorOccuredWhenInsertingDataIntoRedisHalibutPendingRequestQueueHalibutClientException.cs @@ -2,9 +2,9 @@ namespace Halibut.Queue.Redis.Exceptions { - public class ErrorOccuredWhenInsertingDataIntoRedisHalibutPendingRequestQueue : HalibutClientException + public class ErrorOccuredWhenInsertingDataIntoRedisHalibutPendingRequestQueueHalibutClientException : HalibutClientException { - public ErrorOccuredWhenInsertingDataIntoRedisHalibutPendingRequestQueue(string message, Exception inner) : base(message, inner) + public ErrorOccuredWhenInsertingDataIntoRedisHalibutPendingRequestQueueHalibutClientException(string message, Exception inner) : base(message, inner) { } } diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index b6cfe367b..9ed5fb2dc 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -154,7 +154,7 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can catch (Exception ex) { throw CreateCancellationExceptionIfCancelled() - ?? new ErrorOccuredWhenInsertingDataIntoRedisHalibutPendingRequestQueue($"Request {request.ActivityId} failed since an error occured inserting the data into the queue", ex); + ?? new ErrorOccuredWhenInsertingDataIntoRedisHalibutPendingRequestQueueHalibutClientException($"Request {request.ActivityId} failed since an error occured inserting the data into the queue", ex); } Interlocked.Increment(ref numberOfInFlightRequestsThatHaveReachedTheStageOfBeingReadyForCollection); From 129c1f30ed29f658bf9ed78ac36dc291f6b5a102 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Thu, 21 Aug 2025 10:48:42 +1000 Subject: [PATCH 129/137] . --- docs/RedisQueue.md | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/docs/RedisQueue.md b/docs/RedisQueue.md index 29ad1306b..710d70d21 100644 --- a/docs/RedisQueue.md +++ b/docs/RedisQueue.md @@ -60,8 +60,15 @@ At a high level steps the Redis Queue goes through to execute an RPC are: 10. Client B Pulses the `ResponseMessage channel` keyed by the RequestMessage `GUID`, that a Response is available. 11. Client A receives a pulse on the `ResponseMessage channel` and so knows a Response is available, it reads the response from Redis and returns from the `QueueAndWait()` method. +## Cancellation support. -## Pub/Sub and Poll. +The Redis PRQ supports cancellation, even for collected requests. This is done by the RequestReceiverNode (ie the node connected to the Service) subscribing to the request cancellation channel and polling for request cancellation. + +## Dealing with minor network interruptions to Redis. + +All operations to redis are retried for up to 30s, this allows connections to Redis to go down briefly with impacting RPCs even for non idempotent RPCs. + +### Pub/Sub and Poll. Since Pub/Sub does not have guaranteed delivery in Redis, in any place that we do Pub/Sub we must also have a form of polling. For example: - When Dequeuing work not only are we subscribed but when `Dequeue()` is called we also check for work on the queue anyway. (Note that Dequeue() returns every 30s if there is no work, and thus we have polling.) @@ -89,6 +96,14 @@ Since redis can lose data at anytime the queue is able to detect data lose and c Message serialisation is provided by re-using the serialiser halibut uses for transferring requests/responses over the wire. +## Cleanup of old data in Redis. + +All values in redis have a TTL applied, so redis will automatically clean up old keys if Halibut does not. + +Request message TTL: request pickup timeout + 2 minutes. +Response TTL: default 20 minutes. +Pending GUID list TTL: 1 day. +Heartbeat rates: 15s; timeouts: sender 90s, processor 60s. ### DataStream From 2c48c761eb1dbb6793c7471d5d3c41c456570b34 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Thu, 21 Aug 2025 10:49:13 +1000 Subject: [PATCH 130/137] . --- docs/RedisQueue.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/RedisQueue.md b/docs/RedisQueue.md index 710d70d21..2ed95bb8b 100644 --- a/docs/RedisQueue.md +++ b/docs/RedisQueue.md @@ -21,8 +21,6 @@ docker run -v `pwd`/redis-conf:/usr/local/etc/redis -p 6379:6379 --name redis -d Note that Redis is configured to have no backup, everything must be in memory. The queue makes this assumption to function. -## TODO design. - ### Context: Pending Request Queue. Halibut turns an RPC call into a RequestMessage which is placed into the Pending Request Queue. This is done by calling: `ResponseMessage QueueAndWait(RequestMessage)`. Which is a blocking call that queues the RequestMessage and waits for the ResponseMessage before returning. From dd82b2d7a88cbdd702b0506f6d7740a48801e556 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Thu, 21 Aug 2025 10:51:00 +1000 Subject: [PATCH 131/137] . --- docs/RedisQueue.md | 27 +++++++++++++++------------ 1 file changed, 15 insertions(+), 12 deletions(-) diff --git a/docs/RedisQueue.md b/docs/RedisQueue.md index 2ed95bb8b..51782e401 100644 --- a/docs/RedisQueue.md +++ b/docs/RedisQueue.md @@ -21,7 +21,10 @@ docker run -v `pwd`/redis-conf:/usr/local/etc/redis -p 6379:6379 --name redis -d Note that Redis is configured to have no backup, everything must be in memory. The queue makes this assumption to function. -### Context: Pending Request Queue. +# Design + +## Background +### What is a Pending Request Queue. Halibut turns an RPC call into a RequestMessage which is placed into the Pending Request Queue. This is done by calling: `ResponseMessage QueueAndWait(RequestMessage)`. Which is a blocking call that queues the RequestMessage and waits for the ResponseMessage before returning. @@ -29,7 +32,7 @@ Polling service, e.g, Tentacle, call into the `Dequeue` method of the queue to g The Redis Pending Request Queue solves the problem where we have multiple clients, that wish to execute RPC calls to a single Polling Service that is connected to exactly one client. For example Client A makes an RPC call, but the service is connected to Client B. The Redis Pending Request Queue is what moves the `RequestMessage` from Client A to Client B to be sent to the service. -### Context: Redis +### Redis specific details relevant to the queue. First we need to understand just a little about Redis and how we are using redis: - Redis may have data lose. @@ -47,16 +50,16 @@ At a high level steps the Redis Queue goes through to execute an RPC are: 1. Client B subscribes to the unique "RequestMessage Pulse Channel", as the client service is connected to it. The channel is keyed by the polling client id e.g. "poll://123" 2. Client A executes an RPC and so Calls QueueAndWait with a RequestMessage. Each RequestMessage has a unique `GUID`. - 2.1 Client A subscribes to the `ResponseMessage channel` keyed by `GUID` to be notified when a response is available. - 3. Client A serialises the message and places the message into a hash in Redis keyed by the RequestMessage `Guid`. - 4. Client A Adds the `GUID` to the polling clients unique Redis list (aka queue). The key is the polling client id e.g. "poll://123". - 5. Client A pulses the polling clients unique "RequestMessage Pulse Channel", to alert to it that it has work to do. - 6. Client B receives the Pulse message and tries to dequeue a `GUID` from the polling clients unique Redis list (aka queue). - 7. Client B now has the `GUID` of the request and so atomically gets and deletes the RequestMessage from the Redis Hash using that guid. - 8. Client B sends the request to the tentacle, waits for the response, and calls `ApplyResponse()` with the ResponseMessage. - 9. Client B writes the `ResponseMessage` to redis in a hash using the `GUID` as the key. - 10. Client B Pulses the `ResponseMessage channel` keyed by the RequestMessage `GUID`, that a Response is available. - 11. Client A receives a pulse on the `ResponseMessage channel` and so knows a Response is available, it reads the response from Redis and returns from the `QueueAndWait()` method. + 3. Client A subscribes to the `ResponseMessage channel` keyed by `GUID` to be notified when a response is available. + 4. Client A serialises the message and places the message into a hash in Redis keyed by the RequestMessage `Guid`. + 5. Client A Adds the `GUID` to the polling clients unique Redis list (aka queue). The key is the polling client id e.g. "poll://123". + 6. Client A pulses the polling clients unique "RequestMessage Pulse Channel", to alert to it that it has work to do. + 7. Client B receives the Pulse message and tries to dequeue a `GUID` from the polling clients unique Redis list (aka queue). + 8. Client B now has the `GUID` of the request and so atomically gets and deletes the RequestMessage from the Redis Hash using that guid. + 9. Client B sends the request to the tentacle, waits for the response, and calls `ApplyResponse()` with the ResponseMessage. + 10. Client B writes the `ResponseMessage` to redis in a hash using the `GUID` as the key. + 11. Client B Pulses the `ResponseMessage channel` keyed by the RequestMessage `GUID`, that a Response is available. + 12. Client A receives a pulse on the `ResponseMessage channel` and so knows a Response is available, it reads the response from Redis and returns from the `QueueAndWait()` method. ## Cancellation support. From d86653a3e075c9215a0008e846fc861e3c00b12e Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Thu, 21 Aug 2025 13:13:23 +1000 Subject: [PATCH 132/137] Geoff is willing to risk it --- source/Halibut/Queue/QueueMessageSerializer.cs | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/source/Halibut/Queue/QueueMessageSerializer.cs b/source/Halibut/Queue/QueueMessageSerializer.cs index 8b55594bd..5e3858571 100644 --- a/source/Halibut/Queue/QueueMessageSerializer.cs +++ b/source/Halibut/Queue/QueueMessageSerializer.cs @@ -53,17 +53,17 @@ public QueueMessageSerializer(Func createStreamCa return (result.Message, streamCapturingSerializer.DataStreams); } - - // This class is copied from `MessageSerializer`, since this class tries to - // use what it can from that battle tested class. That class had this envelope, - // and so shall we, since we don't want to learn the hard way why it was like that :D. + // By making this a generic type, each message specifies the exact type it sends/expects // And it is impossible to deserialize the wrong type - any mismatched type will refuse to deserialize class MessageEnvelope { -#pragma warning disable CS8618 // Non-nullable field must contain a non-null value when exiting constructor. Consider declaring as nullable. - public T Message { get; set; } -#pragma warning restore CS8618 // Non-nullable field must contain a non-null value when exiting constructor. Consider declaring as nullable. + public MessageEnvelope(T message) + { + Message = message; + } + + public T Message { get; private set; } } } } \ No newline at end of file From 96c42324c88c7d3079b5d1890637946f36d99a5a Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Thu, 21 Aug 2025 13:15:27 +1000 Subject: [PATCH 133/137] . --- source/Halibut/Queue/QueueMessageSerializer.cs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/Halibut/Queue/QueueMessageSerializer.cs b/source/Halibut/Queue/QueueMessageSerializer.cs index 5e3858571..7e663dfc9 100644 --- a/source/Halibut/Queue/QueueMessageSerializer.cs +++ b/source/Halibut/Queue/QueueMessageSerializer.cs @@ -33,7 +33,7 @@ public QueueMessageSerializer(Func createStreamCa using (var jsonTextWriter = new JsonTextWriter(sw) { CloseOutput = false }) { var streamCapturingSerializer = createStreamCapturingSerializer(); - streamCapturingSerializer.Serializer.Serialize(jsonTextWriter, new MessageEnvelope { Message = message! }); + streamCapturingSerializer.Serializer.Serialize(jsonTextWriter, new MessageEnvelope(message)); dataStreams = streamCapturingSerializer.DataStreams; } From c6d3fa5caa81acaf13ab6a56018266c13d9553f0 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Thu, 21 Aug 2025 14:00:25 +1000 Subject: [PATCH 134/137] Don't immediatly poll for a response --- .../PollAndSubscribeToResponse.cs | 17 +++++++++++------ 1 file changed, 11 insertions(+), 6 deletions(-) diff --git a/source/Halibut/Queue/Redis/ResponseMessageTransfer/PollAndSubscribeToResponse.cs b/source/Halibut/Queue/Redis/ResponseMessageTransfer/PollAndSubscribeToResponse.cs index c77c996a3..691e22cbf 100644 --- a/source/Halibut/Queue/Redis/ResponseMessageTransfer/PollAndSubscribeToResponse.cs +++ b/source/Halibut/Queue/Redis/ResponseMessageTransfer/PollAndSubscribeToResponse.cs @@ -64,11 +64,22 @@ async Task WaitForResponse(CancellationToken token) log.Write(EventType.Diagnostic, "Starting polling loop for response - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); + // We actually want a delay before we actually have a go at polling for the response, since it makes + // no sense to send a Request and expect an immediate reply. + pollBackoffStrategy.Try(); + // Also poll to see if the value is set since we can miss the publication. while (!token.IsCancellationRequested) { + var delay = pollBackoffStrategy.GetSleepPeriod(); + log.Write(EventType.Diagnostic, "Waiting {0} seconds before next poll for response - Endpoint: {1}, ActivityId: {2}", delay.TotalSeconds, endpoint, activityId); + await Try.IgnoringError(async () => await Task.Delay(delay, token)); + if(token.IsCancellationRequested) break; + log.Write(EventType.Diagnostic, "Done waiting going to poll for response - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); + try { + pollBackoffStrategy.Try(); if (await TryGetResponseFromRedis("polling", token)) { break; @@ -78,12 +89,6 @@ async Task WaitForResponse(CancellationToken token) { log.Write(EventType.Diagnostic, "Error while polling for response - Endpoint: {0}, ActivityId: {1}, Error: {2}", endpoint, activityId, ex.Message); } - - pollBackoffStrategy.Try(); - var delay = pollBackoffStrategy.GetSleepPeriod(); - log.Write(EventType.Diagnostic, "Waiting {0} seconds before next poll for response - Endpoint: {1}, ActivityId: {2}", delay.TotalSeconds, endpoint, activityId); - await Try.IgnoringError(async () => await Task.Delay(delay, token)); - log.Write(EventType.Diagnostic, "Done waiting going around the loop response - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); } log.Write(EventType.Diagnostic, "Exiting watch loop for response - Endpoint: {0}, ActivityId: {1}", endpoint, activityId); From dd989f2a91a23dee6c75a2a2b29eb25cab4af7bd Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Thu, 21 Aug 2025 14:03:12 +1000 Subject: [PATCH 135/137] Don't immediatly poll for request cancellation --- .../Queue/Redis/Cancellation/WatchForRequestCancellation.cs | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/source/Halibut/Queue/Redis/Cancellation/WatchForRequestCancellation.cs b/source/Halibut/Queue/Redis/Cancellation/WatchForRequestCancellation.cs index 3ed4849bc..b7915c347 100644 --- a/source/Halibut/Queue/Redis/Cancellation/WatchForRequestCancellation.cs +++ b/source/Halibut/Queue/Redis/Cancellation/WatchForRequestCancellation.cs @@ -48,6 +48,10 @@ async Task WatchForCancellation(Uri endpoint, Guid requestActivityId, IHalibutRe // Also poll to see if the request is cancelled since we can miss the publication. while (!token.IsCancellationRequested) { + await Try.IgnoringError(async () => await Task.Delay(TimeSpan.FromSeconds(60), token)); + + if(token.IsCancellationRequested) return; + try { if (await halibutRedisTransport.IsRequestMarkedAsCancelled(endpoint, requestActivityId, token)) @@ -61,7 +65,6 @@ async Task WatchForCancellation(Uri endpoint, Guid requestActivityId, IHalibutRe { log.Write(EventType.Diagnostic, "Error while polling for request cancellation - Endpoint: {0}, ActivityId: {1}, Error: {2}", endpoint, requestActivityId, ex.Message); } - await Try.IgnoringError(async () => await Task.Delay(TimeSpan.FromSeconds(60), token)); } log.Write(EventType.Diagnostic, "Exiting watch loop for request cancellation - Endpoint: {0}, ActivityId: {1}", endpoint, requestActivityId); From 9349e6410de40c8ba457e8bb00dddacb27b7f4a0 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Thu, 21 Aug 2025 14:05:36 +1000 Subject: [PATCH 136/137] Don't immediatly check the request was collected --- .../Redis/NodeHeartBeat/NodeHeartBeatWatcher.cs | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/source/Halibut/Queue/Redis/NodeHeartBeat/NodeHeartBeatWatcher.cs b/source/Halibut/Queue/Redis/NodeHeartBeat/NodeHeartBeatWatcher.cs index a09033626..5d552e42b 100644 --- a/source/Halibut/Queue/Redis/NodeHeartBeat/NodeHeartBeatWatcher.cs +++ b/source/Halibut/Queue/Redis/NodeHeartBeat/NodeHeartBeatWatcher.cs @@ -134,6 +134,15 @@ static async Task WaitForRequestToBeCollected( while (!cancellationToken.IsCancellationRequested) { + await Try.IgnoringError(async () => + { + await Task.WhenAny( + Task.Delay(timeBetweenCheckingIfRequestWasCollected, cancellationToken), + redisPending.WaitForRequestToBeMarkedAsCollected(cancellationToken)); + }); + + if(cancellationToken.IsCancellationRequested) break; + try { // Has something else determined the request was collected? @@ -156,13 +165,6 @@ static async Task WaitForRequestToBeCollected( { log.WriteException(EventType.Diagnostic, "Error checking if request {0} is still on queue", ex, request.ActivityId); } - - await Try.IgnoringError(async () => - { - await Task.WhenAny( - Task.Delay(timeBetweenCheckingIfRequestWasCollected, cancellationToken), - redisPending.WaitForRequestToBeMarkedAsCollected(cancellationToken)); - }); } log.Write(EventType.Diagnostic, "Stopped waiting for request {0} to be collected (cancelled)", request.ActivityId); From 7713ea06287e48f2931d2b8561f71610f03b5bf5 Mon Sep 17 00:00:00 2001 From: Luke Butters Date: Thu, 21 Aug 2025 14:30:27 +1000 Subject: [PATCH 137/137] Data Store now has MetaData --- .../Redis/RedisHelpers/RedisFacadeFixture.cs | 52 +++++++++------- .../RedisFacadeWhenRedisGoesDownAwayTests.cs | 19 +++--- .../Redis/RedisPendingRequestQueueFixture.cs | 2 +- .../HalibutRedisTransportWithVirtuals.cs | 11 ++-- ...oryStoreDataStreamsForDistributedQueues.cs | 6 +- .../MessageReaderWriterExtensionsMethods.cs | 14 ++--- .../IStoreDataStreamsForDistributedQueues.cs | 10 ++-- .../IMessageSerialiserAndDataStreamStorage.cs | 9 +-- .../MessageSerialiserAndDataStreamStorage.cs | 29 ++++----- .../RedisHelpers/HalibutRedisTransport.cs | 60 ++++++++++++++----- .../RedisHelpers/IHalibutRedisTransport.cs | 11 ++-- .../Queue/Redis/RedisHelpers/RedisFacade.cs | 57 ++++++++++++++---- .../Redis/RedisHelpers/RedisStoredMessage.cs | 15 +++++ .../Queue/Redis/RedisPendingRequestQueue.cs | 17 +++--- .../PollAndSubscribeToResponse.cs | 7 ++- .../ResponseMessageSender.cs | 3 +- 16 files changed, 216 insertions(+), 106 deletions(-) create mode 100644 source/Halibut/Queue/Redis/RedisHelpers/RedisStoredMessage.cs diff --git a/source/Halibut.Tests/Queue/Redis/RedisHelpers/RedisFacadeFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisHelpers/RedisFacadeFixture.cs index 108ec76fb..70d03cd17 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisHelpers/RedisFacadeFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisHelpers/RedisFacadeFixture.cs @@ -57,13 +57,15 @@ public async Task SetInHash_ShouldStoreValueInHash() var key = Guid.NewGuid().ToString(); var field = "test-field"; var payload = "test-payload"; + var values = new Dictionary { { field, payload } }; // Act - await redisFacade.SetInHash(key, field, payload, TimeSpan.FromMinutes(1), CancellationToken); + await redisFacade.SetInHash(key, values, TimeSpan.FromMinutes(1), CancellationToken); // Assert - We'll verify by trying to get and delete it - var retrievedValue = await redisFacade.TryGetAndDeleteFromHash(key, field, CancellationToken); - retrievedValue.Should().Be(payload); + var retrievedValues = await redisFacade.TryGetAndDeleteFromHash(key, new[] { field }, CancellationToken); + retrievedValues.Should().NotBeNull(); + retrievedValues![field].Should().Be(payload); } [Test] @@ -74,14 +76,16 @@ public async Task TryGetAndDeleteFromHash_WithExistingValue_ShouldReturnValueAnd var key = Guid.NewGuid().ToString(); var field = "test-field"; var payload = "test-payload"; + var values = new Dictionary { { field, payload } }; - await redisFacade.SetInHash(key, field, payload, TimeSpan.FromMinutes(1), CancellationToken); + await redisFacade.SetInHash(key, values, TimeSpan.FromMinutes(1), CancellationToken); // Act - var retrievedValue = await redisFacade.TryGetAndDeleteFromHash(key, field, CancellationToken); + var retrievedValues = await redisFacade.TryGetAndDeleteFromHash(key, new[] { field }, CancellationToken); // Assert - retrievedValue.Should().Be(payload); + retrievedValues.Should().NotBeNull(); + retrievedValues![field].Should().Be(payload); } [Test] @@ -92,8 +96,9 @@ public async Task HashContainsKey_WithExistingField_ShouldReturnTrue() var key = Guid.NewGuid().ToString(); var field = "test-field"; var payload = "test-payload"; + var values = new Dictionary { { field, payload } }; - await redisFacade.SetInHash(key, field, payload, TimeSpan.FromMinutes(1), CancellationToken); + await redisFacade.SetInHash(key, values, TimeSpan.FromMinutes(1), CancellationToken); // Act var exists = await redisFacade.HashContainsKey(key, field, CancellationToken); @@ -140,25 +145,27 @@ public async Task TryGetAndDeleteFromHash_ShouldDeleteTheEntireKey() var key = Guid.NewGuid().ToString(); var field = "test-field"; var payload = "test-payload"; + var values = new Dictionary { { field, payload } }; - await redisFacade.SetInHash(key, field, payload, TimeSpan.FromMinutes(1), CancellationToken); + await redisFacade.SetInHash(key, values, TimeSpan.FromMinutes(1), CancellationToken); // Verify the hash field exists var existsBefore = await redisFacade.HashContainsKey(key, field, CancellationToken); existsBefore.Should().BeTrue(); // Act - var retrievedValue = await redisFacade.TryGetAndDeleteFromHash(key, field, CancellationToken); + var retrievedValues = await redisFacade.TryGetAndDeleteFromHash(key, new[] { field }, CancellationToken); // Assert - retrievedValue.Should().Be(payload); + retrievedValues.Should().NotBeNull(); + retrievedValues![field].Should().Be(payload); // Verify the entire key was deleted (not just the field) var existsAfter = await redisFacade.HashContainsKey(key, field, CancellationToken); existsAfter.Should().BeFalse(); // Verify trying to get it again returns null - var secondRetrieval = await redisFacade.TryGetAndDeleteFromHash(key, field, CancellationToken); + var secondRetrieval = await redisFacade.TryGetAndDeleteFromHash(key, new[] { field }, CancellationToken); secondRetrieval.Should().BeNull(); } @@ -333,20 +340,22 @@ public async Task SetInHash_WithTTL_ShouldExpireAfterSpecifiedTime() var key = Guid.NewGuid().ToString(); var field = "test-field"; var payload = "test-payload"; + var values = new Dictionary { { field, payload } }; // Act - Set a value in hash with short TTL that we can actually test - await redisFacade.SetInHash(key, field, payload, TimeSpan.FromMinutes(3), CancellationToken); + await redisFacade.SetInHash(key, values, TimeSpan.FromMinutes(3), CancellationToken); // Immediately verify it exists var immediateExists = await redisFacade.HashContainsKey(key, field, CancellationToken); immediateExists.Should().BeTrue(); // Also verify we can retrieve the value immediately - var immediateValue = await redisFacade.TryGetAndDeleteFromHash(key, field, CancellationToken); - immediateValue.Should().Be(payload); + var immediateValues = await redisFacade.TryGetAndDeleteFromHash(key, new[] { field }, CancellationToken); + immediateValues.Should().NotBeNull(); + immediateValues![field].Should().Be(payload); // Set the value again to test expiration (since TryGetAndDeleteFromHash removes it) - await redisFacade.SetInHash(key, field, payload, TimeSpan.FromMilliseconds(3), CancellationToken); + await redisFacade.SetInHash(key, values, TimeSpan.FromMilliseconds(3), CancellationToken); // Assert - Should eventually expire await ShouldEventually.Eventually(async () => @@ -356,8 +365,8 @@ await ShouldEventually.Eventually(async () => }, TimeSpan.FromSeconds(5), CancellationToken); // Verify TryGetAndDeleteFromHash also returns null for expired key - var expiredValue = await redisFacade.TryGetAndDeleteFromHash(key, field, CancellationToken); - expiredValue.Should().BeNull(); + var expiredValues = await redisFacade.TryGetAndDeleteFromHash(key, new[] { field }, CancellationToken); + expiredValues.Should().BeNull(); } [Test] @@ -554,22 +563,23 @@ public async Task TryGetAndDeleteFromHash_WithConcurrentCalls_ShouldReturnValueT var key = Guid.NewGuid().ToString(); var field = "test-field"; var payload = "test-payload"; + var values = new Dictionary { { field, payload } }; const int concurrentCallCount = 20; // Set a value in the hash - await redisFacade.SetInHash(key, field, payload, TimeSpan.FromMinutes(1), CancellationToken); + await redisFacade.SetInHash(key, values, TimeSpan.FromMinutes(1), CancellationToken); var countDownLatch = new AsyncCountdownEvent(concurrentCallCount); // Act - Make multiple concurrent calls to TryGetAndDeleteFromHash - var concurrentTasks = new Task[concurrentCallCount]; + var concurrentTasks = new Task?>[concurrentCallCount]; for (int i = 0; i < concurrentCallCount; i++) { concurrentTasks[i] = Task.Run(async () => { countDownLatch.Signal(); await countDownLatch.WaitAsync(); - return await redisFacade.TryGetAndDeleteFromHash(key, field, CancellationToken); + return await redisFacade.TryGetAndDeleteFromHash(key, new[] { field }, CancellationToken); }); } @@ -580,7 +590,7 @@ public async Task TryGetAndDeleteFromHash_WithConcurrentCalls_ShouldReturnValueT var nullResults = results.Where(result => result == null).ToArray(); nonNullResults.Should().HaveCount(1, "exactly one concurrent call should retrieve the value"); - nonNullResults[0].Should().Be(payload, "the successful call should return the correct payload"); + nonNullResults[0]![field].Should().Be(payload, "the successful call should return the correct payload"); nullResults.Should().HaveCount(concurrentCallCount - 1, "all other concurrent calls should return null"); // Verify the hash key no longer exists diff --git a/source/Halibut.Tests/Queue/Redis/RedisHelpers/RedisFacadeWhenRedisGoesDownAwayTests.cs b/source/Halibut.Tests/Queue/Redis/RedisHelpers/RedisFacadeWhenRedisGoesDownAwayTests.cs index 84f59be21..9a4c1fb0f 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisHelpers/RedisFacadeWhenRedisGoesDownAwayTests.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisHelpers/RedisFacadeWhenRedisGoesDownAwayTests.cs @@ -1,6 +1,7 @@ #if NET8_0_OR_GREATER using System; using System.Collections.Concurrent; +using System.Collections.Generic; using System.Threading.Tasks; using FluentAssertions; using Halibut.Tests.Queue.Redis.Utils; @@ -74,11 +75,13 @@ public async Task WhenTheEstablishedConnectionToRedisBrieflyGoesDown_WeCanImmedi portForwarder.ReturnToNormalMode(); // Assert - await redisFacade.SetInHash("test-hash", "test-field", "test-value", TimeSpan.FromMinutes(1), CancellationToken); + await redisFacade.SetInHash("test-hash", new Dictionary(){{"test-field", "test-value"}}, TimeSpan.FromMinutes(1), CancellationToken); // Check that the value was set. - var retrievedValue = await redisFacade.TryGetAndDeleteFromHash("test-hash", "test-field", CancellationToken); - retrievedValue.Should().Be("test-value"); + var retrievedValue = await redisFacade.TryGetAndDeleteFromHash("test-hash", new []{"test-field"}, CancellationToken); + retrievedValue.Should().NotBeNull(); + retrievedValue.Should().ContainKey("test-field"); + retrievedValue!["test-field"].Should().Be("test-value"); } [Test] @@ -89,13 +92,15 @@ public async Task WhenTheEstablishedConnectionToRedisBrieflyGoesDown_WeCanImmedi await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder); // Establish connection and set up test data - await redisFacade.SetInHash("test-hash", "test-field", "test-value", TimeSpan.FromMinutes(1), CancellationToken); + await redisFacade.SetInHash("test-hash", new Dictionary(){{"test-field", "test-value"}}, TimeSpan.FromMinutes(1), CancellationToken); portForwarder.EnterKillNewAndExistingConnectionsMode(); portForwarder.ReturnToNormalMode(); - var result = await redisFacade.TryGetAndDeleteFromHash("test-hash", "test-field", CancellationToken); - result.Should().Be("test-value"); + var retrievedValue = await redisFacade.TryGetAndDeleteFromHash("test-hash", new []{"test-field"}, CancellationToken); + retrievedValue.Should().NotBeNull(); + retrievedValue.Should().ContainKey("test-field"); + retrievedValue!["test-field"].Should().Be("test-value"); } [Test] @@ -180,7 +185,7 @@ public async Task WhenTheEstablishedConnectionToRedisBrieflyGoesDown_WeCanImmedi await using var redisFacade = RedisFacadeBuilder.CreateRedisFacade(portForwarder); // Establish connection and set up test data - await redisFacade.SetInHash("test-hash", "test-field", "test-value", TimeSpan.FromMinutes(1), CancellationToken); + await redisFacade.SetInHash("test-hash", new Dictionary(){{"test-field", "test-value"}}, TimeSpan.FromMinutes(1), CancellationToken); portForwarder.EnterKillNewAndExistingConnectionsMode(); portForwarder.ReturnToNormalMode(); diff --git a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs index b068c96aa..0c528845f 100644 --- a/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs +++ b/source/Halibut.Tests/Queue/Redis/RedisPendingRequestQueueFixture.cs @@ -217,7 +217,7 @@ public async Task WhenEnteringTheQueue_AndRedisIsUnavailableAndDataLoseOccurs_AR var redisDataLoseDetector = new CancellableDataLossWatchForRedisLosingAllItsData(); var redisTransport = Substitute.ForPartsOf(new HalibutRedisTransport(redisFacade)); - redisTransport.Configure().PutRequest(Arg.Any(), Arg.Any(), Arg.Any(), Arg.Any(), Arg.Any()) + redisTransport.Configure().PutRequest(Arg.Any(), Arg.Any(), Arg.Any(), Arg.Any(), Arg.Any()) .Returns(async callInfo => { await redisDataLoseDetector.DataLossHasOccured(); diff --git a/source/Halibut.Tests/Queue/Redis/Utils/HalibutRedisTransportWithVirtuals.cs b/source/Halibut.Tests/Queue/Redis/Utils/HalibutRedisTransportWithVirtuals.cs index 2cd5b1f6d..b5577513c 100644 --- a/source/Halibut.Tests/Queue/Redis/Utils/HalibutRedisTransportWithVirtuals.cs +++ b/source/Halibut.Tests/Queue/Redis/Utils/HalibutRedisTransportWithVirtuals.cs @@ -5,6 +5,7 @@ using System.Threading.Tasks; using Halibut.Queue.Redis; using Halibut.Queue.Redis.NodeHeartBeat; +using Halibut.Queue.Redis.RedisHelpers; using StackExchange.Redis; namespace Halibut.Tests.Queue.Redis.Utils @@ -38,12 +39,12 @@ public Task PushRequestGuidOnToQueue(Uri endpoint, Guid guid, CancellationToken return halibutRedisTransport.TryPopNextRequestGuid(endpoint, cancellationToken); } - public virtual Task PutRequest(Uri endpoint, Guid requestId, string requestMessage, TimeSpan requestPickupTimeout, CancellationToken cancellationToken) + public virtual Task PutRequest(Uri endpoint, Guid requestId, RedisStoredMessage requestMessage, TimeSpan requestPickupTimeout, CancellationToken cancellationToken) { return halibutRedisTransport.PutRequest(endpoint, requestId, requestMessage, requestPickupTimeout, cancellationToken); } - public Task TryGetAndRemoveRequest(Uri endpoint, Guid requestId, CancellationToken cancellationToken) + public Task TryGetAndRemoveRequest(Uri endpoint, Guid requestId, CancellationToken cancellationToken) { return halibutRedisTransport.TryGetAndRemoveRequest(endpoint, requestId, cancellationToken); } @@ -93,17 +94,17 @@ public Task PublishThatResponseIsAvailable(Uri endpoint, Guid identifier, Cancel return halibutRedisTransport.PublishThatResponseIsAvailable(endpoint, identifier, cancellationToken); } - public Task SetResponseMessage(Uri endpoint, Guid identifier, string responseMessage, TimeSpan ttl, CancellationToken cancellationToken) + public Task SetResponseMessage(Uri endpoint, Guid identifier, RedisStoredMessage responseMessage, TimeSpan ttl, CancellationToken cancellationToken) { return halibutRedisTransport.SetResponseMessage(endpoint, identifier, responseMessage, ttl, cancellationToken); } - public Task GetResponseMessage(Uri endpoint, Guid identifier, CancellationToken cancellationToken) + public Task GetResponseMessage(Uri endpoint, Guid identifier, CancellationToken cancellationToken) { return halibutRedisTransport.GetResponseMessage(endpoint, identifier, cancellationToken); } - public Task DeleteResponseMessage(Uri endpoint, Guid identifier, CancellationToken cancellationToken) + public Task DeleteResponseMessage(Uri endpoint, Guid identifier, CancellationToken cancellationToken) { return halibutRedisTransport.DeleteResponseMessage(endpoint, identifier, cancellationToken); } diff --git a/source/Halibut.Tests/Queue/Redis/Utils/InMemoryStoreDataStreamsForDistributedQueues.cs b/source/Halibut.Tests/Queue/Redis/Utils/InMemoryStoreDataStreamsForDistributedQueues.cs index 9ab04540f..f9181547c 100644 --- a/source/Halibut.Tests/Queue/Redis/Utils/InMemoryStoreDataStreamsForDistributedQueues.cs +++ b/source/Halibut.Tests/Queue/Redis/Utils/InMemoryStoreDataStreamsForDistributedQueues.cs @@ -10,7 +10,7 @@ namespace Halibut.Tests.Queue.Redis.Utils public class InMemoryStoreDataStreamsForDistributedQueues : IStoreDataStreamsForDistributedQueues { readonly IDictionary dataStreamsStored = new Dictionary(); - public async Task StoreDataStreams(IReadOnlyList dataStreams, CancellationToken cancellationToken) + public async Task StoreDataStreams(IReadOnlyList dataStreams, CancellationToken cancellationToken) { foreach (var dataStream in dataStreams) { @@ -18,9 +18,11 @@ public async Task StoreDataStreams(IReadOnlyList dataStreams, Cancel await dataStream.WriteData(memoryStream, cancellationToken); dataStreamsStored[dataStream.Id] = memoryStream.ToArray(); } + + return ""; } - public async Task ReHydrateDataStreams(IReadOnlyList dataStreams, CancellationToken cancellationToken) + public async Task ReHydrateDataStreams(string _, IReadOnlyList dataStreams, CancellationToken cancellationToken) { await Task.CompletedTask; foreach (var dataStream in dataStreams) diff --git a/source/Halibut.Tests/Queue/Redis/Utils/MessageReaderWriterExtensionsMethods.cs b/source/Halibut.Tests/Queue/Redis/Utils/MessageReaderWriterExtensionsMethods.cs index 126313b2d..6139c53bf 100644 --- a/source/Halibut.Tests/Queue/Redis/Utils/MessageReaderWriterExtensionsMethods.cs +++ b/source/Halibut.Tests/Queue/Redis/Utils/MessageReaderWriterExtensionsMethods.cs @@ -1,8 +1,8 @@ using System; using System.Threading; using System.Threading.Tasks; -using Halibut.Queue.Redis; using Halibut.Queue.Redis.MessageStorage; +using Halibut.Queue.Redis.RedisHelpers; using Halibut.Transport.Protocol; namespace Halibut.Tests.Queue.Redis.Utils @@ -29,22 +29,22 @@ public MessageSerialiserAndDataStreamStorageWithVirtualMethods(IMessageSerialise this.messageSerialiserAndDataStreamStorage = messageSerialiserAndDataStreamStorage; } - public virtual Task PrepareRequest(RequestMessage request, CancellationToken cancellationToken) + public virtual Task PrepareRequest(RequestMessage request, CancellationToken cancellationToken) { return messageSerialiserAndDataStreamStorage.PrepareRequest(request, cancellationToken); } - public virtual Task ReadRequest(string jsonRequest, CancellationToken cancellationToken) + public virtual Task ReadRequest(RedisStoredMessage jsonRequest, CancellationToken cancellationToken) { return messageSerialiserAndDataStreamStorage.ReadRequest(jsonRequest, cancellationToken); } - public virtual Task PrepareResponse(ResponseMessage response, CancellationToken cancellationToken) + public virtual Task PrepareResponse(ResponseMessage response, CancellationToken cancellationToken) { return messageSerialiserAndDataStreamStorage.PrepareResponse(response, cancellationToken); } - public virtual Task ReadResponse(string jsonResponse, CancellationToken cancellationToken) + public virtual Task ReadResponse(RedisStoredMessage jsonResponse, CancellationToken cancellationToken) { return messageSerialiserAndDataStreamStorage.ReadResponse(jsonResponse, cancellationToken); } @@ -59,7 +59,7 @@ public MessageSerialiserAndDataStreamStorageThatThrowsWhenReadingResponse(IMessa this.exception = exception; } - public override Task ReadResponse(string jsonResponse, CancellationToken cancellationToken) + public override Task ReadResponse(RedisStoredMessage jsonResponse, CancellationToken cancellationToken) { throw exception(); } @@ -74,7 +74,7 @@ public MessageSerialiserAndDataStreamStorageThatThrowsOnPrepareRequest(IMessageS this.exception = exception; } - public override Task PrepareRequest(RequestMessage request, CancellationToken cancellationToken) + public override Task PrepareRequest(RequestMessage request, CancellationToken cancellationToken) { throw exception(); } diff --git a/source/Halibut/Queue/QueuedDataStreams/IStoreDataStreamsForDistributedQueues.cs b/source/Halibut/Queue/QueuedDataStreams/IStoreDataStreamsForDistributedQueues.cs index ab3b37a79..de390a705 100644 --- a/source/Halibut/Queue/QueuedDataStreams/IStoreDataStreamsForDistributedQueues.cs +++ b/source/Halibut/Queue/QueuedDataStreams/IStoreDataStreamsForDistributedQueues.cs @@ -21,16 +21,18 @@ public interface IStoreDataStreamsForDistributedQueues /// /// /// - /// - public Task StoreDataStreams(IReadOnlyList dataStreams, CancellationToken cancellationToken); - + /// A string, DataStreamMetadata, containing a small amount of data that will be stored in redis, this will be + /// given to ReHydrateDataStreams + public Task StoreDataStreams(IReadOnlyList dataStreams, CancellationToken cancellationToken); + /// /// Updates the dataStreams `writerAsync` to write the previously stored data. Using /// the SetWriterAsync method. /// + /// /// /// /// - public Task ReHydrateDataStreams(IReadOnlyList dataStreams, CancellationToken cancellationToken); + public Task ReHydrateDataStreams(string dataStreamMetadata, IReadOnlyList dataStreams, CancellationToken cancellationToken); } } \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/MessageStorage/IMessageSerialiserAndDataStreamStorage.cs b/source/Halibut/Queue/Redis/MessageStorage/IMessageSerialiserAndDataStreamStorage.cs index 160267af2..519e04e1e 100644 --- a/source/Halibut/Queue/Redis/MessageStorage/IMessageSerialiserAndDataStreamStorage.cs +++ b/source/Halibut/Queue/Redis/MessageStorage/IMessageSerialiserAndDataStreamStorage.cs @@ -1,6 +1,7 @@ using System; using System.Threading; using System.Threading.Tasks; +using Halibut.Queue.Redis.RedisHelpers; using Halibut.Transport.Protocol; namespace Halibut.Queue.Redis.MessageStorage @@ -15,9 +16,9 @@ namespace Halibut.Queue.Redis.MessageStorage /// public interface IMessageSerialiserAndDataStreamStorage { - Task PrepareRequest(RequestMessage request, CancellationToken cancellationToken); - Task ReadRequest(string jsonRequest, CancellationToken cancellationToken); - Task PrepareResponse(ResponseMessage response, CancellationToken cancellationToken); - Task ReadResponse(string jsonResponse, CancellationToken cancellationToken); + Task PrepareRequest(RequestMessage request, CancellationToken cancellationToken); + Task ReadRequest(RedisStoredMessage jsonRequest, CancellationToken cancellationToken); + Task PrepareResponse(ResponseMessage response, CancellationToken cancellationToken); + Task ReadResponse(RedisStoredMessage jsonResponse, CancellationToken cancellationToken); } } \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/MessageStorage/MessageSerialiserAndDataStreamStorage.cs b/source/Halibut/Queue/Redis/MessageStorage/MessageSerialiserAndDataStreamStorage.cs index c7ceab190..78fc44302 100644 --- a/source/Halibut/Queue/Redis/MessageStorage/MessageSerialiserAndDataStreamStorage.cs +++ b/source/Halibut/Queue/Redis/MessageStorage/MessageSerialiserAndDataStreamStorage.cs @@ -2,6 +2,7 @@ using System.Threading; using System.Threading.Tasks; using Halibut.Queue.QueuedDataStreams; +using Halibut.Queue.Redis.RedisHelpers; using Halibut.Transport.Protocol; namespace Halibut.Queue.Redis.MessageStorage @@ -17,31 +18,31 @@ public MessageSerialiserAndDataStreamStorage(QueueMessageSerializer queueMessage this.storeDataStreamsForDistributedQueues = storeDataStreamsForDistributedQueues; } - public async Task PrepareRequest(RequestMessage request, CancellationToken cancellationToken) + public async Task PrepareRequest(RequestMessage request, CancellationToken cancellationToken) { - var (payload, dataStreams) = queueMessageSerializer.WriteMessage(request); - await storeDataStreamsForDistributedQueues.StoreDataStreams(dataStreams, cancellationToken); - return payload; + var (jsonRequestMessage, dataStreams) = queueMessageSerializer.WriteMessage(request); + var dataStreamMetaData = await storeDataStreamsForDistributedQueues.StoreDataStreams(dataStreams, cancellationToken); + return new RedisStoredMessage(jsonRequestMessage, dataStreamMetaData); } - public async Task ReadRequest(string jsonRequest, CancellationToken cancellationToken) + public async Task ReadRequest(RedisStoredMessage storedMessage, CancellationToken cancellationToken) { - var (request, dataStreams) = queueMessageSerializer.ReadMessage(jsonRequest); - await storeDataStreamsForDistributedQueues.ReHydrateDataStreams(dataStreams, cancellationToken); + var (request, dataStreams) = queueMessageSerializer.ReadMessage(storedMessage.Message); + await storeDataStreamsForDistributedQueues.ReHydrateDataStreams(storedMessage.DataStreamMetadata, dataStreams, cancellationToken); return request; } - public async Task PrepareResponse(ResponseMessage response, CancellationToken cancellationToken) + public async Task PrepareResponse(ResponseMessage response, CancellationToken cancellationToken) { - var (payload, dataStreams) = queueMessageSerializer.WriteMessage(response); - await storeDataStreamsForDistributedQueues.StoreDataStreams(dataStreams, cancellationToken); - return payload; + var (jsonResponseMessage, dataStreams) = queueMessageSerializer.WriteMessage(response); + var dataStreamMetaData = await storeDataStreamsForDistributedQueues.StoreDataStreams(dataStreams, cancellationToken); + return new RedisStoredMessage(jsonResponseMessage, dataStreamMetaData); } - public async Task ReadResponse(string jsonResponse, CancellationToken cancellationToken) + public async Task ReadResponse(RedisStoredMessage storedMessage, CancellationToken cancellationToken) { - var (response, dataStreams) = queueMessageSerializer.ReadMessage(jsonResponse); - await storeDataStreamsForDistributedQueues.ReHydrateDataStreams(dataStreams, cancellationToken); + var (response, dataStreams) = queueMessageSerializer.ReadMessage(storedMessage.Message); + await storeDataStreamsForDistributedQueues.ReHydrateDataStreams(storedMessage.DataStreamMetadata, dataStreams, cancellationToken); return response; } } diff --git a/source/Halibut/Queue/Redis/RedisHelpers/HalibutRedisTransport.cs b/source/Halibut/Queue/Redis/RedisHelpers/HalibutRedisTransport.cs index 9825620f3..b324d6ce4 100644 --- a/source/Halibut/Queue/Redis/RedisHelpers/HalibutRedisTransport.cs +++ b/source/Halibut/Queue/Redis/RedisHelpers/HalibutRedisTransport.cs @@ -1,10 +1,12 @@ #if NET8_0_OR_GREATER using System; +using System.Collections.Generic; using System.Threading; using System.Threading.Tasks; using Halibut.Queue.Redis.NodeHeartBeat; using Halibut.Util; +using Microsoft.VisualBasic.CompilerServices; using StackExchange.Redis; namespace Halibut.Queue.Redis.RedisHelpers @@ -78,8 +80,6 @@ static string RequestMessageKey(Uri endpoint, Guid requestId) { return $"{Namespace}::RequestMessage::{endpoint}::{requestId}"; } - - static readonly string RequestMessageField = "RequestMessageField"; /// /// The amount of time on top of the requestPickupTimout, the request will stay on the queue @@ -90,15 +90,17 @@ static string RequestMessageKey(Uri endpoint, Guid requestId) /// static readonly TimeSpan AdditionalRequestMessageTtl = TimeSpan.FromMinutes(2); - public async Task PutRequest(Uri endpoint, Guid requestId, string requestMessage, TimeSpan requestPickupTimeout, CancellationToken cancellationToken) + public async Task PutRequest(Uri endpoint, Guid requestId, RedisStoredMessage requestMessage, TimeSpan requestPickupTimeout, CancellationToken cancellationToken) { var requestKey = RequestMessageKey(endpoint, requestId); var ttl = requestPickupTimeout + AdditionalRequestMessageTtl; - await facade.SetInHash(requestKey, RequestMessageField, requestMessage, ttl, cancellationToken); + var dict = RedisStoredMessageToDictionary(requestMessage); + + await facade.SetInHash(requestKey, dict, ttl, cancellationToken); } - + /// /// Atomically Gets and removes the request from the queue. /// Exactly up to one caller of this method will be given the RequestMessage, all @@ -112,11 +114,11 @@ public async Task PutRequest(Uri endpoint, Guid requestId, string requestMessage /// /// /// - public async Task TryGetAndRemoveRequest(Uri endpoint, Guid requestId, CancellationToken cancellationToken) + public async Task TryGetAndRemoveRequest(Uri endpoint, Guid requestId, CancellationToken cancellationToken) { var requestKey = RequestMessageKey(endpoint, requestId); - var requestMessage = await facade.TryGetAndDeleteFromHash(requestKey, RequestMessageField, cancellationToken); - return requestMessage; + var dit = await facade.TryGetAndDeleteFromHash(requestKey, RedisStoredMessageHashFields, cancellationToken); + return DictionaryToRedisStoredMessage(dit); } public async Task IsRequestStillOnQueue(Uri endpoint, Guid requestId, CancellationToken cancellationToken) @@ -253,22 +255,52 @@ static string ResponseMessageKey(Uri endpoint, Guid identifier) return $"{Namespace}::Response::{endpoint}::{identifier}"; } - public async Task SetResponseMessage(Uri endpoint, Guid identifier, string responseMessage, TimeSpan ttl, CancellationToken cancellationToken) + public async Task SetResponseMessage(Uri endpoint, Guid identifier, RedisStoredMessage responseMessage, TimeSpan ttl, CancellationToken cancellationToken) { var key = ResponseMessageKey(endpoint, identifier); - await facade.SetString(key, responseMessage, ttl, cancellationToken); + var dict = RedisStoredMessageToDictionary(responseMessage); + await facade.SetInHash(key, dict, ttl, cancellationToken); } - public async Task GetResponseMessage(Uri endpoint, Guid identifier, CancellationToken cancellationToken) + public async Task GetResponseMessage(Uri endpoint, Guid identifier, CancellationToken cancellationToken) { var key = ResponseMessageKey(endpoint, identifier); - return await facade.GetString(key, cancellationToken); + var dict = await facade.TryGetFromHash(key, RedisStoredMessageHashFields, cancellationToken); + return DictionaryToRedisStoredMessage(dict); } - public async Task DeleteResponseMessage(Uri endpoint, Guid identifier, CancellationToken cancellationToken) + public async Task DeleteResponseMessage(Uri endpoint, Guid identifier, CancellationToken cancellationToken) { var key = ResponseMessageKey(endpoint, identifier); - return await facade.DeleteString(key, cancellationToken); + await facade.DeleteHash(key, cancellationToken); + } + + static readonly string RequestMessageField = "RequestMessageField"; + static readonly string DataStreamMetaDataField = "DataStreamMetaDataField"; + static string[] RedisStoredMessageHashFields => new[] { RequestMessageField, DataStreamMetaDataField }; + + static RedisStoredMessage? DictionaryToRedisStoredMessage(Dictionary? dit) + { + if(dit == null) return null; + var requestMessage = dit[RequestMessageField]!; + + // As it turns out Redis or our client seems to treat "" as null, which is insane + // and results in us needing to deal with that here. + var dataStreamMetaData = ""; + if(dit.TryGetValue(DataStreamMetaDataField, out var dataStreamMetaDataFromRedis)) + { + dataStreamMetaData = dataStreamMetaDataFromRedis ?? ""; + } + + return new RedisStoredMessage(requestMessage, dataStreamMetaData); + } + + static Dictionary RedisStoredMessageToDictionary(RedisStoredMessage requestMessage) + { + var dict = new Dictionary(); + dict[RequestMessageField] = requestMessage.Message; + dict[DataStreamMetaDataField] = requestMessage.DataStreamMetadata; + return dict; } } } diff --git a/source/Halibut/Queue/Redis/RedisHelpers/IHalibutRedisTransport.cs b/source/Halibut/Queue/Redis/RedisHelpers/IHalibutRedisTransport.cs index 24f14f263..30fb4b771 100644 --- a/source/Halibut/Queue/Redis/RedisHelpers/IHalibutRedisTransport.cs +++ b/source/Halibut/Queue/Redis/RedisHelpers/IHalibutRedisTransport.cs @@ -4,6 +4,7 @@ using System.Threading; using System.Threading.Tasks; using Halibut.Queue.Redis.NodeHeartBeat; +using Halibut.Queue.Redis.RedisHelpers; using StackExchange.Redis; namespace Halibut.Queue.Redis @@ -18,8 +19,8 @@ public interface IHalibutRedisTransport Task TryPopNextRequestGuid(Uri endpoint, CancellationToken cancellationToken); - Task PutRequest(Uri endpoint, Guid requestId, string requestMessage, TimeSpan requestPickupTimeout, CancellationToken cancellationToken); - Task TryGetAndRemoveRequest(Uri endpoint, Guid requestId, CancellationToken cancellationToken); + Task PutRequest(Uri endpoint, Guid requestId, RedisStoredMessage requestMessage, TimeSpan requestPickupTimeout, CancellationToken cancellationToken); + Task TryGetAndRemoveRequest(Uri endpoint, Guid requestId, CancellationToken cancellationToken); Task IsRequestStillOnQueue(Uri endpoint, Guid requestId, CancellationToken cancellationToken); Task SubscribeToRequestCancellation( @@ -51,9 +52,9 @@ Task SubscribeToResponseChannel( Task PublishThatResponseIsAvailable(Uri endpoint, Guid identifier, CancellationToken cancellationToken); - Task SetResponseMessage(Uri endpoint, Guid identifier, string responseMessage, TimeSpan ttl, CancellationToken cancellationToken); - Task GetResponseMessage(Uri endpoint, Guid identifier, CancellationToken cancellationToken); - Task DeleteResponseMessage(Uri endpoint, Guid identifier, CancellationToken cancellationToken); + Task SetResponseMessage(Uri endpoint, Guid identifier, RedisStoredMessage responseMessage, TimeSpan ttl, CancellationToken cancellationToken); + Task GetResponseMessage(Uri endpoint, Guid identifier, CancellationToken cancellationToken); + Task DeleteResponseMessage(Uri endpoint, Guid identifier, CancellationToken cancellationToken); } } #endif \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/RedisHelpers/RedisFacade.cs b/source/Halibut/Queue/Redis/RedisHelpers/RedisFacade.cs index 9279f83bd..89617d018 100644 --- a/source/Halibut/Queue/Redis/RedisHelpers/RedisFacade.cs +++ b/source/Halibut/Queue/Redis/RedisHelpers/RedisFacade.cs @@ -1,5 +1,7 @@ using System; +using System.Collections.Generic; using System.Diagnostics; +using System.Linq; using System.Threading; using System.Threading.Tasks; using Halibut.Diagnostics; @@ -199,14 +201,16 @@ await ExecuteWithRetry(async () => }, cancellationToken); } - public async Task SetInHash(string key, string field, string payload, TimeSpan ttl, CancellationToken cancellationToken) + public async Task SetInHash(string key, Dictionary values, TimeSpan ttl, CancellationToken cancellationToken) { var hashKey = ToHashKey(key); + + var hashEntries = values.Select(v => new HashEntry(v.Key, v.Value)).ToArray(); await ExecuteWithRetry(async () => { var database = Connection.GetDatabase(); - await database.HashSetAsync(hashKey, new RedisValue(field), new RedisValue(payload)); + await database.HashSetAsync(hashKey, hashEntries); }, cancellationToken); await SetTtlForKeyRaw(hashKey, ttl, cancellationToken); @@ -227,16 +231,11 @@ public async Task HashContainsKey(string key, string field, CancellationTo }, cancellationToken); } - public async Task TryGetAndDeleteFromHash(string key, string field, CancellationToken cancellationToken) + public async Task?> TryGetAndDeleteFromHash(string key, string[] fields, CancellationToken cancellationToken) { var hashKey = ToHashKey(key); - // Retry each operation independently - var value = await ExecuteWithRetry(async () => - { - var database = Connection.GetDatabase(); - return await database.HashGetAsync(hashKey, new RedisValue(field)); - }, cancellationToken); + Dictionary? dict = await RawKeyReadHashFieldsToDictionary(hashKey, fields, cancellationToken); // Retry does make this non-idempotent, what can happen is the key is deleted on redis. // But we do not get a response saying it is deleted. We try again and get told @@ -254,9 +253,47 @@ public async Task HashContainsKey(string key, string field, CancellationTo // Someone else deleted this, so return nothing to make the get and delete appear to be atomic. return null; } - return value; + return dict; + } + + public async Task?> TryGetFromHash(string key, string[] fields, CancellationToken cancellationToken) + { + var hashKey = ToHashKey(key); + + return await RawKeyReadHashFieldsToDictionary(hashKey, fields, cancellationToken); + } + + + public async Task DeleteHash(string key, CancellationToken cancellationToken) + { + var hashKey = ToHashKey(key); + + await ExecuteWithRetry(async () => + { + var database = Connection.GetDatabase(); + return await database.KeyDeleteAsync(hashKey); + }, cancellationToken); } + async Task?> RawKeyReadHashFieldsToDictionary(RedisKey hashKey, string[] fields, CancellationToken cancellationToken) + { + var dict = new Dictionary(); + foreach (var field in fields) + { + // Retry each operation independently + var value = await ExecuteWithRetry(async () => + { + var database = Connection.GetDatabase(); + return await database.HashGetAsync(hashKey, new RedisValue(field)); + }, cancellationToken); + if(value.HasValue) dict[field] = value; + } + + if (dict.Count == 0) return null; + + return dict; + } + RedisKey ToListKey(string key) { return "list:" + keyPrefix + ":" + key; diff --git a/source/Halibut/Queue/Redis/RedisHelpers/RedisStoredMessage.cs b/source/Halibut/Queue/Redis/RedisHelpers/RedisStoredMessage.cs new file mode 100644 index 000000000..059aa6a12 --- /dev/null +++ b/source/Halibut/Queue/Redis/RedisHelpers/RedisStoredMessage.cs @@ -0,0 +1,15 @@ +namespace Halibut.Queue.Redis.RedisHelpers +{ + public class RedisStoredMessage + { + public RedisStoredMessage(string message, string dataStreamMetadata) + { + Message = message; + DataStreamMetadata = dataStreamMetadata; + } + + public string Message { get; } + + public string DataStreamMetadata { get; } + } +} \ No newline at end of file diff --git a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs index 9ed5fb2dc..e55f399f5 100644 --- a/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs +++ b/source/Halibut/Queue/Redis/RedisPendingRequestQueue.cs @@ -10,6 +10,7 @@ using Halibut.Queue.Redis.MessageStorage; using Halibut.Queue.Redis.NodeHeartBeat; using Halibut.Queue.Redis.RedisDataLossDetection; +using Halibut.Queue.Redis.RedisHelpers; using Halibut.Queue.Redis.ResponseMessageTransfer; using Halibut.ServiceModel; using Halibut.Transport.Protocol; @@ -125,10 +126,10 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can using var pending = new RedisPendingRequest(request, log); - string payload; + RedisStoredMessage messageToStore; try { - payload = await messageSerialiserAndDataStreamStorage.PrepareRequest(request, cancellationToken); + messageToStore = await messageSerialiserAndDataStreamStorage.PrepareRequest(request, cancellationToken); } catch (Exception ex) { @@ -147,7 +148,7 @@ public async Task QueueAndWaitAsync(RequestMessage request, Can // Make the request available before we tell people it is available. try { - await halibutRedisTransport.PutRequest(endpoint, request.ActivityId, payload, request.Destination.PollingRequestQueueTimeout, cancellationToken); + await halibutRedisTransport.PutRequest(endpoint, request.ActivityId, messageToStore, request.Destination.PollingRequestQueueTimeout, cancellationToken); await halibutRedisTransport.PushRequestGuidOnToQueue(endpoint, request.ActivityId, cancellationToken); await halibutRedisTransport.PulseRequestPushedToEndpoint(endpoint, cancellationToken); } @@ -283,8 +284,8 @@ async Task TryClearRequestFromQueue(RedisPendingRequest redisPending) } await using var cts = new CancelOnDisposeCancellationToken(); cts.CancelAfter(TimeSpan.FromMinutes(2)); // Best efforts. - var requestJson = await halibutRedisTransport.TryGetAndRemoveRequest(endpoint, request.ActivityId, cts.Token); - if (requestJson != null) + var requestMessage = await halibutRedisTransport.TryGetAndRemoveRequest(endpoint, request.ActivityId, cts.Token); + if (requestMessage != null) { log.Write(EventType.Diagnostic, "Successfully removed request {0} from queue - request was never collected by a processing node", request.ActivityId); return true; @@ -309,7 +310,7 @@ async Task TryClearRequestFromQueue(RedisPendingRequest redisPending) { await Task.Yield(); var activityId = requestMessage.ActivityId; - string responseJson; + RedisStoredMessage responseJson; try { log.Write(EventType.Diagnostic, "Waiting for response for request {0}", activityId); @@ -421,9 +422,9 @@ public async Task ApplyResponse(ResponseMessage response, Guid requestActivityId response = ResponseMessage.FromException(response, new HalibutClientException(RequestAbandonedMessage)); } } - var responseJson = await messageSerialiserAndDataStreamStorage.PrepareResponse(response, cancellationToken); + var responseStoredMessage = await messageSerialiserAndDataStreamStorage.PrepareResponse(response, cancellationToken); log.Write(EventType.MessageExchange, "Sending response message for request {0}", requestActivityId); - await ResponseMessageSender.SendResponse(halibutRedisTransport, endpoint, requestActivityId, responseJson, TTLOfResponseMessage, log); + await ResponseMessageSender.SendResponse(halibutRedisTransport, endpoint, requestActivityId, responseStoredMessage, TTLOfResponseMessage, log); log.Write(EventType.MessageExchange, "Successfully applied response for request {0}", requestActivityId); } catch (Exception ex) diff --git a/source/Halibut/Queue/Redis/ResponseMessageTransfer/PollAndSubscribeToResponse.cs b/source/Halibut/Queue/Redis/ResponseMessageTransfer/PollAndSubscribeToResponse.cs index 691e22cbf..cd561d31b 100644 --- a/source/Halibut/Queue/Redis/ResponseMessageTransfer/PollAndSubscribeToResponse.cs +++ b/source/Halibut/Queue/Redis/ResponseMessageTransfer/PollAndSubscribeToResponse.cs @@ -4,6 +4,7 @@ using System.Threading; using System.Threading.Tasks; using Halibut.Diagnostics; +using Halibut.Queue.Redis.RedisHelpers; using Halibut.Util; using Nito.AsyncEx; @@ -18,12 +19,12 @@ public class PollAndSubscribeToResponse : IAsyncDisposable readonly Guid activityId; readonly LinearBackoffStrategy pollBackoffStrategy; - readonly TaskCompletionSource responseJsonCompletionSource = new(); + readonly TaskCompletionSource responseJsonCompletionSource = new(); /// /// An awaitable task that returns when the response is available. /// - public Task ResponseJson => responseJsonCompletionSource.Task; + public Task ResponseJson => responseJsonCompletionSource.Task; public PollAndSubscribeToResponse(Uri endpoint, Guid activityId, IHalibutRedisTransport halibutRedisTransport, ILog log) { @@ -145,7 +146,7 @@ async Task DeleteResponseFromRedis(string detectedBy, CancellationToken token) } } - void TrySetResponse(string value) + void TrySetResponse(RedisStoredMessage value) { try { diff --git a/source/Halibut/Queue/Redis/ResponseMessageTransfer/ResponseMessageSender.cs b/source/Halibut/Queue/Redis/ResponseMessageTransfer/ResponseMessageSender.cs index 40cfb6985..5625f1365 100644 --- a/source/Halibut/Queue/Redis/ResponseMessageTransfer/ResponseMessageSender.cs +++ b/source/Halibut/Queue/Redis/ResponseMessageTransfer/ResponseMessageSender.cs @@ -2,6 +2,7 @@ using System; using System.Threading.Tasks; using Halibut.Diagnostics; +using Halibut.Queue.Redis.RedisHelpers; using Halibut.Util; namespace Halibut.Queue.Redis.ResponseMessageTransfer @@ -12,7 +13,7 @@ public static async Task SendResponse( IHalibutRedisTransport halibutRedisTransport, Uri endpoint, Guid activityId, - string responseMessage, + RedisStoredMessage responseMessage, TimeSpan ttl, ILog log) {