Skip to content

Commit

Permalink
Timeouts: Improve Backlog handling and errors for users + GC rooting …
Browse files Browse the repository at this point in the history
…fixes for outstanding scenarios (#2408)

This combination PR is both fixing a GC issue (see below, and #2413 for details) and improves timeout exception. Basically if a timeout happens for a message that was in the backlog but was never sent, the user now gets a much more informative message like this:

> Exception: The message timed out in the backlog attempting to send because no connection became available - Last Connection Exception: InternalFailure on 127.0.0.1:6379/Interactive, Initializing/NotStarted, last: GET, origin: ConnectedAsync, outstanding: 0, last-read: 0s ago, last-write: 0s ago, keep-alive: 500s, state: Connecting, mgr: 10 of 10 available, last-heartbeat: never, last-mbeat: 0s ago, global: 0s ago, v: 2.6.99.22667, command=PING, inst: 0, qu: 0, qs: 0, aw: False, bw: CheckingForTimeout, last-in: 0, cur-in: 0, sync-ops: 1, async-ops: 1, serverEndpoint: 127.0.0.1:6379, conn-sec: n/a, aoc: 0, mc: 1/1/0, mgr: 10 of 10 available, clientName: NAMISTOU-3(SE.Redis-v2.6.99.22667), IOCP: (Busy=0,Free=1000,Min=32,Max=1000), WORKER: (Busy=2,Free=32765,Min=32,Max=32767), POOL: (Threads=18,QueuedItems=0,CompletedItems=65), v: 2.6.99.22667 (Please take a look at this article for some common client-side issues that can cause timeouts: https://stackexchange.github.io/StackExchange.Redis/Timeouts)

Today, this isn't intuitive especially for connections with `AbortOnConnectFail` set to `false`. What happens is a multiplexer _never_ connects successfully, but the user just gets generic timeouts. This makes the error more specific and includes the inner exception (also as `.InnerException`) for more details, informing the user of a config/auth/whatever error underneath as to why things are never successfully sending.

Also adds `aoc: (0|1)` to the exception message for easier advice in issues (reflecting what `AbortOnConnectFail` is set to).

Co-authored-by: Nick Craver <[email protected]>
Co-authored-by: Marc Gravell <[email protected]>
  • Loading branch information
NickCraver and mgravell authored Mar 30, 2023
1 parent f690d16 commit 129d59f
Show file tree
Hide file tree
Showing 15 changed files with 414 additions and 45 deletions.
2 changes: 1 addition & 1 deletion Directory.Packages.props
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@
<PackageVersion Include="Nerdbank.GitVersioning" Version="3.4.255" />
<PackageVersion Include="Newtonsoft.Json" Version="13.0.1" />
<PackageVersion Include="NSubstitute" Version="4.3.0" />
<PackageVersion Include="StackExchange.Redis" Version="2.5.43" />
<PackageVersion Include="StackExchange.Redis" Version="2.6.96" />
<!-- For binding redirect testing, main package gets this transitively -->
<PackageVersion Include="System.IO.Pipelines" Version="5.0.1" />
<PackageVersion Include="System.Runtime.Caching" Version="5.0.0" />
Expand Down
4 changes: 3 additions & 1 deletion docs/ReleaseNotes.md
Original file line number Diff line number Diff line change
Expand Up @@ -8,9 +8,11 @@ Current package versions:

## Unreleased

- Fix [#2412](https://github.com/StackExchange/StackExchange.Redis/issues/2412): Critical (but rare) GC bug that can lead to async tasks never completing if the multiplexer is not held by the consumer ([#2408 by mgravell](https://github.com/StackExchange/StackExchange.Redis/pull/2408))
- Add: Better error messages (over generic timeout) when commands are backlogged and unable to write to any connection ([#2408 by NickCraver](https://github.com/StackExchange/StackExchange.Redis/pull/2408))
- Fix [#2392](https://github.com/StackExchange/StackExchange.Redis/issues/2392): Dequeue *all* timed out messages from the backlog when not connected (including Fire+Forget) ([#2397 by kornelpal](https://github.com/StackExchange/StackExchange.Redis/pull/2397))
- Fix [#2400](https://github.com/StackExchange/StackExchange.Redis/issues/2400): Expose `ChannelMessageQueue` as `IAsyncEnumerable<ChannelMessage>` ([#2402 by mgravell](https://github.com/StackExchange/StackExchange.Redis/pull/2402))
- Add: support for `CLIENT SETINFO` (lib name/version) during handshake; opt-out is via `ConfigurationOptions`; also support read of `resp`, `lib-ver` and `lib-name` via `CLIENT LIST` ([#2414 by mgravell](https://github.com/StackExchange/StackExchange.Redis/pull/2414))
- Add: Support for `CLIENT SETINFO` (lib name/version) during handshake; opt-out is via `ConfigurationOptions`; also support read of `resp`, `lib-ver` and `lib-name` via `CLIENT LIST` ([#2414 by mgravell](https://github.com/StackExchange/StackExchange.Redis/pull/2414))
- Documentation: clarify the meaning of `RedisValue.IsInteger` re [#2418](https://github.com/StackExchange/StackExchange.Redis/issues/2418) ([#2420 by mgravell](https://github.com/StackExchange/StackExchange.Redis/pull/2420))

## 2.6.96
Expand Down
1 change: 1 addition & 0 deletions src/Directory.Build.props
Original file line number Diff line number Diff line change
Expand Up @@ -3,6 +3,7 @@
<PropertyGroup>
<GenerateDocumentationFile>true</GenerateDocumentationFile>
<IsPackable>true</IsPackable>
<CheckEolTargetFramework>false</CheckEolTargetFramework>
</PropertyGroup>
<ItemGroup>
<PackageReference Include="Microsoft.CodeAnalysis.PublicApiAnalyzers" PrivateAssets="all" />
Expand Down
141 changes: 123 additions & 18 deletions src/StackExchange.Redis/ConnectionMultiplexer.cs
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@ public sealed partial class ConnectionMultiplexer : IInternalConnectionMultiplex
internal long syncOps, asyncOps;
private long syncTimeouts, fireAndForgets, asyncTimeouts;
private string? failureMessage, activeConfigCause;
private IDisposable? pulse;
private TimerToken? pulse;

private readonly Hashtable servers = new Hashtable();
private volatile ServerSnapshot _serverSnapshot = ServerSnapshot.Empty;
Expand Down Expand Up @@ -874,7 +874,7 @@ public ServerSnapshotFiltered(ServerEndPoint[] endpoints, int count, Func<Server
}
}

[return: NotNullIfNotNull("endpoint")]
[return: NotNullIfNotNull(nameof(endpoint))]
internal ServerEndPoint? GetServerEndPoint(EndPoint? endpoint, LogProxy? log = null, bool activate = true)
{
if (endpoint == null) return null;
Expand Down Expand Up @@ -909,40 +909,130 @@ public ServerSnapshotFiltered(ServerEndPoint[] endpoints, int count, Func<Server
return server;
}

private sealed class TimerToken
internal void Root() => pulse?.Root(this);

// note that this also acts (conditionally) as the GC root for the multiplexer
// when there are in-flight messages; the timer can then acts as the heartbeat
// to make sure that everything *eventually* completes
private sealed class TimerToken : IDisposable
{
private TimerToken(ConnectionMultiplexer muxer)
{
_ref = new WeakReference(muxer);
_weakRef = new(muxer);
}
private Timer? _timer;
public void SetTimer(Timer timer) => _timer = timer;
private readonly WeakReference _ref;

private readonly WeakReference<ConnectionMultiplexer> _weakRef;

private object StrongRefSyncLock => _weakRef; // private and readonly? it'll do
private ConnectionMultiplexer? _strongRef;
private int _strongRefToken;

private static readonly TimerCallback Heartbeat = state =>
{
var token = (TimerToken)state!;
var muxer = (ConnectionMultiplexer?)(token._ref?.Target);
if (muxer != null)
if (token._weakRef.TryGetTarget(out var muxer))
{
muxer.OnHeartbeat();
}
else
{
// the muxer got disposed from out of us; kill the timer
var tmp = token._timer;
token._timer = null;
if (tmp != null) try { tmp.Dispose(); } catch { }
token.Dispose();
}
};

internal static IDisposable Create(ConnectionMultiplexer connection)
internal static TimerToken Create(ConnectionMultiplexer connection)
{
var token = new TimerToken(connection);
var heartbeatMilliseconds = (int)connection.RawConfig.HeartbeatInterval.TotalMilliseconds;
var timer = new Timer(Heartbeat, token, heartbeatMilliseconds, heartbeatMilliseconds);
token.SetTimer(timer);
return timer;
return token;
}

public void Dispose()
{
var tmp = _timer;
_timer = null;
if (tmp is not null) try { tmp.Dispose(); } catch { }

_strongRef = null; // note that this shouldn't be relevant since we've unrooted the TimerToken
}


// explanation of rooting model:
//
// the timer has a reference to the TimerToken; this *always* has a weak-ref,
// and *may* sometimes have a strong-ref; this is so that if a consumer
// drops a multiplexer, it can be garbage collected, i.e. the heartbeat timer
// doesn't keep the entire thing alive forever; instead, if the heartbeat detects
// the weak-ref has been collected, it can cancel the timer and *itself* go away;
// however: this leaves a problem where there is *in flight work* when the consumer
// drops the multiplexer; in particular, if that happens when disconnected, there
// could be consumer-visible pending TCS items *in the backlog queue*; we don't want
// to leave those incomplete, as that fails the contractual expectations of async/await;
// instead we need to root ourselves. The natural place to do this is by rooting the
// multiplexer, allowing the heartbeat to keep poking things, so that the usual
// message-processing and timeout rules apply. This is why we *sometimes* also keep
// a strong-ref to the same multiplexer.
//
// The TimerToken is rooted by the timer callback; this then roots the multiplexer,
// which keeps our bridges and connections in scope - until we're sure we're done
// with them.
//
// 1) any bridge or connection will trigger rooting by calling Root when
// they change from "empty" to "non-empty" i.e. whenever there
// in-flight items; this always changes the token; this includes both the
// backlog and awaiting-reply queues.
//
// 2) the heartbeat is responsible for unrooting, after processing timeouts
// etc; first it checks whether it is needed (IsRooted), which also gives
// it the current token.
//
// 3) if so, the heartbeat will (outside of the lock) query all sources to
// see if they still have outstanding work; if everyone reports negatively,
// then the heartbeat calls UnRoot passing in the old token; if this still
// matches (i.e. no new work came in while we were looking away), then the
// strong reference is removed; note that "has outstanding work" ignores
// internal-call messages; we are only interested in consumer-facing items
// (but we need to check this *here* rather than when adding, as otherwise
// the definition of "is empty, should root" becomes more complicated, which
// impacts the write path, rather than the heartbeat path.
//
// This means that the multiplexer (via the timer) lasts as long as there are
// outstanding messages; if the consumer has dropped the multiplexer, then
// there will be no new incoming messages, and after timeouts: everything
// should drop.

public void Root(ConnectionMultiplexer multiplexer)
{
lock (StrongRefSyncLock)
{
_strongRef = multiplexer;
_strongRefToken++;
}
}

public bool IsRooted(out int token)
{
lock (StrongRefSyncLock)
{
token = _strongRefToken;
return _strongRef is not null;
}
}

public void UnRoot(int token)
{
lock (StrongRefSyncLock)
{
if (token == _strongRefToken)
{
_strongRef = null;
}
}
}
}

Expand All @@ -956,8 +1046,21 @@ private void OnHeartbeat()
Trace("heartbeat");

var tmp = GetServerSnapshot();
int token = 0;
bool isRooted = pulse?.IsRooted(out token) ?? false, hasPendingCallerFacingItems = false;

for (int i = 0; i < tmp.Length; i++)
{
tmp[i].OnHeartbeat();
if (isRooted && !hasPendingCallerFacingItems)
{
hasPendingCallerFacingItems = tmp[i].HasPendingCallerFacingItems();
}
}
if (isRooted && !hasPendingCallerFacingItems)
{ // release the GC root on the heartbeat *if* the token still matches
pulse?.UnRoot(token);
}
}
catch (Exception ex)
{
Expand Down Expand Up @@ -1909,11 +2012,11 @@ private WriteResult TryPushMessageToBridgeSync<T>(Message message, ResultProcess
/// </summary>
public override string ToString() => string.IsNullOrWhiteSpace(ClientName) ? GetType().Name : ClientName;

internal Exception GetException(WriteResult result, Message message, ServerEndPoint? server) => result switch
internal Exception GetException(WriteResult result, Message message, ServerEndPoint? server, PhysicalBridge? bridge = null) => result switch
{
WriteResult.Success => throw new ArgumentOutOfRangeException(nameof(result), "Be sure to check result isn't successful before calling GetException."),
WriteResult.NoConnectionAvailable => ExceptionFactory.NoConnectionAvailable(this, message, server),
WriteResult.TimeoutBeforeWrite => ExceptionFactory.Timeout(this, "The timeout was reached before the message could be written to the output buffer, and it was not sent", message, server, result),
WriteResult.TimeoutBeforeWrite => ExceptionFactory.Timeout(this, null, message, server, result, bridge),
_ => ExceptionFactory.ConnectionFailure(RawConfig.IncludeDetailInExceptions, ConnectionFailureType.ProtocolFailure, "An unknown error occurred when writing the message", server),
};

Expand All @@ -1935,7 +2038,7 @@ internal static void ThrowFailed<T>(TaskCompletionSource<T>? source, Exception u
}
}

[return: NotNullIfNotNull("defaultValue")]
[return: NotNullIfNotNull(nameof(defaultValue))]
internal T? ExecuteSyncImpl<T>(Message message, ResultProcessor<T>? processor, ServerEndPoint? server, T? defaultValue = default)
{
if (_isDisposed) throw new ObjectDisposedException(ToString());
Expand All @@ -1960,10 +2063,11 @@ internal static void ThrowFailed<T>(TaskCompletionSource<T>? source, Exception u
var source = SimpleResultBox<T>.Get();

bool timeout = false;
WriteResult result;
lock (source)
{
#pragma warning disable CS0618 // Type or member is obsolete
var result = TryPushMessageToBridgeSync(message, processor, source, ref server);
result = TryPushMessageToBridgeSync(message, processor, source, ref server);
#pragma warning restore CS0618
if (result != WriteResult.Success)
{
Expand All @@ -1985,7 +2089,8 @@ internal static void ThrowFailed<T>(TaskCompletionSource<T>? source, Exception u
{
Interlocked.Increment(ref syncTimeouts);
// Very important not to return "source" to the pool here
throw ExceptionFactory.Timeout(this, null, message, server);
// Also note we return "success" when queueing a messages to the backlog, so we need to manually fake it back here when timing out in the backlog
throw ExceptionFactory.Timeout(this, null, message, server, message.IsBacklogged ? WriteResult.TimeoutBeforeWrite : result, server?.GetBridge(message.Command, create: false));
}
// Snapshot these so that we can recycle the box
var val = source.GetResult(out var ex, canRecycle: true); // now that we aren't locking it...
Expand Down Expand Up @@ -2047,7 +2152,7 @@ static async Task<T> ExecuteAsyncImpl_Awaited(ConnectionMultiplexer @this, Value

internal Task<T?> ExecuteAsyncImpl<T>(Message? message, ResultProcessor<T>? processor, object? state, ServerEndPoint? server)
{
[return: NotNullIfNotNull("tcs")]
[return: NotNullIfNotNull(nameof(tcs))]
static async Task<T?> ExecuteAsyncImpl_Awaited(ConnectionMultiplexer @this, ValueTask<WriteResult> write, TaskCompletionSource<T?>? tcs, Message message, ServerEndPoint? server)
{
var result = await write.ForAwait();
Expand Down
6 changes: 5 additions & 1 deletion src/StackExchange.Redis/Enums/CommandStatus.cs
Original file line number Diff line number Diff line change
Expand Up @@ -10,12 +10,16 @@ public enum CommandStatus
/// </summary>
Unknown,
/// <summary>
/// ConnectionMultiplexer has not yet started writing this command to redis.
/// ConnectionMultiplexer has not yet started writing this command to Redis.
/// </summary>
WaitingToBeSent,
/// <summary>
/// Command has been sent to Redis.
/// </summary>
Sent,
/// <summary>
/// Command is in the backlog, waiting to be processed and written to Redis.
/// </summary>
WaitingInBacklog,
}
}
44 changes: 35 additions & 9 deletions src/StackExchange.Redis/ExceptionFactory.cs
Original file line number Diff line number Diff line change
Expand Up @@ -212,13 +212,32 @@ private static void Add(List<Tuple<string, string>> data, StringBuilder sb, stri
}
}

internal static Exception Timeout(ConnectionMultiplexer multiplexer, string? baseErrorMessage, Message message, ServerEndPoint? server, WriteResult? result = null)
internal static Exception Timeout(ConnectionMultiplexer multiplexer, string? baseErrorMessage, Message message, ServerEndPoint? server, WriteResult? result = null, PhysicalBridge? bridge = null)
{
List<Tuple<string, string>> data = new List<Tuple<string, string>> { Tuple.Create("Message", message.CommandAndKey) };
var sb = new StringBuilder();

// We timeout writing messages in quite different ways sync/async - so centralize messaging here.
if (string.IsNullOrEmpty(baseErrorMessage) && result == WriteResult.TimeoutBeforeWrite)
{
baseErrorMessage = message.IsBacklogged
? "The message timed out in the backlog attempting to send because no connection became available"
: "The timeout was reached before the message could be written to the output buffer, and it was not sent";
}

var lastConnectionException = bridge?.LastException as RedisConnectionException;
var logConnectionException = message.IsBacklogged && lastConnectionException is not null;

if (!string.IsNullOrEmpty(baseErrorMessage))
{
sb.Append(baseErrorMessage);

// If we're in the situation where we've never connected
if (logConnectionException && lastConnectionException is not null)
{
sb.Append(" - Last Connection Exception: ").Append(lastConnectionException.Message);
}

if (message != null)
{
sb.Append(", command=").Append(message.Command); // no key here, note
Expand Down Expand Up @@ -252,17 +271,23 @@ internal static Exception Timeout(ConnectionMultiplexer multiplexer, string? bas
}
catch { }
}

AddCommonDetail(data, sb, message, multiplexer, server);

sb.Append(" (Please take a look at this article for some common client-side issues that can cause timeouts: ");
sb.Append(TimeoutHelpLink);
sb.Append(')');
sb.Append(" (Please take a look at this article for some common client-side issues that can cause timeouts: ")
.Append(TimeoutHelpLink)
.Append(')');

var ex = new RedisTimeoutException(sb.ToString(), message?.Status ?? CommandStatus.Unknown)
{
HelpLink = TimeoutHelpLink
};
// If we're from a backlog timeout scenario, we log a more intuitive connection exception for the timeout...because the timeout was a symptom
// and we have a more direct cause: we had no connection to send it on.
Exception ex = logConnectionException && lastConnectionException is not null
? new RedisConnectionException(lastConnectionException.FailureType, sb.ToString(), lastConnectionException, message?.Status ?? CommandStatus.Unknown)
{
HelpLink = TimeoutHelpLink
}
: new RedisTimeoutException(sb.ToString(), message?.Status ?? CommandStatus.Unknown)
{
HelpLink = TimeoutHelpLink
};
CopyDataToException(data, ex);

if (multiplexer.RawConfig.IncludeDetailInExceptions) AddExceptionDetail(ex, message, server, null);
Expand Down Expand Up @@ -333,6 +358,7 @@ private static void AddCommonDetail(
}
Add(data, sb, "Server-Endpoint", "serverEndpoint", (server.EndPoint.ToString() ?? "Unknown").Replace("Unspecified/", ""));
Add(data, sb, "Server-Connected-Seconds", "conn-sec", bs.ConnectedAt is DateTime dt ? (DateTime.UtcNow - dt).TotalSeconds.ToString("0.##") : "n/a");
Add(data, sb, "Abort-On-Connect", "aoc", multiplexer.RawConfig.AbortOnConnectFail ? "1" : "0");
}
Add(data, sb, "Multiplexer-Connects", "mc", $"{multiplexer._connectAttemptCount}/{multiplexer._connectCompletedCount}/{multiplexer._connectionCloseCount}");
Add(data, sb, "Manager", "mgr", multiplexer.SocketManager?.GetState());
Expand Down
Loading

0 comments on commit 129d59f

Please sign in to comment.