Skip to content

Commit fa8f66b

Browse files
authored
Increase the backoff strategy time (#345)
* Increase the backoff strategy time * The back-off reconnect strategy is increased sensibly. The previous value was too aggressive. It caused a lot of tentative and a lot of server requests. The TCP port can be ready during the restart, but the stream cannot be ready due to the sync. * Introduce a random delay on the strategy to avoid having the same reconnection time in case the client has more producers and consumers. * Introduce also a random delay on the disconnected part on disconnect for the same above reason. --------- Signed-off-by: Gabriele Santomaggio <[email protected]>
1 parent 083ff20 commit fa8f66b

15 files changed

+122
-62
lines changed

RabbitMQ.Stream.Client/ClientExceptions.cs

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -49,6 +49,15 @@ internal static bool IsStreamNotAvailable(Exception exception)
4949
return exception is CreateException { ResponseCode: ResponseCode.StreamNotAvailable };
5050
}
5151

52+
internal static void CheckLeader(StreamInfo metaStreamInfo)
53+
{
54+
if (metaStreamInfo.Leader.Equals(default(Broker)))
55+
{
56+
throw new LeaderNotFoundException(
57+
$"No leader found for streams {string.Join(" ", metaStreamInfo.Stream)}");
58+
}
59+
}
60+
5261
public static void MaybeThrowException(ResponseCode responseCode, string message)
5362
{
5463
if (responseCode is ResponseCode.Ok)

RabbitMQ.Stream.Client/PublicAPI.Unshipped.txt

Lines changed: 6 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -180,9 +180,6 @@ RabbitMQ.Stream.Client.RawSuperStreamProducerConfig.ConnectionClosedHandler.get
180180
RabbitMQ.Stream.Client.RawSuperStreamProducerConfig.ConnectionClosedHandler.set -> void
181181
RabbitMQ.Stream.Client.RawSuperStreamProducerConfig.RoutingStrategyType.get -> RabbitMQ.Stream.Client.RoutingStrategyType
182182
RabbitMQ.Stream.Client.RawSuperStreamProducerConfig.RoutingStrategyType.set -> void
183-
RabbitMQ.Stream.Client.Reconnect.IReconnectStrategy
184-
RabbitMQ.Stream.Client.Reconnect.IReconnectStrategy.WhenConnected(string itemIdentifier) -> System.Threading.Tasks.ValueTask
185-
RabbitMQ.Stream.Client.Reconnect.IReconnectStrategy.WhenDisconnected(string itemIdentifier) -> System.Threading.Tasks.ValueTask<bool>
186183
RabbitMQ.Stream.Client.Reliable.Consumer.Info.get -> RabbitMQ.Stream.Client.ConsumerInfo
187184
RabbitMQ.Stream.Client.Reliable.ConsumerConfig.Crc32.get -> RabbitMQ.Stream.Client.ICrc32
188185
RabbitMQ.Stream.Client.Reliable.ConsumerConfig.Crc32.set -> void
@@ -199,6 +196,9 @@ RabbitMQ.Stream.Client.Reliable.DeduplicatingProducer.IsOpen() -> bool
199196
RabbitMQ.Stream.Client.Reliable.DeduplicatingProducer.Send(ulong publishing, RabbitMQ.Stream.Client.Message message) -> System.Threading.Tasks.ValueTask
200197
RabbitMQ.Stream.Client.Reliable.DeduplicatingProducerConfig
201198
RabbitMQ.Stream.Client.Reliable.DeduplicatingProducerConfig.DeduplicatingProducerConfig(RabbitMQ.Stream.Client.StreamSystem streamSystem, string stream, string reference) -> void
199+
RabbitMQ.Stream.Client.Reliable.IReconnectStrategy
200+
RabbitMQ.Stream.Client.Reliable.IReconnectStrategy.WhenConnected(string itemIdentifier) -> System.Threading.Tasks.ValueTask
201+
RabbitMQ.Stream.Client.Reliable.IReconnectStrategy.WhenDisconnected(string itemIdentifier) -> System.Threading.Tasks.ValueTask<bool>
202202
RabbitMQ.Stream.Client.Reliable.Producer.Info.get -> RabbitMQ.Stream.Client.ProducerInfo
203203
RabbitMQ.Stream.Client.Reliable.ProducerConfig.Filter.get -> RabbitMQ.Stream.Client.ProducerFilter
204204
RabbitMQ.Stream.Client.Reliable.ProducerConfig.Filter.set -> void
@@ -208,8 +208,8 @@ RabbitMQ.Stream.Client.Reliable.ProducerFactory._producer -> RabbitMQ.Stream.Cli
208208
RabbitMQ.Stream.Client.Reliable.ReliableBase.CompareStatus(RabbitMQ.Stream.Client.Reliable.ReliableEntityStatus toTest) -> bool
209209
RabbitMQ.Stream.Client.Reliable.ReliableBase.UpdateStatus(RabbitMQ.Stream.Client.Reliable.ReliableEntityStatus status) -> void
210210
RabbitMQ.Stream.Client.Reliable.ReliableBase._status -> RabbitMQ.Stream.Client.Reliable.ReliableEntityStatus
211-
RabbitMQ.Stream.Client.Reliable.ReliableConfig.ReconnectStrategy.get -> RabbitMQ.Stream.Client.Reconnect.IReconnectStrategy
212-
RabbitMQ.Stream.Client.Reliable.ReliableConfig.ResourceAvailableReconnectStrategy.get -> RabbitMQ.Stream.Client.Reconnect.IReconnectStrategy
211+
RabbitMQ.Stream.Client.Reliable.ReliableConfig.ReconnectStrategy.get -> RabbitMQ.Stream.Client.Reliable.IReconnectStrategy
212+
RabbitMQ.Stream.Client.Reliable.ReliableConfig.ResourceAvailableReconnectStrategy.get -> RabbitMQ.Stream.Client.Reliable.IReconnectStrategy
213213
RabbitMQ.Stream.Client.Reliable.ReliableConfig.ResourceAvailableReconnectStrategy.set -> void
214214
RabbitMQ.Stream.Client.Reliable.ReliableEntityStatus
215215
RabbitMQ.Stream.Client.Reliable.ReliableEntityStatus.Closed = 3 -> RabbitMQ.Stream.Client.Reliable.ReliableEntityStatus
@@ -264,5 +264,6 @@ static RabbitMQ.Stream.Client.RawProducer.Create(RabbitMQ.Stream.Client.ClientPa
264264
static RabbitMQ.Stream.Client.RawSuperStreamConsumer.Create(RabbitMQ.Stream.Client.RawSuperStreamConsumerConfig rawSuperStreamConsumerConfig, System.Collections.Generic.IDictionary<string, RabbitMQ.Stream.Client.StreamInfo> streamInfos, RabbitMQ.Stream.Client.ClientParameters clientParameters, Microsoft.Extensions.Logging.ILogger logger = null) -> RabbitMQ.Stream.Client.ISuperStreamConsumer
265265
static RabbitMQ.Stream.Client.RawSuperStreamProducer.Create(RabbitMQ.Stream.Client.RawSuperStreamProducerConfig rawSuperStreamProducerConfig, System.Collections.Generic.IDictionary<string, RabbitMQ.Stream.Client.StreamInfo> streamInfos, RabbitMQ.Stream.Client.ClientParameters clientParameters, Microsoft.Extensions.Logging.ILogger logger = null) -> RabbitMQ.Stream.Client.ISuperStreamProducer
266266
static RabbitMQ.Stream.Client.Reliable.DeduplicatingProducer.Create(RabbitMQ.Stream.Client.Reliable.DeduplicatingProducerConfig producerConfig, Microsoft.Extensions.Logging.ILogger<RabbitMQ.Stream.Client.Reliable.Producer> logger = null) -> System.Threading.Tasks.Task<RabbitMQ.Stream.Client.Reliable.DeduplicatingProducer>
267+
static RabbitMQ.Stream.Client.Reliable.ReliableBase.RandomWait() -> System.Threading.Tasks.Task
267268
static RabbitMQ.Stream.Client.RoutingHelper<T>.LookupLeaderConnection(RabbitMQ.Stream.Client.ClientParameters clientParameters, RabbitMQ.Stream.Client.StreamInfo metaDataInfo, RabbitMQ.Stream.Client.ConnectionsPool pool, Microsoft.Extensions.Logging.ILogger logger = null) -> System.Threading.Tasks.Task<RabbitMQ.Stream.Client.IClient>
268269
static RabbitMQ.Stream.Client.RoutingHelper<T>.LookupRandomConnection(RabbitMQ.Stream.Client.ClientParameters clientParameters, RabbitMQ.Stream.Client.StreamInfo metaDataInfo, RabbitMQ.Stream.Client.ConnectionsPool pool, Microsoft.Extensions.Logging.ILogger logger = null) -> System.Threading.Tasks.Task<RabbitMQ.Stream.Client.IClient>

RabbitMQ.Stream.Client/RawSuperStreamConsumer.cs

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -95,7 +95,6 @@ private RawConsumerConfig FromStreamConfig(string stream)
9595
}
9696

9797
consumer?.Dispose();
98-
_streamInfos.Remove(stream);
9998

10099
if (_config.ConnectionClosedHandler != null)
101100
{
@@ -117,7 +116,6 @@ private RawConsumerConfig FromStreamConfig(string stream)
117116
{
118117
_consumers.TryRemove(update.Stream, out var consumer);
119118
consumer?.Close();
120-
_streamInfos.Remove(update.Stream);
121119
if (_config.MetadataHandler != null)
122120
{
123121
await _config.MetadataHandler(update).ConfigureAwait(false);
@@ -159,12 +157,13 @@ private async Task StartConsumers()
159157

160158
public async Task ReconnectPartition(StreamInfo streamInfo)
161159
{
160+
ClientExceptions.CheckLeader(streamInfo);
162161
await _semaphoreSlim.WaitAsync().ConfigureAwait(false);
163162
try
164163
{
165164
_consumers.TryRemove(streamInfo.Stream, out var consumer);
166165
consumer?.Dispose();
167-
_streamInfos.TryAdd(streamInfo.Stream, streamInfo); // add the new stream infos
166+
_streamInfos[streamInfo.Stream] = streamInfo;
168167
await MaybeAddConsumer(streamInfo.Stream).ConfigureAwait(false);
169168
}
170169
finally

RabbitMQ.Stream.Client/RawSuperStreamProducer.cs

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -163,6 +163,7 @@ private async Task<IProducer> MaybeAddAndGetProducer(string stream)
163163

164164
public async Task ReconnectPartition(StreamInfo streamInfo)
165165
{
166+
ClientExceptions.CheckLeader(streamInfo);
166167
await _semaphoreSlim.WaitAsync().ConfigureAwait(false);
167168
try
168169
{

RabbitMQ.Stream.Client/Reliable/Consumer.cs

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -6,7 +6,6 @@
66
using System.Threading.Tasks;
77
using Microsoft.Extensions.Logging;
88
using Microsoft.Extensions.Logging.Abstractions;
9-
using RabbitMQ.Stream.Client.Reconnect;
109

1110
namespace RabbitMQ.Stream.Client.Reliable;
1211

RabbitMQ.Stream.Client/Reliable/ConsumerFactory.cs

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -122,6 +122,7 @@ private async Task<IConsumer> SuperConsumer(bool boot)
122122
OffsetSpec = offsetSpecs,
123123
ConnectionClosedHandler = async (closeReason, partitionStream) =>
124124
{
125+
await RandomWait().ConfigureAwait(false);
125126
if (closeReason == ConnectionClosedReason.Normal)
126127
{
127128
BaseLogger.LogInformation("{Identity} is closed normally", ToString());
@@ -134,6 +135,7 @@ await OnEntityClosed(_consumerConfig.StreamSystem, partitionStream, r)
134135
},
135136
MetadataHandler = async update =>
136137
{
138+
await RandomWait().ConfigureAwait(false);
137139
var r = ((RawSuperStreamConsumer)(_consumer)).ReconnectPartition;
138140
await OnEntityClosed(_consumerConfig.StreamSystem, update.Stream, r)
139141
.ConfigureAwait(false);

RabbitMQ.Stream.Client/Reliable/IReconnectStrategy.cs

Lines changed: 6 additions & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -7,7 +7,7 @@
77
using Microsoft.Extensions.Logging;
88
using Microsoft.Extensions.Logging.Abstractions;
99

10-
namespace RabbitMQ.Stream.Client.Reconnect;
10+
namespace RabbitMQ.Stream.Client.Reliable;
1111

1212
/// <summary>
1313
/// IReconnectStrategy is the interface to reconnect the TCP client
@@ -48,36 +48,24 @@ public BackOffReconnectStrategy(ILogger logger = null)
4848
// else the backoff will be too long
4949
private void MaybeResetTentatives()
5050
{
51-
if (Tentatives > 5)
51+
if (Tentatives > 4)
5252
{
5353
Tentatives = 1;
5454
}
5555
}
5656

5757
public async ValueTask<bool> WhenDisconnected(string connectionIdentifier)
5858
{
59-
6059
Tentatives <<= 1;
60+
var next = Random.Shared.Next(Tentatives * 1000, Tentatives * 3000);
6161
_logger.LogInformation(
6262
"{ConnectionIdentifier} disconnected, check if reconnection needed in {ReconnectionDelayMs} ms",
6363
connectionIdentifier,
64-
Tentatives * 100
64+
next
6565
);
66-
await Task.Delay(TimeSpan.FromMilliseconds(Tentatives * 100)).ConfigureAwait(false);
66+
await Task.Delay(TimeSpan.FromMilliseconds(next)).ConfigureAwait(false);
6767
MaybeResetTentatives();
6868
return true;
69-
// this will be in another commit
70-
// Tentatives <<= 1;
71-
// var next = Random.Shared.Next(Tentatives * 1000, Tentatives * 2000);
72-
// _logger.LogInformation(
73-
// "{ConnectionIdentifier} disconnected, check if reconnection needed in {ReconnectionDelayMs} ms",
74-
// connectionIdentifier,
75-
// next
76-
// );
77-
//
78-
// await Task.Delay(TimeSpan.FromMilliseconds(next)).ConfigureAwait(false);
79-
// MaybeResetTentatives();
80-
// return true;
8169
}
8270

8371
public ValueTask WhenConnected(string connectionIdentifier)
@@ -112,7 +100,7 @@ public async ValueTask<bool> WhenDisconnected(string resourceIdentifier)
112100
{
113101
Tentatives <<= 1;
114102
_logger.LogInformation(
115-
"{ConnectionIdentifier} resource not available, retry in {ReconnectionDelayMs} seconds",
103+
"{ConnectionIdentifier} resource not available, retry in {ReconnectionDelayS} seconds",
116104
resourceIdentifier,
117105
Tentatives
118106
);

RabbitMQ.Stream.Client/Reliable/Producer.cs

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -9,7 +9,6 @@
99
using System.Threading.Tasks;
1010
using Microsoft.Extensions.Logging;
1111
using Microsoft.Extensions.Logging.Abstractions;
12-
using RabbitMQ.Stream.Client.Reconnect;
1312

1413
namespace RabbitMQ.Stream.Client.Reliable;
1514

RabbitMQ.Stream.Client/Reliable/ProducerFactory.cs

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -47,6 +47,7 @@ private async Task<IProducer> SuperStreamProducer(bool boot)
4747
Filter = _producerConfig.Filter,
4848
ConnectionClosedHandler = async (closeReason, partitionStream) =>
4949
{
50+
await RandomWait().ConfigureAwait(false);
5051
if (closeReason == ConnectionClosedReason.Normal)
5152
{
5253
BaseLogger.LogDebug("{Identity} is closed normally", ToString());
@@ -59,6 +60,7 @@ await OnEntityClosed(_producerConfig.StreamSystem, partitionStream, r)
5960
},
6061
MetadataHandler = async update =>
6162
{
63+
await RandomWait().ConfigureAwait(false);
6264
var r = ((RawSuperStreamProducer)(_producer)).ReconnectPartition;
6365
await OnEntityClosed(_producerConfig.StreamSystem, update.Stream, r)
6466
.ConfigureAwait(false);
@@ -99,10 +101,12 @@ private async Task<IProducer> StandardProducer()
99101
Filter = _producerConfig.Filter,
100102
MetadataHandler = async _ =>
101103
{
104+
await RandomWait().ConfigureAwait(false);
102105
await OnEntityClosed(_producerConfig.StreamSystem, _producerConfig.Stream).ConfigureAwait(false);
103106
},
104107
ConnectionClosedHandler = async (closeReason) =>
105108
{
109+
await RandomWait().ConfigureAwait(false);
106110
if (closeReason == ConnectionClosedReason.Normal)
107111
{
108112
BaseLogger.LogDebug("{Identity} is closed normally", ToString());

RabbitMQ.Stream.Client/Reliable/ReliableBase.cs

Lines changed: 22 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -6,7 +6,6 @@
66
using System.Threading;
77
using System.Threading.Tasks;
88
using Microsoft.Extensions.Logging;
9-
using RabbitMQ.Stream.Client.Reconnect;
109

1110
namespace RabbitMQ.Stream.Client.Reliable;
1211

@@ -50,6 +49,11 @@ public abstract class ReliableBase
5049
private readonly object _lock = new();
5150
protected ReliableEntityStatus _status = ReliableEntityStatus.Initialization;
5251

52+
protected static async Task RandomWait()
53+
{
54+
await Task.Delay(Consts.RandomMid()).ConfigureAwait(false);
55+
}
56+
5357
protected void UpdateStatus(ReliableEntityStatus status)
5458
{
5559
lock (_lock)
@@ -112,7 +116,6 @@ private async Task MaybeInit(bool boot)
112116

113117
reconnect = true;
114118
LogException(e);
115-
116119
}
117120

118121
if (reconnect)
@@ -130,6 +133,7 @@ private async Task Init(bool boot)
130133
BaseLogger.LogDebug("{Identity} is already closed. The init will be skipped", ToString());
131134
return;
132135
}
136+
133137
// each time that the client is initialized, we need to reset the status
134138
// if we hare here it means that the entity is not open for some reason like:
135139
// first time initialization or reconnect due of a IsAKnownException
@@ -156,7 +160,8 @@ private async Task Init(bool boot)
156160
/// <summary>
157161
/// When the clients receives a meta data update, it doesn't know
158162
/// If the stream exists or not. It just knows that the stream topology has changed.
159-
/// the method CheckIfStreamIsAvailable checks if the stream exists.
163+
/// the method CheckIfStreamIsAvailable checks if the stream exists
164+
/// and if the leader is available.
160165
/// </summary>
161166
/// <param name="stream">stream name</param>
162167
/// <param name="system">stream system</param>
@@ -172,7 +177,16 @@ private async Task<bool> CheckIfStreamIsAvailable(string stream, StreamSystem sy
172177
{
173178
exists = await system.StreamExists(stream).ConfigureAwait(false);
174179
var available = exists ? "available" : "not available";
175-
await _resourceAvailableReconnectStrategy.WhenConnected($"{stream} is {available}")
180+
if (exists)
181+
{
182+
// It is not enough to check if the stream exists
183+
// we need to check if the stream has the leader
184+
var streamInfo = await system.StreamInfo(stream).ConfigureAwait(false);
185+
ClientExceptions.CheckLeader(streamInfo);
186+
available += " and has a valid leader";
187+
}
188+
189+
await _resourceAvailableReconnectStrategy.WhenConnected($"{stream} for {ToString()} is {available}")
176190
.ConfigureAwait(false);
177191
break;
178192
}
@@ -229,7 +243,8 @@ private async Task MaybeReconnect()
229243
}
230244
}
231245

232-
private async Task MaybeReconnectPartition(StreamInfo streamInfo, string info, Func<StreamInfo, Task> reconnectPartitionFunc)
246+
private async Task MaybeReconnectPartition(StreamInfo streamInfo, string info,
247+
Func<StreamInfo, Task> reconnectPartitionFunc)
233248
{
234249
var reconnect = await _reconnectStrategy
235250
.WhenDisconnected($"Super Stream partition: {streamInfo.Stream} for {info}").ConfigureAwait(false);
@@ -285,7 +300,8 @@ private void LogException(Exception exception)
285300
/// <param name="system">Stream System</param>
286301
/// <param name="stream">Partition Stream</param>
287302
/// <param name="reconnectPartitionFunc">Function to reconnect the partition</param>
288-
internal async Task OnEntityClosed(StreamSystem system, string stream, Func<StreamInfo, Task> reconnectPartitionFunc)
303+
internal async Task OnEntityClosed(StreamSystem system, string stream,
304+
Func<StreamInfo, Task> reconnectPartitionFunc)
289305
{
290306
var streamExists = false;
291307
await SemaphoreSlim.WaitAsync().ConfigureAwait(false);

RabbitMQ.Stream.Client/StreamSystem.cs

Lines changed: 7 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -153,15 +153,6 @@ private async Task MayBeReconnectLocator()
153153
}
154154
}
155155

156-
private static void CheckLeader(StreamInfo metaStreamInfo)
157-
{
158-
if (metaStreamInfo.Leader.Equals(default(Broker)))
159-
{
160-
throw new LeaderNotFoundException(
161-
$"No leader found for streams {string.Join(" ", metaStreamInfo.Stream)}");
162-
}
163-
}
164-
165156
public async Task<ISuperStreamProducer> CreateRawSuperStreamProducer(
166157
RawSuperStreamProducerConfig rawSuperStreamProducerConfig, ILogger logger = null)
167158
{
@@ -197,6 +188,11 @@ public async Task<ISuperStreamProducer> CreateRawSuperStreamProducer(
197188
streamInfos[partitionsStream] = await StreamInfo(partitionsStream).ConfigureAwait(false);
198189
}
199190

191+
foreach (var (_, value) in streamInfos)
192+
{
193+
ClientExceptions.CheckLeader(value);
194+
}
195+
200196
var r = RawSuperStreamProducer.Create(rawSuperStreamProducerConfig,
201197
streamInfos,
202198
_clientParameters with { ClientProvidedName = rawSuperStreamProducerConfig.ClientProvidedName },
@@ -273,7 +269,7 @@ public async Task<IProducer> CreateRawProducer(RawProducerConfig rawProducerConf
273269
throw new CreateProducerException($"producer could not be created code: {metaStreamInfo.ResponseCode}");
274270
}
275271

276-
CheckLeader(metaStreamInfo);
272+
ClientExceptions.CheckLeader(metaStreamInfo);
277273

278274
try
279275
{
@@ -432,7 +428,7 @@ public async Task<IConsumer> CreateRawConsumer(RawConsumerConfig rawConsumerConf
432428
metaStreamInfo.ResponseCode);
433429
}
434430

435-
CheckLeader(metaStreamInfo);
431+
ClientExceptions.CheckLeader(metaStreamInfo);
436432

437433
try
438434
{

0 commit comments

Comments
 (0)