Skip to content

Commit

Permalink
Add support for super stream key routing (#270)
Browse files Browse the repository at this point in the history
* Add support for super stream key routing
---------

Signed-off-by: Gabriele Santomaggio <[email protected]>
  • Loading branch information
Gsantomaggio authored May 31, 2023
1 parent 0f5cd64 commit cdbbfc5
Show file tree
Hide file tree
Showing 12 changed files with 533 additions and 14 deletions.
11 changes: 10 additions & 1 deletion RabbitMQ.Stream.Client/Client.cs
Original file line number Diff line number Diff line change
Expand Up @@ -341,6 +341,12 @@ public async Task<PartitionsQueryResponse> QueryPartition(string superStream)
new PartitionsQueryRequest(corr, superStream)).ConfigureAwait(false);
}

public async Task<RouteQueryResponse> QueryRoute(string superStream, string routingKey)
{
return await Request<RouteQueryRequest, RouteQueryResponse>(corr =>
new RouteQueryRequest(corr, superStream, routingKey)).ConfigureAwait(false);
}

private async ValueTask<TOut> Request<TIn, TOut>(Func<uint, TIn> request, TimeSpan? timeout = null)
where TIn : struct, ICommand where TOut : struct, ICommand
{
Expand Down Expand Up @@ -526,7 +532,10 @@ private void HandleCorrelatedCommand(ushort tag, ref ReadOnlySequence<byte> fram
PartitionsQueryResponse.Read(frame, out var partitionsQueryResponse);
HandleCorrelatedResponse(partitionsQueryResponse);
break;

case RouteQueryResponse.Key:
RouteQueryResponse.Read(frame, out var routeQueryResponse);
HandleCorrelatedResponse(routeQueryResponse);
break;
default:
if (MemoryMarshal.TryGetArray(frame.First, out var segment))
{
Expand Down
11 changes: 11 additions & 0 deletions RabbitMQ.Stream.Client/ClientExceptions.cs
Original file line number Diff line number Diff line change
Expand Up @@ -82,4 +82,15 @@ public OffsetNotFoundException(string s)
{
}
}

// RouteNotFoundException the exception for super stream publish
// RouteNotFoundException is raised when the message can't be routed to any stream.
// In this case the user will receive a timeout error and this exception is raised
public class RouteNotFoundException : ProtocolException
{
public RouteNotFoundException(string s)
: base(s)
{
}
}
}
2 changes: 0 additions & 2 deletions RabbitMQ.Stream.Client/PublicAPI.Shipped.txt
Original file line number Diff line number Diff line change
Expand Up @@ -317,7 +317,6 @@ RabbitMQ.Stream.Client.Hash.Murmur3.Murmur3(uint seed) -> void
RabbitMQ.Stream.Client.Hash.Murmur3.Seed.get -> uint
RabbitMQ.Stream.Client.HashRoutingMurmurStrategy
RabbitMQ.Stream.Client.HashRoutingMurmurStrategy.HashRoutingMurmurStrategy(System.Func<RabbitMQ.Stream.Client.Message, string> routingKeyExtractor) -> void
RabbitMQ.Stream.Client.HashRoutingMurmurStrategy.Route(RabbitMQ.Stream.Client.Message message, System.Collections.Generic.List<string> partitions) -> System.Collections.Generic.List<string>
RabbitMQ.Stream.Client.HeartBeatHandler
RabbitMQ.Stream.Client.HeartBeatHandler.HeartBeatHandler(System.Func<System.Threading.Tasks.ValueTask<bool>> sendHeartbeatFunc, System.Func<string, System.Threading.Tasks.Task<RabbitMQ.Stream.Client.CloseResponse>> close, int heartbeat, Microsoft.Extensions.Logging.ILogger<RabbitMQ.Stream.Client.HeartBeatHandler> logger = null) -> void
RabbitMQ.Stream.Client.IClient
Expand Down Expand Up @@ -390,7 +389,6 @@ RabbitMQ.Stream.Client.IRouting.ValidateDns.get -> bool
RabbitMQ.Stream.Client.IRouting.ValidateDns.set -> void
RabbitMQ.Stream.Client.IRoutingConfiguration
RabbitMQ.Stream.Client.IRoutingStrategy
RabbitMQ.Stream.Client.IRoutingStrategy.Route(RabbitMQ.Stream.Client.Message message, System.Collections.Generic.List<string> partitions) -> System.Collections.Generic.List<string>
RabbitMQ.Stream.Client.LeaderLocator
RabbitMQ.Stream.Client.LeaderLocator.LeaderLocator() -> void
RabbitMQ.Stream.Client.LeaderNotFoundException
Expand Down
24 changes: 24 additions & 0 deletions RabbitMQ.Stream.Client/PublicAPI.Unshipped.txt
Original file line number Diff line number Diff line change
@@ -1,11 +1,20 @@
const RabbitMQ.Stream.Client.RouteQueryResponse.Key = 24 -> ushort
const RabbitMQ.Stream.Client.StreamStatsResponse.Key = 28 -> ushort
RabbitMQ.Stream.Client.AbstractEntity.MaybeCancelToken() -> void
RabbitMQ.Stream.Client.AbstractEntity.Token.get -> System.Threading.CancellationToken
RabbitMQ.Stream.Client.Chunk.Data.get -> System.Memory<byte>
RabbitMQ.Stream.Client.Chunk.MagicVersion.get -> byte
RabbitMQ.Stream.Client.Client.QueryRoute(string superStream, string routingKey) -> System.Threading.Tasks.Task<RabbitMQ.Stream.Client.RouteQueryResponse>
RabbitMQ.Stream.Client.Client.StreamStats(string stream) -> System.Threading.Tasks.ValueTask<RabbitMQ.Stream.Client.StreamStatsResponse>
RabbitMQ.Stream.Client.HashRoutingMurmurStrategy.Route(RabbitMQ.Stream.Client.Message message, System.Collections.Generic.List<string> partitions) -> System.Threading.Tasks.Task<System.Collections.Generic.List<string>>
RabbitMQ.Stream.Client.IConsumerConfig.InitialCredits.get -> ushort
RabbitMQ.Stream.Client.IConsumerConfig.InitialCredits.set -> void
RabbitMQ.Stream.Client.IRoutingStrategy.Route(RabbitMQ.Stream.Client.Message message, System.Collections.Generic.List<string> partitions) -> System.Threading.Tasks.Task<System.Collections.Generic.List<string>>
RabbitMQ.Stream.Client.KeyRoutingStrategy
RabbitMQ.Stream.Client.KeyRoutingStrategy.KeyRoutingStrategy(System.Func<RabbitMQ.Stream.Client.Message, string> routingKeyExtractor, System.Func<string, string, System.Threading.Tasks.Task<RabbitMQ.Stream.Client.RouteQueryResponse>> routingKeyQFunc, string superStream) -> void
RabbitMQ.Stream.Client.KeyRoutingStrategy.Route(RabbitMQ.Stream.Client.Message message, System.Collections.Generic.List<string> partitions) -> System.Threading.Tasks.Task<System.Collections.Generic.List<string>>
RabbitMQ.Stream.Client.RawSuperStreamProducerConfig.RoutingStrategyType.get -> RabbitMQ.Stream.Client.RoutingStrategyType
RabbitMQ.Stream.Client.RawSuperStreamProducerConfig.RoutingStrategyType.set -> void
RabbitMQ.Stream.Client.Reliable.ConsumerConfig.InitialCredits.get -> ushort
RabbitMQ.Stream.Client.Reliable.ConsumerConfig.InitialCredits.set -> void
RabbitMQ.Stream.Client.Reliable.DeduplicatingProducer
Expand All @@ -16,6 +25,21 @@ RabbitMQ.Stream.Client.Reliable.DeduplicatingProducer.Send(ulong publishing, Rab
RabbitMQ.Stream.Client.Reliable.DeduplicatingProducerConfig
RabbitMQ.Stream.Client.Reliable.DeduplicatingProducerConfig.DeduplicatingProducerConfig(RabbitMQ.Stream.Client.StreamSystem streamSystem, string stream, string reference) -> void
RabbitMQ.Stream.Client.Reliable.ProducerConfig.Reference.set -> void
RabbitMQ.Stream.Client.Reliable.SuperStreamConfig.RoutingStrategyType.get -> RabbitMQ.Stream.Client.RoutingStrategyType
RabbitMQ.Stream.Client.Reliable.SuperStreamConfig.RoutingStrategyType.set -> void
RabbitMQ.Stream.Client.RouteNotFoundException
RabbitMQ.Stream.Client.RouteNotFoundException.RouteNotFoundException(string s) -> void
RabbitMQ.Stream.Client.RouteQueryResponse
RabbitMQ.Stream.Client.RouteQueryResponse.CorrelationId.get -> uint
RabbitMQ.Stream.Client.RouteQueryResponse.ResponseCode.get -> RabbitMQ.Stream.Client.ResponseCode
RabbitMQ.Stream.Client.RouteQueryResponse.RouteQueryResponse() -> void
RabbitMQ.Stream.Client.RouteQueryResponse.RouteQueryResponse(uint correlationId, RabbitMQ.Stream.Client.ResponseCode responseCode, System.Collections.Generic.List<string> streams) -> void
RabbitMQ.Stream.Client.RouteQueryResponse.SizeNeeded.get -> int
RabbitMQ.Stream.Client.RouteQueryResponse.Streams.get -> System.Collections.Generic.List<string>
RabbitMQ.Stream.Client.RouteQueryResponse.Write(System.Span<byte> span) -> int
RabbitMQ.Stream.Client.RoutingStrategyType
RabbitMQ.Stream.Client.RoutingStrategyType.Hash = 0 -> RabbitMQ.Stream.Client.RoutingStrategyType
RabbitMQ.Stream.Client.RoutingStrategyType.Key = 1 -> RabbitMQ.Stream.Client.RoutingStrategyType
RabbitMQ.Stream.Client.StreamStats
RabbitMQ.Stream.Client.StreamStats.CommittedChunkId() -> ulong
RabbitMQ.Stream.Client.StreamStats.FirstOffset() -> ulong
Expand Down
74 changes: 67 additions & 7 deletions RabbitMQ.Stream.Client/RawSuperStreamProducer.cs
Original file line number Diff line number Diff line change
Expand Up @@ -63,7 +63,13 @@ private RawSuperStreamProducer(
_config = config;
_streamInfos = streamInfos;
_clientParameters = clientParameters;
_defaultRoutingConfiguration.RoutingStrategy = new HashRoutingMurmurStrategy(_config.Routing);
_defaultRoutingConfiguration.RoutingStrategy = _config.RoutingStrategyType switch
{
RoutingStrategyType.Key => new KeyRoutingStrategy(_config.Routing,
_config.Client.QueryRoute, _config.SuperStream),
RoutingStrategyType.Hash => new HashRoutingMurmurStrategy(_config.Routing),
_ => new HashRoutingMurmurStrategy(_config.Routing)
};
_logger = logger ?? NullLogger<RawSuperStreamProducer>.Instance;
}

Expand Down Expand Up @@ -122,7 +128,8 @@ private RawProducerConfig FromStreamConfig(string stream)
// The producer is created on demand when a message is sent to a stream
private async Task<IProducer> InitProducer(string stream)
{
var p = await RawProducer.Create(_clientParameters, FromStreamConfig(stream), _streamInfos[stream], _logger).ConfigureAwait(false);
var p = await RawProducer.Create(_clientParameters, FromStreamConfig(stream), _streamInfos[stream], _logger)
.ConfigureAwait(false);
_logger?.LogDebug("Producer {ProducerReference} created for Stream {StreamIdentifier}", _config.Reference,
stream);
return p;
Expand All @@ -147,8 +154,16 @@ private async Task<IProducer> GetProducerForMessage(Message message)
throw new ObjectDisposedException(nameof(RawSuperStreamProducer));
}

var routes = _defaultRoutingConfiguration.RoutingStrategy.Route(message,
_streamInfos.Keys.ToList());
var routes = await _defaultRoutingConfiguration.RoutingStrategy.Route(message,
_streamInfos.Keys.ToList()).ConfigureAwait(false);

// we should always have a route
// but in case of stream KEY the routing could not exist
if (routes is not { Count: > 0 })
{
throw new RouteNotFoundException("No route found for the message to any stream");
}

return await GetProducer(routes[0]).ConfigureAwait(false);
}

Expand Down Expand Up @@ -263,6 +278,12 @@ public void Dispose()
public int PendingCount => _producers.Sum(x => x.Value.PendingCount);
}

public enum RoutingStrategyType
{
Hash,
Key,
}

public record RawSuperStreamProducerConfig : IProducerConfig
{
public RawSuperStreamProducerConfig(string superStream)
Expand All @@ -285,6 +306,8 @@ public RawSuperStreamProducerConfig(string superStream)
public Func<Message, string> Routing { get; set; } = null;
public Action<(string, Confirmation)> ConfirmHandler { get; set; } = _ => { };

public RoutingStrategyType RoutingStrategyType { get; set; } = RoutingStrategyType.Hash;

internal Client Client { get; set; }
}

Expand All @@ -306,7 +329,7 @@ internal class DefaultRoutingConfiguration : IRoutingConfiguration
/// </summary>
public interface IRoutingStrategy
{
List<string> Route(Message message, List<string> partitions);
Task<List<string>> Route(Message message, List<string> partitions);
}

/// <summary>
Expand All @@ -322,16 +345,53 @@ public class HashRoutingMurmurStrategy : IRoutingStrategy
private const int Seed = 104729; // must be the same to all the clients to be compatible

// Routing based on the Murmur hash function
public List<string> Route(Message message, List<string> partitions)
public Task<List<string>> Route(Message message, List<string> partitions)
{
var key = _routingKeyExtractor(message);
var hash = new Murmur32ManagedX86(Seed).ComputeHash(Encoding.UTF8.GetBytes(key));
var index = BitConverter.ToUInt32(hash, 0) % (uint)partitions.Count;
return new List<string>() { partitions[(int)index] };
var r = new List<string>() { partitions[(int)index] };
return Task.FromResult(r);
}

public HashRoutingMurmurStrategy(Func<Message, string> routingKeyExtractor)
{
_routingKeyExtractor = routingKeyExtractor;
}
}

/// <summary>
/// KeyRoutingStrategy is a routing strategy that uses the routing key to route messages to streams.
/// </summary>
public class KeyRoutingStrategy : IRoutingStrategy
{
private readonly Func<Message, string> _routingKeyExtractor;
private readonly Func<string, string, Task<RouteQueryResponse>> _routingKeyQFunc;
private readonly string _superStream;
private readonly Dictionary<string, List<string>> _cacheStream = new();

public async Task<List<string>> Route(Message message, List<string> partitions)
{
var key = _routingKeyExtractor(message);
// If the stream is already in the cache we return it
// to avoid a query to the server for each send
if (_cacheStream.TryGetValue(key, out var value))
{
return value;
}

var c = await _routingKeyQFunc(_superStream, key).ConfigureAwait(false);
_cacheStream[key] = c.Streams;
return (from resultStream in c.Streams
where partitions.Contains(resultStream)
select new List<string>() { resultStream }).FirstOrDefault();
}

public KeyRoutingStrategy(Func<Message, string> routingKeyExtractor,
Func<string, string, Task<RouteQueryResponse>> routingKeyQFunc, string superStream)
{
_routingKeyExtractor = routingKeyExtractor;
_routingKeyQFunc = routingKeyQFunc;
_superStream = superStream;
}
}
20 changes: 20 additions & 0 deletions RabbitMQ.Stream.Client/Reliable/Producer.cs
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,8 @@ public record SuperStreamConfig
{
public bool Enabled { get; init; } = true;
public Func<Message, string> Routing { get; set; }

public RoutingStrategyType RoutingStrategyType { get; set; } = RoutingStrategyType.Hash;
}

[AttributeUsage(AttributeTargets.Method)]
Expand Down Expand Up @@ -257,6 +259,12 @@ internal async ValueTask SendInternal(ulong publishingId, Message message)
}
}

// see the RouteNotFoundException comment
catch (RouteNotFoundException)
{
throw;
}

catch (Exception e)
{
_logger?.LogError(e, "Error sending message. " +
Expand Down Expand Up @@ -291,6 +299,12 @@ public async ValueTask Send(List<Message> messages, CompressionType compressionT
}
}

// see the RouteNotFoundException comment
catch (RouteNotFoundException)
{
throw;
}

catch (Exception e)
{
_logger?.LogError(e, "Error sending sub-batch messages. " +
Expand Down Expand Up @@ -346,6 +360,12 @@ public async ValueTask Send(List<Message> messages)
}
}

// see the RouteNotFoundException comment
catch (RouteNotFoundException)
{
throw;
}

catch (Exception e)
{
_logger?.LogError(e, "Error sending messages. " +
Expand Down
6 changes: 5 additions & 1 deletion RabbitMQ.Stream.Client/Reliable/ProducerFactory.cs
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@ private async Task<IProducer> SuperStreamProducer()
MessagesBufferSize = _producerConfig.MessagesBufferSize,
MaxInFlight = _producerConfig.MaxInFlight,
Routing = _producerConfig.SuperStreamConfig.Routing,
RoutingStrategyType = _producerConfig.SuperStreamConfig.RoutingStrategyType,
ConfirmHandler = confirmationHandler =>
{
var (stream, confirmation) = confirmationHandler;
Expand Down Expand Up @@ -77,7 +78,10 @@ private async Task<IProducer> StandardProducer()
_producerConfig.StreamSystem).WaitAsync(CancellationToken.None);
});
},
ConnectionClosedHandler = async _ => { await TryToReconnect(_producerConfig.ReconnectStrategy).ConfigureAwait(false); },
ConnectionClosedHandler = async _ =>
{
await TryToReconnect(_producerConfig.ReconnectStrategy).ConfigureAwait(false);
},
ConfirmHandler = confirmation =>
{
var confirmationStatus = confirmation.Code switch
Expand Down
37 changes: 37 additions & 0 deletions RabbitMQ.Stream.Client/RouteQueryRequest.cs
Original file line number Diff line number Diff line change
@@ -0,0 +1,37 @@
// This source code is dual-licensed under the Apache License, version
// 2.0, and the Mozilla Public License, version 2.0.
// Copyright (c) 2007-2023 VMware, Inc.

using System;

namespace RabbitMQ.Stream.Client;

internal readonly struct RouteQueryRequest : ICommand
{
public const ushort Key = 0x0018;
private readonly string _superStream;
private readonly string _routingKey;
private readonly uint _corrId;

public RouteQueryRequest(uint corrId, string superStream, string routingKey)
{
_corrId = corrId;
_superStream = superStream;
_routingKey = routingKey;
}

public int SizeNeeded => 2 + 2 + 4 +
WireFormatting.StringSize(_superStream) +
WireFormatting.StringSize(_routingKey);

public int Write(Span<byte> span)
{
var command = (ICommand)this;
var offset = WireFormatting.WriteUInt16(span, Key);
offset += WireFormatting.WriteUInt16(span[offset..], command.Version);
offset += WireFormatting.WriteUInt32(span[offset..], _corrId);
offset += WireFormatting.WriteString(span[offset..], _routingKey);
offset += WireFormatting.WriteString(span[offset..], _superStream);
return offset;
}
}
47 changes: 47 additions & 0 deletions RabbitMQ.Stream.Client/RouteQueryResponse.cs
Original file line number Diff line number Diff line change
@@ -0,0 +1,47 @@
// This source code is dual-licensed under the Apache License, version
// 2.0, and the Mozilla Public License, version 2.0.
// Copyright (c) 2007-2023 VMware, Inc.

using System;
using System.Buffers;
using System.Collections.Generic;

namespace RabbitMQ.Stream.Client;

public struct RouteQueryResponse : ICommand
{
public const ushort Key = 0x0018;

public RouteQueryResponse(uint correlationId, ResponseCode responseCode, List<string> streams)
{
Streams = streams;
ResponseCode = responseCode;
CorrelationId = correlationId;
}

public List<string> Streams { get; }
public int SizeNeeded => throw new NotImplementedException();
public int Write(Span<byte> span) => throw new NotImplementedException();

public uint CorrelationId { get; }
public ResponseCode ResponseCode { get; }

internal static int Read(ReadOnlySequence<byte> frame, out RouteQueryResponse command)
{
var offset = WireFormatting.ReadUInt16(frame, out _);
offset += WireFormatting.ReadUInt16(frame.Slice(offset), out _);
offset += WireFormatting.ReadUInt32(frame.Slice(offset), out var correlationId);
offset += WireFormatting.ReadUInt16(frame.Slice(offset), out var responseCode);
offset += WireFormatting.ReadUInt32(frame.Slice(offset), out var numOfStreams);

var streams = new List<string>();
for (var i = 0; i < numOfStreams; i++)
{
offset += WireFormatting.ReadString(frame.Slice(offset), out var stream);
streams.Add(stream);
}

command = new RouteQueryResponse(correlationId, (ResponseCode)responseCode, streams);
return offset;
}
}
Loading

0 comments on commit cdbbfc5

Please sign in to comment.