Skip to content

Commit

Permalink
Fix the read in the AMQP parser (#217)
Browse files Browse the repository at this point in the history
* Fix the read in the AMQP parser
- Change the string read: use the offset len coming from the protocol instead
  of encoding.GetByteCount(value). In some case the message properties can't be UFT8.
- Use Encoding.ASCII.GetString in case of FormatCode.Sym8 that's more corret.
- Add the AMQPlite dependency on the Test package to simulate the use case where the
  parser failed.
- Add amqp 1.0 plugin to the CI
- bump rabbitmq version on Windows CI

Signed-off-by: Gabriele Santomaggio <[email protected]>
Co-authored-by: Luke Bakken <[email protected]>
  • Loading branch information
Gsantomaggio and lukebakken authored Jan 21, 2023
1 parent bb71c52 commit aac7819
Show file tree
Hide file tree
Showing 6 changed files with 81 additions and 12 deletions.
2 changes: 1 addition & 1 deletion .ci/install.ps1
Original file line number Diff line number Diff line change
Expand Up @@ -158,4 +158,4 @@ Write-Host '[INFO] Getting RabbitMQ status...'

$ErrorActionPreference = 'Continue'
Write-Host '[INFO] Enabling plugins...'
& $rabbitmq_plugins_path enable rabbitmq_management rabbitmq_stream rabbitmq_stream_management
& $rabbitmq_plugins_path enable rabbitmq_management rabbitmq_stream rabbitmq_stream_management rabbitmq_amqp1_0
2 changes: 1 addition & 1 deletion .ci/versions.json
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
{
"erlang": "25.2",
"rabbitmq": "3.11.6"
"rabbitmq": "3.11.7"
}
2 changes: 1 addition & 1 deletion .github/workflows/build-test.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -68,7 +68,7 @@ jobs:
restore-keys: |
${{ runner.os }}-v2-nuget-
- name: Enable RabbitMQ Plugins
run: docker exec ${{ job.services.rabbitmq.id }} rabbitmq-plugins enable rabbitmq_stream rabbitmq_stream_management
run: docker exec ${{ job.services.rabbitmq.id }} rabbitmq-plugins enable rabbitmq_stream rabbitmq_stream_management rabbitmq_amqp1_0
- name: Restore
run: dotnet restore --verbosity=normal
- name: Build
Expand Down
19 changes: 15 additions & 4 deletions RabbitMQ.Stream.Client/AMQP/AmqpWireFormattingRead.cs
Original file line number Diff line number Diff line change
Expand Up @@ -169,23 +169,34 @@ internal static int ReadString(ref SequenceReader<byte> reader, out string value
var offset = ReadType(ref reader, out var type);
switch (type)
{
case FormatCode.Str8:
case FormatCode.Sym8:
offset += WireFormatting.ReadByte(ref reader, out var lenAscii);
Span<byte> tempSpanAscii = stackalloc byte[lenAscii];
reader.TryCopyTo(tempSpanAscii);
reader.Advance(lenAscii);
value = Encoding.ASCII.GetString(tempSpanAscii);
return offset + lenAscii;
case FormatCode.Str8:
offset += WireFormatting.ReadByte(ref reader, out var lenC);
Span<byte> tempSpan = stackalloc byte[lenC];
reader.TryCopyTo(tempSpan);
reader.Advance(lenC);
value = Encoding.UTF8.GetString(tempSpan);
return offset + s_encoding.GetByteCount(value);

return offset + lenC;
case FormatCode.Sym32:
offset += WireFormatting.ReadInt32(ref reader, out var lenAscii32);
var tempSpanAscii32 = lenAscii32 <= 64 ? stackalloc byte[lenAscii32] : new byte[lenAscii32];
reader.TryCopyTo(tempSpanAscii32);
reader.Advance(lenAscii32);
value = Encoding.ASCII.GetString(tempSpanAscii32);
return offset + lenAscii32;
case FormatCode.Str32:
offset += WireFormatting.ReadInt32(ref reader, out var len);
var tempSpan32 = len <= 64 ? stackalloc byte[len] : new byte[len];
reader.TryCopyTo(tempSpan32);
reader.Advance(len);
value = Encoding.UTF8.GetString(tempSpan32);
return offset + s_encoding.GetByteCount(value);
return offset + len;
}

throw new AMQP.AmqpParseException($"ReadString invalid type {type}");
Expand Down
67 changes: 62 additions & 5 deletions Tests/FromToAMQPTests.cs
Original file line number Diff line number Diff line change
Expand Up @@ -8,12 +8,15 @@
using System.Text;
using System.Threading;
using System.Threading.Tasks;
using Amqp;
using RabbitMQ.Client;
using RabbitMQ.Client.Events;
using RabbitMQ.Stream.Client;
using RabbitMQ.Stream.Client.AMQP;
using RabbitMQ.Stream.Client.Reliable;
using Xunit.Abstractions;
using ConnectionFactory = RabbitMQ.Client.ConnectionFactory;
using Message = RabbitMQ.Stream.Client.Message;

namespace Tests;

Expand Down Expand Up @@ -102,8 +105,7 @@ public async void Amqp10ShouldReadTheAmqp019Properties()
properties.UserId = "guest";
properties.Headers = new Dictionary<string, object>()
{
{"stream_key", "stream_value"},
{"stream_key4", "Alan Mathison Turing(1912 年 6 月 23 日"},
{"stream_key", "stream_value"}, {"stream_key4", "Alan Mathison Turing(1912 年 6 月 23 日"},
};
channel.BasicPublish("", stream, properties, Encoding.ASCII.GetBytes("FromAMQP"));
var tcs = new TaskCompletionSource<Message>();
Expand Down Expand Up @@ -170,20 +172,20 @@ await producer.Send(new Message(Encoding.ASCII.GetBytes($"FromStream{i}"))
{
Properties = new Properties()
{
MessageId = $"Alan Mathison Turing(1912 年 6 月 23 日 - 1954 年 6 月 7 日)是英国数学家、计算机科学家、逻辑学家、密码分析家、哲学家和理论生物学家。 [6] 图灵在理论计算机科学的发展中具有很大的影响力,用图灵机提供了算法和计算概念的形式化,可以被认为是通用计算机的模型。[7][8][9] 他被广泛认为是理论计算机科学和人工智能之父{i}",
MessageId =
$"Alan Mathison Turing(1912 年 6 月 23 日 - 1954 年 6 月 7 日)是英国数学家、计算机科学家、逻辑学家、密码分析家、哲学家和理论生物学家。 [6] 图灵在理论计算机科学的发展中具有很大的影响力,用图灵机提供了算法和计算概念的形式化,可以被认为是通用计算机的模型。[7][8][9] 他被广泛认为是理论计算机科学和人工智能之父{i}",
CorrelationId = "10000_00000",
ContentType = "text/plain",
ContentEncoding = "utf-8",
UserId = Encoding.ASCII.GetBytes("MY_USER_ID"),
GroupSequence = 601,
ReplyToGroupId = "ReplyToGroupId",
GroupId = "GroupId",

},
ApplicationProperties = new ApplicationProperties()
{
{"stream_key", "stream_value"},
{"stream_key2", 100},
{"stream_key2", 100},
{"stream_key3", 10_000_009},
{"stream_key4", "Alan Mathison Turing(1912 年 6 月 23 日"},
}
Expand Down Expand Up @@ -258,4 +260,59 @@ public async void Amqp10ShouldReadTheAmqp019Properties1000Messages()
await system.DeleteStream(stream);
await system.Close();
}

/// <summary>
/// In this test se send 1 message using the Amqp10 Producer https://github.com/Azure/amqpnetlite to
/// a stream and then we read it using.
/// See https://github.com/rabbitmq/rabbitmq-stream-dotnet-client/pull/217
/// </summary>
[Fact]
public async void StreamShouldReadTheAmqp10PropertiesMessages()
{
SystemUtils.InitStreamSystemWithRandomStream(out var system, out var stream);

var address = new Address("amqp://guest:guest@localhost:5672");
var connection = new Amqp.Connection(address);
var session = new Session(connection);

var message = new Amqp.Message("msg from amqp 1.0");
message.Properties = new Amqp.Framing.Properties()
{
MessageId = "1",
Subject = "test",
ContentType = "text/plain"
};
message.ApplicationProperties = new Amqp.Framing.ApplicationProperties()
{
Map = { { "key1", "value1" }, { "key2", 2 } }
};

var sender = new SenderLink(session, "mixing", $"/amq/queue/{stream}");
await sender.SendAsync(message);
await sender.CloseAsync();
await session.CloseAsync();
await connection.CloseAsync();

var tcs = new TaskCompletionSource<Message>();
var consumer = await Consumer.Create(new ConsumerConfig(system, stream)
{
OffsetSpec = new OffsetTypeFirst(),
MessageHandler = async (_, _, _, streamMessage) =>
{
tcs.SetResult(streamMessage);
await Task.CompletedTask;
}
});

new Utils<Message>(_testOutputHelper).WaitUntilTaskCompletes(tcs);
var result = tcs.Task.Result;
// Why do we need result.Data.Contents.ToArray()[5..]?
// Because of https://github.com/rabbitmq/rabbitmq-server/issues/6937
// When it will be fixed we can remove the [5..]
// For the moment we leave it as it is because it is not a problem for the client
Assert.Equal("msg from amqp 1.0", Encoding.UTF8.GetString(result.Data.Contents.ToArray()[5..]));
await consumer.Close();
await system.DeleteStream(stream);
await system.Close();
}
}
1 change: 1 addition & 0 deletions Tests/Tests.csproj
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,7 @@

<ItemGroup>
<PackageReference Include="AltCover" Version="8.2.837" />
<PackageReference Include="AmqpNetLite" Version="2.4.5" />
<PackageReference Include="Microsoft.NET.Test.Sdk" Version="17.1.0" />
<PackageReference Include="RabbitMQ.Client" Version="6.4.0" />
<PackageReference Include="xunit" Version="2.4.1" />
Expand Down

0 comments on commit aac7819

Please sign in to comment.