Fixed #34 Reconnect not working when restarting RabbitMQ

- Fixed deadlock issue when connection is lost
- Fixed Ack and Cancel being attempted on wrong connection causing channel disconnects
This commit is contained in:
Mark van Renswoude 2021-09-21 16:17:09 +02:00
parent ad7314c42f
commit b22c5200f4
4 changed files with 213 additions and 136 deletions

View File

@ -78,13 +78,13 @@ namespace Tapeti.Connection
/// <param name="queueName"></param> /// <param name="queueName"></param>
/// <param name="consumer">The consumer implementation which will receive the messages from the queue</param> /// <param name="consumer">The consumer implementation which will receive the messages from the queue</param>
/// <returns>The consumer tag as returned by BasicConsume.</returns> /// <returns>The consumer tag as returned by BasicConsume.</returns>
Task<string> Consume(CancellationToken cancellationToken, string queueName, IConsumer consumer); Task<TapetiConsumerTag> Consume(CancellationToken cancellationToken, string queueName, IConsumer consumer);
/// <summary> /// <summary>
/// Stops the consumer with the specified tag. /// Stops the consumer with the specified tag.
/// </summary> /// </summary>
/// <param name="consumerTag">The consumer tag as returned by Consume.</param> /// <param name="consumerTag">The consumer tag as returned by Consume.</param>
Task Cancel(string consumerTag); Task Cancel(TapetiConsumerTag consumerTag);
/// <summary> /// <summary>
/// Creates a durable queue if it does not already exist, and updates the bindings. /// Creates a durable queue if it does not already exist, and updates the bindings.
@ -129,4 +129,31 @@ namespace Tapeti.Connection
/// </summary> /// </summary>
Task Close(); Task Close();
} }
/// <summary>
/// Represents a consumer for a specific connection.
/// </summary>
public class TapetiConsumerTag
{
/// <summary>
/// The consumer tag as determined by the AMQP protocol.
/// </summary>
public string ConsumerTag { get; }
/// <summary>
/// An internal reference to the connection on which the consume was started.
/// </summary>
public long ConnectionReference { get;}
/// <summary>
/// Creates a new instance of the TapetiConsumerTag class.
/// </summary>
public TapetiConsumerTag(long connectionReference, string consumerTag)
{
ConnectionReference = connectionReference;
ConsumerTag = consumerTag;
}
}
} }

View File

@ -5,6 +5,9 @@ using Tapeti.Default;
namespace Tapeti.Connection namespace Tapeti.Connection
{ {
public delegate Task ResponseFunc(long expectedConnectionReference, ulong deliveryTag, ConsumeResult result);
/// <inheritdoc /> /// <inheritdoc />
/// <summary> /// <summary>
/// Implements the bridge between the RabbitMQ Client consumer and a Tapeti Consumer /// Implements the bridge between the RabbitMQ Client consumer and a Tapeti Consumer
@ -12,13 +15,15 @@ namespace Tapeti.Connection
internal class TapetiBasicConsumer : DefaultBasicConsumer internal class TapetiBasicConsumer : DefaultBasicConsumer
{ {
private readonly IConsumer consumer; private readonly IConsumer consumer;
private readonly Func<ulong, ConsumeResult, Task> onRespond; private readonly long connectionReference;
private readonly ResponseFunc onRespond;
/// <inheritdoc /> /// <inheritdoc />
public TapetiBasicConsumer(IConsumer consumer, Func<ulong, ConsumeResult, Task> onRespond) public TapetiBasicConsumer(IConsumer consumer, long connectionReference, ResponseFunc onRespond)
{ {
this.consumer = consumer; this.consumer = consumer;
this.connectionReference = connectionReference;
this.onRespond = onRespond; this.onRespond = onRespond;
} }
@ -45,11 +50,11 @@ namespace Tapeti.Connection
try try
{ {
var response = await consumer.Consume(exchange, routingKey, new RabbitMQMessageProperties(properties), bodyArray); var response = await consumer.Consume(exchange, routingKey, new RabbitMQMessageProperties(properties), bodyArray);
await onRespond(deliveryTag, response); await onRespond(connectionReference, deliveryTag, response);
} }
catch catch
{ {
await onRespond(deliveryTag, ConsumeResult.Error); await onRespond(connectionReference, deliveryTag, ConsumeResult.Error);
} }
}); });
} }

View File

@ -51,6 +51,7 @@ namespace Tapeti.Connection
// These fields must be locked using connectionLock // These fields must be locked using connectionLock
private readonly object connectionLock = new(); private readonly object connectionLock = new();
private long connectionReference;
private RabbitMQ.Client.IConnection connection; private RabbitMQ.Client.IConnection connection;
private IModel consumeChannelModel; private IModel consumeChannelModel;
private IModel publishChannelModel; private IModel publishChannelModel;
@ -200,7 +201,7 @@ namespace Tapeti.Connection
/// <inheritdoc /> /// <inheritdoc />
public async Task<string> Consume(CancellationToken cancellationToken, string queueName, IConsumer consumer) public async Task<TapetiConsumerTag> Consume(CancellationToken cancellationToken, string queueName, IConsumer consumer)
{ {
if (deletedQueues.Contains(queueName)) if (deletedQueues.Contains(queueName))
return null; return null;
@ -209,6 +210,7 @@ namespace Tapeti.Connection
throw new ArgumentNullException(nameof(queueName)); throw new ArgumentNullException(nameof(queueName));
long capturedConnectionReference = -1;
string consumerTag = null; string consumerTag = null;
await GetTapetiChannel(TapetiChannelType.Consume).QueueRetryable(channel => await GetTapetiChannel(TapetiChannelType.Consume).QueueRetryable(channel =>
@ -216,33 +218,52 @@ namespace Tapeti.Connection
if (cancellationToken.IsCancellationRequested) if (cancellationToken.IsCancellationRequested)
return; return;
var basicConsumer = new TapetiBasicConsumer(consumer, Respond); capturedConnectionReference = Interlocked.Read(ref connectionReference);
var basicConsumer = new TapetiBasicConsumer(consumer, capturedConnectionReference, Respond);
consumerTag = channel.BasicConsume(queueName, false, basicConsumer); consumerTag = channel.BasicConsume(queueName, false, basicConsumer);
}); });
return consumerTag; return new TapetiConsumerTag(capturedConnectionReference, consumerTag);
} }
/// <inheritdoc /> /// <inheritdoc />
public async Task Cancel(string consumerTag) public async Task Cancel(TapetiConsumerTag consumerTag)
{ {
if (isClosing || string.IsNullOrEmpty(consumerTag)) if (isClosing || string.IsNullOrEmpty(consumerTag.ConsumerTag))
return;
var capturedConnectionReference = Interlocked.Read(ref connectionReference);
// If the connection was re-established in the meantime, don't respond with an
// invalid deliveryTag. The message will be requeued.
if (capturedConnectionReference != consumerTag.ConnectionReference)
return; return;
// No need for a retryable channel here, if the connection is lost // No need for a retryable channel here, if the connection is lost
// so is the consumer. // so is the consumer.
await GetTapetiChannel(TapetiChannelType.Consume).Queue(channel => await GetTapetiChannel(TapetiChannelType.Consume).Queue(channel =>
{ {
channel.BasicCancel(consumerTag); // Check again as a reconnect may have occured in the meantime
var currentConnectionReference = Interlocked.Read(ref connectionReference);
if (currentConnectionReference != consumerTag.ConnectionReference)
return;
channel.BasicCancel(consumerTag.ConsumerTag);
}); });
} }
private async Task Respond(ulong deliveryTag, ConsumeResult result) private async Task Respond(long expectedConnectionReference, ulong deliveryTag, ConsumeResult result)
{ {
await GetTapetiChannel(TapetiChannelType.Consume).Queue(channel => await GetTapetiChannel(TapetiChannelType.Consume).Queue(channel =>
{ {
// If the connection was re-established in the meantime, don't respond with an
// invalid deliveryTag. The message will be requeued.
var currentConnectionReference = Interlocked.Read(ref connectionReference);
if (currentConnectionReference != connectionReference)
return;
// No need for a retryable channel here, if the connection is lost we can't // No need for a retryable channel here, if the connection is lost we can't
// use the deliveryTag anymore. // use the deliveryTag anymore.
switch (result) switch (result)
@ -487,8 +508,8 @@ namespace Tapeti.Connection
await publishChannel.Reset(); await publishChannel.Reset();
// No need to close the channels as the connection will be closed // No need to close the channels as the connection will be closed
capturedConsumeModel.Dispose(); capturedConsumeModel?.Dispose();
capturedPublishModel.Dispose(); capturedPublishModel?.Dispose();
// ReSharper disable once InvertIf // ReSharper disable once InvertIf
if (capturedConnection != null) if (capturedConnection != null)
@ -695,56 +716,76 @@ namespace Tapeti.Connection
if (channel != null && channel.IsOpen) if (channel != null && channel.IsOpen)
return channel; return channel;
}
// If the Disconnect quickly follows the Connect (when an error occurs that is reported back by RabbitMQ // If the Disconnect quickly follows the Connect (when an error occurs that is reported back by RabbitMQ
// not related to the connection), wait for a bit to avoid spamming the connection // not related to the connection), wait for a bit to avoid spamming the connection
if ((DateTime.UtcNow - connectedDateTime).TotalMilliseconds <= MinimumConnectedReconnectDelay) if ((DateTime.UtcNow - connectedDateTime).TotalMilliseconds <= MinimumConnectedReconnectDelay)
Thread.Sleep(ReconnectDelay); Thread.Sleep(ReconnectDelay);
var connectionFactory = new ConnectionFactory var connectionFactory = new ConnectionFactory
{
HostName = connectionParams.HostName,
Port = connectionParams.Port,
VirtualHost = connectionParams.VirtualHost,
UserName = connectionParams.Username,
Password = connectionParams.Password,
AutomaticRecoveryEnabled = false,
TopologyRecoveryEnabled = false,
RequestedHeartbeat = TimeSpan.FromSeconds(30)
};
if (connectionParams.ClientProperties != null)
foreach (var pair in connectionParams.ClientProperties)
{ {
HostName = connectionParams.HostName, if (connectionFactory.ClientProperties.ContainsKey(pair.Key))
Port = connectionParams.Port, connectionFactory.ClientProperties[pair.Key] = Encoding.UTF8.GetBytes(pair.Value);
VirtualHost = connectionParams.VirtualHost, else
UserName = connectionParams.Username, connectionFactory.ClientProperties.Add(pair.Key, Encoding.UTF8.GetBytes(pair.Value));
Password = connectionParams.Password, }
AutomaticRecoveryEnabled = false,
TopologyRecoveryEnabled = false,
RequestedHeartbeat = TimeSpan.FromSeconds(30)
};
if (connectionParams.ClientProperties != null)
foreach (var pair in connectionParams.ClientProperties) while (true)
{
try
{
RabbitMQ.Client.IConnection capturedConnection;
IModel capturedConsumeChannelModel;
IModel capturedPublishChannelModel;
lock (connectionLock)
{ {
if (connectionFactory.ClientProperties.ContainsKey(pair.Key)) capturedConnection = connection;
connectionFactory.ClientProperties[pair.Key] = Encoding.UTF8.GetBytes(pair.Value);
else
connectionFactory.ClientProperties.Add(pair.Key, Encoding.UTF8.GetBytes(pair.Value));
} }
if (capturedConnection != null)
while (true)
{
try
{ {
if (connection != null) try
{ {
try if (connection.IsOpen)
{
connection.Close(); connection.Close();
} }
finally catch (AlreadyClosedException)
{ {
connection.Dispose(); }
} finally
{
connection = null; connection.Dispose();
} }
logger.Connect(new ConnectContext(connectionParams, isReconnect)); connection = null;
}
logger.Connect(new ConnectContext(connectionParams, isReconnect));
Interlocked.Increment(ref connectionReference);
lock (connectionLock)
{
connection = connectionFactory.CreateConnection(); connection = connectionFactory.CreateConnection();
capturedConnection = connection;
consumeChannelModel = connection.CreateModel(); consumeChannelModel = connection.CreateModel();
if (consumeChannel == null) if (consumeChannel == null)
throw new BrokerUnreachableException(null); throw new BrokerUnreachableException(null);
@ -753,98 +794,102 @@ namespace Tapeti.Connection
if (publishChannel == null) if (publishChannel == null)
throw new BrokerUnreachableException(null); throw new BrokerUnreachableException(null);
capturedConsumeChannelModel = consumeChannelModel;
capturedPublishChannelModel = publishChannelModel;
}
if (config.Features.PublisherConfirms)
if (config.Features.PublisherConfirms)
{
lastDeliveryTag = 0;
Monitor.Enter(confirmLock);
try
{ {
lastDeliveryTag = 0; foreach (var pair in confirmMessages)
pair.Value.CompletionSource.SetCanceled();
Monitor.Enter(confirmLock); confirmMessages.Clear();
try }
{ finally
foreach (var pair in confirmMessages) {
pair.Value.CompletionSource.SetCanceled(); Monitor.Exit(confirmLock);
confirmMessages.Clear();
}
finally
{
Monitor.Exit(confirmLock);
}
publishChannelModel.ConfirmSelect();
} }
if (connectionParams.PrefetchCount > 0) capturedPublishChannelModel.ConfirmSelect();
consumeChannelModel.BasicQos(0, connectionParams.PrefetchCount, false);
var capturedConsumeChannelModel = consumeChannelModel;
consumeChannelModel.ModelShutdown += (_, e) =>
{
lock (connectionLock)
{
if (consumeChannelModel == null || consumeChannelModel != capturedConsumeChannelModel)
return;
consumeChannelModel = null;
}
ConnectionEventListener?.Disconnected(new DisconnectedEventArgs
{
ReplyCode = e.ReplyCode,
ReplyText = e.ReplyText
});
logger.Disconnect(new DisconnectContext(connectionParams, e.ReplyCode, e.ReplyText));
// Reconnect if the disconnect was unexpected
if (!isClosing)
GetTapetiChannel(TapetiChannelType.Consume).QueueRetryable(_ => { });
};
var capturedPublishChannelModel = publishChannelModel;
publishChannelModel.ModelShutdown += (_, _) =>
{
lock (connectionLock)
{
if (publishChannelModel == null || publishChannelModel != capturedPublishChannelModel)
return;
publishChannelModel = null;
}
// No need to reconnect, the next Publish will
};
publishChannelModel.BasicReturn += HandleBasicReturn;
publishChannelModel.BasicAcks += HandleBasicAck;
publishChannelModel.BasicNacks += HandleBasicNack;
connectedDateTime = DateTime.UtcNow;
var connectedEventArgs = new ConnectedEventArgs
{
ConnectionParams = connectionParams,
LocalPort = connection.LocalPort
};
if (isReconnect)
ConnectionEventListener?.Reconnected(connectedEventArgs);
else
ConnectionEventListener?.Connected(connectedEventArgs);
logger.ConnectSuccess(new ConnectContext(connectionParams, isReconnect, connection.LocalPort));
isReconnect = true;
break;
} }
catch (BrokerUnreachableException e)
if (connectionParams.PrefetchCount > 0)
capturedPublishChannelModel.BasicQos(0, connectionParams.PrefetchCount, false);
capturedPublishChannelModel.ModelShutdown += (_, e) =>
{ {
logger.ConnectFailed(new ConnectContext(connectionParams, isReconnect, exception: e)); lock (connectionLock)
Thread.Sleep(ReconnectDelay); {
} if (consumeChannelModel == null || consumeChannelModel != capturedConsumeChannelModel)
return;
consumeChannelModel = null;
}
ConnectionEventListener?.Disconnected(new DisconnectedEventArgs
{
ReplyCode = e.ReplyCode,
ReplyText = e.ReplyText
});
logger.Disconnect(new DisconnectContext(connectionParams, e.ReplyCode, e.ReplyText));
// Reconnect if the disconnect was unexpected
if (!isClosing)
GetTapetiChannel(TapetiChannelType.Consume).QueueRetryable(_ => { });
};
capturedPublishChannelModel.ModelShutdown += (_, _) =>
{
lock (connectionLock)
{
if (publishChannelModel == null || publishChannelModel != capturedPublishChannelModel)
return;
publishChannelModel = null;
}
// No need to reconnect, the next Publish will
};
capturedPublishChannelModel.BasicReturn += HandleBasicReturn;
capturedPublishChannelModel.BasicAcks += HandleBasicAck;
capturedPublishChannelModel.BasicNacks += HandleBasicNack;
connectedDateTime = DateTime.UtcNow;
var connectedEventArgs = new ConnectedEventArgs
{
ConnectionParams = connectionParams,
LocalPort = capturedConnection.LocalPort
};
if (isReconnect)
ConnectionEventListener?.Reconnected(connectedEventArgs);
else
ConnectionEventListener?.Connected(connectedEventArgs);
logger.ConnectSuccess(new ConnectContext(connectionParams, isReconnect, capturedConnection.LocalPort));
isReconnect = true;
break;
} }
catch (BrokerUnreachableException e)
{
logger.ConnectFailed(new ConnectContext(connectionParams, isReconnect, exception: e));
Thread.Sleep(ReconnectDelay);
}
}
lock (connectionLock)
{
return channelType == TapetiChannelType.Publish return channelType == TapetiChannelType.Publish
? publishChannelModel ? publishChannelModel
: consumeChannelModel; : consumeChannelModel;

View File

@ -1,4 +1,4 @@
using System; using System;
using System.Collections.Generic; using System.Collections.Generic;
using System.Linq; using System.Linq;
using System.Threading; using System.Threading;
@ -13,7 +13,7 @@ namespace Tapeti.Connection
private readonly Func<ITapetiClient> clientFactory; private readonly Func<ITapetiClient> clientFactory;
private readonly ITapetiConfig config; private readonly ITapetiConfig config;
private bool consuming; private bool consuming;
private readonly List<string> consumerTags = new(); private readonly List<TapetiConsumerTag> consumerTags = new();
private CancellationTokenSource initializeCancellationTokenSource; private CancellationTokenSource initializeCancellationTokenSource;