From 0210ac9675d16338de990f925418f51423d880fa Mon Sep 17 00:00:00 2001 From: bollhals Date: Wed, 15 Sep 2021 22:46:23 +0200 Subject: [PATCH 1/2] introduce struct version of BasicProperties for basicPublish --- .../AsyncBasicConsumerFake.cs | 4 +- .../ConsumerDispatching/ConsumerDispatcher.cs | 2 +- .../Networking_BasicDeliver_Commons.cs | 7 +- ...orking_BasicDeliver_LongLivedConnection.cs | 4 +- .../WireFormatting/MethodFraming.cs | 15 +- .../WireFormatting/MethodSerialization.cs | 26 +- .../client/api/AsyncDefaultBasicConsumer.cs | 4 +- .../client/api/BasicGetResult.cs | 8 +- .../client/api/BasicProperties.cs | 262 +++++++++++++ .../client/api/CachedString.cs | 2 +- .../client/api/DefaultBasicConsumer.cs | 2 +- .../api/{IContentHeader.cs => IAmqpHeader.cs} | 12 +- .../client/api/IAmqpWriteable.cs} | 28 +- .../client/api/IAsyncBasicConsumer.cs | 2 +- .../client/api/IBasicConsumer.cs | 2 +- .../client/api/IBasicProperties.cs | 263 ++++++++----- projects/RabbitMQ.Client/client/api/IModel.cs | 13 +- .../client/api/IModelExtensions.cs | 40 +- .../client/api/IStreamProperties.cs | 129 ------- .../client/api/ReadonlyBasicProperties.cs | 123 ++++++ .../events/AsyncEventingBasicConsumer.cs | 2 +- .../client/events/BasicDeliverEventArgs.cs | 4 +- .../client/events/BasicReturnEventArgs.cs | 2 +- .../client/events/EventingBasicConsumer.cs | 2 +- .../client/framing/BasicAck.cs | 2 +- .../client/framing/BasicCancel.cs | 2 +- .../client/framing/BasicConsume.cs | 2 +- .../client/framing/BasicGet.cs | 2 +- .../client/framing/BasicNack.cs | 2 +- .../client/framing/BasicProperties.cs | 362 ------------------ .../client/framing/BasicPublish.cs | 4 +- .../client/framing/BasicQos.cs | 2 +- .../client/framing/BasicRecover.cs | 2 +- .../client/framing/BasicRecoverAsync.cs | 2 +- .../client/framing/BasicReject.cs | 2 +- .../client/framing/ChannelClose.cs | 2 +- .../client/framing/ChannelCloseOk.cs | 2 +- .../client/framing/ChannelFlowOk.cs | 2 +- .../client/framing/ChannelOpen.cs | 2 +- .../client/framing/ConfirmSelect.cs | 2 +- .../client/framing/ConnectionClose.cs | 2 +- .../client/framing/ConnectionCloseOk.cs | 2 +- .../client/framing/ConnectionOpen.cs | 2 +- .../client/framing/ConnectionSecureOk.cs | 2 +- .../client/framing/ConnectionStartOk.cs | 2 +- .../client/framing/ConnectionTuneOk.cs | 2 +- .../client/framing/ConnectionUpdateSecret.cs | 2 +- .../client/framing/ExchangeBind.cs | 2 +- .../client/framing/ExchangeDeclare.cs | 2 +- .../client/framing/ExchangeDelete.cs | 2 +- .../client/framing/ExchangeUnbind.cs | 2 +- .../client/framing/IAmqpMethod.cs | 8 +- .../RabbitMQ.Client/client/framing/Model.cs | 18 - .../client/framing/Protocol.cs | 9 - .../client/framing/QueueBind.cs | 2 +- .../client/framing/QueueDeclare.cs | 2 +- .../client/framing/QueueDelete.cs | 2 +- .../client/framing/QueuePurge.cs | 2 +- .../client/framing/QueueUnbind.cs | 2 +- .../client/framing/TxCommit.cs | 2 +- .../client/framing/TxRollback.cs | 2 +- .../client/framing/TxSelect.cs | 2 +- .../client/impl/AutorecoveringModel.cs | 13 +- .../client/impl/BasicProperties.cs | 284 -------------- .../client/impl/CommandAssembler.cs | 60 +-- .../ConsumerDispatcherChannelBase.cs | 6 +- .../ConsumerDispatching/FallbackConsumer.cs | 4 +- .../IConsumerDispatcher.cs | 2 +- .../client/impl/ContentHeaderBase.cs | 57 --- .../client/impl/EmptyBasicProperty.cs | 55 +++ projects/RabbitMQ.Client/client/impl/Frame.cs | 19 +- .../RabbitMQ.Client/client/impl/ISession.cs | 4 +- .../client/impl/IncomingCommand.cs | 17 +- .../RabbitMQ.Client/client/impl/ModelBase.cs | 55 ++- .../client/impl/ProtocolBase.cs | 1 - .../RabbitMQ.Client/client/impl/Session.cs | 2 +- .../client/impl/SessionBase.cs | 6 +- .../client/impl/StreamProperties.cs | 67 ---- .../TestApplications/MassPublish/Program.cs | 9 +- .../Unit/APIApproval.Approve.verified.txt | 234 +++++++---- projects/Unit/Fixtures.cs | 4 +- projects/Unit/RabbitMQCtl.cs | 2 +- projects/Unit/TestAsyncConsumer.cs | 21 +- projects/Unit/TestAsyncConsumerExceptions.cs | 4 +- projects/Unit/TestBasicProperties.cs | 63 ++- projects/Unit/TestBasicPublish.cs | 13 +- ...estConcurrentAccessWithSharedConnection.cs | 2 +- projects/Unit/TestConfirmSelect.cs | 2 +- projects/Unit/TestConnectionRecovery.cs | 34 +- projects/Unit/TestConsumer.cs | 5 +- projects/Unit/TestConsumerExceptions.cs | 4 +- .../Unit/TestConsumerOperationDispatch.cs | 8 +- projects/Unit/TestContentHeaderCodec.cs | 12 +- projects/Unit/TestEventingConsumer.cs | 2 +- projects/Unit/TestExtensions.cs | 6 +- projects/Unit/TestFloodPublishing.cs | 5 +- projects/Unit/TestFrameFormatting.cs | 6 +- projects/Unit/TestMainLoop.cs | 6 +- projects/Unit/TestMessageCount.cs | 2 +- projects/Unit/TestPropertiesClone.cs | 170 -------- projects/Unit/TestPublishSharedModel.cs | 2 +- projects/Unit/TestPublisherConfirms.cs | 4 +- projects/Unit/TestRecoverAfterCancel.cs | 2 +- projects/Unit/TestSsl.cs | 2 +- 104 files changed, 1118 insertions(+), 1583 deletions(-) create mode 100644 projects/RabbitMQ.Client/client/api/BasicProperties.cs rename projects/RabbitMQ.Client/client/api/{IContentHeader.cs => IAmqpHeader.cs} (83%) rename projects/{Unit/TestPublishValidation.cs => RabbitMQ.Client/client/api/IAmqpWriteable.cs} (70%) delete mode 100644 projects/RabbitMQ.Client/client/api/IStreamProperties.cs create mode 100644 projects/RabbitMQ.Client/client/api/ReadonlyBasicProperties.cs delete mode 100644 projects/RabbitMQ.Client/client/framing/BasicProperties.cs delete mode 100644 projects/RabbitMQ.Client/client/impl/BasicProperties.cs delete mode 100644 projects/RabbitMQ.Client/client/impl/ContentHeaderBase.cs create mode 100644 projects/RabbitMQ.Client/client/impl/EmptyBasicProperty.cs delete mode 100644 projects/RabbitMQ.Client/client/impl/StreamProperties.cs delete mode 100644 projects/Unit/TestPropertiesClone.cs diff --git a/projects/Benchmarks/ConsumerDispatching/AsyncBasicConsumerFake.cs b/projects/Benchmarks/ConsumerDispatching/AsyncBasicConsumerFake.cs index f37929e4d7..38e38a8a1a 100644 --- a/projects/Benchmarks/ConsumerDispatching/AsyncBasicConsumerFake.cs +++ b/projects/Benchmarks/ConsumerDispatching/AsyncBasicConsumerFake.cs @@ -18,7 +18,7 @@ public AsyncBasicConsumerFake(ManualResetEventSlim autoResetEvent) _autoResetEvent = autoResetEvent; } - public Task HandleBasicDeliver(string consumerTag, ulong deliveryTag, bool redelivered, string exchange, string routingKey, IBasicProperties properties, ReadOnlyMemory body) + public Task HandleBasicDeliver(string consumerTag, ulong deliveryTag, bool redelivered, string exchange, string routingKey, in ReadOnlyBasicProperties properties, ReadOnlyMemory body) { if (Interlocked.Increment(ref _current) == Count) { @@ -29,7 +29,7 @@ public Task HandleBasicDeliver(string consumerTag, ulong deliveryTag, bool redel } void IBasicConsumer.HandleBasicDeliver(string consumerTag, ulong deliveryTag, bool redelivered, string exchange, string routingKey, - IBasicProperties properties, ReadOnlyMemory body) + in ReadOnlyBasicProperties properties, ReadOnlyMemory body) { if (Interlocked.Increment(ref _current) == Count) { diff --git a/projects/Benchmarks/ConsumerDispatching/ConsumerDispatcher.cs b/projects/Benchmarks/ConsumerDispatching/ConsumerDispatcher.cs index d1792131ce..b751091c4d 100644 --- a/projects/Benchmarks/ConsumerDispatching/ConsumerDispatcher.cs +++ b/projects/Benchmarks/ConsumerDispatching/ConsumerDispatcher.cs @@ -17,7 +17,7 @@ public class ConsumerDispatcherBase protected readonly ulong _deliveryTag = 500UL; protected readonly string _exchange = "Exchange"; protected readonly string _routingKey = "RoutingKey"; - protected readonly IBasicProperties _properties = new Client.Framing.BasicProperties(); + protected readonly ReadOnlyBasicProperties _properties = new ReadOnlyBasicProperties(); protected readonly byte[] _body = new byte[512]; } diff --git a/projects/Benchmarks/Networking/Networking_BasicDeliver_Commons.cs b/projects/Benchmarks/Networking/Networking_BasicDeliver_Commons.cs index 678d370502..fb8489fff7 100644 --- a/projects/Benchmarks/Networking/Networking_BasicDeliver_Commons.cs +++ b/projects/Benchmarks/Networking/Networking_BasicDeliver_Commons.cs @@ -9,10 +9,7 @@ namespace Benchmarks.Networking [MemoryDiagnoser] public class Networking_BasicDeliver_Commons { - private const int messageCount = 10000; - - - public static async Task Publish_Hello_World(IConnection connection, uint n, byte[] body) + public static async Task Publish_Hello_World(IConnection connection, uint messageCount, byte[] body) { var tcs = new TaskCompletionSource(TaskCreationOptions.RunContinuationsAsynchronously); using (var model = connection.CreateModel()) @@ -31,7 +28,7 @@ public static async Task Publish_Hello_World(IConnection connection, uint n, byt for (int i = 0; i < messageCount; i++) { - model.BasicPublish("", queue.QueueName, null, body); + model.BasicPublish("", queue.QueueName, body); } await tcs.Task; diff --git a/projects/Benchmarks/Networking/Networking_BasicDeliver_LongLivedConnection.cs b/projects/Benchmarks/Networking/Networking_BasicDeliver_LongLivedConnection.cs index b37ba8d591..4743919e0c 100644 --- a/projects/Benchmarks/Networking/Networking_BasicDeliver_LongLivedConnection.cs +++ b/projects/Benchmarks/Networking/Networking_BasicDeliver_LongLivedConnection.cs @@ -32,9 +32,9 @@ public void GlobalCleanup() } [Benchmark(Baseline = true)] - public async Task Publish_Hello_World() + public Task Publish_Hello_World() { - await Networking_BasicDeliver_Commons.Publish_Hello_World(_connection, messageCount, _body); + return Networking_BasicDeliver_Commons.Publish_Hello_World(_connection, messageCount, _body); } } } diff --git a/projects/Benchmarks/WireFormatting/MethodFraming.cs b/projects/Benchmarks/WireFormatting/MethodFraming.cs index 22a43a4c7d..e2f032341e 100644 --- a/projects/Benchmarks/WireFormatting/MethodFraming.cs +++ b/projects/Benchmarks/WireFormatting/MethodFraming.cs @@ -2,12 +2,11 @@ using System.Text; using BenchmarkDotNet.Attributes; - +using RabbitMQ.Client; +using RabbitMQ.Client.client.impl; using RabbitMQ.Client.Framing.Impl; using RabbitMQ.Client.Impl; -using BasicProperties = RabbitMQ.Client.Framing.BasicProperties; - namespace RabbitMQ.Benchmarks { [Config(typeof(Config))] @@ -30,8 +29,8 @@ public class MethodFramingBasicPublish private const string StringValue = "Exchange_OR_RoutingKey"; private BasicPublish _basicPublish = new BasicPublish(StringValue, StringValue, false, false); private BasicPublishMemory _basicPublishMemory = new BasicPublishMemory(Encoding.UTF8.GetBytes(StringValue), Encoding.UTF8.GetBytes(StringValue), false, false); - private readonly BasicProperties _propertiesEmpty = new BasicProperties(); - private readonly BasicProperties _properties = new BasicProperties { AppId = "Application id", MessageId = "Random message id" }; + private EmptyBasicProperty _propertiesEmpty = new EmptyBasicProperty(); + private BasicProperties _properties = new BasicProperties { AppId = "Application id", MessageId = "Random message id" }; private readonly ReadOnlyMemory _bodyEmpty = ReadOnlyMemory.Empty; private readonly ReadOnlyMemory _body = new byte[512]; @@ -42,13 +41,13 @@ public class MethodFramingBasicPublish public int FrameMax { get; set; } [Benchmark] - public ReadOnlyMemory BasicPublishWriteNonEmpty() => Framing.SerializeToFrames(ref _basicPublish, _properties, _body, Channel, FrameMax); + public ReadOnlyMemory BasicPublishWriteNonEmpty() => Framing.SerializeToFrames(ref _basicPublish, ref _properties, _body, Channel, FrameMax); [Benchmark] - public ReadOnlyMemory BasicPublishWrite() => Framing.SerializeToFrames(ref _basicPublish, _propertiesEmpty, _bodyEmpty, Channel, FrameMax); + public ReadOnlyMemory BasicPublishWrite() => Framing.SerializeToFrames(ref _basicPublish, ref _propertiesEmpty, _bodyEmpty, Channel, FrameMax); [Benchmark] - public ReadOnlyMemory BasicPublishMemoryWrite() => Framing.SerializeToFrames(ref _basicPublishMemory, _propertiesEmpty, _bodyEmpty, Channel, FrameMax); + public ReadOnlyMemory BasicPublishMemoryWrite() => Framing.SerializeToFrames(ref _basicPublishMemory, ref _propertiesEmpty, _bodyEmpty, Channel, FrameMax); } [Config(typeof(Config))] diff --git a/projects/Benchmarks/WireFormatting/MethodSerialization.cs b/projects/Benchmarks/WireFormatting/MethodSerialization.cs index 084fef073b..1a67657c80 100644 --- a/projects/Benchmarks/WireFormatting/MethodSerialization.cs +++ b/projects/Benchmarks/WireFormatting/MethodSerialization.cs @@ -2,8 +2,7 @@ using System.Text; using BenchmarkDotNet.Attributes; - -using RabbitMQ.Client.Framing; +using RabbitMQ.Client; using RabbitMQ.Client.Framing.Impl; namespace RabbitMQ.Benchmarks @@ -21,13 +20,13 @@ public virtual void SetUp() { } public class MethodBasicAck : MethodSerializationBase { private readonly BasicAck _basicAck = new BasicAck(ulong.MaxValue, true); - public override void SetUp() => _basicAck.WriteArgumentsTo(_buffer.Span); + public override void SetUp() => _basicAck.WriteTo(_buffer.Span); [Benchmark] public ulong BasicAckRead() => new BasicAck(_buffer.Span)._deliveryTag; // return one property to not box when returning an object instead [Benchmark] - public int BasicAckWrite() => _basicAck.WriteArgumentsTo(_buffer.Span); + public int BasicAckWrite() => _basicAck.WriteTo(_buffer.Span); } public class MethodBasicDeliver : MethodSerializationBase @@ -38,7 +37,6 @@ public class MethodBasicDeliver : MethodSerializationBase public override void SetUp() { - int length = _buffer.Length; int offset = Client.Impl.WireFormatting.WriteShortstr(ref _buffer.Span.GetStart(), string.Empty); offset += Client.Impl.WireFormatting.WriteLonglong(ref _buffer.Span.GetOffset(offset), 0); offset += Client.Impl.WireFormatting.WriteBits(ref _buffer.Span.GetOffset(offset), false); @@ -50,10 +48,10 @@ public override void SetUp() public object BasicDeliverRead() => new BasicDeliver(_buffer.Span)._consumerTag; // return one property to not box when returning an object instead [Benchmark] - public int BasicPublishWrite() => _basicPublish.WriteArgumentsTo(_buffer.Span); + public int BasicPublishWrite() => _basicPublish.WriteTo(_buffer.Span); [Benchmark] - public int BasicPublishMemoryWrite() => _basicPublishMemory.WriteArgumentsTo(_buffer.Span); + public int BasicPublishMemoryWrite() => _basicPublishMemory.WriteTo(_buffer.Span); [Benchmark] public int BasicPublishSize() => _basicPublish.GetRequiredBufferSize(); @@ -66,27 +64,27 @@ public class MethodChannelClose : MethodSerializationBase { private readonly ChannelClose _channelClose = new ChannelClose(333, string.Empty, 0099, 2999); - public override void SetUp() => _channelClose.WriteArgumentsTo(_buffer.Span); + public override void SetUp() => _channelClose.WriteTo(_buffer.Span); [Benchmark] public object ChannelCloseRead() => new ChannelClose(_buffer.Span)._replyText; // return one property to not box when returning an object instead [Benchmark] - public int ChannelCloseWrite() => _channelClose.WriteArgumentsTo(_buffer.Span); + public int ChannelCloseWrite() => _channelClose.WriteTo(_buffer.Span); } public class MethodBasicProperties : MethodSerializationBase { - private readonly BasicProperties _basicProperties = new BasicProperties { Persistent = true, AppId = "AppId", ContentEncoding = "content", }; - public override void SetUp() => _basicProperties.WritePropertiesTo(_buffer.Span); + private readonly IAmqpWriteable _basicProperties = new BasicProperties { Persistent = true, AppId = "AppId", ContentEncoding = "content", }; + public override void SetUp() => _basicProperties.WriteTo(_buffer.Span); [Benchmark] - public object BasicPropertiesRead() => new BasicProperties(_buffer.Span); + public ReadOnlyBasicProperties BasicPropertiesRead() => new ReadOnlyBasicProperties(_buffer.Span); [Benchmark] - public int BasicPropertiesWrite() => _basicProperties.WritePropertiesTo(_buffer.Span); + public int BasicPropertiesWrite() => _basicProperties.WriteTo(_buffer.Span); [Benchmark] - public int BasicDeliverSize() => _basicProperties.GetRequiredPayloadBufferSize(); + public int BasicDeliverSize() => _basicProperties.GetRequiredBufferSize(); } } diff --git a/projects/RabbitMQ.Client/client/api/AsyncDefaultBasicConsumer.cs b/projects/RabbitMQ.Client/client/api/AsyncDefaultBasicConsumer.cs index 65c46c4bea..2be678e00f 100644 --- a/projects/RabbitMQ.Client/client/api/AsyncDefaultBasicConsumer.cs +++ b/projects/RabbitMQ.Client/client/api/AsyncDefaultBasicConsumer.cs @@ -112,7 +112,7 @@ public virtual Task HandleBasicDeliver(string consumerTag, bool redelivered, string exchange, string routingKey, - IBasicProperties properties, + in ReadOnlyBasicProperties properties, ReadOnlyMemory body) { // Nothing to do here. @@ -165,7 +165,7 @@ void IBasicConsumer.HandleBasicConsumeOk(string consumerTag) throw new InvalidOperationException("Should never be called."); } - void IBasicConsumer.HandleBasicDeliver(string consumerTag, ulong deliveryTag, bool redelivered, string exchange, string routingKey, IBasicProperties properties, ReadOnlyMemory body) + void IBasicConsumer.HandleBasicDeliver(string consumerTag, ulong deliveryTag, bool redelivered, string exchange, string routingKey, in ReadOnlyBasicProperties properties, ReadOnlyMemory body) { throw new InvalidOperationException("Should never be called."); } diff --git a/projects/RabbitMQ.Client/client/api/BasicGetResult.cs b/projects/RabbitMQ.Client/client/api/BasicGetResult.cs index 2ab3746349..04dfa7f07e 100644 --- a/projects/RabbitMQ.Client/client/api/BasicGetResult.cs +++ b/projects/RabbitMQ.Client/client/api/BasicGetResult.cs @@ -53,7 +53,7 @@ public sealed class BasicGetResult : IDisposable /// The Basic-class content header properties for the message. /// The body public BasicGetResult(ulong deliveryTag, bool redelivered, string exchange, string routingKey, - uint messageCount, IBasicProperties basicProperties, ReadOnlyMemory body) + uint messageCount, in ReadOnlyBasicProperties basicProperties, ReadOnlyMemory body) { DeliveryTag = deliveryTag; Redelivered = redelivered; @@ -76,7 +76,7 @@ public BasicGetResult(ulong deliveryTag, bool redelivered, string exchange, stri /// The body /// The rented array which body is part of. public BasicGetResult(ulong deliveryTag, bool redelivered, string exchange, string routingKey, - uint messageCount, IBasicProperties basicProperties, ReadOnlyMemory body, byte[] rentedArray) + uint messageCount, in ReadOnlyBasicProperties basicProperties, ReadOnlyMemory body, byte[] rentedArray) { DeliveryTag = deliveryTag; Redelivered = redelivered; @@ -91,7 +91,7 @@ public BasicGetResult(ulong deliveryTag, bool redelivered, string exchange, stri /// /// Retrieves the Basic-class content header properties for this message. /// - public IBasicProperties BasicProperties { get; } + public ReadOnlyBasicProperties BasicProperties { get; } /// /// Retrieves the body of this message. @@ -130,7 +130,7 @@ public BasicGetResult(ulong deliveryTag, bool redelivered, string exchange, stri /// public void Dispose() { - if (!(_rentedArray is null)) + if (_rentedArray is not null) { ArrayPool.Shared.Return(_rentedArray); } diff --git a/projects/RabbitMQ.Client/client/api/BasicProperties.cs b/projects/RabbitMQ.Client/client/api/BasicProperties.cs new file mode 100644 index 0000000000..fb0bb7b986 --- /dev/null +++ b/projects/RabbitMQ.Client/client/api/BasicProperties.cs @@ -0,0 +1,262 @@ +// This source code is dual-licensed under the Apache License, version +// 2.0, and the Mozilla Public License, version 2.0. +// +// The APL v2.0: +// +//--------------------------------------------------------------------------- +// Copyright (c) 2007-2020 VMware, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// https://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +//--------------------------------------------------------------------------- +// +// The MPL v2.0: +// +//--------------------------------------------------------------------------- +// This Source Code Form is subject to the terms of the Mozilla Public +// License, v. 2.0. If a copy of the MPL was not distributed with this +// file, You can obtain one at https://mozilla.org/MPL/2.0/. +// +// Copyright (c) 2007-2020 VMware, Inc. All rights reserved. +//--------------------------------------------------------------------------- + +using System; +using System.Collections.Generic; +using RabbitMQ.Client.Framing.Impl; +using RabbitMQ.Client.Impl; + +namespace RabbitMQ.Client +{ +#nullable enable + /// + /// AMQP specification content header properties for content class "basic". + /// + public struct BasicProperties : IBasicProperties, IAmqpHeader + { + public string? ContentType { get; set; } + public string? ContentEncoding { get; set; } + public IDictionary? Headers { get; set; } + public byte DeliveryMode { get; set; } + public byte Priority { get; set; } + public string? CorrelationId { get; set; } + public string? ReplyTo { get; set; } + public string? Expiration { get; set; } + public string? MessageId { get; set; } + public AmqpTimestamp Timestamp { get; set; } + public string? Type { get; set; } + public string? UserId { get; set; } + public string? AppId { get; set; } + public string? ClusterId { get; set; } + + public bool Persistent + { + readonly get { return DeliveryMode == 2; } + set { DeliveryMode = value ? (byte)2 : (byte)1; } + } + + public PublicationAddress? ReplyToAddress + { + readonly get + { + PublicationAddress.TryParse(ReplyTo, out PublicationAddress result); + return result; + } + set { ReplyTo = value?.ToString(); } + } + + public BasicProperties(in ReadOnlyBasicProperties input) + { + ContentType = input.ContentType; + ContentEncoding = input.ContentEncoding; + Headers = input.Headers; + DeliveryMode = input.DeliveryMode; + Priority = input.Priority; + CorrelationId = input.CorrelationId; + ReplyTo = input.ReplyTo; + Expiration = input.Expiration; + MessageId = input.MessageId; + Timestamp = input.Timestamp; + Type = input.Type; + UserId = input.UserId; + AppId = input.AppId; + ClusterId = input.ClusterId; + } + + public void ClearContentType() => ContentType = default; + public void ClearContentEncoding() => ContentEncoding = default; + public void ClearHeaders() => Headers = default; + public void ClearDeliveryMode() => DeliveryMode = default; + public void ClearPriority() => Priority = default; + public void ClearCorrelationId() => CorrelationId = default; + public void ClearReplyTo() => ReplyTo = default; + public void ClearExpiration() => Expiration = default; + public void ClearMessageId() => MessageId = default; + public void ClearTimestamp() => Timestamp = default; + public void ClearType() => Type = default; + public void ClearUserId() => UserId = default; + public void ClearAppId() => AppId = default; + public void ClearClusterId() => ClusterId = default; + + public readonly bool IsContentTypePresent() => ContentType != default; + public readonly bool IsContentEncodingPresent() => ContentEncoding != default; + public readonly bool IsHeadersPresent() => Headers != default; + public readonly bool IsDeliveryModePresent() => DeliveryMode != default; + public readonly bool IsPriorityPresent() => Priority != default; + public readonly bool IsCorrelationIdPresent() => CorrelationId != default; + public readonly bool IsReplyToPresent() => ReplyTo != default; + public readonly bool IsExpirationPresent() => Expiration != default; + public readonly bool IsMessageIdPresent() => MessageId != default; + public readonly bool IsTimestampPresent() => Timestamp != default; + public readonly bool IsTypePresent() => Type != default; + public readonly bool IsUserIdPresent() => UserId != default; + public readonly bool IsAppIdPresent() => AppId != default; + public readonly bool IsClusterIdPresent() => ClusterId != default; + + ushort IAmqpHeader.ProtocolClassId => ClassConstants.Basic; + + //---------------------------------- + // First byte + //---------------------------------- + internal const byte ContentTypeBit = 7; + internal const byte ContentEncodingBit = 6; + internal const byte HeaderBit = 5; + internal const byte DeliveryModeBit = 4; + internal const byte PriorityBit = 3; + internal const byte CorrelationIdBit = 2; + internal const byte ReplyToBit = 1; + internal const byte ExpirationBit = 0; + + //---------------------------------- + // Second byte + //---------------------------------- + internal const byte MessageIdBit = 7; + internal const byte TimestampBit = 6; + internal const byte TypeBit = 5; + internal const byte UserIdBit = 4; + internal const byte AppIdBit = 3; + internal const byte ClusterIdBit = 2; + + readonly int IAmqpWriteable.WriteTo(Span span) + { + int offset = 2; + ref byte bitValue = ref span.GetStart(); + bitValue = 0; + if (IsContentTypePresent()) + { + bitValue.SetBit(ContentTypeBit); + offset += WireFormatting.WriteShortstr(ref span.GetOffset(offset), ContentType); + } + + if (IsContentEncodingPresent()) + { + bitValue.SetBit(ContentEncodingBit); + offset += WireFormatting.WriteShortstr(ref span.GetOffset(offset), ContentEncoding); + } + + if (IsHeadersPresent()) + { + bitValue.SetBit(HeaderBit); + offset += WireFormatting.WriteTable(ref span.GetOffset(offset), Headers); + } + + if (IsDeliveryModePresent()) + { + bitValue.SetBit(DeliveryModeBit); + span.GetOffset(offset++) = DeliveryMode; + } + + if (IsPriorityPresent()) + { + bitValue.SetBit(PriorityBit); + span.GetOffset(offset++) = Priority; + } + + if (IsCorrelationIdPresent()) + { + bitValue.SetBit(CorrelationIdBit); + offset += WireFormatting.WriteShortstr(ref span.GetOffset(offset), CorrelationId); + } + + if (IsReplyToPresent()) + { + bitValue.SetBit(ReplyToBit); + offset += WireFormatting.WriteShortstr(ref span.GetOffset(offset), ReplyTo); + } + + if (IsExpirationPresent()) + { + bitValue.SetBit(ExpirationBit); + offset += WireFormatting.WriteShortstr(ref span.GetOffset(offset), Expiration); + } + + bitValue = ref span.GetOffset(1); + bitValue = 0; + if (IsMessageIdPresent()) + { + bitValue.SetBit(MessageIdBit); + offset += WireFormatting.WriteShortstr(ref span.GetOffset(offset), MessageId); + } + + if (IsTimestampPresent()) + { + bitValue.SetBit(TimestampBit); + offset += WireFormatting.WriteTimestamp(ref span.GetOffset(offset), Timestamp); + } + + if (IsTypePresent()) + { + bitValue.SetBit(TypeBit); + offset += WireFormatting.WriteShortstr(ref span.GetOffset(offset), Type); + } + + if (IsUserIdPresent()) + { + bitValue.SetBit(UserIdBit); + offset += WireFormatting.WriteShortstr(ref span.GetOffset(offset), UserId); + } + + if (IsAppIdPresent()) + { + bitValue.SetBit(AppIdBit); + offset += WireFormatting.WriteShortstr(ref span.GetOffset(offset), AppId); + } + + if (IsClusterIdPresent()) + { + bitValue.SetBit(ClusterIdBit); + offset += WireFormatting.WriteShortstr(ref span.GetOffset(offset), ClusterId); + } + + return offset; + } + + readonly int IAmqpWriteable.GetRequiredBufferSize() + { + int bufferSize = 2; // number of presence fields (14) in 2 bytes blocks + if (IsContentTypePresent()) { bufferSize += 1 + WireFormatting.GetByteCount(ContentType); } // _contentType in bytes + if (IsContentEncodingPresent()) { bufferSize += 1 + WireFormatting.GetByteCount(ContentEncoding); } // _contentEncoding in bytes + if (IsHeadersPresent()) { bufferSize += WireFormatting.GetTableByteCount(Headers); } // _headers in bytes + if (IsDeliveryModePresent()) { bufferSize++; } // _deliveryMode in bytes + if (IsPriorityPresent()) { bufferSize++; } // _priority in bytes + if (IsCorrelationIdPresent()) { bufferSize += 1 + WireFormatting.GetByteCount(CorrelationId); } // _correlationId in bytes + if (IsReplyToPresent()) { bufferSize += 1 + WireFormatting.GetByteCount(ReplyTo); } // _replyTo in bytes + if (IsExpirationPresent()) { bufferSize += 1 + WireFormatting.GetByteCount(Expiration); } // _expiration in bytes + if (IsMessageIdPresent()) { bufferSize += 1 + WireFormatting.GetByteCount(MessageId); } // _messageId in bytes + if (IsTimestampPresent()) { bufferSize += 8; } // _timestamp in bytes + if (IsTypePresent()) { bufferSize += 1 + WireFormatting.GetByteCount(Type); } // _type in bytes + if (IsUserIdPresent()) { bufferSize += 1 + WireFormatting.GetByteCount(UserId); } // _userId in bytes + if (IsAppIdPresent()) { bufferSize += 1 + WireFormatting.GetByteCount(AppId); } // _appId in bytes + if (IsClusterIdPresent()) { bufferSize += 1 + WireFormatting.GetByteCount(ClusterId); } // _clusterId in bytes + return bufferSize; + } + } +} diff --git a/projects/RabbitMQ.Client/client/api/CachedString.cs b/projects/RabbitMQ.Client/client/api/CachedString.cs index a2af7cacbc..16e33a1fb7 100644 --- a/projects/RabbitMQ.Client/client/api/CachedString.cs +++ b/projects/RabbitMQ.Client/client/api/CachedString.cs @@ -4,7 +4,7 @@ namespace RabbitMQ.Client { /// - /// Caches a string's byte representation to be used for certain methods like . + /// Caches a string's byte representation to be used for certain methods like IModel.BasicPublish/>. /// public sealed class CachedString { diff --git a/projects/RabbitMQ.Client/client/api/DefaultBasicConsumer.cs b/projects/RabbitMQ.Client/client/api/DefaultBasicConsumer.cs index c7b0bb8211..9e1b92beb0 100644 --- a/projects/RabbitMQ.Client/client/api/DefaultBasicConsumer.cs +++ b/projects/RabbitMQ.Client/client/api/DefaultBasicConsumer.cs @@ -152,7 +152,7 @@ public virtual void HandleBasicDeliver(string consumerTag, bool redelivered, string exchange, string routingKey, - IBasicProperties properties, + in ReadOnlyBasicProperties properties, ReadOnlyMemory body) { // Nothing to do here. diff --git a/projects/RabbitMQ.Client/client/api/IContentHeader.cs b/projects/RabbitMQ.Client/client/api/IAmqpHeader.cs similarity index 83% rename from projects/RabbitMQ.Client/client/api/IContentHeader.cs rename to projects/RabbitMQ.Client/client/api/IAmqpHeader.cs index 24ccacc01a..319de11ace 100644 --- a/projects/RabbitMQ.Client/client/api/IContentHeader.cs +++ b/projects/RabbitMQ.Client/client/api/IAmqpHeader.cs @@ -31,19 +31,15 @@ namespace RabbitMQ.Client { +#nullable enable /// - /// A decoded AMQP content header frame. + /// A AMQP header. /// - public interface IContentHeader// : ICloneable + public interface IAmqpHeader : IAmqpWriteable { /// - /// Retrieve the AMQP class ID of this content header. + /// The protocol class id. /// ushort ProtocolClassId { get; } - - /// - /// Retrieve the AMQP class name of this content header. - /// - string ProtocolClassName { get; } } } diff --git a/projects/Unit/TestPublishValidation.cs b/projects/RabbitMQ.Client/client/api/IAmqpWriteable.cs similarity index 70% rename from projects/Unit/TestPublishValidation.cs rename to projects/RabbitMQ.Client/client/api/IAmqpWriteable.cs index ca55bb05f0..db72f2813b 100644 --- a/projects/Unit/TestPublishValidation.cs +++ b/projects/RabbitMQ.Client/client/api/IAmqpWriteable.cs @@ -31,19 +31,25 @@ using System; -using Xunit; - -namespace RabbitMQ.Client.Unit +namespace RabbitMQ.Client { - - public class TestPublishValidation : IntegrationFixture +#nullable enable + /// + /// A AMQP writeable. + /// + public interface IAmqpWriteable { + /// + /// Gets the minimum required buffer size. + /// + /// The minimum required buffer size. + int GetRequiredBufferSize(); - [Fact] - public void TestNullRoutingKeyIsRejected() - { - IModel ch = _conn.CreateModel(); - Assert.Throws(() => ch.BasicPublish("", null, null, _encoding.GetBytes("msg"))); - } + /// + /// Write this instance to the provided span. + /// + /// The span to write it to. + /// The amount of bytes written. + int WriteTo(Span span); } } diff --git a/projects/RabbitMQ.Client/client/api/IAsyncBasicConsumer.cs b/projects/RabbitMQ.Client/client/api/IAsyncBasicConsumer.cs index 10c0db9a15..24e787d691 100644 --- a/projects/RabbitMQ.Client/client/api/IAsyncBasicConsumer.cs +++ b/projects/RabbitMQ.Client/client/api/IAsyncBasicConsumer.cs @@ -51,7 +51,7 @@ Task HandleBasicDeliver(string consumerTag, bool redelivered, string exchange, string routingKey, - IBasicProperties properties, + in ReadOnlyBasicProperties properties, ReadOnlyMemory body); /// diff --git a/projects/RabbitMQ.Client/client/api/IBasicConsumer.cs b/projects/RabbitMQ.Client/client/api/IBasicConsumer.cs index 7b46667d82..94627c5a35 100644 --- a/projects/RabbitMQ.Client/client/api/IBasicConsumer.cs +++ b/projects/RabbitMQ.Client/client/api/IBasicConsumer.cs @@ -94,7 +94,7 @@ void HandleBasicDeliver(string consumerTag, bool redelivered, string exchange, string routingKey, - IBasicProperties properties, + in ReadOnlyBasicProperties properties, ReadOnlyMemory body); /// diff --git a/projects/RabbitMQ.Client/client/api/IBasicProperties.cs b/projects/RabbitMQ.Client/client/api/IBasicProperties.cs index bf777de5b8..c2d59171cb 100644 --- a/projects/RabbitMQ.Client/client/api/IBasicProperties.cs +++ b/projects/RabbitMQ.Client/client/api/IBasicProperties.cs @@ -33,244 +33,331 @@ namespace RabbitMQ.Client { - /// Common AMQP Basic content-class headers interface, +#nullable enable + /// + /// The AMQP Basic headers class interface, /// spanning the union of the functionality offered by versions - /// 0-8, 0-8qpid, 0-9 and 0-9-1 of AMQP. - /// - /// - /// The specification code generator provides - /// protocol-version-specific implementations of this interface. To - /// obtain an implementation of this interface in a - /// protocol-version-neutral way, use . - /// - /// - /// Each property is readable, writable and clearable: a cleared - /// property will not be transmitted over the wire. Properties on a - /// fresh instance are clear by default. - /// - /// - public interface IBasicProperties : IContentHeader + /// 0-8, 0-8qpid, 0-9 and 0-9-1 of AMQP. + /// + public interface IReadOnlyBasicProperties { /// /// Application Id. /// - string AppId { get; set; } + string? AppId { get; } /// /// Intra-cluster routing identifier (cluster id is deprecated in AMQP 0-9-1). /// - string ClusterId { get; set; } + string? ClusterId { get; } /// /// MIME content encoding. /// - string ContentEncoding { get; set; } + string? ContentEncoding { get; } /// /// MIME content type. /// - string ContentType { get; set; } + string? ContentType { get; } /// /// Application correlation identifier. /// - string CorrelationId { get; set; } + string? CorrelationId { get; } /// /// Non-persistent (1) or persistent (2). /// - byte DeliveryMode { get; set; } + byte DeliveryMode { get; } /// /// Message expiration specification. /// - string Expiration { get; set; } + string? Expiration { get; } /// /// Message header field table. Is of type . /// - IDictionary Headers { get; set; } + IDictionary? Headers { get; } /// /// Application message Id. /// - string MessageId { get; set; } + string? MessageId { get; } /// /// Sets to either persistent (2) or non-persistent (1). /// - bool Persistent { get; set; } + bool Persistent { get; } /// /// Message priority, 0 to 9. /// - byte Priority { get; set; } + byte Priority { get; } /// /// Destination to reply to. /// - string ReplyTo { get; set; } + string? ReplyTo { get; } /// /// Convenience property; parses property using , /// and serializes it using . /// Returns null if property cannot be parsed by . /// - PublicationAddress ReplyToAddress { get; set; } + PublicationAddress? ReplyToAddress { get; } /// /// Message timestamp. /// - AmqpTimestamp Timestamp { get; set; } + AmqpTimestamp Timestamp { get; } /// /// Message type name. /// - string Type { get; set; } + string? Type { get; } /// /// User Id. /// - string UserId { get; set; } + string? UserId { get; } /// - /// Clear the property. + /// Returns true if the property is present. /// - void ClearAppId(); + bool IsAppIdPresent(); /// - /// Clear the property (cluster id is deprecated in AMQP 0-9-1). + /// Returns true if the property is present (cluster id is deprecated in AMQP 0-9-1). /// - void ClearClusterId(); + bool IsClusterIdPresent(); /// - /// Clear the property. + /// Returns true if the property is present. /// - void ClearContentEncoding(); + bool IsContentEncodingPresent(); /// - /// Clear the property. + /// Returns true if the property is present. /// - void ClearContentType(); + bool IsContentTypePresent(); /// - /// Clear the property. + /// Returns true if the property is present. /// - void ClearCorrelationId(); + bool IsCorrelationIdPresent(); /// - /// Clear the property. + /// Returns true if the property is present. /// - void ClearDeliveryMode(); + bool IsDeliveryModePresent(); /// - /// Clear the property. + /// Returns true if the property is present. /// - void ClearExpiration(); + bool IsExpirationPresent(); /// - /// Clear the property. + /// Returns true if the property is present. /// - void ClearHeaders(); + bool IsHeadersPresent(); /// - /// Clear the property. + /// Returns true if the property is present. /// - void ClearMessageId(); + bool IsMessageIdPresent(); /// - /// Clear the property. + /// Returns true if the property is present. /// - void ClearPriority(); + bool IsPriorityPresent(); /// - /// Clear the property. + /// Returns true if the property is present. /// - void ClearReplyTo(); + bool IsReplyToPresent(); /// - /// Clear the property. + /// Returns true if the property is present. /// - void ClearTimestamp(); + bool IsTimestampPresent(); /// - /// Clear the Type property. + /// Returns true if the property is present. /// - void ClearType(); + bool IsTypePresent(); /// - /// Clear the property. + /// Returns true if the property is present. /// - void ClearUserId(); + bool IsUserIdPresent(); + } + /// + /// The AMQP Basic headers class interface, + /// spanning the union of the functionality offered by versions + /// 0-8, 0-8qpid, 0-9 and 0-9-1 of AMQP. + /// + /// + /// + /// Each property is readable, writable and clearable: a cleared + /// property will not be transmitted over the wire. Properties on a + /// fresh instance are clear by default. + /// + /// + public interface IBasicProperties : IReadOnlyBasicProperties + { /// - /// Returns true if the property is present. + /// Application Id. /// - bool IsAppIdPresent(); + new string? AppId { get; set; } /// - /// Returns true if the property is present (cluster id is deprecated in AMQP 0-9-1). + /// Intra-cluster routing identifier (cluster id is deprecated in AMQP 0-9-1). /// - bool IsClusterIdPresent(); + new string? ClusterId { get; set; } /// - /// Returns true if the property is present. + /// MIME content encoding. /// - bool IsContentEncodingPresent(); + new string? ContentEncoding { get; set; } /// - /// Returns true if the property is present. + /// MIME content type. /// - bool IsContentTypePresent(); + new string? ContentType { get; set; } /// - /// Returns true if the property is present. + /// Application correlation identifier. /// - bool IsCorrelationIdPresent(); + new string? CorrelationId { get; set; } /// - /// Returns true if the property is present. + /// Non-persistent (1) or persistent (2). /// - bool IsDeliveryModePresent(); + new byte DeliveryMode { get; set; } /// - /// Returns true if the property is present. + /// Message expiration specification. /// - bool IsExpirationPresent(); + new string? Expiration { get; set; } /// - /// Returns true if the property is present. + /// Message header field table. Is of type . /// - bool IsHeadersPresent(); + new IDictionary? Headers { get; set; } /// - /// Returns true if the property is present. + /// Application message Id. /// - bool IsMessageIdPresent(); + new string? MessageId { get; set; } /// - /// Returns true if the property is present. + /// Sets to either persistent (2) or non-persistent (1). /// - bool IsPriorityPresent(); + new bool Persistent { get; set; } /// - /// Returns true if the property is present. + /// Message priority, 0 to 9. /// - bool IsReplyToPresent(); + new byte Priority { get; set; } /// - /// Returns true if the property is present. + /// Destination to reply to. /// - bool IsTimestampPresent(); + new string? ReplyTo { get; set; } /// - /// Returns true if the Type property is present. + /// Convenience property; parses property using , + /// and serializes it using . + /// Returns null if property cannot be parsed by . /// - bool IsTypePresent(); + new PublicationAddress? ReplyToAddress { get; set; } /// - /// Returns true if the UserId property is present. + /// Message timestamp. /// - bool IsUserIdPresent(); + new AmqpTimestamp Timestamp { get; set; } + + /// + /// Message type name. + /// + new string? Type { get; set; } + + /// + /// User Id. + /// + new string? UserId { get; set; } + + /// + /// Clear the property. + /// + void ClearAppId(); + + /// + /// Clear the property (cluster id is deprecated in AMQP 0-9-1). + /// + void ClearClusterId(); + + /// + /// Clear the property. + /// + void ClearContentEncoding(); + + /// + /// Clear the property. + /// + void ClearContentType(); + + /// + /// Clear the property. + /// + void ClearCorrelationId(); + + /// + /// Clear the property. + /// + void ClearDeliveryMode(); + + /// + /// Clear the property. + /// + void ClearExpiration(); + + /// + /// Clear the property. + /// + void ClearHeaders(); + + /// + /// Clear the property. + /// + void ClearMessageId(); + + /// + /// Clear the property. + /// + void ClearPriority(); + + /// + /// Clear the property. + /// + void ClearReplyTo(); + + /// + /// Clear the property. + /// + void ClearTimestamp(); + + /// + /// Clear the property. + /// + void ClearType(); + + /// + /// Clear the property. + /// + void ClearUserId(); } } diff --git a/projects/RabbitMQ.Client/client/api/IModel.cs b/projects/RabbitMQ.Client/client/api/IModel.cs index fcdff69875..7417b59e02 100644 --- a/projects/RabbitMQ.Client/client/api/IModel.cs +++ b/projects/RabbitMQ.Client/client/api/IModel.cs @@ -179,6 +179,7 @@ string BasicConsume( /// Reject one or more delivered message(s). void BasicNack(ulong deliveryTag, bool multiple, bool requeue); +#nullable enable /// /// Publishes a message. /// @@ -187,7 +188,8 @@ string BasicConsume( /// Routing key must be shorter than 255 bytes. /// /// - void BasicPublish(string exchange, string routingKey, bool mandatory, IBasicProperties basicProperties, ReadOnlyMemory body); + void BasicPublish(string exchange, string routingKey, in TProperties basicProperties, ReadOnlyMemory body = default, bool mandatory = false) + where TProperties : IReadOnlyBasicProperties, IAmqpHeader; /// /// Publishes a message. /// @@ -196,7 +198,9 @@ string BasicConsume( /// Routing key must be shorter than 255 bytes. /// /// - void BasicPublish(CachedString exchange, CachedString routingKey, bool mandatory, IBasicProperties basicProperties, ReadOnlyMemory body); + void BasicPublish(CachedString exchange, CachedString routingKey, in TProperties basicProperties, ReadOnlyMemory body = default, bool mandatory = false) + where TProperties : IReadOnlyBasicProperties, IAmqpHeader; +#nullable disable /// /// Configures QoS parameters of the Basic content-class. @@ -229,11 +233,6 @@ string BasicConsume( /// void ConfirmSelect(); - /// - /// Construct a completely empty content header for use with the Basic content class. - /// - IBasicProperties CreateBasicProperties(); - /// /// Bind an exchange to an exchange. /// diff --git a/projects/RabbitMQ.Client/client/api/IModelExtensions.cs b/projects/RabbitMQ.Client/client/api/IModelExtensions.cs index d003ead6a1..ff84bfb185 100644 --- a/projects/RabbitMQ.Client/client/api/IModelExtensions.cs +++ b/projects/RabbitMQ.Client/client/api/IModelExtensions.cs @@ -31,6 +31,7 @@ using System; using System.Collections.Generic; +using RabbitMQ.Client.client.impl; namespace RabbitMQ.Client { @@ -74,46 +75,25 @@ public static string BasicConsume(this IModel model, string queue, return model.BasicConsume(queue, autoAck, consumerTag, false, false, arguments, consumer); } +#nullable enable /// /// (Extension method) Convenience overload of BasicPublish. /// /// /// The publication occurs with mandatory=false and immediate=false. /// - public static void BasicPublish(this IModel model, PublicationAddress addr, IBasicProperties basicProperties, ReadOnlyMemory body) + public static void BasicPublish(this IModel model, PublicationAddress addr, in T basicProperties, ReadOnlyMemory body) + where T : IReadOnlyBasicProperties, IAmqpHeader { - model.BasicPublish(addr.ExchangeName, addr.RoutingKey, basicProperties: basicProperties, body: body); + model.BasicPublish(addr.ExchangeName, addr.RoutingKey, basicProperties, body); } - /// - /// (Extension method) Convenience overload of BasicPublish. - /// - /// - /// The publication occurs with mandatory=false - /// - public static void BasicPublish(this IModel model, CachedString exchange, CachedString routingKey, IBasicProperties basicProperties, ReadOnlyMemory body) - { - model.BasicPublish(exchange, routingKey, false, basicProperties, body); - } + public static void BasicPublish(this IModel model, string exchange, string routingKey, ReadOnlyMemory body = default, bool mandatory = false) + => model.BasicPublish(exchange, routingKey, default(EmptyBasicProperty), body, mandatory); - /// - /// (Extension method) Convenience overload of BasicPublish. - /// - /// - /// The publication occurs with mandatory=false - /// - public static void BasicPublish(this IModel model, string exchange, string routingKey, IBasicProperties basicProperties, ReadOnlyMemory body) - { - model.BasicPublish(exchange, routingKey, false, basicProperties, body); - } - - /// - /// (Spec method) Convenience overload of BasicPublish. - /// - public static void BasicPublish(this IModel model, string exchange, string routingKey, bool mandatory = false, IBasicProperties basicProperties = null, ReadOnlyMemory body = default) - { - model.BasicPublish(exchange, routingKey, mandatory, basicProperties, body); - } + public static void BasicPublish(this IModel model, CachedString exchange, CachedString routingKey, ReadOnlyMemory body = default, bool mandatory = false) + => model.BasicPublish(exchange, routingKey, default(EmptyBasicProperty), body, mandatory); +#nullable disable /// /// (Spec method) Declare a queue. diff --git a/projects/RabbitMQ.Client/client/api/IStreamProperties.cs b/projects/RabbitMQ.Client/client/api/IStreamProperties.cs deleted file mode 100644 index 89686802f3..0000000000 --- a/projects/RabbitMQ.Client/client/api/IStreamProperties.cs +++ /dev/null @@ -1,129 +0,0 @@ -// This source code is dual-licensed under the Apache License, version -// 2.0, and the Mozilla Public License, version 2.0. -// -// The APL v2.0: -// -//--------------------------------------------------------------------------- -// Copyright (c) 2007-2020 VMware, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// https://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. -//--------------------------------------------------------------------------- -// -// The MPL v2.0: -// -//--------------------------------------------------------------------------- -// This Source Code Form is subject to the terms of the Mozilla Public -// License, v. 2.0. If a copy of the MPL was not distributed with this -// file, You can obtain one at https://mozilla.org/MPL/2.0/. -// -// Copyright (c) 2007-2020 VMware, Inc. All rights reserved. -//--------------------------------------------------------------------------- - -using System.Collections.Generic; - -namespace RabbitMQ.Client -{ - /// - /// Common AMQP Stream content-class headers interface, - ///spanning the union of the functionality offered by versions 0-8, 0-8qpid, 0-9 and 0-9-1 of AMQP. - /// - /// - /// - /// The specification code generator provides - /// protocol-version-specific implementations of this interface. To - /// obtain an implementation of this interface in a - /// protocol-version-neutral way, use IModel.CreateStreamProperties(). - /// - /// - /// Each property is readable, writable and clearable: a cleared - /// property will not be transmitted over the wire. Properties on a fresh instance are clear by default. - /// - /// - public interface IStreamProperties : IContentHeader - { - /// - /// MIME content encoding. - /// - string ContentEncoding { get; set; } - - /// - /// MIME content type. - /// - string ContentType { get; set; } - - /// - /// Message header field table. - /// - IDictionary Headers { get; set; } - - /// - /// Message priority, 0 to 9. - /// - byte Priority { get; set; } - - /// - /// Message timestamp. - /// - AmqpTimestamp Timestamp { get; set; } - - /// - /// Clear the property. - /// - void ClearContentEncoding(); - - /// - /// Clear the property. - /// - void ClearContentType(); - - /// - /// Clear the property. - /// - void ClearHeaders(); - - /// - /// Clear the property. - /// - void ClearPriority(); - - /// - /// Clear the property. - /// - void ClearTimestamp(); - - /// - /// Returns true if the property is present. - /// - bool IsContentEncodingPresent(); - - /// - /// Returns true if the property is present. - /// - bool IsContentTypePresent(); - - /// - /// Returns true if the property is present. - /// - bool IsHeadersPresent(); - - /// - /// Returns true if the property is present. - /// - bool IsPriorityPresent(); - - /// - /// Returns true if the property is present. - /// - bool IsTimestampPresent(); - } -} diff --git a/projects/RabbitMQ.Client/client/api/ReadonlyBasicProperties.cs b/projects/RabbitMQ.Client/client/api/ReadonlyBasicProperties.cs new file mode 100644 index 0000000000..a3c7bd5b03 --- /dev/null +++ b/projects/RabbitMQ.Client/client/api/ReadonlyBasicProperties.cs @@ -0,0 +1,123 @@ +// This source code is dual-licensed under the Apache License, version +// 2.0, and the Mozilla Public License, version 2.0. +// +// The APL v2.0: +// +//--------------------------------------------------------------------------- +// Copyright (c) 2007-2020 VMware, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// https://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +//--------------------------------------------------------------------------- +// +// The MPL v2.0: +// +//--------------------------------------------------------------------------- +// This Source Code Form is subject to the terms of the Mozilla Public +// License, v. 2.0. If a copy of the MPL was not distributed with this +// file, You can obtain one at https://mozilla.org/MPL/2.0/. +// +// Copyright (c) 2007-2020 VMware, Inc. All rights reserved. +//--------------------------------------------------------------------------- + +using System; +using System.Collections.Generic; +using RabbitMQ.Client.Impl; + +namespace RabbitMQ.Client +{ +#nullable enable + /// + /// AMQP specification content header properties for content class "basic" + /// + public readonly struct ReadOnlyBasicProperties : IReadOnlyBasicProperties + { + private readonly string? _contentType; + private readonly string? _contentEncoding; + private readonly IDictionary? _headers; + private readonly byte _deliveryMode; + private readonly byte _priority; + private readonly string? _correlationId; + private readonly string? _replyTo; + private readonly string? _expiration; + private readonly string? _messageId; + private readonly AmqpTimestamp _timestamp; + private readonly string? _type; + private readonly string? _userId; + private readonly string? _appId; + private readonly string? _clusterId; + + public string? ContentType => _contentType; + public string? ContentEncoding => _contentEncoding; + public IDictionary? Headers => _headers; + public byte DeliveryMode => _deliveryMode; + public byte Priority => _priority; + public string? CorrelationId => _correlationId; + public string? ReplyTo => _replyTo; + public string? Expiration => _expiration; + public string? MessageId => _messageId; + public AmqpTimestamp Timestamp => _timestamp; + public string? Type => _type; + public string? UserId => _userId; + public string? AppId => _appId; + public string? ClusterId => _clusterId; + + public bool Persistent => DeliveryMode == 2; + + public PublicationAddress? ReplyToAddress + { + get + { + PublicationAddress.TryParse(ReplyTo, out PublicationAddress result); + return result; + } + } + + public ReadOnlyBasicProperties(ReadOnlySpan span) + : this() + { + int offset = 2; + ref readonly byte bits = ref span[0]; + if (bits.IsBitSet(BasicProperties.ContentTypeBit)) { offset += WireFormatting.ReadShortstr(span.Slice(offset), out _contentType); } + if (bits.IsBitSet(BasicProperties.ContentEncodingBit)) { offset += WireFormatting.ReadShortstr(span.Slice(offset), out _contentEncoding); } + if (bits.IsBitSet(BasicProperties.HeaderBit)) { offset += WireFormatting.ReadDictionary(span.Slice(offset), out var tmpDirectory); _headers = tmpDirectory; } + if (bits.IsBitSet(BasicProperties.DeliveryModeBit)) { _deliveryMode = span[offset++]; } + if (bits.IsBitSet(BasicProperties.PriorityBit)) { _priority = span[offset++]; } + if (bits.IsBitSet(BasicProperties.CorrelationIdBit)) { offset += WireFormatting.ReadShortstr(span.Slice(offset), out _correlationId); } + if (bits.IsBitSet(BasicProperties.ReplyToBit)) { offset += WireFormatting.ReadShortstr(span.Slice(offset), out _replyTo); } + if (bits.IsBitSet(BasicProperties.ExpirationBit)) { offset += WireFormatting.ReadShortstr(span.Slice(offset), out _expiration); } + + bits = ref span[1]; + if (bits.IsBitSet(BasicProperties.MessageIdBit)) { offset += WireFormatting.ReadShortstr(span.Slice(offset), out _messageId); } + if (bits.IsBitSet(BasicProperties.TimestampBit)) { offset += WireFormatting.ReadTimestamp(span.Slice(offset), out _timestamp); } + if (bits.IsBitSet(BasicProperties.TypeBit)) { offset += WireFormatting.ReadShortstr(span.Slice(offset), out _type); } + if (bits.IsBitSet(BasicProperties.UserIdBit)) { offset += WireFormatting.ReadShortstr(span.Slice(offset), out _userId); } + if (bits.IsBitSet(BasicProperties.AppIdBit)) { offset += WireFormatting.ReadShortstr(span.Slice(offset), out _appId); } + if (bits.IsBitSet(BasicProperties.ClusterIdBit)) { WireFormatting.ReadShortstr(span.Slice(offset), out _clusterId); } + } + + public bool IsContentTypePresent() => ContentType != default; + public bool IsContentEncodingPresent() => ContentEncoding != default; + public bool IsHeadersPresent() => Headers != default; + public bool IsDeliveryModePresent() => DeliveryMode != default; + public bool IsPriorityPresent() => Priority != default; + public bool IsCorrelationIdPresent() => CorrelationId != default; + public bool IsReplyToPresent() => ReplyTo != default; + public bool IsExpirationPresent() => Expiration != default; + public bool IsMessageIdPresent() => MessageId != default; + public bool IsTimestampPresent() => Timestamp != default; + public bool IsTypePresent() => Type != default; + public bool IsUserIdPresent() => UserId != default; + public bool IsAppIdPresent() => AppId != default; + public bool IsClusterIdPresent() => ClusterId != default; + } +} diff --git a/projects/RabbitMQ.Client/client/events/AsyncEventingBasicConsumer.cs b/projects/RabbitMQ.Client/client/events/AsyncEventingBasicConsumer.cs index 8d9d784e4d..ac83df6b25 100644 --- a/projects/RabbitMQ.Client/client/events/AsyncEventingBasicConsumer.cs +++ b/projects/RabbitMQ.Client/client/events/AsyncEventingBasicConsumer.cs @@ -71,7 +71,7 @@ public override async Task HandleBasicConsumeOk(string consumerTag) } ///Fires the Received event. - public override Task HandleBasicDeliver(string consumerTag, ulong deliveryTag, bool redelivered, string exchange, string routingKey, IBasicProperties properties, ReadOnlyMemory body) + public override Task HandleBasicDeliver(string consumerTag, ulong deliveryTag, bool redelivered, string exchange, string routingKey, in ReadOnlyBasicProperties properties, ReadOnlyMemory body) { // No need to call base, it's empty. return _receivedWrapper.InvokeAsync(this, new BasicDeliverEventArgs(consumerTag, deliveryTag, redelivered, exchange, routingKey, properties, body)); diff --git a/projects/RabbitMQ.Client/client/events/BasicDeliverEventArgs.cs b/projects/RabbitMQ.Client/client/events/BasicDeliverEventArgs.cs index 867726df2e..96e90f4cdf 100644 --- a/projects/RabbitMQ.Client/client/events/BasicDeliverEventArgs.cs +++ b/projects/RabbitMQ.Client/client/events/BasicDeliverEventArgs.cs @@ -49,7 +49,7 @@ public BasicDeliverEventArgs(string consumerTag, bool redelivered, string exchange, string routingKey, - IBasicProperties properties, + in ReadOnlyBasicProperties properties, ReadOnlyMemory body) { ConsumerTag = consumerTag; @@ -62,7 +62,7 @@ public BasicDeliverEventArgs(string consumerTag, } ///The content header of the message. - public IBasicProperties BasicProperties { get; set; } + public ReadOnlyBasicProperties BasicProperties { get; set; } ///The message body. public ReadOnlyMemory Body { get; set; } diff --git a/projects/RabbitMQ.Client/client/events/BasicReturnEventArgs.cs b/projects/RabbitMQ.Client/client/events/BasicReturnEventArgs.cs index 8ec1311665..67220f1244 100644 --- a/projects/RabbitMQ.Client/client/events/BasicReturnEventArgs.cs +++ b/projects/RabbitMQ.Client/client/events/BasicReturnEventArgs.cs @@ -38,7 +38,7 @@ namespace RabbitMQ.Client.Events public class BasicReturnEventArgs : EventArgs { ///The content header of the message. - public IBasicProperties BasicProperties { get; set; } + public ReadOnlyBasicProperties BasicProperties { get; set; } ///The message body. public ReadOnlyMemory Body { get; set; } diff --git a/projects/RabbitMQ.Client/client/events/EventingBasicConsumer.cs b/projects/RabbitMQ.Client/client/events/EventingBasicConsumer.cs index 77cac48119..842ce37a11 100644 --- a/projects/RabbitMQ.Client/client/events/EventingBasicConsumer.cs +++ b/projects/RabbitMQ.Client/client/events/EventingBasicConsumer.cs @@ -84,7 +84,7 @@ public override void HandleBasicConsumeOk(string consumerTag) /// Accessing the body at a later point is unsafe as its memory can /// be already released. /// - public override void HandleBasicDeliver(string consumerTag, ulong deliveryTag, bool redelivered, string exchange, string routingKey, IBasicProperties properties, ReadOnlyMemory body) + public override void HandleBasicDeliver(string consumerTag, ulong deliveryTag, bool redelivered, string exchange, string routingKey, in ReadOnlyBasicProperties properties, ReadOnlyMemory body) { base.HandleBasicDeliver(consumerTag, deliveryTag, redelivered, exchange, routingKey, properties, body); Received?.Invoke( diff --git a/projects/RabbitMQ.Client/client/framing/BasicAck.cs b/projects/RabbitMQ.Client/client/framing/BasicAck.cs index 6476947ddb..027257e525 100644 --- a/projects/RabbitMQ.Client/client/framing/BasicAck.cs +++ b/projects/RabbitMQ.Client/client/framing/BasicAck.cs @@ -57,7 +57,7 @@ public BasicAck(ReadOnlySpan span) public ProtocolCommandId ProtocolCommandId => ProtocolCommandId.BasicAck; - public int WriteArgumentsTo(Span span) + public int WriteTo(Span span) { int offset = WireFormatting.WriteLonglong(ref span.GetStart(), _deliveryTag); return offset + WireFormatting.WriteBits(ref span.GetOffset(offset), _multiple); diff --git a/projects/RabbitMQ.Client/client/framing/BasicCancel.cs b/projects/RabbitMQ.Client/client/framing/BasicCancel.cs index 556b87f073..bec131c4e7 100644 --- a/projects/RabbitMQ.Client/client/framing/BasicCancel.cs +++ b/projects/RabbitMQ.Client/client/framing/BasicCancel.cs @@ -55,7 +55,7 @@ public BasicCancel(ReadOnlySpan span) public ProtocolCommandId ProtocolCommandId => ProtocolCommandId.BasicCancel; - public int WriteArgumentsTo(Span span) + public int WriteTo(Span span) { int offset = WireFormatting.WriteShortstr(ref span.GetStart(), _consumerTag); return offset + WireFormatting.WriteBits(ref span.GetOffset(offset), _nowait); diff --git a/projects/RabbitMQ.Client/client/framing/BasicConsume.cs b/projects/RabbitMQ.Client/client/framing/BasicConsume.cs index 830ed0b535..05e0234704 100644 --- a/projects/RabbitMQ.Client/client/framing/BasicConsume.cs +++ b/projects/RabbitMQ.Client/client/framing/BasicConsume.cs @@ -62,7 +62,7 @@ public BasicConsume(string Queue, string ConsumerTag, bool NoLocal, bool NoAck, public ProtocolCommandId ProtocolCommandId => ProtocolCommandId.BasicConsume; - public int WriteArgumentsTo(Span span) + public int WriteTo(Span span) { int offset = WireFormatting.WriteShort(ref span.GetStart(), default); offset += WireFormatting.WriteShortstr(ref span.GetOffset(offset), _queue); diff --git a/projects/RabbitMQ.Client/client/framing/BasicGet.cs b/projects/RabbitMQ.Client/client/framing/BasicGet.cs index 31fc917533..52e7cf2088 100644 --- a/projects/RabbitMQ.Client/client/framing/BasicGet.cs +++ b/projects/RabbitMQ.Client/client/framing/BasicGet.cs @@ -51,7 +51,7 @@ public BasicGet(string Queue, bool NoAck) public ProtocolCommandId ProtocolCommandId => ProtocolCommandId.BasicGet; - public int WriteArgumentsTo(Span span) + public int WriteTo(Span span) { int offset = WireFormatting.WriteShort(ref span.GetStart(), default); offset += WireFormatting.WriteShortstr(ref span.GetOffset(offset), _queue); diff --git a/projects/RabbitMQ.Client/client/framing/BasicNack.cs b/projects/RabbitMQ.Client/client/framing/BasicNack.cs index ac10a7d893..6f8a54aa5b 100644 --- a/projects/RabbitMQ.Client/client/framing/BasicNack.cs +++ b/projects/RabbitMQ.Client/client/framing/BasicNack.cs @@ -56,7 +56,7 @@ public BasicNack(ReadOnlySpan span) public ProtocolCommandId ProtocolCommandId => ProtocolCommandId.BasicNack; - public int WriteArgumentsTo(Span span) + public int WriteTo(Span span) { int offset = WireFormatting.WriteLonglong(ref span.GetStart(), _deliveryTag); return offset + WireFormatting.WriteBits(ref span.GetOffset(offset), _multiple, _requeue); diff --git a/projects/RabbitMQ.Client/client/framing/BasicProperties.cs b/projects/RabbitMQ.Client/client/framing/BasicProperties.cs deleted file mode 100644 index 4bc9704e76..0000000000 --- a/projects/RabbitMQ.Client/client/framing/BasicProperties.cs +++ /dev/null @@ -1,362 +0,0 @@ -// This source code is dual-licensed under the Apache License, version -// 2.0, and the Mozilla Public License, version 2.0. -// -// The APL v2.0: -// -//--------------------------------------------------------------------------- -// Copyright (c) 2007-2020 VMware, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// https://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. -//--------------------------------------------------------------------------- -// -// The MPL v2.0: -// -//--------------------------------------------------------------------------- -// This Source Code Form is subject to the terms of the Mozilla Public -// License, v. 2.0. If a copy of the MPL was not distributed with this -// file, You can obtain one at https://mozilla.org/MPL/2.0/. -// -// Copyright (c) 2007-2020 VMware, Inc. All rights reserved. -//--------------------------------------------------------------------------- - -using System; -using System.Collections.Generic; - -using RabbitMQ.Client.Impl; - -namespace RabbitMQ.Client.Framing -{ - /// Autogenerated type. AMQP specification content header properties for content class "basic" - internal sealed class BasicProperties : RabbitMQ.Client.Impl.BasicProperties - { - private string _contentType; - private string _contentEncoding; - private IDictionary _headers; - private byte _deliveryMode; - private byte _priority; - private string _correlationId; - private string _replyTo; - private string _expiration; - private string _messageId; - private AmqpTimestamp _timestamp; - private string _type; - private string _userId; - private string _appId; - private string _clusterId; - - public override string ContentType - { - get => _contentType; - set => _contentType = value; - } - - public override string ContentEncoding - { - get => _contentEncoding; - set => _contentEncoding = value; - } - - public override IDictionary Headers - { - get => _headers; - set => _headers = value; - } - - public override byte DeliveryMode - { - get => _deliveryMode; - set => _deliveryMode = value; - } - - public override byte Priority - { - get => _priority; - set => _priority = value; - } - - public override string CorrelationId - { - get => _correlationId; - set => _correlationId = value; - } - - public override string ReplyTo - { - get => _replyTo; - set => _replyTo = value; - } - - public override string Expiration - { - get => _expiration; - set => _expiration = value; - } - - public override string MessageId - { - get => _messageId; - set => _messageId = value; - } - - public override AmqpTimestamp Timestamp - { - get => _timestamp; - set => _timestamp = value; - } - - public override string Type - { - get => _type; - set => _type = value; - } - - public override string UserId - { - get => _userId; - set => _userId = value; - } - - public override string AppId - { - get => _appId; - set => _appId = value; - } - - public override string ClusterId - { - get => _clusterId; - set => _clusterId = value; - } - - public override void ClearContentType() => _contentType = default; - - public override void ClearContentEncoding() => _contentEncoding = default; - - public override void ClearHeaders() => _headers = default; - - public override void ClearDeliveryMode() => _deliveryMode = default; - - public override void ClearPriority() => _priority = default; - - public override void ClearCorrelationId() => _correlationId = default; - - public override void ClearReplyTo() => _replyTo = default; - - public override void ClearExpiration() => _expiration = default; - - public override void ClearMessageId() => _messageId = default; - - public override void ClearTimestamp() => _timestamp = default; - - public override void ClearType() => _type = default; - - public override void ClearUserId() => _userId = default; - - public override void ClearAppId() => _appId = default; - - public override void ClearClusterId() => _clusterId = default; - - public override bool IsContentTypePresent() => _contentType != default; - - public override bool IsContentEncodingPresent() => _contentEncoding != default; - - public override bool IsHeadersPresent() => _headers != default; - - public override bool IsDeliveryModePresent() => _deliveryMode != default; - - public override bool IsPriorityPresent() => _priority != default; - - public override bool IsCorrelationIdPresent() => _correlationId != default; - - public override bool IsReplyToPresent() => _replyTo != default; - - public override bool IsExpirationPresent() => _expiration != default; - - public override bool IsMessageIdPresent() => _messageId != default; - - public override bool IsTimestampPresent() => _timestamp != default; - - public override bool IsTypePresent() => _type != default; - - public override bool IsUserIdPresent() => _userId != default; - - public override bool IsAppIdPresent() => _appId != default; - - public override bool IsClusterIdPresent() => _clusterId != default; - - public BasicProperties() - { - } - - public BasicProperties(ReadOnlySpan span) - { - int offset = 2; - ref readonly byte bits = ref span[0]; - if (bits.IsBitSet(ContentTypeBit)) { offset += WireFormatting.ReadShortstr(span.Slice(offset), out _contentType); } - if (bits.IsBitSet(ContentEncodingBit)) { offset += WireFormatting.ReadShortstr(span.Slice(offset), out _contentEncoding); } - if (bits.IsBitSet(HeaderBit)) { offset += WireFormatting.ReadDictionary(span.Slice(offset), out var tmpDirectory); _headers = tmpDirectory; } - if (bits.IsBitSet(DeliveryModeBit)) { _deliveryMode = span[offset++]; } - if (bits.IsBitSet(PriorityBit)) { _priority = span[offset++]; } - if (bits.IsBitSet(CorrelationIdBit)) { offset += WireFormatting.ReadShortstr(span.Slice(offset), out _correlationId); } - if (bits.IsBitSet(ReplyToBit)) { offset += WireFormatting.ReadShortstr(span.Slice(offset), out _replyTo); } - if (bits.IsBitSet(ExpirationBit)) { offset += WireFormatting.ReadShortstr(span.Slice(offset), out _expiration); } - - bits = ref span[1]; - if (bits.IsBitSet(MessageIdBit)) { offset += WireFormatting.ReadShortstr(span.Slice(offset), out _messageId); } - if (bits.IsBitSet(TimestampBit)) { offset += WireFormatting.ReadTimestamp(span.Slice(offset), out _timestamp); } - if (bits.IsBitSet(TypeBit)) { offset += WireFormatting.ReadShortstr(span.Slice(offset), out _type); } - if (bits.IsBitSet(UserIdBit)) { offset += WireFormatting.ReadShortstr(span.Slice(offset), out _userId); } - if (bits.IsBitSet(AppIdBit)) { offset += WireFormatting.ReadShortstr(span.Slice(offset), out _appId); } - if (bits.IsBitSet(ClusterIdBit)) { WireFormatting.ReadShortstr(span.Slice(offset), out _clusterId); } - } - - public override ushort ProtocolClassId => 60; - public override string ProtocolClassName => "basic"; - - //---------------------------------- - // First byte - //---------------------------------- - private const byte ContentTypeBit = 7; - private const byte ContentEncodingBit = 6; - private const byte HeaderBit = 5; - private const byte DeliveryModeBit = 4; - private const byte PriorityBit = 3; - private const byte CorrelationIdBit = 2; - private const byte ReplyToBit = 1; - private const byte ExpirationBit = 0; - - //---------------------------------- - // Second byte - //---------------------------------- - private const byte MessageIdBit = 7; - private const byte TimestampBit = 6; - private const byte TypeBit = 5; - private const byte UserIdBit = 4; - private const byte AppIdBit = 3; - private const byte ClusterIdBit = 2; - - internal override int WritePropertiesTo(Span span) - { - int offset = 2; - ref byte bitValue = ref span.GetStart(); - bitValue = 0; - if (IsContentTypePresent()) - { - bitValue.SetBit(ContentTypeBit); - offset += WireFormatting.WriteShortstr(ref span.GetOffset(offset), _contentType); - } - - if (IsContentEncodingPresent()) - { - bitValue.SetBit(ContentEncodingBit); - offset += WireFormatting.WriteShortstr(ref span.GetOffset(offset), _contentEncoding); - } - - if (IsHeadersPresent()) - { - bitValue.SetBit(HeaderBit); - offset += WireFormatting.WriteTable(ref span.GetOffset(offset), _headers); - } - - if (IsDeliveryModePresent()) - { - bitValue.SetBit(DeliveryModeBit); - span.GetOffset(offset++) = _deliveryMode; - } - - if (IsPriorityPresent()) - { - bitValue.SetBit(PriorityBit); - span.GetOffset(offset++) = _priority; - } - - if (IsCorrelationIdPresent()) - { - bitValue.SetBit(CorrelationIdBit); - offset += WireFormatting.WriteShortstr(ref span.GetOffset(offset), _correlationId); - } - - if (IsReplyToPresent()) - { - bitValue.SetBit(ReplyToBit); - offset += WireFormatting.WriteShortstr(ref span.GetOffset(offset), _replyTo); - } - - if (IsExpirationPresent()) - { - bitValue.SetBit(ExpirationBit); - offset += WireFormatting.WriteShortstr(ref span.GetOffset(offset), _expiration); - } - - bitValue = ref span.GetOffset(1); - bitValue = 0; - if (IsMessageIdPresent()) - { - bitValue.SetBit(MessageIdBit); - offset += WireFormatting.WriteShortstr(ref span.GetOffset(offset), _messageId); - } - - if (IsTimestampPresent()) - { - bitValue.SetBit(TimestampBit); - offset += WireFormatting.WriteTimestamp(ref span.GetOffset(offset), _timestamp); - } - - if (IsTypePresent()) - { - bitValue.SetBit(TypeBit); - offset += WireFormatting.WriteShortstr(ref span.GetOffset(offset), _type); - } - - if (IsUserIdPresent()) - { - bitValue.SetBit(UserIdBit); - offset += WireFormatting.WriteShortstr(ref span.GetOffset(offset), _userId); - } - - if (IsAppIdPresent()) - { - bitValue.SetBit(AppIdBit); - offset += WireFormatting.WriteShortstr(ref span.GetOffset(offset), _appId); - } - - if (IsClusterIdPresent()) - { - bitValue.SetBit(ClusterIdBit); - offset += WireFormatting.WriteShortstr(ref span.GetOffset(offset), _clusterId); - } - - return offset; - } - - public override int GetRequiredPayloadBufferSize() - { - int bufferSize = 2; // number of presence fields (14) in 2 bytes blocks - if (IsContentTypePresent()) { bufferSize += 1 + WireFormatting.GetByteCount(_contentType); } // _contentType in bytes - if (IsContentEncodingPresent()) { bufferSize += 1 + WireFormatting.GetByteCount(_contentEncoding); } // _contentEncoding in bytes - if (IsHeadersPresent()) { bufferSize += WireFormatting.GetTableByteCount(_headers); } // _headers in bytes - if (IsDeliveryModePresent()) { bufferSize++; } // _deliveryMode in bytes - if (IsPriorityPresent()) { bufferSize++; } // _priority in bytes - if (IsCorrelationIdPresent()) { bufferSize += 1 + WireFormatting.GetByteCount(_correlationId); } // _correlationId in bytes - if (IsReplyToPresent()) { bufferSize += 1 + WireFormatting.GetByteCount(_replyTo); } // _replyTo in bytes - if (IsExpirationPresent()) { bufferSize += 1 + WireFormatting.GetByteCount(_expiration); } // _expiration in bytes - if (IsMessageIdPresent()) { bufferSize += 1 + WireFormatting.GetByteCount(_messageId); } // _messageId in bytes - if (IsTimestampPresent()) { bufferSize += 8; } // _timestamp in bytes - if (IsTypePresent()) { bufferSize += 1 + WireFormatting.GetByteCount(_type); } // _type in bytes - if (IsUserIdPresent()) { bufferSize += 1 + WireFormatting.GetByteCount(_userId); } // _userId in bytes - if (IsAppIdPresent()) { bufferSize += 1 + WireFormatting.GetByteCount(_appId); } // _appId in bytes - if (IsClusterIdPresent()) { bufferSize += 1 + WireFormatting.GetByteCount(_clusterId); } // _clusterId in bytes - return bufferSize; - } - } -} diff --git a/projects/RabbitMQ.Client/client/framing/BasicPublish.cs b/projects/RabbitMQ.Client/client/framing/BasicPublish.cs index a3dd2d2f50..9969a6a0c5 100644 --- a/projects/RabbitMQ.Client/client/framing/BasicPublish.cs +++ b/projects/RabbitMQ.Client/client/framing/BasicPublish.cs @@ -55,7 +55,7 @@ public BasicPublish(string Exchange, string RoutingKey, bool Mandatory, bool Imm public ProtocolCommandId ProtocolCommandId => ProtocolCommandId.BasicPublish; - public int WriteArgumentsTo(Span span) + public int WriteTo(Span span) { int offset = WireFormatting.WriteShort(ref span.GetStart(), default); offset += WireFormatting.WriteShortstr(ref span.GetOffset(offset), _exchange); @@ -91,7 +91,7 @@ public BasicPublishMemory(ReadOnlyMemory Exchange, ReadOnlyMemory Ro public ProtocolCommandId ProtocolCommandId => ProtocolCommandId.BasicPublish; - public int WriteArgumentsTo(Span span) + public int WriteTo(Span span) { int offset = WireFormatting.WriteShort(ref span.GetStart(), default); offset += WireFormatting.WriteShortstr(ref span.GetOffset(offset), _exchange.Span); diff --git a/projects/RabbitMQ.Client/client/framing/BasicQos.cs b/projects/RabbitMQ.Client/client/framing/BasicQos.cs index ac2ff32271..eea6d1f581 100644 --- a/projects/RabbitMQ.Client/client/framing/BasicQos.cs +++ b/projects/RabbitMQ.Client/client/framing/BasicQos.cs @@ -50,7 +50,7 @@ public BasicQos(uint PrefetchSize, ushort PrefetchCount, bool Global) public ProtocolCommandId ProtocolCommandId => ProtocolCommandId.BasicQos; - public int WriteArgumentsTo(Span span) + public int WriteTo(Span span) { int offset = WireFormatting.WriteLong(ref span.GetStart(), _prefetchSize); offset += WireFormatting.WriteShort(ref span.GetOffset(offset), _prefetchCount); diff --git a/projects/RabbitMQ.Client/client/framing/BasicRecover.cs b/projects/RabbitMQ.Client/client/framing/BasicRecover.cs index 0ed8ff142b..471e9b2274 100644 --- a/projects/RabbitMQ.Client/client/framing/BasicRecover.cs +++ b/projects/RabbitMQ.Client/client/framing/BasicRecover.cs @@ -46,7 +46,7 @@ public BasicRecover(bool Requeue) public ProtocolCommandId ProtocolCommandId => ProtocolCommandId.BasicRecover; - public int WriteArgumentsTo(Span span) + public int WriteTo(Span span) { return WireFormatting.WriteBits(ref span.GetStart(), _requeue); } diff --git a/projects/RabbitMQ.Client/client/framing/BasicRecoverAsync.cs b/projects/RabbitMQ.Client/client/framing/BasicRecoverAsync.cs index d48bb9ec51..e2c76097ea 100644 --- a/projects/RabbitMQ.Client/client/framing/BasicRecoverAsync.cs +++ b/projects/RabbitMQ.Client/client/framing/BasicRecoverAsync.cs @@ -46,7 +46,7 @@ public BasicRecoverAsync(bool Requeue) public ProtocolCommandId ProtocolCommandId => ProtocolCommandId.BasicRecoverAsync; - public int WriteArgumentsTo(Span span) + public int WriteTo(Span span) { return WireFormatting.WriteBits(ref span.GetStart(), _requeue); } diff --git a/projects/RabbitMQ.Client/client/framing/BasicReject.cs b/projects/RabbitMQ.Client/client/framing/BasicReject.cs index 8a914bf585..74fb7b5283 100644 --- a/projects/RabbitMQ.Client/client/framing/BasicReject.cs +++ b/projects/RabbitMQ.Client/client/framing/BasicReject.cs @@ -48,7 +48,7 @@ public BasicReject(ulong DeliveryTag, bool Requeue) public ProtocolCommandId ProtocolCommandId => ProtocolCommandId.BasicReject; - public int WriteArgumentsTo(Span span) + public int WriteTo(Span span) { int offset = WireFormatting.WriteLonglong(ref span.GetStart(), _deliveryTag); return offset + WireFormatting.WriteBits(ref span.GetOffset(offset), _requeue); diff --git a/projects/RabbitMQ.Client/client/framing/ChannelClose.cs b/projects/RabbitMQ.Client/client/framing/ChannelClose.cs index a96a5e84a8..3d39bd6df1 100644 --- a/projects/RabbitMQ.Client/client/framing/ChannelClose.cs +++ b/projects/RabbitMQ.Client/client/framing/ChannelClose.cs @@ -61,7 +61,7 @@ public ChannelClose(ReadOnlySpan span) public ProtocolCommandId ProtocolCommandId => ProtocolCommandId.ChannelClose; - public int WriteArgumentsTo(Span span) + public int WriteTo(Span span) { int offset = WireFormatting.WriteShort(ref span.GetStart(), _replyCode); offset += WireFormatting.WriteShortstr(ref span.GetOffset(offset), _replyText); diff --git a/projects/RabbitMQ.Client/client/framing/ChannelCloseOk.cs b/projects/RabbitMQ.Client/client/framing/ChannelCloseOk.cs index cfac32c4a1..bb4ebf80cf 100644 --- a/projects/RabbitMQ.Client/client/framing/ChannelCloseOk.cs +++ b/projects/RabbitMQ.Client/client/framing/ChannelCloseOk.cs @@ -38,7 +38,7 @@ namespace RabbitMQ.Client.Framing.Impl { public ProtocolCommandId ProtocolCommandId => ProtocolCommandId.ChannelCloseOk; - public int WriteArgumentsTo(Span span) + public int WriteTo(Span span) { return 0; } diff --git a/projects/RabbitMQ.Client/client/framing/ChannelFlowOk.cs b/projects/RabbitMQ.Client/client/framing/ChannelFlowOk.cs index 0208380c4e..45fc55d42a 100644 --- a/projects/RabbitMQ.Client/client/framing/ChannelFlowOk.cs +++ b/projects/RabbitMQ.Client/client/framing/ChannelFlowOk.cs @@ -46,7 +46,7 @@ public ChannelFlowOk(bool Active) public ProtocolCommandId ProtocolCommandId => ProtocolCommandId.ChannelFlowOk; - public int WriteArgumentsTo(Span span) + public int WriteTo(Span span) { return WireFormatting.WriteBits(ref span.GetStart(), _active); } diff --git a/projects/RabbitMQ.Client/client/framing/ChannelOpen.cs b/projects/RabbitMQ.Client/client/framing/ChannelOpen.cs index 20b43b1f6a..88d3fb42dd 100644 --- a/projects/RabbitMQ.Client/client/framing/ChannelOpen.cs +++ b/projects/RabbitMQ.Client/client/framing/ChannelOpen.cs @@ -41,7 +41,7 @@ namespace RabbitMQ.Client.Framing.Impl public ProtocolCommandId ProtocolCommandId => ProtocolCommandId.ChannelOpen; - public int WriteArgumentsTo(Span span) + public int WriteTo(Span span) { span[0] = 0; // _reserved1 return 1; diff --git a/projects/RabbitMQ.Client/client/framing/ConfirmSelect.cs b/projects/RabbitMQ.Client/client/framing/ConfirmSelect.cs index d95809c875..e3e558a35f 100644 --- a/projects/RabbitMQ.Client/client/framing/ConfirmSelect.cs +++ b/projects/RabbitMQ.Client/client/framing/ConfirmSelect.cs @@ -46,7 +46,7 @@ public ConfirmSelect(bool Nowait) public ProtocolCommandId ProtocolCommandId => ProtocolCommandId.ConfirmSelect; - public int WriteArgumentsTo(Span span) + public int WriteTo(Span span) { return WireFormatting.WriteBits(ref span.GetStart(), _nowait); } diff --git a/projects/RabbitMQ.Client/client/framing/ConnectionClose.cs b/projects/RabbitMQ.Client/client/framing/ConnectionClose.cs index 349c08f032..fb9fcfc7c9 100644 --- a/projects/RabbitMQ.Client/client/framing/ConnectionClose.cs +++ b/projects/RabbitMQ.Client/client/framing/ConnectionClose.cs @@ -61,7 +61,7 @@ public ConnectionClose(ReadOnlySpan span) public ProtocolCommandId ProtocolCommandId => ProtocolCommandId.ConnectionClose; - public int WriteArgumentsTo(Span span) + public int WriteTo(Span span) { int offset = WireFormatting.WriteShort(ref span.GetStart(), _replyCode); offset += WireFormatting.WriteShortstr(ref span.GetOffset(offset), _replyText); diff --git a/projects/RabbitMQ.Client/client/framing/ConnectionCloseOk.cs b/projects/RabbitMQ.Client/client/framing/ConnectionCloseOk.cs index 2c76a23d06..321f7b8c3d 100644 --- a/projects/RabbitMQ.Client/client/framing/ConnectionCloseOk.cs +++ b/projects/RabbitMQ.Client/client/framing/ConnectionCloseOk.cs @@ -38,7 +38,7 @@ namespace RabbitMQ.Client.Framing.Impl { public ProtocolCommandId ProtocolCommandId => ProtocolCommandId.ConnectionCloseOk; - public int WriteArgumentsTo(Span span) + public int WriteTo(Span span) { return 0; } diff --git a/projects/RabbitMQ.Client/client/framing/ConnectionOpen.cs b/projects/RabbitMQ.Client/client/framing/ConnectionOpen.cs index 73c3059f37..599ab9022d 100644 --- a/projects/RabbitMQ.Client/client/framing/ConnectionOpen.cs +++ b/projects/RabbitMQ.Client/client/framing/ConnectionOpen.cs @@ -50,7 +50,7 @@ public ConnectionOpen(string VirtualHost) public ProtocolCommandId ProtocolCommandId => ProtocolCommandId.ConnectionOpen; - public int WriteArgumentsTo(Span span) + public int WriteTo(Span span) { int offset = WireFormatting.WriteShortstr(ref span.GetStart(), _virtualHost); span[offset++] = 0; // _reserved1 diff --git a/projects/RabbitMQ.Client/client/framing/ConnectionSecureOk.cs b/projects/RabbitMQ.Client/client/framing/ConnectionSecureOk.cs index a9e51fd264..70c53119b3 100644 --- a/projects/RabbitMQ.Client/client/framing/ConnectionSecureOk.cs +++ b/projects/RabbitMQ.Client/client/framing/ConnectionSecureOk.cs @@ -46,7 +46,7 @@ public ConnectionSecureOk(byte[] Response) public ProtocolCommandId ProtocolCommandId => ProtocolCommandId.ConnectionSecureOk; - public int WriteArgumentsTo(Span span) + public int WriteTo(Span span) { return WireFormatting.WriteLongstr(ref span.GetStart(), _response); } diff --git a/projects/RabbitMQ.Client/client/framing/ConnectionStartOk.cs b/projects/RabbitMQ.Client/client/framing/ConnectionStartOk.cs index 6c4eb1cce9..66c322108b 100644 --- a/projects/RabbitMQ.Client/client/framing/ConnectionStartOk.cs +++ b/projects/RabbitMQ.Client/client/framing/ConnectionStartOk.cs @@ -54,7 +54,7 @@ public ConnectionStartOk(IDictionary ClientProperties, string Me public ProtocolCommandId ProtocolCommandId => ProtocolCommandId.ConnectionStartOk; - public int WriteArgumentsTo(Span span) + public int WriteTo(Span span) { int offset = WireFormatting.WriteTable(ref span.GetStart(), _clientProperties); offset += WireFormatting.WriteShortstr(ref span.GetOffset(offset), _mechanism); diff --git a/projects/RabbitMQ.Client/client/framing/ConnectionTuneOk.cs b/projects/RabbitMQ.Client/client/framing/ConnectionTuneOk.cs index 5106e21472..f17ec31dc6 100644 --- a/projects/RabbitMQ.Client/client/framing/ConnectionTuneOk.cs +++ b/projects/RabbitMQ.Client/client/framing/ConnectionTuneOk.cs @@ -50,7 +50,7 @@ public ConnectionTuneOk(ushort ChannelMax, uint FrameMax, ushort Heartbeat) public ProtocolCommandId ProtocolCommandId => ProtocolCommandId.ConnectionTuneOk; - public int WriteArgumentsTo(Span span) + public int WriteTo(Span span) { int offset = WireFormatting.WriteShort(ref span.GetStart(), _channelMax); offset += WireFormatting.WriteLong(ref span.GetOffset(offset), _frameMax); diff --git a/projects/RabbitMQ.Client/client/framing/ConnectionUpdateSecret.cs b/projects/RabbitMQ.Client/client/framing/ConnectionUpdateSecret.cs index 7b1a55673d..c318136f3e 100644 --- a/projects/RabbitMQ.Client/client/framing/ConnectionUpdateSecret.cs +++ b/projects/RabbitMQ.Client/client/framing/ConnectionUpdateSecret.cs @@ -49,7 +49,7 @@ public ConnectionUpdateSecret(byte[] NewSecret, string Reason) public ProtocolCommandId ProtocolCommandId => ProtocolCommandId.ConnectionUpdateSecret; - public int WriteArgumentsTo(Span span) + public int WriteTo(Span span) { int offset = WireFormatting.WriteLongstr(ref span.GetStart(), _newSecret); return offset + WireFormatting.WriteShortstr(ref span.GetOffset(offset), _reason); diff --git a/projects/RabbitMQ.Client/client/framing/ExchangeBind.cs b/projects/RabbitMQ.Client/client/framing/ExchangeBind.cs index 1b1610523e..3fd120e8fa 100644 --- a/projects/RabbitMQ.Client/client/framing/ExchangeBind.cs +++ b/projects/RabbitMQ.Client/client/framing/ExchangeBind.cs @@ -58,7 +58,7 @@ public ExchangeBind(string Destination, string Source, string RoutingKey, bool N public ProtocolCommandId ProtocolCommandId => ProtocolCommandId.ExchangeBind; - public int WriteArgumentsTo(Span span) + public int WriteTo(Span span) { int offset = WireFormatting.WriteShort(ref span.GetStart(), default); offset += WireFormatting.WriteShortstr(ref span.GetOffset(offset), _destination); diff --git a/projects/RabbitMQ.Client/client/framing/ExchangeDeclare.cs b/projects/RabbitMQ.Client/client/framing/ExchangeDeclare.cs index 5638e50a5e..25e730a218 100644 --- a/projects/RabbitMQ.Client/client/framing/ExchangeDeclare.cs +++ b/projects/RabbitMQ.Client/client/framing/ExchangeDeclare.cs @@ -64,7 +64,7 @@ public ExchangeDeclare(string Exchange, string Type, bool Passive, bool Durable, public ProtocolCommandId ProtocolCommandId => ProtocolCommandId.ExchangeDeclare; - public int WriteArgumentsTo(Span span) + public int WriteTo(Span span) { int offset = WireFormatting.WriteShort(ref span.GetStart(), default); offset += WireFormatting.WriteShortstr(ref span.GetOffset(offset), _exchange); diff --git a/projects/RabbitMQ.Client/client/framing/ExchangeDelete.cs b/projects/RabbitMQ.Client/client/framing/ExchangeDelete.cs index cae296f201..4943798866 100644 --- a/projects/RabbitMQ.Client/client/framing/ExchangeDelete.cs +++ b/projects/RabbitMQ.Client/client/framing/ExchangeDelete.cs @@ -53,7 +53,7 @@ public ExchangeDelete(string Exchange, bool IfUnused, bool Nowait) public ProtocolCommandId ProtocolCommandId => ProtocolCommandId.ExchangeDelete; - public int WriteArgumentsTo(Span span) + public int WriteTo(Span span) { int offset = WireFormatting.WriteShort(ref span.GetStart(), default); offset += WireFormatting.WriteShortstr(ref span.GetOffset(offset), _exchange); diff --git a/projects/RabbitMQ.Client/client/framing/ExchangeUnbind.cs b/projects/RabbitMQ.Client/client/framing/ExchangeUnbind.cs index 944e3cf542..fdd768e8d3 100644 --- a/projects/RabbitMQ.Client/client/framing/ExchangeUnbind.cs +++ b/projects/RabbitMQ.Client/client/framing/ExchangeUnbind.cs @@ -58,7 +58,7 @@ public ExchangeUnbind(string Destination, string Source, string RoutingKey, bool public ProtocolCommandId ProtocolCommandId => ProtocolCommandId.ExchangeUnbind; - public int WriteArgumentsTo(Span span) + public int WriteTo(Span span) { int offset = WireFormatting.WriteShort(ref span.GetStart(), default); offset += WireFormatting.WriteShortstr(ref span.GetOffset(offset), _destination); diff --git a/projects/RabbitMQ.Client/client/framing/IAmqpMethod.cs b/projects/RabbitMQ.Client/client/framing/IAmqpMethod.cs index 5d14cf416d..550af226e4 100644 --- a/projects/RabbitMQ.Client/client/framing/IAmqpMethod.cs +++ b/projects/RabbitMQ.Client/client/framing/IAmqpMethod.cs @@ -1,5 +1,3 @@ -using System; -using System.Diagnostics.Contracts; using RabbitMQ.Client.client.framing; namespace RabbitMQ.Client.Framing.Impl @@ -9,11 +7,7 @@ internal interface IAmqpMethod ProtocolCommandId ProtocolCommandId { get; } } - internal interface IOutgoingAmqpMethod : IAmqpMethod + internal interface IOutgoingAmqpMethod : IAmqpMethod, IAmqpWriteable { - [Pure] - int WriteArgumentsTo(Span span); - [Pure] - int GetRequiredBufferSize(); } } diff --git a/projects/RabbitMQ.Client/client/framing/Model.cs b/projects/RabbitMQ.Client/client/framing/Model.cs index c79c973777..ef92870017 100644 --- a/projects/RabbitMQ.Client/client/framing/Model.cs +++ b/projects/RabbitMQ.Client/client/framing/Model.cs @@ -29,7 +29,6 @@ // Copyright (c) 2007-2020 VMware, Inc. All rights reserved. //--------------------------------------------------------------------------- -using System; using System.Collections.Generic; using RabbitMQ.Client.client.framing; using RabbitMQ.Client.Impl; @@ -66,18 +65,6 @@ public override void _Private_BasicGet(string queue, bool autoAck) ModelSend(ref cmd); } - public override void _Private_BasicPublish(string exchange, string routingKey, bool mandatory, IBasicProperties basicProperties, ReadOnlyMemory body) - { - var cmd = new BasicPublish(exchange, routingKey, mandatory, default); - ModelSend(ref cmd, (BasicProperties) basicProperties, body); - } - - public override void _Private_BasicPublishMemory(ReadOnlyMemory exchange, ReadOnlyMemory routingKey, bool mandatory, IBasicProperties basicProperties, ReadOnlyMemory body) - { - var cmd = new BasicPublishMemory(exchange, routingKey, mandatory, default); - ModelSend(ref cmd, (BasicProperties) basicProperties, body); - } - public override void _Private_BasicRecover(bool requeue) { var cmd = new BasicRecover(requeue); @@ -283,11 +270,6 @@ public override void BasicReject(ulong deliveryTag, bool requeue) ModelSend(ref cmd); } - public override IBasicProperties CreateBasicProperties() - { - return new BasicProperties(); - } - public override void QueueUnbind(string queue, string exchange, string routingKey, IDictionary arguments) { var cmd = new QueueUnbind(queue, exchange, routingKey, arguments); diff --git a/projects/RabbitMQ.Client/client/framing/Protocol.cs b/projects/RabbitMQ.Client/client/framing/Protocol.cs index 4678427580..9f4a8d900e 100644 --- a/projects/RabbitMQ.Client/client/framing/Protocol.cs +++ b/projects/RabbitMQ.Client/client/framing/Protocol.cs @@ -57,14 +57,5 @@ internal override ProtocolCommandId DecodeCommandIdFrom(ReadOnlySpan span) { return (ProtocolCommandId)Util.NetworkOrderDeserializer.ReadUInt32(span); } - - internal override Client.Impl.ContentHeaderBase DecodeContentHeaderFrom(ushort classId, ReadOnlySpan span) - { - switch (classId) - { - case 60: return new BasicProperties(span); - default: throw new Exceptions.UnknownClassOrMethodException(classId, 0); - } - } } } diff --git a/projects/RabbitMQ.Client/client/framing/QueueBind.cs b/projects/RabbitMQ.Client/client/framing/QueueBind.cs index 9556acaacd..1f0246b806 100644 --- a/projects/RabbitMQ.Client/client/framing/QueueBind.cs +++ b/projects/RabbitMQ.Client/client/framing/QueueBind.cs @@ -58,7 +58,7 @@ public QueueBind(string Queue, string Exchange, string RoutingKey, bool Nowait, public ProtocolCommandId ProtocolCommandId => ProtocolCommandId.QueueBind; - public int WriteArgumentsTo(Span span) + public int WriteTo(Span span) { int offset = WireFormatting.WriteShort(ref span.GetStart(), default); offset += WireFormatting.WriteShortstr(ref span.GetOffset(offset), _queue); diff --git a/projects/RabbitMQ.Client/client/framing/QueueDeclare.cs b/projects/RabbitMQ.Client/client/framing/QueueDeclare.cs index 999e62834a..1fe6307bcf 100644 --- a/projects/RabbitMQ.Client/client/framing/QueueDeclare.cs +++ b/projects/RabbitMQ.Client/client/framing/QueueDeclare.cs @@ -62,7 +62,7 @@ public QueueDeclare(string Queue, bool Passive, bool Durable, bool Exclusive, bo public ProtocolCommandId ProtocolCommandId => ProtocolCommandId.QueueDeclare; - public int WriteArgumentsTo(Span span) + public int WriteTo(Span span) { int offset = WireFormatting.WriteShort(ref span.GetStart(), default); offset += WireFormatting.WriteShortstr(ref span.GetOffset(offset), _queue); diff --git a/projects/RabbitMQ.Client/client/framing/QueueDelete.cs b/projects/RabbitMQ.Client/client/framing/QueueDelete.cs index be83a7d6a1..7a776a5336 100644 --- a/projects/RabbitMQ.Client/client/framing/QueueDelete.cs +++ b/projects/RabbitMQ.Client/client/framing/QueueDelete.cs @@ -55,7 +55,7 @@ public QueueDelete(string Queue, bool IfUnused, bool IfEmpty, bool Nowait) public ProtocolCommandId ProtocolCommandId => ProtocolCommandId.QueueDelete; - public int WriteArgumentsTo(Span span) + public int WriteTo(Span span) { int offset = WireFormatting.WriteShort(ref span.GetStart(), default); offset += WireFormatting.WriteShortstr(ref span.GetOffset(offset), _queue); diff --git a/projects/RabbitMQ.Client/client/framing/QueuePurge.cs b/projects/RabbitMQ.Client/client/framing/QueuePurge.cs index 271e2a727d..d589d8ee24 100644 --- a/projects/RabbitMQ.Client/client/framing/QueuePurge.cs +++ b/projects/RabbitMQ.Client/client/framing/QueuePurge.cs @@ -51,7 +51,7 @@ public QueuePurge(string Queue, bool Nowait) public ProtocolCommandId ProtocolCommandId => ProtocolCommandId.QueuePurge; - public int WriteArgumentsTo(Span span) + public int WriteTo(Span span) { int offset = WireFormatting.WriteShort(ref span.GetStart(), default); offset += WireFormatting.WriteShortstr(ref span.GetOffset(offset), _queue); diff --git a/projects/RabbitMQ.Client/client/framing/QueueUnbind.cs b/projects/RabbitMQ.Client/client/framing/QueueUnbind.cs index 92fe785313..40fc505ed0 100644 --- a/projects/RabbitMQ.Client/client/framing/QueueUnbind.cs +++ b/projects/RabbitMQ.Client/client/framing/QueueUnbind.cs @@ -56,7 +56,7 @@ public QueueUnbind(string Queue, string Exchange, string RoutingKey, IDictionary public ProtocolCommandId ProtocolCommandId => ProtocolCommandId.QueueUnbind; - public int WriteArgumentsTo(Span span) + public int WriteTo(Span span) { int offset = WireFormatting.WriteShort(ref span.GetStart(), default); offset += WireFormatting.WriteShortstr(ref span.GetOffset(offset), _queue); diff --git a/projects/RabbitMQ.Client/client/framing/TxCommit.cs b/projects/RabbitMQ.Client/client/framing/TxCommit.cs index 6565094514..013999fa5f 100644 --- a/projects/RabbitMQ.Client/client/framing/TxCommit.cs +++ b/projects/RabbitMQ.Client/client/framing/TxCommit.cs @@ -38,7 +38,7 @@ namespace RabbitMQ.Client.Framing.Impl { public ProtocolCommandId ProtocolCommandId => ProtocolCommandId.TxCommit; - public int WriteArgumentsTo(Span span) + public int WriteTo(Span span) { return 0; } diff --git a/projects/RabbitMQ.Client/client/framing/TxRollback.cs b/projects/RabbitMQ.Client/client/framing/TxRollback.cs index a32b30739d..027713b6e4 100644 --- a/projects/RabbitMQ.Client/client/framing/TxRollback.cs +++ b/projects/RabbitMQ.Client/client/framing/TxRollback.cs @@ -38,7 +38,7 @@ namespace RabbitMQ.Client.Framing.Impl { public ProtocolCommandId ProtocolCommandId => ProtocolCommandId.TxRollback; - public int WriteArgumentsTo(Span span) + public int WriteTo(Span span) { return 0; } diff --git a/projects/RabbitMQ.Client/client/framing/TxSelect.cs b/projects/RabbitMQ.Client/client/framing/TxSelect.cs index 8b813ed6ac..dd0ea866fe 100644 --- a/projects/RabbitMQ.Client/client/framing/TxSelect.cs +++ b/projects/RabbitMQ.Client/client/framing/TxSelect.cs @@ -38,7 +38,7 @@ namespace RabbitMQ.Client.Framing.Impl { public ProtocolCommandId ProtocolCommandId => ProtocolCommandId.TxSelect; - public int WriteArgumentsTo(Span span) + public int WriteTo(Span span) { return 0; } diff --git a/projects/RabbitMQ.Client/client/impl/AutorecoveringModel.cs b/projects/RabbitMQ.Client/client/impl/AutorecoveringModel.cs index 347f2113d7..00b5ae22bf 100644 --- a/projects/RabbitMQ.Client/client/impl/AutorecoveringModel.cs +++ b/projects/RabbitMQ.Client/client/impl/AutorecoveringModel.cs @@ -242,11 +242,13 @@ public BasicGetResult BasicGet(string queue, bool autoAck) public void BasicNack(ulong deliveryTag, bool multiple, bool requeue) => InnerChannel.BasicNack(deliveryTag, multiple, requeue); - public void BasicPublish(string exchange, string routingKey, bool mandatory, IBasicProperties basicProperties, ReadOnlyMemory body) - => InnerChannel.BasicPublish(exchange, routingKey, mandatory, basicProperties, body); + public void BasicPublish(string exchange, string routingKey, in TProperties basicProperties, ReadOnlyMemory body, bool mandatory) + where TProperties : IReadOnlyBasicProperties, IAmqpHeader + => InnerChannel.BasicPublish(exchange, routingKey, basicProperties, body, mandatory); - public void BasicPublish(CachedString exchange, CachedString routingKey, bool mandatory, IBasicProperties basicProperties, ReadOnlyMemory body) - => InnerChannel.BasicPublish(exchange, routingKey, mandatory, basicProperties, body); + public void BasicPublish(CachedString exchange, CachedString routingKey, in TProperties basicProperties, ReadOnlyMemory body, bool mandatory) + where TProperties : IReadOnlyBasicProperties, IAmqpHeader + => InnerChannel.BasicPublish(exchange, routingKey, basicProperties, body, mandatory); public void BasicQos(uint prefetchSize, ushort prefetchCount, bool global) { @@ -277,9 +279,6 @@ public void ConfirmSelect() _usesPublisherConfirms = true; } - public IBasicProperties CreateBasicProperties() - => InnerChannel.CreateBasicProperties(); - public void ExchangeBind(string destination, string source, string routingKey, IDictionary arguments) { ThrowIfDisposed(); diff --git a/projects/RabbitMQ.Client/client/impl/BasicProperties.cs b/projects/RabbitMQ.Client/client/impl/BasicProperties.cs deleted file mode 100644 index 581c64913a..0000000000 --- a/projects/RabbitMQ.Client/client/impl/BasicProperties.cs +++ /dev/null @@ -1,284 +0,0 @@ -// This source code is dual-licensed under the Apache License, version -// 2.0, and the Mozilla Public License, version 2.0. -// -// The APL v2.0: -// -//--------------------------------------------------------------------------- -// Copyright (c) 2007-2020 VMware, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// https://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. -//--------------------------------------------------------------------------- -// -// The MPL v2.0: -// -//--------------------------------------------------------------------------- -// This Source Code Form is subject to the terms of the Mozilla Public -// License, v. 2.0. If a copy of the MPL was not distributed with this -// file, You can obtain one at https://mozilla.org/MPL/2.0/. -// -// Copyright (c) 2007-2020 VMware, Inc. All rights reserved. -//--------------------------------------------------------------------------- - -using System.Collections.Generic; - -namespace RabbitMQ.Client.Impl -{ - internal abstract class BasicProperties : ContentHeaderBase, IBasicProperties - { - /// - /// Application Id. - /// - public abstract string AppId { get; set; } - - /// - /// Intra-cluster routing identifier (cluster id is deprecated in AMQP 0-9-1). - /// - public abstract string ClusterId { get; set; } - - /// - /// MIME content encoding. - /// - public abstract string ContentEncoding { get; set; } - - /// - /// MIME content type. - /// - public abstract string ContentType { get; set; } - - /// - /// Application correlation identifier. - /// - public abstract string CorrelationId { get; set; } - - /// - /// Non-persistent (1) or persistent (2). - /// - public abstract byte DeliveryMode { get; set; } - - /// - /// Message expiration specification. - /// - public abstract string Expiration { get; set; } - - /// - /// Message header field table. Is of type . - /// - public abstract IDictionary Headers { get; set; } - - /// - /// Application message Id. - /// - public abstract string MessageId { get; set; } - - /// - /// Sets to either persistent (2) or non-persistent (1). - /// - public bool Persistent - { - get { return DeliveryMode == 2; } - set { DeliveryMode = value ? (byte)2 : (byte)1; } - } - - /// - /// Message priority, 0 to 9. - /// - public abstract byte Priority { get; set; } - - /// - /// Destination to reply to. - /// - public abstract string ReplyTo { get; set; } - - /// - /// Convenience property; parses property using , - /// and serializes it using . - /// Returns null if property cannot be parsed by . - /// - public PublicationAddress ReplyToAddress - { - get { - PublicationAddress.TryParse(ReplyTo, out PublicationAddress result); - return result; - } - set { ReplyTo = value.ToString(); } - } - - /// - /// Message timestamp. - /// - public abstract AmqpTimestamp Timestamp { get; set; } - - /// - /// Message type name. - /// - public abstract string Type { get; set; } - - /// - /// User Id. - /// - public abstract string UserId { get; set; } - - /// - /// Clear the property. - /// - public abstract void ClearAppId(); - - /// - /// Clear the property (cluster id is deprecated in AMQP 0-9-1). - /// - public abstract void ClearClusterId(); - - /// - /// Clear the property. - /// - public abstract void ClearContentEncoding(); - - /// - /// Clear the property. - /// - public abstract void ClearContentType(); - - /// - /// Clear the property. - /// - public abstract void ClearCorrelationId(); - - /// - /// Clear the property. - /// - public abstract void ClearDeliveryMode(); - - /// - /// Clear the property. - /// - public abstract void ClearExpiration(); - - /// - /// Clear the property. - /// - public abstract void ClearHeaders(); - - /// - /// Clear the property. - /// - public abstract void ClearMessageId(); - - /// - /// Clear the property. - /// - public abstract void ClearPriority(); - - /// - /// Clear the property. - /// - public abstract void ClearReplyTo(); - - /// - /// Clear the property. - /// - public abstract void ClearTimestamp(); - - /// - /// Clear the Type property. - /// - public abstract void ClearType(); - - /// - /// Clear the property. - /// - public abstract void ClearUserId(); - - /// - /// Returns true if the property is present. - /// - public abstract bool IsAppIdPresent(); - - /// - /// Returns true if the property is present (cluster id is deprecated in AMQP 0-9-1). - /// - public abstract bool IsClusterIdPresent(); - - /// - /// Returns true if the property is present. - /// - public abstract bool IsContentEncodingPresent(); - - /// - /// Returns true if the property is present. - /// - public abstract bool IsContentTypePresent(); - - /// - /// Returns true if the property is present. - /// - public abstract bool IsCorrelationIdPresent(); - - /// - /// Returns true if the property is present. - /// - public abstract bool IsDeliveryModePresent(); - - /// - /// Returns true if the property is present. - /// - public abstract bool IsExpirationPresent(); - - /// - /// Returns true if the property is present. - /// - public abstract bool IsHeadersPresent(); - - /// - /// Returns true if the property is present. - /// - public abstract bool IsMessageIdPresent(); - - /// - /// Returns true if the property is present. - /// - public abstract bool IsPriorityPresent(); - - /// - /// Returns true if the property is present. - /// - public abstract bool IsReplyToPresent(); - - /// - /// Returns true if the property is present. - /// - public abstract bool IsTimestampPresent(); - - /// - /// Returns true if the Type property is present. - /// - public abstract bool IsTypePresent(); - - /// - /// Returns true if the UserId property is present. - /// - public abstract bool IsUserIdPresent(); - - public abstract override int GetRequiredPayloadBufferSize(); - - public override object Clone() - { - var clone = MemberwiseClone() as BasicProperties; - if (IsHeadersPresent()) - { - clone.Headers = new Dictionary(Headers); - } - - return clone; - } - } -} diff --git a/projects/RabbitMQ.Client/client/impl/CommandAssembler.cs b/projects/RabbitMQ.Client/client/impl/CommandAssembler.cs index de5eaecd8f..d52bffe630 100644 --- a/projects/RabbitMQ.Client/client/impl/CommandAssembler.cs +++ b/projects/RabbitMQ.Client/client/impl/CommandAssembler.cs @@ -39,36 +39,36 @@ namespace RabbitMQ.Client.Impl { +#nullable enable internal sealed class CommandAssembler { private const int MaxArrayOfBytesSize = 2_147_483_591; - private readonly ProtocolBase _protocol; - - private ReadOnlyMemory _methodBytes; - private byte[] _rentedMethodArray; private ProtocolCommandId _commandId; - private ContentHeaderBase _header; - private byte[] _bodyBytes; - private ReadOnlyMemory _body; + private ReadOnlyMemory _methodBytes; + private byte[]? _rentedMethodArray; + private ReadOnlyMemory _headerBytes; + private byte[]? _rentedHeaderArray; + private ReadOnlyMemory _bodyBytes; + private byte[]? _rentedBodyArray; private int _remainingBodyBytes; private int _offset; private AssemblyState _state; - public CommandAssembler(ProtocolBase protocol) + public CommandAssembler() { - _protocol = protocol; Reset(); } private void Reset() { + _commandId = default; _methodBytes = ReadOnlyMemory.Empty; _rentedMethodArray = null; - _commandId = default; - _header = null; - _bodyBytes = null; - _body = ReadOnlyMemory.Empty; + _headerBytes = ReadOnlyMemory.Empty; + _rentedHeaderArray = null; + _bodyBytes = ReadOnlyMemory.Empty; + _rentedBodyArray = null; _remainingBodyBytes = 0; _offset = 0; _state = AssemblyState.ExpectingMethod; @@ -84,7 +84,7 @@ public bool HandleFrame(in InboundFrame frame, out IncomingCommand command) shallReturn = false; break; case AssemblyState.ExpectingContentHeader: - ParseHeaderFrame(in frame); + shallReturn = ParseHeaderFrame(in frame); break; case AssemblyState.ExpectingContentBody: shallReturn = ParseBodyFrame(in frame); @@ -98,7 +98,7 @@ public bool HandleFrame(in InboundFrame frame, out IncomingCommand command) } RabbitMqClientEventSource.Log.CommandReceived(); - command = new IncomingCommand(_commandId, _methodBytes, _rentedMethodArray, _header, _body, _bodyBytes); + command = new IncomingCommand(_commandId, _methodBytes, _rentedMethodArray, _headerBytes, _rentedHeaderArray, _bodyBytes, _rentedBodyArray); Reset(); return shallReturn; } @@ -128,7 +128,7 @@ private void ParseMethodFrame(in InboundFrame frame) } } - private void ParseHeaderFrame(in InboundFrame frame) + private bool ParseHeaderFrame(in InboundFrame frame) { if (frame.Type != FrameType.FrameHeader) { @@ -136,20 +136,24 @@ private void ParseHeaderFrame(in InboundFrame frame) } ReadOnlySpan span = frame.Payload.Span; - _header = _protocol.DecodeContentHeaderFrom(NetworkOrderDeserializer.ReadUInt16(span), span.Slice(12)); + var classId = NetworkOrderDeserializer.ReadUInt16(span); + if (classId != ClassConstants.Basic) + { + throw new UnknownClassOrMethodException(classId, 0); + } + ulong totalBodyBytes = NetworkOrderDeserializer.ReadUInt64(span.Slice(4)); if (totalBodyBytes > MaxArrayOfBytesSize) { throw new UnexpectedFrameException(frame.Type); } - if (totalBodyBytes == 0) - { - // If the body size is 0, there is no body frame coming, so assign an empty array - _bodyBytes = Array.Empty(); - } + _rentedHeaderArray = totalBodyBytes != 0 ? frame.TakeoverPayload() : Array.Empty(); + + _headerBytes = frame.Payload; _remainingBodyBytes = (int)totalBodyBytes; UpdateContentBodyState(); + return _rentedHeaderArray.Length == 0; } private bool ParseBodyFrame(in InboundFrame frame) @@ -165,23 +169,23 @@ private bool ParseBodyFrame(in InboundFrame frame) throw new MalformedFrameException($"Overlong content body received - {_remainingBodyBytes} bytes remaining, {payloadLength} bytes received"); } - if (_bodyBytes is null) + if (_rentedBodyArray is null) { // check for single frame payload for an early exit if (payloadLength == _remainingBodyBytes) { - _bodyBytes = frame.TakeoverPayload(); - _body = frame.Payload; + _rentedBodyArray = frame.TakeoverPayload(); + _bodyBytes = frame.Payload; _state = AssemblyState.Complete; return false; } // Is returned by IncomingCommand.ReturnPayload in Session.HandleFrame - _bodyBytes = ArrayPool.Shared.Rent(_remainingBodyBytes); - _body = new ReadOnlyMemory(_bodyBytes, 0, _remainingBodyBytes); + _rentedBodyArray = ArrayPool.Shared.Rent(_remainingBodyBytes); + _bodyBytes = new ReadOnlyMemory(_rentedBodyArray, 0, _remainingBodyBytes); } - frame.Payload.Span.CopyTo(_bodyBytes.AsSpan(_offset)); + frame.Payload.Span.CopyTo(_rentedBodyArray.AsSpan(_offset)); _remainingBodyBytes -= payloadLength; _offset += payloadLength; UpdateContentBodyState(); diff --git a/projects/RabbitMQ.Client/client/impl/ConsumerDispatching/ConsumerDispatcherChannelBase.cs b/projects/RabbitMQ.Client/client/impl/ConsumerDispatching/ConsumerDispatcherChannelBase.cs index bc56e037a5..ff5d3535f4 100644 --- a/projects/RabbitMQ.Client/client/impl/ConsumerDispatching/ConsumerDispatcherChannelBase.cs +++ b/projects/RabbitMQ.Client/client/impl/ConsumerDispatching/ConsumerDispatcherChannelBase.cs @@ -54,7 +54,7 @@ public void HandleBasicConsumeOk(IBasicConsumer consumer, string consumerTag) } public void HandleBasicDeliver(string consumerTag, ulong deliveryTag, bool redelivered, - string exchange, string routingKey, IBasicProperties basicProperties, ReadOnlyMemory body, byte[] rentedArray) + string exchange, string routingKey, in ReadOnlyBasicProperties basicProperties, ReadOnlyMemory body, byte[] rentedArray) { if (!IsShutdown) { @@ -110,7 +110,7 @@ protected readonly struct WorkStruct public readonly bool Redelivered; public readonly string? Exchange; public readonly string? RoutingKey; - public readonly IBasicProperties? BasicProperties; + public readonly ReadOnlyBasicProperties BasicProperties; public readonly ReadOnlyMemory Body; public readonly byte[]? RentedArray; public readonly ShutdownEventArgs? Reason; @@ -133,7 +133,7 @@ public WorkStruct(IBasicConsumer consumer, ShutdownEventArgs reason) } public WorkStruct(IBasicConsumer consumer, string consumerTag, ulong deliveryTag, bool redelivered, - string exchange, string routingKey, IBasicProperties basicProperties, ReadOnlyMemory body, byte[] rentedArray) + string exchange, string routingKey, in ReadOnlyBasicProperties basicProperties, ReadOnlyMemory body, byte[] rentedArray) { WorkType = WorkType.Deliver; Consumer = consumer; diff --git a/projects/RabbitMQ.Client/client/impl/ConsumerDispatching/FallbackConsumer.cs b/projects/RabbitMQ.Client/client/impl/ConsumerDispatching/FallbackConsumer.cs index 8b91fe3c41..cf7a613bc5 100644 --- a/projects/RabbitMQ.Client/client/impl/ConsumerDispatching/FallbackConsumer.cs +++ b/projects/RabbitMQ.Client/client/impl/ConsumerDispatching/FallbackConsumer.cs @@ -37,7 +37,7 @@ void IBasicConsumer.HandleBasicConsumeOk(string consumerTag) ESLog.Info($"Unhandled {nameof(IBasicConsumer.HandleBasicConsumeOk)} for tag {consumerTag}"); } - void IBasicConsumer.HandleBasicDeliver(string consumerTag, ulong deliveryTag, bool redelivered, string exchange, string routingKey, IBasicProperties properties, + void IBasicConsumer.HandleBasicDeliver(string consumerTag, ulong deliveryTag, bool redelivered, string exchange, string routingKey, in ReadOnlyBasicProperties properties, ReadOnlyMemory body) { ESLog.Info($"Unhandled {nameof(IBasicConsumer.HandleBasicDeliver)} for tag {consumerTag}"); @@ -66,7 +66,7 @@ Task IAsyncBasicConsumer.HandleBasicConsumeOk(string consumerTag) return Task.CompletedTask; } - Task IAsyncBasicConsumer.HandleBasicDeliver(string consumerTag, ulong deliveryTag, bool redelivered, string exchange, string routingKey, IBasicProperties properties, + Task IAsyncBasicConsumer.HandleBasicDeliver(string consumerTag, ulong deliveryTag, bool redelivered, string exchange, string routingKey, in ReadOnlyBasicProperties properties, ReadOnlyMemory body) { ((IBasicConsumer)this).HandleBasicDeliver(consumerTag, deliveryTag, redelivered, exchange, routingKey, properties, body); diff --git a/projects/RabbitMQ.Client/client/impl/ConsumerDispatching/IConsumerDispatcher.cs b/projects/RabbitMQ.Client/client/impl/ConsumerDispatching/IConsumerDispatcher.cs index 5946f6e58e..8104257bc1 100644 --- a/projects/RabbitMQ.Client/client/impl/ConsumerDispatching/IConsumerDispatcher.cs +++ b/projects/RabbitMQ.Client/client/impl/ConsumerDispatching/IConsumerDispatcher.cs @@ -50,7 +50,7 @@ void HandleBasicDeliver(string consumerTag, bool redelivered, string exchange, string routingKey, - IBasicProperties basicProperties, + in ReadOnlyBasicProperties basicProperties, ReadOnlyMemory body, byte[] rentedArray); diff --git a/projects/RabbitMQ.Client/client/impl/ContentHeaderBase.cs b/projects/RabbitMQ.Client/client/impl/ContentHeaderBase.cs deleted file mode 100644 index ca9213825d..0000000000 --- a/projects/RabbitMQ.Client/client/impl/ContentHeaderBase.cs +++ /dev/null @@ -1,57 +0,0 @@ -// This source code is dual-licensed under the Apache License, version -// 2.0, and the Mozilla Public License, version 2.0. -// -// The APL v2.0: -// -//--------------------------------------------------------------------------- -// Copyright (c) 2007-2020 VMware, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// https://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. -//--------------------------------------------------------------------------- -// -// The MPL v2.0: -// -//--------------------------------------------------------------------------- -// This Source Code Form is subject to the terms of the Mozilla Public -// License, v. 2.0. If a copy of the MPL was not distributed with this -// file, You can obtain one at https://mozilla.org/MPL/2.0/. -// -// Copyright (c) 2007-2020 VMware, Inc. All rights reserved. -//--------------------------------------------------------------------------- - -using System; - -namespace RabbitMQ.Client.Impl -{ - internal abstract class ContentHeaderBase : IContentHeader - { - /// - /// Retrieve the AMQP class ID of this content header. - /// - public abstract ushort ProtocolClassId { get; } - - /// - /// Retrieve the AMQP class name of this content header. - /// - public abstract string ProtocolClassName { get; } - - public virtual object Clone() - { - throw new NotImplementedException(); - } - - internal abstract int WritePropertiesTo(Span span); - - public abstract int GetRequiredPayloadBufferSize(); - } -} diff --git a/projects/RabbitMQ.Client/client/impl/EmptyBasicProperty.cs b/projects/RabbitMQ.Client/client/impl/EmptyBasicProperty.cs new file mode 100644 index 0000000000..f0f95ab3b6 --- /dev/null +++ b/projects/RabbitMQ.Client/client/impl/EmptyBasicProperty.cs @@ -0,0 +1,55 @@ +using System; +using System.Collections.Generic; +using RabbitMQ.Client.Framing.Impl; +using RabbitMQ.Client.Impl; + +namespace RabbitMQ.Client.client.impl +{ +#nullable enable + internal readonly struct EmptyBasicProperty : IReadOnlyBasicProperties, IAmqpHeader + { + ushort IAmqpHeader.ProtocolClassId => ClassConstants.Basic; + + int IAmqpWriteable.WriteTo(Span span) + { + return WireFormatting.WriteShort(ref span.GetStart(), 0); + } + + int IAmqpWriteable.GetRequiredBufferSize() + { + return 2; // number of presence fields (14) in 2 bytes blocks + } + + public string? AppId => default; + public string? ClusterId => default; + public string? ContentEncoding => default; + public string? ContentType => default; + public string? CorrelationId => default; + public byte DeliveryMode => default; + public string? Expiration => default; + public IDictionary? Headers => default; + public string? MessageId => default; + public bool Persistent => default; + public byte Priority => default; + public string? ReplyTo => default; + public PublicationAddress? ReplyToAddress => default; + public AmqpTimestamp Timestamp => default; + public string? Type => default; + public string? UserId => default; + + public bool IsAppIdPresent() => false; + public bool IsClusterIdPresent() => false; + public bool IsContentEncodingPresent() => false; + public bool IsContentTypePresent() => false; + public bool IsCorrelationIdPresent() => false; + public bool IsDeliveryModePresent() => false; + public bool IsExpirationPresent() => false; + public bool IsHeadersPresent() => false; + public bool IsMessageIdPresent() => false; + public bool IsPriorityPresent() => false; + public bool IsReplyToPresent() => false; + public bool IsTimestampPresent() => false; + public bool IsTypePresent() => false; + public bool IsUserIdPresent() => false; + } +} diff --git a/projects/RabbitMQ.Client/client/impl/Frame.cs b/projects/RabbitMQ.Client/client/impl/Frame.cs index 100600e42c..156dd3eed2 100644 --- a/projects/RabbitMQ.Client/client/impl/Frame.cs +++ b/projects/RabbitMQ.Client/client/impl/Frame.cs @@ -63,14 +63,14 @@ internal static class Method * | 2 bytes | 2 bytes | | * +----------+-----------+-----------+ */ public const int FrameSize = BaseFrameSize + 2 + 2; - + [MethodImpl(MethodImplOptions.AggressiveInlining)] public static int WriteTo(Span span, ushort channel, ref T method) where T : struct, IOutgoingAmqpMethod { const int StartClassId = StartPayload; const int StartMethodArguments = StartClassId + 4; - int payloadLength = method.WriteArgumentsTo(span.Slice(StartMethodArguments)) + 4; + int payloadLength = method.WriteTo(span.Slice(StartMethodArguments)) + 4; NetworkOrderSerializer.WriteUInt64(ref span.GetStart(), ((ulong)Constants.FrameMethod << 56) | ((ulong)channel << 40) | ((ulong)payloadLength << 8)); NetworkOrderSerializer.WriteUInt32(ref span.GetOffset(StartClassId), (uint)method.ProtocolCommandId); span[payloadLength + StartPayload] = Constants.FrameEnd; @@ -86,15 +86,15 @@ internal static class Header * | 2 bytes | 2 bytes | 8 bytes | x bytes | * +----------+----------+-------------------+-----------+ */ public const int FrameSize = BaseFrameSize + 2 + 2 + 8; - + [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static int WriteTo(Span span, ushort channel, T header, int bodyLength) where T : ContentHeaderBase + public static int WriteTo(Span span, ushort channel, ref T header, int bodyLength) where T : IAmqpHeader { const int StartClassId = StartPayload; const int StartBodyLength = StartPayload + 4; const int StartHeaderArguments = StartPayload + 12; - int payloadLength = 12 + header.WritePropertiesTo(span.Slice(StartHeaderArguments)); + int payloadLength = 12 + header.WriteTo(span.Slice(StartHeaderArguments)); NetworkOrderSerializer.WriteUInt64(ref span.GetStart(), ((ulong)Constants.FrameHeader << 56) | ((ulong)channel << 40) | ((ulong)payloadLength << 8)); NetworkOrderSerializer.WriteUInt32(ref span.GetOffset(StartClassId), (uint)header.ProtocolClassId << 16); // The last 16 bytes (Weight) aren't used NetworkOrderSerializer.WriteUInt64(ref span.GetOffset(StartBodyLength), (ulong)bodyLength); @@ -163,19 +163,20 @@ public static ReadOnlyMemory SerializeToFrames(ref T method, ushort cha } [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static ReadOnlyMemory SerializeToFrames(ref T method, ContentHeaderBase header, ReadOnlyMemory body, ushort channelNumber, int maxBodyPayloadBytes) - where T : struct, IOutgoingAmqpMethod + public static ReadOnlyMemory SerializeToFrames(ref TMethod method, ref THeader header, ReadOnlyMemory body, ushort channelNumber, int maxBodyPayloadBytes) + where TMethod : struct, IOutgoingAmqpMethod + where THeader : IAmqpHeader { int remainingBodyBytes = body.Length; int size = Method.FrameSize + Header.FrameSize + - method.GetRequiredBufferSize() + header.GetRequiredPayloadBufferSize() + + method.GetRequiredBufferSize() + header.GetRequiredBufferSize() + BodySegment.FrameSize * GetBodyFrameCount(maxBodyPayloadBytes, remainingBodyBytes) + remainingBodyBytes; // Will be returned by SocketFrameWriter.WriteLoop var array = ArrayPool.Shared.Rent(size); int offset = Method.WriteTo(array, channelNumber, ref method); - offset += Header.WriteTo(array.AsSpan(offset), channelNumber, header, remainingBodyBytes); + offset += Header.WriteTo(array.AsSpan(offset), channelNumber, ref header, remainingBodyBytes); var bodySpan = body.Span; while (remainingBodyBytes > 0) { diff --git a/projects/RabbitMQ.Client/client/impl/ISession.cs b/projects/RabbitMQ.Client/client/impl/ISession.cs index a1b2e99bc7..1ad103fe13 100644 --- a/projects/RabbitMQ.Client/client/impl/ISession.cs +++ b/projects/RabbitMQ.Client/client/impl/ISession.cs @@ -73,6 +73,8 @@ internal interface ISession bool HandleFrame(in InboundFrame frame); void Notify(); void Transmit(ref T cmd) where T : struct, IOutgoingAmqpMethod; - void Transmit(ref T cmd, ContentHeaderBase header, ReadOnlyMemory body) where T : struct, IOutgoingAmqpMethod; + void Transmit(ref TMethod cmd, ref THeader header, ReadOnlyMemory body) + where TMethod : struct, IOutgoingAmqpMethod + where THeader : IAmqpHeader; } } diff --git a/projects/RabbitMQ.Client/client/impl/IncomingCommand.cs b/projects/RabbitMQ.Client/client/impl/IncomingCommand.cs index a6396078d2..5f3aa7fbb8 100644 --- a/projects/RabbitMQ.Client/client/impl/IncomingCommand.cs +++ b/projects/RabbitMQ.Client/client/impl/IncomingCommand.cs @@ -13,28 +13,35 @@ internal readonly struct IncomingCommand public readonly ReadOnlyMemory MethodBytes; private readonly byte[] _rentedMethodBytes; - public readonly ContentHeaderBase Header; + public readonly ReadOnlyMemory HeaderBytes; + private readonly byte[] _rentedHeaderArray; public readonly ReadOnlyMemory Body; private readonly byte[] _rentedBodyArray; public bool IsEmpty => CommandId is default(ProtocolCommandId); - public IncomingCommand(ProtocolCommandId commandId, ReadOnlyMemory methodBytes, byte[] rentedMethodBytes, ContentHeaderBase header, ReadOnlyMemory body, byte[] rentedBodyArray) + public IncomingCommand(ProtocolCommandId commandId, ReadOnlyMemory methodBytes, byte[] rentedMethodArray, ReadOnlyMemory headerBytes, byte[] rentedHeaderArray, ReadOnlyMemory body, byte[] rentedBodyArray) { CommandId = commandId; MethodBytes = methodBytes; - _rentedMethodBytes = rentedMethodBytes; - Header = header; + _rentedMethodBytes = rentedMethodArray; + HeaderBytes = headerBytes; + _rentedHeaderArray = rentedHeaderArray; Body = body; _rentedBodyArray = rentedBodyArray; } - public byte[] TakeoverPayload() + public byte[] TakeoverBody() { return _rentedBodyArray; } + public void ReturnHeaderBuffer() + { + ArrayPool.Shared.Return(_rentedHeaderArray); + } + public void ReturnMethodBuffer() { ArrayPool.Shared.Return(_rentedMethodBytes); diff --git a/projects/RabbitMQ.Client/client/impl/ModelBase.cs b/projects/RabbitMQ.Client/client/impl/ModelBase.cs index b14d809b9b..5432cc4977 100644 --- a/projects/RabbitMQ.Client/client/impl/ModelBase.cs +++ b/projects/RabbitMQ.Client/client/impl/ModelBase.cs @@ -51,7 +51,6 @@ internal abstract class ModelBase : IModel, IRecoverable ///Only used to kick-start a connection open ///sequence. See internal BlockingCell m_connectionStartCell; - internal readonly IBasicProperties _emptyBasicProperties; private readonly RpcContinuationQueue _continuationQueue = new RpcContinuationQueue(); private readonly ManualResetEventSlim _flowControlBlock = new ManualResetEventSlim(true); @@ -73,7 +72,6 @@ protected ModelBase(bool dispatchAsync, int concurrency, ISession session) (IConsumerDispatcher)new AsyncConsumerDispatcher(this, concurrency) : new ConsumerDispatcher(this, concurrency); - _emptyBasicProperties = CreateBasicProperties(); Action onException = (exception, context) => OnCallbackException(CallbackExceptionEventArgs.Build(exception, context)); _basicAcksWrapper = new EventingWrapper("OnBasicAck", onException); _basicNacksWrapper = new EventingWrapper("OnBasicNack", onException); @@ -385,13 +383,15 @@ protected void ModelSend(ref T method) where T : struct, IOutgoingAmqpMethod } [MethodImpl(MethodImplOptions.AggressiveInlining)] - protected void ModelSend(ref T method, ContentHeaderBase header, ReadOnlyMemory body) where T : struct, IOutgoingAmqpMethod + protected void ModelSend(ref TMethod method, ref THeader header, ReadOnlyMemory body) + where TMethod : struct, IOutgoingAmqpMethod + where THeader : IAmqpHeader { if (!_flowControlBlock.IsSet) { _flowControlBlock.Wait(); } - Session.Transmit(ref method, header, body); + Session.Transmit(ref method, ref header, body); } internal void OnCallbackException(CallbackExceptionEventArgs args) @@ -574,6 +574,8 @@ protected void HandleBasicDeliver(in IncomingCommand cmd) { var method = new Client.Framing.Impl.BasicDeliver(cmd.MethodBytes.Span); cmd.ReturnMethodBuffer(); + var header = new ReadOnlyBasicProperties(cmd.HeaderBytes.Span); + cmd.ReturnHeaderBuffer(); ConsumerDispatcher.HandleBasicDeliver( method._consumerTag, @@ -581,15 +583,18 @@ protected void HandleBasicDeliver(in IncomingCommand cmd) method._redelivered, method._exchange, method._routingKey, - (IBasicProperties)cmd.Header, + header, cmd.Body, - cmd.TakeoverPayload()); + cmd.TakeoverBody()); } protected void HandleBasicGetOk(in IncomingCommand cmd) { var method = new BasicGetOk(cmd.MethodBytes.Span); cmd.ReturnMethodBuffer(); + var header = new ReadOnlyBasicProperties(cmd.HeaderBytes.Span); + cmd.ReturnHeaderBuffer(); + var k = (BasicGetRpcContinuation)_continuationQueue.Next(); k.m_result = new BasicGetResult( AdjustDeliveryTag(method._deliveryTag), @@ -597,9 +602,9 @@ protected void HandleBasicGetOk(in IncomingCommand cmd) method._exchange, method._routingKey, method._messageCount, - (IBasicProperties)cmd.Header, + header, cmd.Body, - cmd.TakeoverPayload()); + cmd.TakeoverBody()); k.HandleCommand(IncomingCommand.Empty); // release the continuation. } @@ -633,13 +638,14 @@ protected void HandleBasicReturn(in IncomingCommand cmd) ReplyText = basicReturn._replyText, Exchange = basicReturn._exchange, RoutingKey = basicReturn._routingKey, - BasicProperties = (IBasicProperties)cmd.Header, + BasicProperties = new ReadOnlyBasicProperties(cmd.HeaderBytes.Span), Body = cmd.Body }; _basicReturnWrapper.Invoke(this, e); } cmd.ReturnMethodBuffer(); - ArrayPool.Shared.Return(cmd.TakeoverPayload()); + cmd.ReturnHeaderBuffer(); + ArrayPool.Shared.Return(cmd.TakeoverBody()); } protected void HandleChannelClose(in IncomingCommand cmd) @@ -790,10 +796,6 @@ protected void HandleQueueDeclareOk(in IncomingCommand cmd) public abstract void _Private_BasicGet(string queue, bool autoAck); - public abstract void _Private_BasicPublish(string exchange, string routingKey, bool mandatory, IBasicProperties basicProperties, ReadOnlyMemory body); - - public abstract void _Private_BasicPublishMemory(ReadOnlyMemory exchange, ReadOnlyMemory routingKey, bool mandatory, IBasicProperties basicProperties, ReadOnlyMemory body); - public abstract void _Private_BasicRecover(bool requeue); public abstract void _Private_ChannelClose(ushort replyCode, string replyText, ushort classId, ushort methodId); @@ -895,13 +897,9 @@ public BasicGetResult BasicGet(string queue, bool autoAck) public abstract void BasicNack(ulong deliveryTag, bool multiple, bool requeue); - public void BasicPublish(string exchange, string routingKey, bool mandatory, IBasicProperties basicProperties, ReadOnlyMemory body) + public void BasicPublish(string exchange, string routingKey, in TProperties basicProperties, ReadOnlyMemory body, bool mandatory) + where TProperties : IReadOnlyBasicProperties, IAmqpHeader { - if (routingKey is null) - { - throw new ArgumentNullException(nameof(routingKey)); - } - if (NextPublishSeqNo > 0) { lock (_confirmLock) @@ -910,14 +908,13 @@ public void BasicPublish(string exchange, string routingKey, bool mandatory, IBa } } - _Private_BasicPublish(exchange, - routingKey, - mandatory, - basicProperties ?? _emptyBasicProperties, - body); + var cmd = new BasicPublish(exchange, routingKey, mandatory, default); + var props = basicProperties; + ModelSend(ref cmd, ref props, body); } - public void BasicPublish(CachedString exchange, CachedString routingKey, bool mandatory, IBasicProperties basicProperties, ReadOnlyMemory body) + public void BasicPublish(CachedString exchange, CachedString routingKey, in TProperties basicProperties, ReadOnlyMemory body, bool mandatory) + where TProperties : IReadOnlyBasicProperties, IAmqpHeader { if (NextPublishSeqNo > 0) { @@ -927,7 +924,9 @@ public void BasicPublish(CachedString exchange, CachedString routingKey, bool ma } } - _Private_BasicPublishMemory(exchange.Bytes, routingKey.Bytes, mandatory, basicProperties ?? _emptyBasicProperties, body); + var cmd = new BasicPublishMemory(exchange.Bytes, routingKey.Bytes, mandatory, default); + var props = basicProperties; + ModelSend(ref cmd, ref props, body); } public void UpdateSecret(string newSecret, string reason) @@ -974,8 +973,6 @@ public void ConfirmSelect() _Private_ConfirmSelect(false); } - public abstract IBasicProperties CreateBasicProperties(); - public void ExchangeBind(string destination, string source, string routingKey, IDictionary arguments) { _Private_ExchangeBind(destination, source, routingKey, false, arguments); diff --git a/projects/RabbitMQ.Client/client/impl/ProtocolBase.cs b/projects/RabbitMQ.Client/client/impl/ProtocolBase.cs index 7d2702748a..2d473f5cf6 100644 --- a/projects/RabbitMQ.Client/client/impl/ProtocolBase.cs +++ b/projects/RabbitMQ.Client/client/impl/ProtocolBase.cs @@ -65,7 +65,6 @@ public AmqpVersion Version get { return new AmqpVersion(MajorVersion, MinorVersion); } } - internal abstract ContentHeaderBase DecodeContentHeaderFrom(ushort classId, ReadOnlySpan span); internal abstract ProtocolCommandId DecodeCommandIdFrom(ReadOnlySpan span); public override bool Equals(object obj) diff --git a/projects/RabbitMQ.Client/client/impl/Session.cs b/projects/RabbitMQ.Client/client/impl/Session.cs index e101f386dd..d724fe3cca 100644 --- a/projects/RabbitMQ.Client/client/impl/Session.cs +++ b/projects/RabbitMQ.Client/client/impl/Session.cs @@ -40,7 +40,7 @@ internal class Session : SessionBase public Session(Connection connection, ushort channelNumber) : base(connection, channelNumber) { - _assembler = new CommandAssembler(connection.Protocol); + _assembler = new CommandAssembler(); } public override bool HandleFrame(in InboundFrame frame) diff --git a/projects/RabbitMQ.Client/client/impl/SessionBase.cs b/projects/RabbitMQ.Client/client/impl/SessionBase.cs index 6c0d6a5342..c6991f5c70 100644 --- a/projects/RabbitMQ.Client/client/impl/SessionBase.cs +++ b/projects/RabbitMQ.Client/client/impl/SessionBase.cs @@ -139,14 +139,16 @@ public virtual void Transmit(ref T cmd) where T : struct, IOutgoingAmqpMethod Connection.Write(Framing.SerializeToFrames(ref cmd, ChannelNumber)); } - public void Transmit(ref T cmd, ContentHeaderBase header, ReadOnlyMemory body) where T : struct, IOutgoingAmqpMethod + public void Transmit(ref TMethod cmd, ref THeader header, ReadOnlyMemory body) + where TMethod : struct, IOutgoingAmqpMethod + where THeader : IAmqpHeader { if (!IsOpen && cmd.ProtocolCommandId != ProtocolCommandId.ChannelCloseOk) { ThrowAlreadyClosedException(); } - Connection.Write(Framing.SerializeToFrames(ref cmd, header, body, ChannelNumber, Connection.MaxPayloadSize)); + Connection.Write(Framing.SerializeToFrames(ref cmd, ref header, body, ChannelNumber, Connection.MaxPayloadSize)); } private void ThrowAlreadyClosedException() diff --git a/projects/RabbitMQ.Client/client/impl/StreamProperties.cs b/projects/RabbitMQ.Client/client/impl/StreamProperties.cs deleted file mode 100644 index 0109e00d99..0000000000 --- a/projects/RabbitMQ.Client/client/impl/StreamProperties.cs +++ /dev/null @@ -1,67 +0,0 @@ -// This source code is dual-licensed under the Apache License, version -// 2.0, and the Mozilla Public License, version 2.0. -// -// The APL v2.0: -// -//--------------------------------------------------------------------------- -// Copyright (c) 2007-2020 VMware, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// https://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. -//--------------------------------------------------------------------------- -// -// The MPL v2.0: -// -//--------------------------------------------------------------------------- -// This Source Code Form is subject to the terms of the Mozilla Public -// License, v. 2.0. If a copy of the MPL was not distributed with this -// file, You can obtain one at https://mozilla.org/MPL/2.0/. -// -// Copyright (c) 2007-2020 VMware, Inc. All rights reserved. -//--------------------------------------------------------------------------- - -using System.Collections.Generic; - -namespace RabbitMQ.Client.Impl -{ - internal abstract class StreamProperties : ContentHeaderBase, IStreamProperties - { - public abstract string ContentEncoding { get; set; } - public abstract string ContentType { get; set; } - public abstract IDictionary Headers { get; set; } - public abstract byte Priority { get; set; } - public abstract AmqpTimestamp Timestamp { get; set; } - - public override object Clone() - { - var clone = MemberwiseClone() as StreamProperties; - if (IsHeadersPresent()) - { - clone.Headers = new Dictionary(Headers); - } - - return clone; - } - - public abstract void ClearContentEncoding(); - public abstract void ClearContentType(); - public abstract void ClearHeaders(); - public abstract void ClearPriority(); - public abstract void ClearTimestamp(); - - public abstract bool IsContentEncodingPresent(); - public abstract bool IsContentTypePresent(); - public abstract bool IsHeadersPresent(); - public abstract bool IsPriorityPresent(); - public abstract bool IsTimestampPresent(); - } -} diff --git a/projects/TestApplications/MassPublish/Program.cs b/projects/TestApplications/MassPublish/Program.cs index 1adc547403..c6cd9ab06a 100644 --- a/projects/TestApplications/MassPublish/Program.cs +++ b/projects/TestApplications/MassPublish/Program.cs @@ -44,10 +44,11 @@ public static void Main() { for (int i = 0; i < ItemsPerBatch; i++) { - var properties = publisher.CreateBasicProperties(); - properties.AppId = "testapp"; - properties.CorrelationId = Guid.NewGuid().ToString(); - publisher.BasicPublish("test", "myawesome.routing.key", false, properties, payload); + var properties = new BasicProperties + { + AppId = "testapp", + }; + publisher.BasicPublish("test", "myawesome.routing.key", properties, payload); } messagesSent += ItemsPerBatch; await publisher.WaitForConfirmsOrDieAsync().ConfigureAwait(false); diff --git a/projects/Unit/APIApproval.Approve.verified.txt b/projects/Unit/APIApproval.Approve.verified.txt index 5ca48dd94c..01700fd9f4 100644 --- a/projects/Unit/APIApproval.Approve.verified.txt +++ b/projects/Unit/APIApproval.Approve.verified.txt @@ -47,15 +47,15 @@ namespace RabbitMQ.Client public virtual System.Threading.Tasks.Task HandleBasicCancel(string consumerTag) { } public virtual System.Threading.Tasks.Task HandleBasicCancelOk(string consumerTag) { } public virtual System.Threading.Tasks.Task HandleBasicConsumeOk(string consumerTag) { } - public virtual System.Threading.Tasks.Task HandleBasicDeliver(string consumerTag, ulong deliveryTag, bool redelivered, string exchange, string routingKey, RabbitMQ.Client.IBasicProperties properties, System.ReadOnlyMemory body) { } + public virtual System.Threading.Tasks.Task HandleBasicDeliver(string consumerTag, ulong deliveryTag, bool redelivered, string exchange, string routingKey, in RabbitMQ.Client.ReadOnlyBasicProperties properties, System.ReadOnlyMemory body) { } public virtual System.Threading.Tasks.Task HandleModelShutdown(object model, RabbitMQ.Client.ShutdownEventArgs reason) { } public virtual System.Threading.Tasks.Task OnCancel(params string[] consumerTags) { } } public sealed class BasicGetResult : System.IDisposable { - public BasicGetResult(ulong deliveryTag, bool redelivered, string exchange, string routingKey, uint messageCount, RabbitMQ.Client.IBasicProperties basicProperties, System.ReadOnlyMemory body) { } - public BasicGetResult(ulong deliveryTag, bool redelivered, string exchange, string routingKey, uint messageCount, RabbitMQ.Client.IBasicProperties basicProperties, System.ReadOnlyMemory body, byte[] rentedArray) { } - public RabbitMQ.Client.IBasicProperties BasicProperties { get; } + public BasicGetResult(ulong deliveryTag, bool redelivered, string exchange, string routingKey, uint messageCount, in RabbitMQ.Client.ReadOnlyBasicProperties basicProperties, System.ReadOnlyMemory body) { } + public BasicGetResult(ulong deliveryTag, bool redelivered, string exchange, string routingKey, uint messageCount, in RabbitMQ.Client.ReadOnlyBasicProperties basicProperties, System.ReadOnlyMemory body, byte[] rentedArray) { } + public RabbitMQ.Client.ReadOnlyBasicProperties BasicProperties { get; } public System.ReadOnlyMemory Body { get; } public ulong DeliveryTag { get; } public string Exchange { get; } @@ -64,6 +64,54 @@ namespace RabbitMQ.Client public string RoutingKey { get; } public void Dispose() { } } + public struct BasicProperties : RabbitMQ.Client.IAmqpHeader, RabbitMQ.Client.IAmqpWriteable, RabbitMQ.Client.IBasicProperties, RabbitMQ.Client.IReadOnlyBasicProperties + { + public BasicProperties(in RabbitMQ.Client.ReadOnlyBasicProperties input) { } + public string? AppId { get; set; } + public string? ClusterId { get; set; } + public string? ContentEncoding { get; set; } + public string? ContentType { get; set; } + public string? CorrelationId { get; set; } + public byte DeliveryMode { get; set; } + public string? Expiration { get; set; } + public System.Collections.Generic.IDictionary? Headers { get; set; } + public string? MessageId { get; set; } + public bool Persistent { get; set; } + public byte Priority { get; set; } + public string? ReplyTo { get; set; } + public RabbitMQ.Client.PublicationAddress? ReplyToAddress { get; set; } + public RabbitMQ.Client.AmqpTimestamp Timestamp { get; set; } + public string? Type { get; set; } + public string? UserId { get; set; } + public void ClearAppId() { } + public void ClearClusterId() { } + public void ClearContentEncoding() { } + public void ClearContentType() { } + public void ClearCorrelationId() { } + public void ClearDeliveryMode() { } + public void ClearExpiration() { } + public void ClearHeaders() { } + public void ClearMessageId() { } + public void ClearPriority() { } + public void ClearReplyTo() { } + public void ClearTimestamp() { } + public void ClearType() { } + public void ClearUserId() { } + public bool IsAppIdPresent() { } + public bool IsClusterIdPresent() { } + public bool IsContentEncodingPresent() { } + public bool IsContentTypePresent() { } + public bool IsCorrelationIdPresent() { } + public bool IsDeliveryModePresent() { } + public bool IsExpirationPresent() { } + public bool IsHeadersPresent() { } + public bool IsMessageIdPresent() { } + public bool IsPriorityPresent() { } + public bool IsReplyToPresent() { } + public bool IsTimestampPresent() { } + public bool IsTypePresent() { } + public bool IsUserIdPresent() { } + } public class BinaryTableValue { public BinaryTableValue() { } @@ -172,7 +220,7 @@ namespace RabbitMQ.Client public virtual void HandleBasicCancel(string consumerTag) { } public virtual void HandleBasicCancelOk(string consumerTag) { } public virtual void HandleBasicConsumeOk(string consumerTag) { } - public virtual void HandleBasicDeliver(string consumerTag, ulong deliveryTag, bool redelivered, string exchange, string routingKey, RabbitMQ.Client.IBasicProperties properties, System.ReadOnlyMemory body) { } + public virtual void HandleBasicDeliver(string consumerTag, ulong deliveryTag, bool redelivered, string exchange, string routingKey, in RabbitMQ.Client.ReadOnlyBasicProperties properties, System.ReadOnlyMemory body) { } public virtual void HandleModelShutdown(object model, RabbitMQ.Client.ShutdownEventArgs reason) { } public virtual void OnCancel(params string[] consumerTags) { } } @@ -221,6 +269,15 @@ namespace RabbitMQ.Client public const string XQuorumInitialGroupSize = "x-quorum-initial-group-size"; public const string XSingleActiveConsumer = "x-single-active-consumer"; } + public interface IAmqpHeader : RabbitMQ.Client.IAmqpWriteable + { + ushort ProtocolClassId { get; } + } + public interface IAmqpWriteable + { + int GetRequiredBufferSize(); + int WriteTo(System.Span span); + } public interface IAsyncBasicConsumer { RabbitMQ.Client.IModel Model { get; } @@ -228,7 +285,7 @@ namespace RabbitMQ.Client System.Threading.Tasks.Task HandleBasicCancel(string consumerTag); System.Threading.Tasks.Task HandleBasicCancelOk(string consumerTag); System.Threading.Tasks.Task HandleBasicConsumeOk(string consumerTag); - System.Threading.Tasks.Task HandleBasicDeliver(string consumerTag, ulong deliveryTag, bool redelivered, string exchange, string routingKey, RabbitMQ.Client.IBasicProperties properties, System.ReadOnlyMemory body); + System.Threading.Tasks.Task HandleBasicDeliver(string consumerTag, ulong deliveryTag, bool redelivered, string exchange, string routingKey, in RabbitMQ.Client.ReadOnlyBasicProperties properties, System.ReadOnlyMemory body); System.Threading.Tasks.Task HandleModelShutdown(object model, RabbitMQ.Client.ShutdownEventArgs reason); } public interface IAuthMechanism @@ -247,27 +304,27 @@ namespace RabbitMQ.Client void HandleBasicCancel(string consumerTag); void HandleBasicCancelOk(string consumerTag); void HandleBasicConsumeOk(string consumerTag); - void HandleBasicDeliver(string consumerTag, ulong deliveryTag, bool redelivered, string exchange, string routingKey, RabbitMQ.Client.IBasicProperties properties, System.ReadOnlyMemory body); + void HandleBasicDeliver(string consumerTag, ulong deliveryTag, bool redelivered, string exchange, string routingKey, in RabbitMQ.Client.ReadOnlyBasicProperties properties, System.ReadOnlyMemory body); void HandleModelShutdown(object model, RabbitMQ.Client.ShutdownEventArgs reason); } - public interface IBasicProperties : RabbitMQ.Client.IContentHeader - { - string AppId { get; set; } - string ClusterId { get; set; } - string ContentEncoding { get; set; } - string ContentType { get; set; } - string CorrelationId { get; set; } - byte DeliveryMode { get; set; } - string Expiration { get; set; } - System.Collections.Generic.IDictionary Headers { get; set; } - string MessageId { get; set; } - bool Persistent { get; set; } - byte Priority { get; set; } - string ReplyTo { get; set; } - RabbitMQ.Client.PublicationAddress ReplyToAddress { get; set; } - RabbitMQ.Client.AmqpTimestamp Timestamp { get; set; } - string Type { get; set; } - string UserId { get; set; } + public interface IBasicProperties : RabbitMQ.Client.IReadOnlyBasicProperties + { + new string? AppId { get; set; } + new string? ClusterId { get; set; } + new string? ContentEncoding { get; set; } + new string? ContentType { get; set; } + new string? CorrelationId { get; set; } + new byte DeliveryMode { get; set; } + new string? Expiration { get; set; } + new System.Collections.Generic.IDictionary? Headers { get; set; } + new string? MessageId { get; set; } + new bool Persistent { get; set; } + new byte Priority { get; set; } + new string? ReplyTo { get; set; } + new RabbitMQ.Client.PublicationAddress? ReplyToAddress { get; set; } + new RabbitMQ.Client.AmqpTimestamp Timestamp { get; set; } + new string? Type { get; set; } + new string? UserId { get; set; } void ClearAppId(); void ClearClusterId(); void ClearContentEncoding(); @@ -282,20 +339,6 @@ namespace RabbitMQ.Client void ClearTimestamp(); void ClearType(); void ClearUserId(); - bool IsAppIdPresent(); - bool IsClusterIdPresent(); - bool IsContentEncodingPresent(); - bool IsContentTypePresent(); - bool IsCorrelationIdPresent(); - bool IsDeliveryModePresent(); - bool IsExpirationPresent(); - bool IsHeadersPresent(); - bool IsMessageIdPresent(); - bool IsPriorityPresent(); - bool IsReplyToPresent(); - bool IsTimestampPresent(); - bool IsTypePresent(); - bool IsUserIdPresent(); } public interface IConnection : RabbitMQ.Client.INetworkConnection, System.IDisposable { @@ -356,11 +399,6 @@ namespace RabbitMQ.Client RabbitMQ.Client.IConnection CreateConnection(System.Collections.Generic.IList endpoints, string clientProvidedName); RabbitMQ.Client.IConnection CreateConnection(System.Collections.Generic.IList hostnames, string clientProvidedName); } - public interface IContentHeader - { - ushort ProtocolClassId { get; } - string ProtocolClassName { get; } - } public interface IEndpointResolver { System.Collections.Generic.IEnumerable All(); @@ -387,8 +425,10 @@ namespace RabbitMQ.Client string BasicConsume(string queue, bool autoAck, string consumerTag, bool noLocal, bool exclusive, System.Collections.Generic.IDictionary arguments, RabbitMQ.Client.IBasicConsumer consumer); RabbitMQ.Client.BasicGetResult BasicGet(string queue, bool autoAck); void BasicNack(ulong deliveryTag, bool multiple, bool requeue); - void BasicPublish(RabbitMQ.Client.CachedString exchange, RabbitMQ.Client.CachedString routingKey, bool mandatory, RabbitMQ.Client.IBasicProperties basicProperties, System.ReadOnlyMemory body); - void BasicPublish(string exchange, string routingKey, bool mandatory, RabbitMQ.Client.IBasicProperties basicProperties, System.ReadOnlyMemory body); + void BasicPublish(RabbitMQ.Client.CachedString exchange, RabbitMQ.Client.CachedString routingKey, in TProperties basicProperties, System.ReadOnlyMemory body = default, bool mandatory = false) + where TProperties : RabbitMQ.Client.IReadOnlyBasicProperties, RabbitMQ.Client.IAmqpHeader; + void BasicPublish(string exchange, string routingKey, in TProperties basicProperties, System.ReadOnlyMemory body = default, bool mandatory = false) + where TProperties : RabbitMQ.Client.IReadOnlyBasicProperties, RabbitMQ.Client.IAmqpHeader; void BasicQos(uint prefetchSize, ushort prefetchCount, bool global); void BasicRecover(bool requeue); void BasicRecoverAsync(bool requeue); @@ -396,7 +436,6 @@ namespace RabbitMQ.Client void Close(ushort replyCode, string replyText, bool abort); void ConfirmSelect(); uint ConsumerCount(string queue); - RabbitMQ.Client.IBasicProperties CreateBasicProperties(); void ExchangeBind(string destination, string source, string routingKey, System.Collections.Generic.IDictionary arguments); void ExchangeBindNoWait(string destination, string source, string routingKey, System.Collections.Generic.IDictionary arguments); void ExchangeDeclare(string exchange, string type, bool durable, bool autoDelete, System.Collections.Generic.IDictionary arguments); @@ -430,10 +469,10 @@ namespace RabbitMQ.Client public static string BasicConsume(this RabbitMQ.Client.IModel model, string queue, bool autoAck, string consumerTag, RabbitMQ.Client.IBasicConsumer consumer) { } public static string BasicConsume(this RabbitMQ.Client.IModel model, string queue, bool autoAck, string consumerTag, System.Collections.Generic.IDictionary arguments, RabbitMQ.Client.IBasicConsumer consumer) { } public static string BasicConsume(this RabbitMQ.Client.IModel model, RabbitMQ.Client.IBasicConsumer consumer, string queue, bool autoAck = false, string consumerTag = "", bool noLocal = false, bool exclusive = false, System.Collections.Generic.IDictionary arguments = null) { } - public static void BasicPublish(this RabbitMQ.Client.IModel model, RabbitMQ.Client.PublicationAddress addr, RabbitMQ.Client.IBasicProperties basicProperties, System.ReadOnlyMemory body) { } - public static void BasicPublish(this RabbitMQ.Client.IModel model, RabbitMQ.Client.CachedString exchange, RabbitMQ.Client.CachedString routingKey, RabbitMQ.Client.IBasicProperties basicProperties, System.ReadOnlyMemory body) { } - public static void BasicPublish(this RabbitMQ.Client.IModel model, string exchange, string routingKey, RabbitMQ.Client.IBasicProperties basicProperties, System.ReadOnlyMemory body) { } - public static void BasicPublish(this RabbitMQ.Client.IModel model, string exchange, string routingKey, bool mandatory = false, RabbitMQ.Client.IBasicProperties basicProperties = null, System.ReadOnlyMemory body = default) { } + public static void BasicPublish(this RabbitMQ.Client.IModel model, RabbitMQ.Client.CachedString exchange, RabbitMQ.Client.CachedString routingKey, System.ReadOnlyMemory body = default, bool mandatory = false) { } + public static void BasicPublish(this RabbitMQ.Client.IModel model, string exchange, string routingKey, System.ReadOnlyMemory body = default, bool mandatory = false) { } + public static void BasicPublish(this RabbitMQ.Client.IModel model, RabbitMQ.Client.PublicationAddress addr, in T basicProperties, System.ReadOnlyMemory body) + where T : RabbitMQ.Client.IReadOnlyBasicProperties, RabbitMQ.Client.IAmqpHeader { } public static void Close(this RabbitMQ.Client.IModel model) { } public static void Close(this RabbitMQ.Client.IModel model, ushort replyCode, string replyText) { } public static void ExchangeBind(this RabbitMQ.Client.IModel model, string destination, string source, string routingKey, System.Collections.Generic.IDictionary arguments = null) { } @@ -462,27 +501,42 @@ namespace RabbitMQ.Client int MinorVersion { get; } int Revision { get; } } - public interface IRecoverable - { - event System.EventHandler Recovery; - } - public interface IStreamProperties : RabbitMQ.Client.IContentHeader - { - string ContentEncoding { get; set; } - string ContentType { get; set; } - System.Collections.Generic.IDictionary Headers { get; set; } - byte Priority { get; set; } - RabbitMQ.Client.AmqpTimestamp Timestamp { get; set; } - void ClearContentEncoding(); - void ClearContentType(); - void ClearHeaders(); - void ClearPriority(); - void ClearTimestamp(); + public interface IReadOnlyBasicProperties + { + string? AppId { get; } + string? ClusterId { get; } + string? ContentEncoding { get; } + string? ContentType { get; } + string? CorrelationId { get; } + byte DeliveryMode { get; } + string? Expiration { get; } + System.Collections.Generic.IDictionary? Headers { get; } + string? MessageId { get; } + bool Persistent { get; } + byte Priority { get; } + string? ReplyTo { get; } + RabbitMQ.Client.PublicationAddress? ReplyToAddress { get; } + RabbitMQ.Client.AmqpTimestamp Timestamp { get; } + string? Type { get; } + string? UserId { get; } + bool IsAppIdPresent(); + bool IsClusterIdPresent(); bool IsContentEncodingPresent(); bool IsContentTypePresent(); + bool IsCorrelationIdPresent(); + bool IsDeliveryModePresent(); + bool IsExpirationPresent(); bool IsHeadersPresent(); + bool IsMessageIdPresent(); bool IsPriorityPresent(); + bool IsReplyToPresent(); bool IsTimestampPresent(); + bool IsTypePresent(); + bool IsUserIdPresent(); + } + public interface IRecoverable + { + event System.EventHandler Recovery; } public interface ITcpClient : System.IDisposable { @@ -529,6 +583,40 @@ namespace RabbitMQ.Client public string QueueName { get; } public static string op_Implicit(RabbitMQ.Client.QueueDeclareOk declareOk) { } } + public readonly struct ReadOnlyBasicProperties : RabbitMQ.Client.IReadOnlyBasicProperties + { + public ReadOnlyBasicProperties(System.ReadOnlySpan span) { } + public string? AppId { get; } + public string? ClusterId { get; } + public string? ContentEncoding { get; } + public string? ContentType { get; } + public string? CorrelationId { get; } + public byte DeliveryMode { get; } + public string? Expiration { get; } + public System.Collections.Generic.IDictionary? Headers { get; } + public string? MessageId { get; } + public bool Persistent { get; } + public byte Priority { get; } + public string? ReplyTo { get; } + public RabbitMQ.Client.PublicationAddress? ReplyToAddress { get; } + public RabbitMQ.Client.AmqpTimestamp Timestamp { get; } + public string? Type { get; } + public string? UserId { get; } + public bool IsAppIdPresent() { } + public bool IsClusterIdPresent() { } + public bool IsContentEncodingPresent() { } + public bool IsContentTypePresent() { } + public bool IsCorrelationIdPresent() { } + public bool IsDeliveryModePresent() { } + public bool IsExpirationPresent() { } + public bool IsHeadersPresent() { } + public bool IsMessageIdPresent() { } + public bool IsPriorityPresent() { } + public bool IsReplyToPresent() { } + public bool IsTimestampPresent() { } + public bool IsTypePresent() { } + public bool IsUserIdPresent() { } + } public class ShutdownEventArgs : System.EventArgs { public ShutdownEventArgs(RabbitMQ.Client.ShutdownInitiator initiator, ushort replyCode, string replyText, object cause = null) { } @@ -582,7 +670,7 @@ namespace RabbitMQ.Client.Events public event RabbitMQ.Client.Events.AsyncEventHandler Unregistered; public override System.Threading.Tasks.Task HandleBasicCancelOk(string consumerTag) { } public override System.Threading.Tasks.Task HandleBasicConsumeOk(string consumerTag) { } - public override System.Threading.Tasks.Task HandleBasicDeliver(string consumerTag, ulong deliveryTag, bool redelivered, string exchange, string routingKey, RabbitMQ.Client.IBasicProperties properties, System.ReadOnlyMemory body) { } + public override System.Threading.Tasks.Task HandleBasicDeliver(string consumerTag, ulong deliveryTag, bool redelivered, string exchange, string routingKey, in RabbitMQ.Client.ReadOnlyBasicProperties properties, System.ReadOnlyMemory body) { } public override System.Threading.Tasks.Task HandleModelShutdown(object model, RabbitMQ.Client.ShutdownEventArgs reason) { } } public abstract class BaseExceptionEventArgs : System.EventArgs @@ -600,8 +688,8 @@ namespace RabbitMQ.Client.Events public class BasicDeliverEventArgs : System.EventArgs { public BasicDeliverEventArgs() { } - public BasicDeliverEventArgs(string consumerTag, ulong deliveryTag, bool redelivered, string exchange, string routingKey, RabbitMQ.Client.IBasicProperties properties, System.ReadOnlyMemory body) { } - public RabbitMQ.Client.IBasicProperties BasicProperties { get; set; } + public BasicDeliverEventArgs(string consumerTag, ulong deliveryTag, bool redelivered, string exchange, string routingKey, in RabbitMQ.Client.ReadOnlyBasicProperties properties, System.ReadOnlyMemory body) { } + public RabbitMQ.Client.ReadOnlyBasicProperties BasicProperties { get; set; } public System.ReadOnlyMemory Body { get; set; } public string ConsumerTag { get; set; } public ulong DeliveryTag { get; set; } @@ -619,7 +707,7 @@ namespace RabbitMQ.Client.Events public class BasicReturnEventArgs : System.EventArgs { public BasicReturnEventArgs() { } - public RabbitMQ.Client.IBasicProperties BasicProperties { get; set; } + public RabbitMQ.Client.ReadOnlyBasicProperties BasicProperties { get; set; } public System.ReadOnlyMemory Body { get; set; } public string Exchange { get; set; } public ushort ReplyCode { get; set; } @@ -662,7 +750,7 @@ namespace RabbitMQ.Client.Events public event System.EventHandler Unregistered; public override void HandleBasicCancelOk(string consumerTag) { } public override void HandleBasicConsumeOk(string consumerTag) { } - public override void HandleBasicDeliver(string consumerTag, ulong deliveryTag, bool redelivered, string exchange, string routingKey, RabbitMQ.Client.IBasicProperties properties, System.ReadOnlyMemory body) { } + public override void HandleBasicDeliver(string consumerTag, ulong deliveryTag, bool redelivered, string exchange, string routingKey, in RabbitMQ.Client.ReadOnlyBasicProperties properties, System.ReadOnlyMemory body) { } public override void HandleModelShutdown(object model, RabbitMQ.Client.ShutdownEventArgs reason) { } } public class FlowControlEventArgs : System.EventArgs @@ -821,4 +909,4 @@ namespace RabbitMQ.Client.Logging public string Type { get; } public override string ToString() { } } -} \ No newline at end of file +} diff --git a/projects/Unit/Fixtures.cs b/projects/Unit/Fixtures.cs index a1f2fed913..265ef28025 100644 --- a/projects/Unit/Fixtures.cs +++ b/projects/Unit/Fixtures.cs @@ -35,7 +35,7 @@ using System.Collections.Generic; using System.Text; using System.Threading; - +using RabbitMQ.Client.Framing; using RabbitMQ.Client.Framing.Impl; using Xunit; @@ -260,7 +260,7 @@ internal void WithTemporaryQueueNoWait(IModel model, Action acti internal void EnsureNotEmpty(string q, string body) { - WithTemporaryModel(x => x.BasicPublish("", q, null, _encoding.GetBytes(body))); + WithTemporaryModel(x => x.BasicPublish("", q, _encoding.GetBytes(body))); } internal void WithNonEmptyQueue(Action action) diff --git a/projects/Unit/RabbitMQCtl.cs b/projects/Unit/RabbitMQCtl.cs index b3fde6232a..91ce8d558a 100644 --- a/projects/Unit/RabbitMQCtl.cs +++ b/projects/Unit/RabbitMQCtl.cs @@ -219,7 +219,7 @@ public static void Block(IConnection conn, Encoding encoding) public static void Publish(IConnection conn, Encoding encoding) { IModel ch = conn.CreateModel(); - ch.BasicPublish("amq.fanout", "", null, encoding.GetBytes("message")); + ch.BasicPublish("amq.fanout", "", encoding.GetBytes("message")); } diff --git a/projects/Unit/TestAsyncConsumer.cs b/projects/Unit/TestAsyncConsumer.cs index ffe0ddcc77..34e31cfa18 100644 --- a/projects/Unit/TestAsyncConsumer.cs +++ b/projects/Unit/TestAsyncConsumer.cs @@ -35,7 +35,6 @@ using System.Threading.Tasks; using RabbitMQ.Client.Events; - using Xunit; namespace RabbitMQ.Client.Unit @@ -51,9 +50,8 @@ public void TestBasicRoundtrip() using(IModel m = c.CreateModel()) { QueueDeclareOk q = m.QueueDeclare(); - IBasicProperties bp = m.CreateBasicProperties(); byte[] body = System.Text.Encoding.UTF8.GetBytes("async-hi"); - m.BasicPublish("", q.QueueName, bp, body); + m.BasicPublish("", q.QueueName, body); var consumer = new AsyncEventingBasicConsumer(m); var are = new AutoResetEvent(false); consumer.Received += async (o, a) => @@ -67,7 +65,7 @@ public void TestBasicRoundtrip() Assert.True(waitRes); // unsubscribe and ensure no further deliveries m.BasicCancel(tag); - m.BasicPublish("", q.QueueName, bp, body); + m.BasicPublish("", q.QueueName, body); bool waitResFalse = are.WaitOne(2000); Assert.False(waitResFalse); } @@ -81,13 +79,12 @@ public async Task TestBasicRoundtripConcurrent() using(IModel m = c.CreateModel()) { QueueDeclareOk q = m.QueueDeclare(); - IBasicProperties bp = m.CreateBasicProperties(); const string publish1 = "async-hi-1"; byte[] body = Encoding.UTF8.GetBytes(publish1); - m.BasicPublish("", q.QueueName, bp, body); + m.BasicPublish("", q.QueueName, body); const string publish2 = "async-hi-2"; body = Encoding.UTF8.GetBytes(publish2); - m.BasicPublish("", q.QueueName, bp, body); + m.BasicPublish("", q.QueueName, body); var consumer = new AsyncEventingBasicConsumer(m); @@ -135,9 +132,8 @@ public void TestBasicRoundtripNoWait() using (IModel m = c.CreateModel()) { QueueDeclareOk q = m.QueueDeclare(); - IBasicProperties bp = m.CreateBasicProperties(); byte[] body = System.Text.Encoding.UTF8.GetBytes("async-hi"); - m.BasicPublish("", q.QueueName, bp, body); + m.BasicPublish("", q.QueueName, body); var consumer = new AsyncEventingBasicConsumer(m); var are = new AutoResetEvent(false); consumer.Received += async (o, a) => @@ -151,7 +147,7 @@ public void TestBasicRoundtripNoWait() Assert.True(waitRes); // unsubscribe and ensure no further deliveries m.BasicCancelNoWait(tag); - m.BasicPublish("", q.QueueName, bp, body); + m.BasicPublish("", q.QueueName, body); bool waitResFalse = are.WaitOne(2000); Assert.False(waitResFalse); } @@ -207,7 +203,7 @@ public void ConcurrentEventingTestForReceived() }; // Send message - m.BasicPublish("", q.QueueName, null, ReadOnlyMemory.Empty); + m.BasicPublish("", q.QueueName, ReadOnlyMemory.Empty); are.WaitOne(TimingFixture.TestTimeout); } @@ -223,9 +219,8 @@ public void NonAsyncConsumerShouldThrowInvalidOperationException() using(IModel m = c.CreateModel()) { QueueDeclareOk q = m.QueueDeclare(); - IBasicProperties bp = m.CreateBasicProperties(); byte[] body = System.Text.Encoding.UTF8.GetBytes("async-hi"); - m.BasicPublish("", q.QueueName, bp, body); + m.BasicPublish("", q.QueueName, body); var consumer = new EventingBasicConsumer(m); Assert.Throws(() => m.BasicConsume(q.QueueName, false, consumer)); } diff --git a/projects/Unit/TestAsyncConsumerExceptions.cs b/projects/Unit/TestAsyncConsumerExceptions.cs index f5bbb83096..bdf3ab733a 100644 --- a/projects/Unit/TestAsyncConsumerExceptions.cs +++ b/projects/Unit/TestAsyncConsumerExceptions.cs @@ -109,7 +109,7 @@ public void TestConsumerShutdownExceptionHandling() public void TestDeliveryExceptionHandling() { IBasicConsumer consumer = new ConsumerFailingOnDelivery(_model); - TestExceptionHandlingWith(consumer, (m, q, c, ct) => m.BasicPublish("", q, null, _encoding.GetBytes("msg"))); + TestExceptionHandlingWith(consumer, (m, q, c, ct) => m.BasicPublish("", q, _encoding.GetBytes("msg"))); } private class ConsumerFailingOnDelivery : AsyncEventingBasicConsumer @@ -123,7 +123,7 @@ public override Task HandleBasicDeliver(string consumerTag, bool redelivered, string exchange, string routingKey, - IBasicProperties properties, + in ReadOnlyBasicProperties properties, ReadOnlyMemory body) { return Task.FromException(TestException); diff --git a/projects/Unit/TestBasicProperties.cs b/projects/Unit/TestBasicProperties.cs index 92554fc5bb..8fd4b1e31c 100644 --- a/projects/Unit/TestBasicProperties.cs +++ b/projects/Unit/TestBasicProperties.cs @@ -41,7 +41,7 @@ public class TestBasicProperties public void TestPersistentPropertyChangesDeliveryMode_PersistentTrueDelivery2() { // Arrange - var subject = new Framing.BasicProperties + var subject = new BasicProperties { // Act Persistent = true @@ -50,13 +50,28 @@ public void TestPersistentPropertyChangesDeliveryMode_PersistentTrueDelivery2() // Assert Assert.Equal(2, subject.DeliveryMode); Assert.True(subject.Persistent); + + Span span = new byte[1024]; + int offset = ((IAmqpWriteable)subject).WriteTo(span); + + // Read from Stream + var propertiesFromStream = new ReadOnlyBasicProperties(span.Slice(0, offset)); + + Assert.Equal(2, propertiesFromStream.DeliveryMode); + Assert.True(propertiesFromStream.Persistent); + + // Verify Basic Properties + var basicProperties = new BasicProperties(propertiesFromStream); + + Assert.Equal(2, basicProperties.DeliveryMode); + Assert.True(basicProperties.Persistent); } [Fact] public void TestPersistentPropertyChangesDeliveryMode_PersistentFalseDelivery1() { // Arrange - var subject = new Framing.BasicProperties + var subject = new BasicProperties { // Act @@ -66,6 +81,21 @@ public void TestPersistentPropertyChangesDeliveryMode_PersistentFalseDelivery1() // Assert Assert.Equal(1, subject.DeliveryMode); Assert.False(subject.Persistent); + + Span span = new byte[1024]; + int offset = ((IAmqpWriteable)subject).WriteTo(span); + + // Read from Stream + var propertiesFromStream = new ReadOnlyBasicProperties(span.Slice(0, offset)); + + Assert.Equal(1, propertiesFromStream.DeliveryMode); + Assert.False(propertiesFromStream.Persistent); + + // Verify Basic Properties + var basicProperties = new BasicProperties(propertiesFromStream); + + Assert.Equal(1, basicProperties.DeliveryMode); + Assert.False(basicProperties.Persistent); } [Theory] @@ -80,7 +110,7 @@ public void TestPersistentPropertyChangesDeliveryMode_PersistentFalseDelivery1() public void TestNullableProperties_CanWrite(string clusterId, string correlationId, string messageId) { // Arrange - var subject = new Framing.BasicProperties + var subject = new BasicProperties { // Act ClusterId = clusterId, @@ -99,10 +129,10 @@ public void TestNullableProperties_CanWrite(string clusterId, string correlation Assert.Equal(isMessageIdPresent, subject.IsMessageIdPresent()); Span span = new byte[1024]; - int offset = subject.WritePropertiesTo(span); + int offset = ((IAmqpWriteable)subject).WriteTo(span); // Read from Stream - var propertiesFromStream = new Framing.BasicProperties(span.Slice(0, offset)); + var propertiesFromStream = new ReadOnlyBasicProperties(span.Slice(0, offset)); Assert.Equal(clusterId, propertiesFromStream.ClusterId); Assert.Equal(correlationId, propertiesFromStream.CorrelationId); @@ -110,6 +140,16 @@ public void TestNullableProperties_CanWrite(string clusterId, string correlation Assert.Equal(isClusterIdPresent, propertiesFromStream.IsClusterIdPresent()); Assert.Equal(isCorrelationIdPresent, propertiesFromStream.IsCorrelationIdPresent()); Assert.Equal(isMessageIdPresent, propertiesFromStream.IsMessageIdPresent()); + + // Verify Basic Properties + var basicProperties = new BasicProperties(propertiesFromStream); + + Assert.Equal(clusterId, basicProperties.ClusterId); + Assert.Equal(correlationId, basicProperties.CorrelationId); + Assert.Equal(messageId, basicProperties.MessageId); + Assert.Equal(isClusterIdPresent, basicProperties.IsClusterIdPresent()); + Assert.Equal(isCorrelationIdPresent, basicProperties.IsCorrelationIdPresent()); + Assert.Equal(isMessageIdPresent, basicProperties.IsMessageIdPresent()); } [Theory] @@ -119,7 +159,7 @@ public void TestNullableProperties_CanWrite(string clusterId, string correlation public void TestProperties_ReplyTo(string replyTo) { // Arrange - var subject = new Framing.BasicProperties + var subject = new BasicProperties { // Act ReplyTo = replyTo, @@ -132,14 +172,21 @@ public void TestProperties_ReplyTo(string replyTo) Assert.Equal(isReplyToPresent, subject.IsReplyToPresent()); Span span = new byte[1024]; - int offset = subject.WritePropertiesTo(span); + int offset = ((IAmqpWriteable)subject).WriteTo(span); // Read from Stream - var propertiesFromStream = new Framing.BasicProperties(span.Slice(0, offset)); + var propertiesFromStream = new ReadOnlyBasicProperties(span.Slice(0, offset)); Assert.Equal(replyTo, propertiesFromStream.ReplyTo); Assert.Equal(isReplyToPresent, propertiesFromStream.IsReplyToPresent()); Assert.Equal(replyToAddress, propertiesFromStream.ReplyToAddress?.ToString()); + + // Verify Basic Properties + var basicProperties = new BasicProperties(propertiesFromStream); + + Assert.Equal(replyTo, basicProperties.ReplyTo); + Assert.Equal(isReplyToPresent, basicProperties.IsReplyToPresent()); + Assert.Equal(replyToAddress, basicProperties.ReplyToAddress?.ToString()); } } } diff --git a/projects/Unit/TestBasicPublish.cs b/projects/Unit/TestBasicPublish.cs index 65733368bc..58f7ced4f0 100644 --- a/projects/Unit/TestBasicPublish.cs +++ b/projects/Unit/TestBasicPublish.cs @@ -3,7 +3,7 @@ using System.Threading.Tasks; using RabbitMQ.Client.Events; - +using RabbitMQ.Client.Framing; using Xunit; namespace RabbitMQ.Client.Unit @@ -19,7 +19,7 @@ public void TestBasicRoundtripArray() using(IModel m = c.CreateModel()) { QueueDeclareOk q = m.QueueDeclare(); - IBasicProperties bp = m.CreateBasicProperties(); + var bp = new BasicProperties(); byte[] sendBody = System.Text.Encoding.UTF8.GetBytes("hi"); byte[] consumeBody = null; var consumer = new EventingBasicConsumer(m); @@ -50,7 +50,6 @@ public void TestBasicRoundtripCachedString() { CachedString exchangeName = new CachedString(string.Empty); CachedString queueName = new CachedString(m.QueueDeclare().QueueName); - IBasicProperties bp = m.CreateBasicProperties(); byte[] sendBody = System.Text.Encoding.UTF8.GetBytes("hi"); byte[] consumeBody = null; var consumer = new EventingBasicConsumer(m); @@ -63,7 +62,7 @@ public void TestBasicRoundtripCachedString() }; string tag = m.BasicConsume(queueName.Value, true, consumer); - m.BasicPublish(exchangeName, queueName, bp, sendBody); + m.BasicPublish(exchangeName, queueName, sendBody); bool waitResFalse = are.WaitOne(2000); m.BasicCancel(tag); @@ -80,7 +79,6 @@ public void TestBasicRoundtripReadOnlyMemory() using(IModel m = c.CreateModel()) { QueueDeclareOk q = m.QueueDeclare(); - IBasicProperties bp = m.CreateBasicProperties(); byte[] sendBody = System.Text.Encoding.UTF8.GetBytes("hi"); byte[] consumeBody = null; var consumer = new EventingBasicConsumer(m); @@ -93,7 +91,7 @@ public void TestBasicRoundtripReadOnlyMemory() }; string tag = m.BasicConsume(q.QueueName, true, consumer); - m.BasicPublish("", q.QueueName, bp, new ReadOnlyMemory(sendBody)); + m.BasicPublish("", q.QueueName, new ReadOnlyMemory(sendBody)); bool waitResFalse = are.WaitOne(2000); m.BasicCancel(tag); @@ -110,7 +108,6 @@ public void CanNotModifyPayloadAfterPublish() using(IModel m = c.CreateModel()) { QueueDeclareOk q = m.QueueDeclare(); - IBasicProperties bp = m.CreateBasicProperties(); byte[] sendBody = new byte[1000]; var consumer = new EventingBasicConsumer(m); var are = new AutoResetEvent(false); @@ -125,7 +122,7 @@ public void CanNotModifyPayloadAfterPublish() }; string tag = m.BasicConsume(q.QueueName, true, consumer); - m.BasicPublish("", q.QueueName, bp, sendBody); + m.BasicPublish("", q.QueueName, sendBody); sendBody.AsSpan().Fill(1); Assert.True(are.WaitOne(2000)); diff --git a/projects/Unit/TestConcurrentAccessWithSharedConnection.cs b/projects/Unit/TestConcurrentAccessWithSharedConnection.cs index 894e771cd8..032550ed09 100644 --- a/projects/Unit/TestConcurrentAccessWithSharedConnection.cs +++ b/projects/Unit/TestConcurrentAccessWithSharedConnection.cs @@ -107,7 +107,7 @@ internal void TestConcurrentChannelOpenAndPublishingWithBody(byte[] body, int it ch.ConfirmSelect(); for (int j = 0; j < 200; j++) { - ch.BasicPublish("", "_______", null, body); + ch.BasicPublish("", "_______", body); } using var cts = new CancellationTokenSource(TimeSpan.FromSeconds(40)); ch.WaitForConfirmsAsync(cts.Token).GetAwaiter().GetResult(); diff --git a/projects/Unit/TestConfirmSelect.cs b/projects/Unit/TestConfirmSelect.cs index 888eab8be4..f5c9f17503 100644 --- a/projects/Unit/TestConfirmSelect.cs +++ b/projects/Unit/TestConfirmSelect.cs @@ -58,7 +58,7 @@ public void TestConfirmSelectIdempotency() protected void Publish() { - _model.BasicPublish("", "amq.fanout", null, _encoding.GetBytes("message")); + _model.BasicPublish("", "amq.fanout", _encoding.GetBytes("message")); } } } diff --git a/projects/Unit/TestConnectionRecovery.cs b/projects/Unit/TestConnectionRecovery.cs index da3894fe73..64e874906c 100644 --- a/projects/Unit/TestConnectionRecovery.cs +++ b/projects/Unit/TestConnectionRecovery.cs @@ -91,7 +91,7 @@ public void TestBasicAckAfterBasicGetAndChannelRecovery() string q = GenerateQueueName(); _model.QueueDeclare(q, false, false, false, null); // create an offset - _model.BasicPublish("", q, null, ReadOnlyMemory.Empty); + _model.BasicPublish("", q, ReadOnlyMemory.Empty); Thread.Sleep(50); BasicGetResult g = _model.BasicGet(q, false); CloseAndWaitForRecovery(); @@ -99,7 +99,7 @@ public void TestBasicAckAfterBasicGetAndChannelRecovery() Assert.True(_model.IsOpen); // ack the message after recovery - this should be out of range and ignored _model.BasicAck(g.DeliveryTag, false); - // do a sync operation to 'check' there is no channel exception + // do a sync operation to 'check' there is no channel exception _model.BasicGet(q, false); } @@ -115,7 +115,7 @@ public void TestBasicAckEventHandlerRecovery() CloseAndWaitForRecovery(); Assert.True(_model.IsOpen); - WithTemporaryNonExclusiveQueue(_model, (m, q) => m.BasicPublish("", q, null, _encoding.GetBytes(""))); + WithTemporaryNonExclusiveQueue(_model, (m, q) => m.BasicPublish("", q, _encoding.GetBytes(""))); Wait(latch); } @@ -309,7 +309,7 @@ public void TestConsumerWorkServiceRecovery() var latch = new ManualResetEventSlim(false); cons.Received += (s, args) => latch.Set(); - m.BasicPublish("", q, null, _encoding.GetBytes("msg")); + m.BasicPublish("", q, _encoding.GetBytes("msg")); Wait(latch); m.QueueDelete(q); @@ -349,7 +349,7 @@ public void TestConsumerRecoveryOnClientNamedQueueWithOneRecovery() var latch = new ManualResetEventSlim(false); cons.Received += (s, args) => latch.Set(); - m.BasicPublish("", q1, null, _encoding.GetBytes("msg")); + m.BasicPublish("", q1, _encoding.GetBytes("msg")); Wait(latch); m.QueueDelete(q1); @@ -518,7 +518,7 @@ public void TestExchangeToExchangeBindingRecovery() { CloseAndWaitForRecovery(); Assert.True(_model.IsOpen); - _model.BasicPublish(x2, "", null, _encoding.GetBytes("msg")); + _model.BasicPublish(x2, "", _encoding.GetBytes("msg")); AssertMessageCount(q, 1); } finally @@ -566,7 +566,7 @@ public void TestClientNamedTransientAutoDeleteQueueAndBindingRecovery() ch.ConfirmSelect(); ch.QueuePurge(q); ch.ExchangeDeclare(exchange: x, type: "fanout"); - ch.BasicPublish(exchange: x, routingKey: "", basicProperties: null, body: _encoding.GetBytes("msg")); + ch.BasicPublish(exchange: x, routingKey: "", body: _encoding.GetBytes("msg")); WaitForConfirms(ch); QueueDeclareOk ok = ch.QueueDeclare(queue: q, durable: false, exclusive: false, autoDelete: true, arguments: null); Assert.Equal(1u, ok.MessageCount); @@ -599,7 +599,7 @@ public void TestServerNamedTransientAutoDeleteQueueAndBindingRecovery() Assert.NotEqual(nameBefore, nameAfter); ch.ConfirmSelect(); ch.ExchangeDeclare(exchange: x, type: "fanout"); - ch.BasicPublish(exchange: x, routingKey: "", basicProperties: null, body: _encoding.GetBytes("msg")); + ch.BasicPublish(exchange: x, routingKey: "", body: _encoding.GetBytes("msg")); WaitForConfirms(ch); QueueDeclareOk ok = ch.QueueDeclarePassive(nameAfter); Assert.Equal(1u, ok.MessageCount); @@ -780,7 +780,7 @@ public void TestRecoverTopologyOnDisposedChannel() var latch = new ManualResetEventSlim(false); cons.Received += (s, args) => latch.Set(); - _model.BasicPublish("", q, null, ReadOnlyMemory.Empty); + _model.BasicPublish("", q); Wait(latch); _model.QueueUnbind(q, x, rk); @@ -795,7 +795,7 @@ public void TestPublishRpcRightAfterReconnect() _model.QueueDeclare(testQueueName, false, false, false, null); var replyConsumer = new EventingBasicConsumer(_model); _model.BasicConsume("amq.rabbitmq.reply-to", true, replyConsumer); - var properties = _model.CreateBasicProperties(); + var properties = new BasicProperties(); properties.ReplyTo = "amq.rabbitmq.reply-to"; bool done = false; @@ -817,7 +817,7 @@ public void TestPublishRpcRightAfterReconnect() { try { - _model.BasicPublish(string.Empty, testQueueName, false, properties, ReadOnlyMemory.Empty); + _model.BasicPublish(string.Empty, testQueueName, properties, ReadOnlyMemory.Empty); } catch (Exception e) { @@ -864,7 +864,7 @@ public void TestThatDeletedExchangeBindingsDontReappearOnRecovery() { CloseAndWaitForRecovery(); Assert.True(_model.IsOpen); - _model.BasicPublish(x2, "", null, _encoding.GetBytes("msg")); + _model.BasicPublish(x2, "", _encoding.GetBytes("msg")); AssertMessageCount(q, 0); } finally @@ -914,7 +914,7 @@ public void TestThatDeletedQueueBindingsDontReappearOnRecovery() { CloseAndWaitForRecovery(); Assert.True(_model.IsOpen); - _model.BasicPublish(x2, "", null, _encoding.GetBytes("msg")); + _model.BasicPublish(x2, "", _encoding.GetBytes("msg")); AssertMessageCount(q, 0); } finally @@ -969,7 +969,7 @@ internal void AssertExchangeRecovery(IModel m, string x) string rk = "routing-key"; m.QueueBind(q, x, rk); byte[] mb = RandomMessageBody(); - m.BasicPublish(x, rk, null, mb); + m.BasicPublish(x, rk, mb); Assert.True(WaitForConfirms(m)); m.ExchangeDeclarePassive(x); @@ -987,7 +987,7 @@ internal void AssertQueueRecovery(IModel m, string q, bool exclusive) m.QueueDeclarePassive(q); QueueDeclareOk ok1 = m.QueueDeclare(q, false, exclusive, false, null); Assert.Equal(0u, ok1.MessageCount); - m.BasicPublish("", q, null, _encoding.GetBytes("")); + m.BasicPublish("", q, _encoding.GetBytes("")); Assert.True(WaitForConfirms(m)); QueueDeclareOk ok2 = m.QueueDeclare(q, false, exclusive, false, null); Assert.Equal(1u, ok2.MessageCount); @@ -1064,7 +1064,7 @@ internal void TestDelayedBasicAckNackAfterChannelRecovery(TestBasicConsumer1 con for (int i = 0; i < n; i++) { - publishingModel.BasicPublish("", q, null, _encoding.GetBytes("")); + publishingModel.BasicPublish("", q, _encoding.GetBytes("")); } Wait(latch, TimeSpan.FromSeconds(20)); @@ -1135,7 +1135,7 @@ public override void HandleBasicDeliver(string consumerTag, bool redelivered, string exchange, string routingKey, - IBasicProperties properties, + in ReadOnlyBasicProperties properties, ReadOnlyMemory body) { try diff --git a/projects/Unit/TestConsumer.cs b/projects/Unit/TestConsumer.cs index 20b5b74033..93d14bc9d4 100644 --- a/projects/Unit/TestConsumer.cs +++ b/projects/Unit/TestConsumer.cs @@ -20,13 +20,12 @@ public async Task TestBasicRoundtripConcurrent() using(IModel m = c.CreateModel()) { QueueDeclareOk q = m.QueueDeclare(); - IBasicProperties bp = m.CreateBasicProperties(); const string publish1 = "sync-hi-1"; byte[] body = Encoding.UTF8.GetBytes(publish1); - m.BasicPublish("", q.QueueName, bp, body); + m.BasicPublish("", q.QueueName, body); const string publish2 = "sync-hi-2"; body = Encoding.UTF8.GetBytes(publish2); - m.BasicPublish("", q.QueueName, bp, body); + m.BasicPublish("", q.QueueName, body); var consumer = new EventingBasicConsumer(m); diff --git a/projects/Unit/TestConsumerExceptions.cs b/projects/Unit/TestConsumerExceptions.cs index 5f3c488d30..36bc835876 100644 --- a/projects/Unit/TestConsumerExceptions.cs +++ b/projects/Unit/TestConsumerExceptions.cs @@ -50,7 +50,7 @@ public override void HandleBasicDeliver(string consumerTag, bool redelivered, string exchange, string routingKey, - IBasicProperties properties, + in ReadOnlyBasicProperties properties, ReadOnlyMemory body) { throw new Exception("oops"); @@ -158,7 +158,7 @@ public void TestConsumerShutdownExceptionHandling() public void TestDeliveryExceptionHandling() { IBasicConsumer consumer = new ConsumerFailingOnDelivery(_model); - TestExceptionHandlingWith(consumer, (m, q, c, ct) => m.BasicPublish("", q, null, _encoding.GetBytes("msg"))); + TestExceptionHandlingWith(consumer, (m, q, c, ct) => m.BasicPublish("", q, _encoding.GetBytes("msg"))); } } } diff --git a/projects/Unit/TestConsumerOperationDispatch.cs b/projects/Unit/TestConsumerOperationDispatch.cs index 137db22627..9c6875b516 100644 --- a/projects/Unit/TestConsumerOperationDispatch.cs +++ b/projects/Unit/TestConsumerOperationDispatch.cs @@ -84,7 +84,7 @@ public CollectingConsumer(IModel model) public override void HandleBasicDeliver(string consumerTag, ulong deliveryTag, bool redelivered, string exchange, string routingKey, - IBasicProperties properties, ReadOnlyMemory body) + in ReadOnlyBasicProperties properties, ReadOnlyMemory body) { // we test concurrent dispatch from the moment basic.delivery is returned. // delivery tags have guaranteed ordering and we verify that it is preserved @@ -120,9 +120,7 @@ public void TestDeliveryOrderingWithSingleChannel() for (int i = 0; i < N; i++) { - Ch.BasicPublish(exchange: _x, routingKey: "", - basicProperties: new BasicProperties(), - body: _encoding.GetBytes("msg")); + Ch.BasicPublish(_x, "", _encoding.GetBytes("msg")); } counter.Wait(TimeSpan.FromSeconds(120)); @@ -165,7 +163,7 @@ public void TestChannelShutdownDoesNotShutDownDispatcher() // closing this channel must not affect ch2 ch1.Close(); - ch2.BasicPublish(exchange: _x, basicProperties: null, body: _encoding.GetBytes("msg"), routingKey: ""); + ch2.BasicPublish(_x, "", _encoding.GetBytes("msg")); Wait(latch); } diff --git a/projects/Unit/TestContentHeaderCodec.cs b/projects/Unit/TestContentHeaderCodec.cs index c66ed2984b..ccc2acff09 100644 --- a/projects/Unit/TestContentHeaderCodec.cs +++ b/projects/Unit/TestContentHeaderCodec.cs @@ -60,13 +60,13 @@ private void Check(ReadOnlyMemory actual, ReadOnlyMemory expected) [Fact] public void TestSimpleProperties() { - Framing.BasicProperties prop = new Framing.BasicProperties + IAmqpWriteable prop = new BasicProperties { ContentType = "text/plain" }; - int bytesNeeded = prop.GetRequiredPayloadBufferSize(); + int bytesNeeded = prop.GetRequiredBufferSize(); byte[] bytes = new byte[bytesNeeded]; - int offset = prop.WritePropertiesTo(bytes); + int offset = prop.WriteTo(bytes); Check(bytes.AsMemory().Slice(0, offset), new byte[] { 0x80, 0x00, // props flags 0x0A, // shortstr len @@ -77,7 +77,7 @@ public void TestSimpleProperties() [Fact] public void TestFullProperties() { - Framing.BasicProperties prop = new Framing.BasicProperties + IAmqpWriteable prop = new BasicProperties { AppId = "A", ContentType = "B", @@ -94,9 +94,9 @@ public void TestFullProperties() UserId = "J", Headers = new Dictionary(0) }; - int bytesNeeded = prop.GetRequiredPayloadBufferSize(); + int bytesNeeded = prop.GetRequiredBufferSize(); byte[] bytes = new byte[bytesNeeded]; - int offset = prop.WritePropertiesTo(bytes); + int offset = prop.WriteTo(bytes); Check(bytes.AsMemory().Slice(0, offset), new byte[] { 0b1111_1111, 0b1111_1100, // props flags (all set) 0x01, 0x42, // ContentType diff --git a/projects/Unit/TestEventingConsumer.cs b/projects/Unit/TestEventingConsumer.cs index 74a85da096..a6ab285752 100644 --- a/projects/Unit/TestEventingConsumer.cs +++ b/projects/Unit/TestEventingConsumer.cs @@ -96,7 +96,7 @@ public void TestEventingConsumerDeliveryEvents() }; _model.BasicConsume(q, true, ec); - _model.BasicPublish("", q, null, _encoding.GetBytes("msg")); + _model.BasicPublish("", q, _encoding.GetBytes("msg")); WaitOn(o); Assert.True(receivedInvoked); diff --git a/projects/Unit/TestExtensions.cs b/projects/Unit/TestExtensions.cs index 3f86edc259..16289c37cc 100644 --- a/projects/Unit/TestExtensions.cs +++ b/projects/Unit/TestExtensions.cs @@ -45,7 +45,7 @@ public async Task TestConfirmBarrier() _model.ConfirmSelect(); for (int i = 0; i < 10; i++) { - _model.BasicPublish("", string.Empty, null, new byte[] { }); + _model.BasicPublish("", string.Empty); } Assert.True(await _model.WaitForConfirmsAsync().ConfigureAwait(false)); } @@ -69,12 +69,12 @@ public async Task TestExchangeBinding() _model.ExchangeBind("dest", "src", string.Empty); _model.QueueBind(queue, "dest", string.Empty); - _model.BasicPublish("src", string.Empty, null, new byte[] { }); + _model.BasicPublish("src", string.Empty); await _model.WaitForConfirmsAsync().ConfigureAwait(false); Assert.NotNull(_model.BasicGet(queue, true)); _model.ExchangeUnbind("dest", "src", string.Empty); - _model.BasicPublish("src", string.Empty, null, new byte[] { }); + _model.BasicPublish("src", string.Empty); await _model.WaitForConfirmsAsync().ConfigureAwait(false); Assert.Null(_model.BasicGet(queue, true)); diff --git a/projects/Unit/TestFloodPublishing.cs b/projects/Unit/TestFloodPublishing.cs index 15d04167b0..9c69bfbaab 100644 --- a/projects/Unit/TestFloodPublishing.cs +++ b/projects/Unit/TestFloodPublishing.cs @@ -83,7 +83,7 @@ public void TestUnthrottledFloodPublishing() } } - model.BasicPublish(CachedString.Empty, CachedString.Empty, null, _body); + model.BasicPublish(CachedString.Empty, CachedString.Empty, _body); } } finally @@ -128,10 +128,9 @@ public void TestMultithreadFloodPublishing() { using (IModel m = c.CreateModel()) { - IBasicProperties bp = m.CreateBasicProperties(); for (int i = 0; i < publishCount; i++) { - m.BasicPublish(string.Empty, queueName, bp, sendBody); + m.BasicPublish(string.Empty, queueName, sendBody); } } }); diff --git a/projects/Unit/TestFrameFormatting.cs b/projects/Unit/TestFrameFormatting.cs index c2d75f913d..0536227e6a 100644 --- a/projects/Unit/TestFrameFormatting.cs +++ b/projects/Unit/TestFrameFormatting.cs @@ -74,10 +74,10 @@ public void HeaderFrame() const int Channel = 3; const int BodyLength = 10; - var basicProperties = new Framing.BasicProperties { AppId = "A" }; - int payloadSize = basicProperties.GetRequiredPayloadBufferSize(); + var basicProperties = new BasicProperties { AppId = "A" }; + int payloadSize = ((IAmqpWriteable)basicProperties).GetRequiredBufferSize(); byte[] frameBytes = new byte[Impl.Framing.Header.FrameSize + BodyLength + payloadSize]; - Impl.Framing.Header.WriteTo(frameBytes, Channel, basicProperties, BodyLength); + Impl.Framing.Header.WriteTo(frameBytes, Channel, ref basicProperties, BodyLength); Assert.Equal(20, Impl.Framing.Header.FrameSize); Assert.Equal(Constants.FrameHeader, frameBytes[0]); diff --git a/projects/Unit/TestMainLoop.cs b/projects/Unit/TestMainLoop.cs index 80e6f6122c..65d5d09a96 100644 --- a/projects/Unit/TestMainLoop.cs +++ b/projects/Unit/TestMainLoop.cs @@ -41,7 +41,7 @@ namespace RabbitMQ.Client.Unit public class TestMainLoop : IntegrationFixture { - private class FaultyConsumer : DefaultBasicConsumer + private sealed class FaultyConsumer : DefaultBasicConsumer { public FaultyConsumer(IModel model) : base(model) {} @@ -50,7 +50,7 @@ public override void HandleBasicDeliver(string consumerTag, bool redelivered, string exchange, string routingKey, - IBasicProperties properties, + in ReadOnlyBasicProperties properties, ReadOnlyMemory body) { throw new Exception("I am a bad consumer"); @@ -74,7 +74,7 @@ public void TestCloseWithFaultyConsumer() Monitor.PulseAll(o); }; m.BasicConsume(q, true, new FaultyConsumer(_model)); - m.BasicPublish("", q, null, _encoding.GetBytes("message")); + m.BasicPublish("", q, _encoding.GetBytes("message")); WaitOn(o); Assert.NotNull(ea); diff --git a/projects/Unit/TestMessageCount.cs b/projects/Unit/TestMessageCount.cs index bb760910a7..bbea9c41d9 100644 --- a/projects/Unit/TestMessageCount.cs +++ b/projects/Unit/TestMessageCount.cs @@ -45,7 +45,7 @@ public async Task TestMessageCountMethod() _model.QueueDeclare(queue: q, durable: false, exclusive: true, autoDelete: false, arguments: null); Assert.Equal(0u, _model.MessageCount(q)); - _model.BasicPublish(exchange: "", routingKey: q, basicProperties: null, body: _encoding.GetBytes("msg")); + _model.BasicPublish("", q, _encoding.GetBytes("msg")); await _model.WaitForConfirmsAsync().ConfigureAwait(false); Assert.Equal(1u, _model.MessageCount(q)); } diff --git a/projects/Unit/TestPropertiesClone.cs b/projects/Unit/TestPropertiesClone.cs deleted file mode 100644 index 230008290b..0000000000 --- a/projects/Unit/TestPropertiesClone.cs +++ /dev/null @@ -1,170 +0,0 @@ -// This source code is dual-licensed under the Apache License, version -// 2.0, and the Mozilla Public License, version 2.0. -// -// The APL v2.0: -// -//--------------------------------------------------------------------------- -// Copyright (c) 2007-2020 VMware, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// https://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. -//--------------------------------------------------------------------------- -// -// The MPL v2.0: -// -//--------------------------------------------------------------------------- -// This Source Code Form is subject to the terms of the Mozilla Public -// License, v. 2.0. If a copy of the MPL was not distributed with this -// file, You can obtain one at https://mozilla.org/MPL/2.0/. -// -// Copyright (c) 2007-2020 VMware, Inc. All rights reserved. -//--------------------------------------------------------------------------- - -using System.Collections.Generic; - -using RabbitMQ.Client.Impl; - -using Xunit; - -namespace RabbitMQ.Client.Unit -{ - - public class TestPropertiesClone - { - [Fact] - public void TestBasicPropertiesCloneV0_9_1() - { - TestBasicPropertiesClone(new Framing.BasicProperties()); - } - - [Fact] - public void TestBasicPropertiesNoneCloneV0_9_1() - { - TestBasicPropertiesNoneClone(new Framing.BasicProperties()); - } - - private void TestBasicPropertiesClone(BasicProperties bp) - { - // Set initial values - bp.ContentType = "foo_1"; - bp.ContentEncoding = "foo_2"; - bp.Headers = new Dictionary - { - { "foo_3", "foo_4" }, - { "foo_5", "foo_6" } - }; - bp.DeliveryMode = 2; - // Persistent also changes DeliveryMode's value to 2 - bp.Persistent = true; - bp.Priority = 12; - bp.CorrelationId = "foo_7"; - bp.ReplyTo = "foo_8"; - bp.Expiration = "foo_9"; - bp.MessageId = "foo_10"; - bp.Timestamp = new AmqpTimestamp(123); - bp.Type = "foo_11"; - bp.UserId = "foo_12"; - bp.AppId = "foo_13"; - bp.ClusterId = "foo_14"; - - // Clone - BasicProperties bpClone = bp.Clone() as BasicProperties; - - // Change values in source object - bp.ContentType = "foo_15"; - bp.ContentEncoding = "foo_16"; - bp.Headers.Remove("foo_3"); - bp.Headers.Remove("foo_5"); - bp.Headers.Add("foo_17", "foo_18"); - bp.Headers.Add("foo_19", "foo_20"); - bp.DeliveryMode = 1; - // Persistent also changes DeliveryMode's value to 1 - bp.Persistent = false; - bp.Priority = 23; - bp.CorrelationId = "foo_21"; - bp.ReplyTo = "foo_22"; - bp.Expiration = "foo_23"; - bp.MessageId = "foo_24"; - bp.Timestamp = new AmqpTimestamp(234); - bp.Type = "foo_25"; - bp.UserId = "foo_26"; - bp.AppId = "foo_27"; - bp.ClusterId = "foo_28"; - - // Make sure values have not changed in clone - Assert.Equal("foo_1", bpClone.ContentType); - Assert.Equal("foo_2", bpClone.ContentEncoding); - Assert.Equal(2, bpClone.Headers.Count); - Assert.True(bpClone.Headers.ContainsKey("foo_3")); - Assert.Equal("foo_4", bpClone.Headers["foo_3"]); - Assert.True(bpClone.Headers.ContainsKey("foo_5")); - Assert.Equal("foo_6", bpClone.Headers["foo_5"]); - Assert.Equal(2, bpClone.DeliveryMode); - Assert.True(bpClone.Persistent); - Assert.Equal(12, bpClone.Priority); - Assert.Equal("foo_7", bpClone.CorrelationId); - Assert.Equal("foo_8", bpClone.ReplyTo); - Assert.Null(bpClone.ReplyToAddress); - Assert.Equal("foo_9", bpClone.Expiration); - Assert.Equal("foo_10", bpClone.MessageId); - Assert.Equal(new AmqpTimestamp(123), bpClone.Timestamp); - Assert.Equal("foo_11", bpClone.Type); - Assert.Equal("foo_12", bpClone.UserId); - Assert.Equal("foo_13", bpClone.AppId); - Assert.Equal("foo_14", bpClone.ClusterId); - } - - private void TestBasicPropertiesNoneClone(BasicProperties bp) - { - // Do not set any member and clone - BasicProperties bpClone = bp.Clone() as BasicProperties; - - // Set members in source object - bp.ContentType = "foo_1"; - bp.ContentEncoding = "foo_2"; - bp.Headers = new Dictionary - { - { "foo_3", "foo_4" }, - { "foo_5", "foo_6" } - }; - bp.DeliveryMode = 2; - // Persistent also changes DeliveryMode's value to 2 - bp.Persistent = true; - bp.Priority = 12; - bp.CorrelationId = "foo_7"; - bp.ReplyTo = "foo_8"; - bp.Expiration = "foo_9"; - bp.MessageId = "foo_10"; - bp.Timestamp = new AmqpTimestamp(123); - bp.Type = "foo_11"; - bp.UserId = "foo_12"; - bp.AppId = "foo_13"; - bp.ClusterId = "foo_14"; - - // Check that no member is present in clone - Assert.False(bpClone.IsContentTypePresent()); - Assert.False(bpClone.IsContentEncodingPresent()); - Assert.False(bpClone.IsHeadersPresent()); - Assert.False(bpClone.IsDeliveryModePresent()); - Assert.False(bpClone.IsPriorityPresent()); - Assert.False(bpClone.IsCorrelationIdPresent()); - Assert.False(bpClone.IsReplyToPresent()); - Assert.False(bpClone.IsExpirationPresent()); - Assert.False(bpClone.IsMessageIdPresent()); - Assert.False(bpClone.IsTimestampPresent()); - Assert.False(bpClone.IsTypePresent()); - Assert.False(bpClone.IsUserIdPresent()); - Assert.False(bpClone.IsAppIdPresent()); - Assert.False(bpClone.IsClusterIdPresent()); - } - } -} diff --git a/projects/Unit/TestPublishSharedModel.cs b/projects/Unit/TestPublishSharedModel.cs index a24c847030..1fcf6ad017 100644 --- a/projects/Unit/TestPublishSharedModel.cs +++ b/projects/Unit/TestPublishSharedModel.cs @@ -95,7 +95,7 @@ void NewFunction(IModel model) { for (int j = 0; j < Repeats; j++) { - model.BasicPublish(ExchangeName, PublishKey, false, null, _body); + model.BasicPublish(ExchangeName, PublishKey, _body, false); } Thread.Sleep(1); diff --git a/projects/Unit/TestPublisherConfirms.cs b/projects/Unit/TestPublisherConfirms.cs index 9243b54fab..f9300ec5d5 100644 --- a/projects/Unit/TestPublisherConfirms.cs +++ b/projects/Unit/TestPublisherConfirms.cs @@ -109,7 +109,7 @@ public async Task TestWaitForConfirmsWithEvents() { for (int i = 0; i < n; i++) { - ch.BasicPublish("", QueueName, null, _encoding.GetBytes("msg")); + ch.BasicPublish("", QueueName, _encoding.GetBytes("msg")); } await ch.WaitForConfirmsAsync().ConfigureAwait(false); @@ -135,7 +135,7 @@ protected void TestWaitForConfirms(int numberOfMessagesToPublish, Action ReadOnlyMemory body = _encoding.GetBytes("msg"); for (int i = 0; i < numberOfMessagesToPublish; i++) { - ch.BasicPublish("", QueueName, null, body); + ch.BasicPublish("", QueueName, body); } try diff --git a/projects/Unit/TestRecoverAfterCancel.cs b/projects/Unit/TestRecoverAfterCancel.cs index 87188fca64..3734ebad28 100644 --- a/projects/Unit/TestRecoverAfterCancel.cs +++ b/projects/Unit/TestRecoverAfterCancel.cs @@ -71,7 +71,7 @@ public void Dispose() public void TestRecoverAfterCancel_() { UTF8Encoding enc = new UTF8Encoding(); - _channel.BasicPublish("", _queue, null, enc.GetBytes("message")); + _channel.BasicPublish("", _queue, enc.GetBytes("message")); EventingBasicConsumer Consumer = new EventingBasicConsumer(_channel); BlockingCollection<(bool Redelivered, byte[] Body)> EventQueue = new BlockingCollection<(bool Redelivered, byte[] Body)>(); // Making sure we copy the delivery body since it could be disposed at any time. diff --git a/projects/Unit/TestSsl.cs b/projects/Unit/TestSsl.cs index 20c3d70076..7590c8591c 100644 --- a/projects/Unit/TestSsl.cs +++ b/projects/Unit/TestSsl.cs @@ -52,7 +52,7 @@ private void SendReceive(ConnectionFactory cf) string message = "Hello C# SSL Client World"; byte[] msgBytes = System.Text.Encoding.UTF8.GetBytes(message); - ch.BasicPublish("Exchange_TestSslEndPoint", "Key_TestSslEndpoint", null, msgBytes); + ch.BasicPublish("Exchange_TestSslEndPoint", "Key_TestSslEndpoint", msgBytes); bool autoAck = false; BasicGetResult result = ch.BasicGet(qName, autoAck); From af8c0721d8c7d7c1ed631a490c33596da3e0a580 Mon Sep 17 00:00:00 2001 From: bollhals Date: Sat, 16 Oct 2021 13:39:37 +0200 Subject: [PATCH 2/2] use ref Properties in interface --- projects/RabbitMQ.Client/client/api/IModel.cs | 4 ++-- .../RabbitMQ.Client/client/api/IModelExtensions.cs | 8 ++++---- .../RabbitMQ.Client/client/impl/AutorecoveringModel.cs | 8 ++++---- .../RabbitMQ.Client/client/impl/EmptyBasicProperty.cs | 2 ++ projects/RabbitMQ.Client/client/impl/ModelBase.cs | 10 ++++------ projects/TestApplications/MassPublish/Program.cs | 2 +- projects/Unit/TestBasicPublish.cs | 2 +- projects/Unit/TestConnectionRecovery.cs | 2 +- 8 files changed, 19 insertions(+), 19 deletions(-) diff --git a/projects/RabbitMQ.Client/client/api/IModel.cs b/projects/RabbitMQ.Client/client/api/IModel.cs index 7417b59e02..085dc686e6 100644 --- a/projects/RabbitMQ.Client/client/api/IModel.cs +++ b/projects/RabbitMQ.Client/client/api/IModel.cs @@ -188,7 +188,7 @@ string BasicConsume( /// Routing key must be shorter than 255 bytes. /// /// - void BasicPublish(string exchange, string routingKey, in TProperties basicProperties, ReadOnlyMemory body = default, bool mandatory = false) + void BasicPublish(string exchange, string routingKey, ref TProperties basicProperties, ReadOnlyMemory body = default, bool mandatory = false) where TProperties : IReadOnlyBasicProperties, IAmqpHeader; /// /// Publishes a message. @@ -198,7 +198,7 @@ void BasicPublish(string exchange, string routingKey, in TPropertie /// Routing key must be shorter than 255 bytes. /// /// - void BasicPublish(CachedString exchange, CachedString routingKey, in TProperties basicProperties, ReadOnlyMemory body = default, bool mandatory = false) + void BasicPublish(CachedString exchange, CachedString routingKey, ref TProperties basicProperties, ReadOnlyMemory body = default, bool mandatory = false) where TProperties : IReadOnlyBasicProperties, IAmqpHeader; #nullable disable diff --git a/projects/RabbitMQ.Client/client/api/IModelExtensions.cs b/projects/RabbitMQ.Client/client/api/IModelExtensions.cs index ff84bfb185..77810c6ee4 100644 --- a/projects/RabbitMQ.Client/client/api/IModelExtensions.cs +++ b/projects/RabbitMQ.Client/client/api/IModelExtensions.cs @@ -82,17 +82,17 @@ public static string BasicConsume(this IModel model, string queue, /// /// The publication occurs with mandatory=false and immediate=false. /// - public static void BasicPublish(this IModel model, PublicationAddress addr, in T basicProperties, ReadOnlyMemory body) + public static void BasicPublish(this IModel model, PublicationAddress addr, ref T basicProperties, ReadOnlyMemory body) where T : IReadOnlyBasicProperties, IAmqpHeader { - model.BasicPublish(addr.ExchangeName, addr.RoutingKey, basicProperties, body); + model.BasicPublish(addr.ExchangeName, addr.RoutingKey, ref basicProperties, body); } public static void BasicPublish(this IModel model, string exchange, string routingKey, ReadOnlyMemory body = default, bool mandatory = false) - => model.BasicPublish(exchange, routingKey, default(EmptyBasicProperty), body, mandatory); + => model.BasicPublish(exchange, routingKey, ref EmptyBasicProperty.Empty, body, mandatory); public static void BasicPublish(this IModel model, CachedString exchange, CachedString routingKey, ReadOnlyMemory body = default, bool mandatory = false) - => model.BasicPublish(exchange, routingKey, default(EmptyBasicProperty), body, mandatory); + => model.BasicPublish(exchange, routingKey, ref EmptyBasicProperty.Empty, body, mandatory); #nullable disable /// diff --git a/projects/RabbitMQ.Client/client/impl/AutorecoveringModel.cs b/projects/RabbitMQ.Client/client/impl/AutorecoveringModel.cs index 00b5ae22bf..d826bb149c 100644 --- a/projects/RabbitMQ.Client/client/impl/AutorecoveringModel.cs +++ b/projects/RabbitMQ.Client/client/impl/AutorecoveringModel.cs @@ -242,13 +242,13 @@ public BasicGetResult BasicGet(string queue, bool autoAck) public void BasicNack(ulong deliveryTag, bool multiple, bool requeue) => InnerChannel.BasicNack(deliveryTag, multiple, requeue); - public void BasicPublish(string exchange, string routingKey, in TProperties basicProperties, ReadOnlyMemory body, bool mandatory) + public void BasicPublish(string exchange, string routingKey, ref TProperties basicProperties, ReadOnlyMemory body, bool mandatory) where TProperties : IReadOnlyBasicProperties, IAmqpHeader - => InnerChannel.BasicPublish(exchange, routingKey, basicProperties, body, mandatory); + => InnerChannel.BasicPublish(exchange, routingKey, ref basicProperties, body, mandatory); - public void BasicPublish(CachedString exchange, CachedString routingKey, in TProperties basicProperties, ReadOnlyMemory body, bool mandatory) + public void BasicPublish(CachedString exchange, CachedString routingKey, ref TProperties basicProperties, ReadOnlyMemory body, bool mandatory) where TProperties : IReadOnlyBasicProperties, IAmqpHeader - => InnerChannel.BasicPublish(exchange, routingKey, basicProperties, body, mandatory); + => InnerChannel.BasicPublish(exchange, routingKey, ref basicProperties, body, mandatory); public void BasicQos(uint prefetchSize, ushort prefetchCount, bool global) { diff --git a/projects/RabbitMQ.Client/client/impl/EmptyBasicProperty.cs b/projects/RabbitMQ.Client/client/impl/EmptyBasicProperty.cs index f0f95ab3b6..919b1a4b3e 100644 --- a/projects/RabbitMQ.Client/client/impl/EmptyBasicProperty.cs +++ b/projects/RabbitMQ.Client/client/impl/EmptyBasicProperty.cs @@ -8,6 +8,8 @@ namespace RabbitMQ.Client.client.impl #nullable enable internal readonly struct EmptyBasicProperty : IReadOnlyBasicProperties, IAmqpHeader { + internal static EmptyBasicProperty Empty; + ushort IAmqpHeader.ProtocolClassId => ClassConstants.Basic; int IAmqpWriteable.WriteTo(Span span) diff --git a/projects/RabbitMQ.Client/client/impl/ModelBase.cs b/projects/RabbitMQ.Client/client/impl/ModelBase.cs index 5432cc4977..62f640d78a 100644 --- a/projects/RabbitMQ.Client/client/impl/ModelBase.cs +++ b/projects/RabbitMQ.Client/client/impl/ModelBase.cs @@ -897,7 +897,7 @@ public BasicGetResult BasicGet(string queue, bool autoAck) public abstract void BasicNack(ulong deliveryTag, bool multiple, bool requeue); - public void BasicPublish(string exchange, string routingKey, in TProperties basicProperties, ReadOnlyMemory body, bool mandatory) + public void BasicPublish(string exchange, string routingKey, ref TProperties basicProperties, ReadOnlyMemory body, bool mandatory) where TProperties : IReadOnlyBasicProperties, IAmqpHeader { if (NextPublishSeqNo > 0) @@ -909,11 +909,10 @@ public void BasicPublish(string exchange, string routingKey, in TPr } var cmd = new BasicPublish(exchange, routingKey, mandatory, default); - var props = basicProperties; - ModelSend(ref cmd, ref props, body); + ModelSend(ref cmd, ref basicProperties, body); } - public void BasicPublish(CachedString exchange, CachedString routingKey, in TProperties basicProperties, ReadOnlyMemory body, bool mandatory) + public void BasicPublish(CachedString exchange, CachedString routingKey, ref TProperties basicProperties, ReadOnlyMemory body, bool mandatory) where TProperties : IReadOnlyBasicProperties, IAmqpHeader { if (NextPublishSeqNo > 0) @@ -925,8 +924,7 @@ public void BasicPublish(CachedString exchange, CachedString routin } var cmd = new BasicPublishMemory(exchange.Bytes, routingKey.Bytes, mandatory, default); - var props = basicProperties; - ModelSend(ref cmd, ref props, body); + ModelSend(ref cmd, ref basicProperties, body); } public void UpdateSecret(string newSecret, string reason) diff --git a/projects/TestApplications/MassPublish/Program.cs b/projects/TestApplications/MassPublish/Program.cs index c6cd9ab06a..e489aee3ee 100644 --- a/projects/TestApplications/MassPublish/Program.cs +++ b/projects/TestApplications/MassPublish/Program.cs @@ -48,7 +48,7 @@ public static void Main() { AppId = "testapp", }; - publisher.BasicPublish("test", "myawesome.routing.key", properties, payload); + publisher.BasicPublish("test", "myawesome.routing.key", ref properties, payload); } messagesSent += ItemsPerBatch; await publisher.WaitForConfirmsOrDieAsync().ConfigureAwait(false); diff --git a/projects/Unit/TestBasicPublish.cs b/projects/Unit/TestBasicPublish.cs index 58f7ced4f0..42ea1ff32c 100644 --- a/projects/Unit/TestBasicPublish.cs +++ b/projects/Unit/TestBasicPublish.cs @@ -32,7 +32,7 @@ public void TestBasicRoundtripArray() }; string tag = m.BasicConsume(q.QueueName, true, consumer); - m.BasicPublish("", q.QueueName, bp, sendBody); + m.BasicPublish("", q.QueueName, ref bp, sendBody); bool waitResFalse = are.WaitOne(2000); m.BasicCancel(tag); diff --git a/projects/Unit/TestConnectionRecovery.cs b/projects/Unit/TestConnectionRecovery.cs index 64e874906c..0ec87dd1fd 100644 --- a/projects/Unit/TestConnectionRecovery.cs +++ b/projects/Unit/TestConnectionRecovery.cs @@ -817,7 +817,7 @@ public void TestPublishRpcRightAfterReconnect() { try { - _model.BasicPublish(string.Empty, testQueueName, properties, ReadOnlyMemory.Empty); + _model.BasicPublish(string.Empty, testQueueName, ref properties, ReadOnlyMemory.Empty); } catch (Exception e) {