Skip to content

Commit 0f5cd64

Browse files
authored
Expose initial credits configuration (#265)
* Expose initial credits configuration ref: #264 Signed-off-by: Gabriele Santomaggio <[email protected]> * Add constant for the default consumer credits Signed-off-by: Gabriele Santomaggio <[email protected]>
1 parent e42d8fe commit 0f5cd64

File tree

7 files changed

+65
-6
lines changed

7 files changed

+65
-6
lines changed

RabbitMQ.Stream.Client/Consts.cs

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -14,6 +14,7 @@ internal static class Consts
1414
internal static readonly TimeSpan ShortWait = TimeSpan.FromSeconds(1);
1515
internal static readonly TimeSpan MidWait = TimeSpan.FromSeconds(3);
1616
internal static readonly TimeSpan LongWait = TimeSpan.FromSeconds(10);
17+
internal const ushort ConsumerInitialCredits = 2;
1718

1819
internal static int RandomShort()
1920
{

RabbitMQ.Stream.Client/IConsumer.cs

Lines changed: 23 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -16,6 +16,8 @@ public interface IConsumer
1616

1717
public record IConsumerConfig : INamedEntity
1818
{
19+
private ushort _initialCredits = Consts.ConsumerInitialCredits;
20+
1921
// StoredOffsetSpec configuration it is needed to keep the offset spec.
2022
// since the offset can be decided from the ConsumerConfig.OffsetSpec.
2123
// and from ConsumerConfig.ConsumerUpdateListener.
@@ -40,4 +42,25 @@ public record IConsumerConfig : INamedEntity
4042
public string Reference { get; set; }
4143

4244
public Func<string, Task> ConnectionClosedHandler { get; set; }
45+
46+
// InitialCredits is the initial credits to be used for the consumer.
47+
// if the InitialCredits is not set, the default value will be 2.
48+
// It is the number of the chunks that the consumer will receive at beginning.
49+
// A high value can increase the throughput but could increase the memory usage and server-side CPU usage.
50+
// The RawConsumer uses this value to create the Channel buffer so all the chunks will be stored in the buffer memory.
51+
// The default value it is usually a good value.
52+
public ushort InitialCredits
53+
{
54+
get => _initialCredits;
55+
set
56+
{
57+
if (value < 1)
58+
{
59+
throw new ArgumentException(
60+
$"InitialCredits must be greater than 0. Default value is {Consts.ConsumerInitialCredits}.");
61+
}
62+
63+
_initialCredits = value;
64+
}
65+
}
4366
}

RabbitMQ.Stream.Client/PublicAPI.Unshipped.txt

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -4,6 +4,10 @@ RabbitMQ.Stream.Client.AbstractEntity.Token.get -> System.Threading.Cancellation
44
RabbitMQ.Stream.Client.Chunk.Data.get -> System.Memory<byte>
55
RabbitMQ.Stream.Client.Chunk.MagicVersion.get -> byte
66
RabbitMQ.Stream.Client.Client.StreamStats(string stream) -> System.Threading.Tasks.ValueTask<RabbitMQ.Stream.Client.StreamStatsResponse>
7+
RabbitMQ.Stream.Client.IConsumerConfig.InitialCredits.get -> ushort
8+
RabbitMQ.Stream.Client.IConsumerConfig.InitialCredits.set -> void
9+
RabbitMQ.Stream.Client.Reliable.ConsumerConfig.InitialCredits.get -> ushort
10+
RabbitMQ.Stream.Client.Reliable.ConsumerConfig.InitialCredits.set -> void
711
RabbitMQ.Stream.Client.Reliable.DeduplicatingProducer
812
RabbitMQ.Stream.Client.Reliable.DeduplicatingProducer.Close() -> System.Threading.Tasks.Task
913
RabbitMQ.Stream.Client.Reliable.DeduplicatingProducer.GetLastPublishedId() -> System.Threading.Tasks.Task<ulong>

RabbitMQ.Stream.Client/RawConsumer.cs

Lines changed: 11 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -98,12 +98,20 @@ public class RawConsumer : AbstractEntity, IConsumer, IDisposable
9898
private byte _subscriberId;
9999
private readonly ILogger _logger;
100100
private readonly Channel<Chunk> _chunksBuffer;
101-
private const int InitialCredit = 2;
101+
private readonly ushort _initialCredits;
102102

103103
private RawConsumer(Client client, RawConsumerConfig config, ILogger logger = null)
104104
{
105+
_logger = logger ?? NullLogger.Instance;
106+
_initialCredits = config.InitialCredits;
107+
_logger.LogDebug("creating consumer {Consumer} with initial credits {InitialCredits}, " +
108+
"offset {OffsetSpec}, is single active consumer {IsSingleActiveConsumer}, super stream {SuperStream}, client provided name {ClientProvidedName}, " +
109+
config.Reference,
110+
_initialCredits, config.OffsetSpec, config.SuperStream, config.IsSingleActiveConsumer,
111+
config.ClientProvidedName);
112+
105113
// _chunksBuffer is a channel that is used to buffer the chunks
106-
_chunksBuffer = Channel.CreateBounded<Chunk>(new BoundedChannelOptions(InitialCredit)
114+
_chunksBuffer = Channel.CreateBounded<Chunk>(new BoundedChannelOptions(_initialCredits)
107115
{
108116
AllowSynchronousContinuations = false,
109117
SingleReader = true,
@@ -113,7 +121,6 @@ private RawConsumer(Client client, RawConsumerConfig config, ILogger logger = nu
113121
IsPromotedAsActive = true;
114122
_client = client;
115123
_config = config;
116-
_logger = logger ?? NullLogger.Instance;
117124

118125
ProcessChunks();
119126
}
@@ -387,7 +394,7 @@ private async Task Init()
387394

388395
var (consumerId, response) = await _client.Subscribe(
389396
_config,
390-
InitialCredit,
397+
_initialCredits,
391398
consumerProperties,
392399
async deliver =>
393400
{

RabbitMQ.Stream.Client/Reliable/Consumer.cs

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -95,6 +95,14 @@ public record ConsumerConfig : ReliableConfig
9595
/// </summary>
9696
public Func<string, string, bool, Task<IOffsetType>> ConsumerUpdateListener { get; set; }
9797

98+
// InitialCredits is the initial credits to be used for the consumer.
99+
// if the InitialCredits is not set, the default value will be 2.
100+
// It is the number of the chunks that the consumer will receive at beginning.
101+
// A high value can increase the throughput but could increase the memory usage and server-side CPU usage.
102+
// The RawConsumer uses this value to create the Channel buffer so all the chunks will be stored in the buffer memory.
103+
// The default value it is usually a good value.
104+
public ushort InitialCredits { get; set; } = Consts.ConsumerInitialCredits;
105+
98106
public ConsumerConfig(StreamSystem streamSystem, string stream) : base(streamSystem, stream)
99107
{
100108
}

RabbitMQ.Stream.Client/Reliable/ConsumerFactory.cs

Lines changed: 6 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -48,6 +48,7 @@ private async Task<IConsumer> StandardConsumer(bool boot)
4848
Reference = _consumerConfig.Reference,
4949
ConsumerUpdateListener = _consumerConfig.ConsumerUpdateListener,
5050
IsSingleActiveConsumer = _consumerConfig.IsSingleActiveConsumer,
51+
InitialCredits = _consumerConfig.InitialCredits,
5152
OffsetSpec = offsetSpec,
5253
ConnectionClosedHandler = async _ =>
5354
{
@@ -69,7 +70,8 @@ private async Task<IConsumer> StandardConsumer(bool boot)
6970
_lastOffsetConsumed[_consumerConfig.Stream] = ctx.Offset;
7071
if (_consumerConfig.MessageHandler != null)
7172
{
72-
await _consumerConfig.MessageHandler(_consumerConfig.Stream, consumer, ctx, message).ConfigureAwait(false);
73+
await _consumerConfig.MessageHandler(_consumerConfig.Stream, consumer, ctx, message)
74+
.ConfigureAwait(false);
7375
}
7476
},
7577
}, BaseLogger).ConfigureAwait(false);
@@ -89,7 +91,8 @@ private async Task<IConsumer> SuperConsumer(bool boot)
8991
}
9092
else
9193
{
92-
var partitions = await _consumerConfig.StreamSystem.QueryPartition(_consumerConfig.Stream).ConfigureAwait(false);
94+
var partitions = await _consumerConfig.StreamSystem.QueryPartition(_consumerConfig.Stream)
95+
.ConfigureAwait(false);
9396
foreach (var partition in partitions)
9497
{
9598
offsetSpecs[partition] =
@@ -104,6 +107,7 @@ private async Task<IConsumer> SuperConsumer(bool boot)
104107
Reference = _consumerConfig.Reference,
105108
ConsumerUpdateListener = _consumerConfig.ConsumerUpdateListener,
106109
IsSingleActiveConsumer = _consumerConfig.IsSingleActiveConsumer,
110+
InitialCredits = _consumerConfig.InitialCredits,
107111
OffsetSpec = offsetSpecs,
108112
MessageHandler = async (stream, consumer, ctx, message) =>
109113
{

Tests/RawConsumerSystemTests.cs

Lines changed: 12 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -11,6 +11,7 @@
1111
using System.Threading.Tasks;
1212
using RabbitMQ.Stream.Client;
1313
using RabbitMQ.Stream.Client.AMQP;
14+
using RabbitMQ.Stream.Client.Reliable;
1415
using Xunit;
1516
using Xunit.Abstractions;
1617

@@ -617,6 +618,17 @@ public async void ConsumerMetadataHandlerUpdate()
617618
await system.Close();
618619
}
619620

621+
[Fact]
622+
public async void ValidateInitialCredits()
623+
{
624+
SystemUtils.InitStreamSystemWithRandomStream(out var system, out var stream);
625+
626+
await Assert.ThrowsAsync<ArgumentException>(async () =>
627+
await Consumer.Create(new ConsumerConfig(system, stream) { InitialCredits = 0, }));
628+
629+
await SystemUtils.CleanUpStreamSystem(system, stream);
630+
}
631+
620632
[Fact]
621633
public async void ProducerConsumerMixingDifferentSendTypesCompressAndStandard()
622634
{

0 commit comments

Comments
 (0)