-
Notifications
You must be signed in to change notification settings - Fork 17
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Single active consumer implementation (#248)
* implementing Consumer_Update command * implementing ConsumerUpdateRequest command * SAC: starting implementation * Implementing callback support and consumer_update response * adding basic test * improved test * expand unit test scope * adding example * Adding README * enabling naming for super_stream consumers and setting up sac properties internally * expanding test * few improvements and test for simple SAC * making consumer_update callback able to call async methods * making Delivery export client in order to use store_offset and review super_stream example
- Loading branch information
1 parent
ecd54d7
commit 82f93ad
Showing
18 changed files
with
962 additions
and
36 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,23 @@ | ||
Single active consumer | ||
--- | ||
|
||
This is an example to enable single active consumer functionality for superstream: | ||
https://www.rabbitmq.com/blog/2022/07/05/rabbitmq-3-11-feature-preview-single-active-consumer-for-streams | ||
https://www.rabbitmq.com/blog/2022/07/13/rabbitmq-3-11-feature-preview-super-streams | ||
|
||
This folder contains a consumer and a super-stream consumer configured to enable it. | ||
You can use the example in the super-stream folder to produce messages for a super-stream. | ||
|
||
You can then run the single_active_consumer_super_stream.rs in this folder. | ||
Assuming the super-stream is composed by three streams, you can see that the Consumer will consume messages from all the streams part of the superstream. | ||
|
||
You can then run another consumer in parallel. | ||
now you'll see that one of the two consumers will consume from 2 streams while the other on one stream. | ||
|
||
If you run another you'll see that every Consumer will read from a single stream. | ||
|
||
If you then stop one of the Consumer you'll notice that the related stream is now read from on the Consumer which is still running. | ||
|
||
|
||
|
||
|
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,97 @@ | ||
use futures::StreamExt; | ||
use rabbitmq_stream_client::error::StreamCreateError; | ||
use rabbitmq_stream_client::types::{ | ||
ByteCapacity, OffsetSpecification, ResponseCode, | ||
}; | ||
|
||
|
||
#[tokio::main] | ||
async fn main() -> Result<(), Box<dyn std::error::Error>> { | ||
use rabbitmq_stream_client::Environment; | ||
let environment = Environment::builder().build().await?; | ||
let message_count = 1000000; | ||
let stream = "hello-rust-super-stream-2"; | ||
|
||
let create_response = environment | ||
.stream_creator() | ||
.max_length(ByteCapacity::GB(5)) | ||
.create(stream) | ||
.await; | ||
|
||
if let Err(e) = create_response { | ||
if let StreamCreateError::Create { stream, status } = e { | ||
match status { | ||
// we can ignore this error because the stream already exists | ||
ResponseCode::StreamAlreadyExists => {} | ||
err => { | ||
println!("Error creating stream: {:?} {:?}", stream, err); | ||
} | ||
} | ||
} | ||
} | ||
println!( | ||
"Super stream consumer example, consuming messages from the super stream {}", | ||
stream | ||
); | ||
|
||
let mut consumer = environment | ||
.consumer() | ||
// Mandatory if sac is enabled | ||
.name("consumer-group-1") | ||
.offset(OffsetSpecification::First) | ||
.enable_single_active_consumer(true) | ||
.client_provided_name("my super stream consumer for hello rust") | ||
.consumer_update(move |active, message_context| async move { | ||
let name = message_context.name(); | ||
let stream = message_context.stream(); | ||
let client = message_context.client(); | ||
|
||
println!( | ||
"single active consumer: is active: {} on stream: {} with consumer_name: {}", | ||
active, stream, name | ||
); | ||
let stored_offset = client.query_offset(name, stream.as_str()).await; | ||
|
||
if let Err(e) = stored_offset { | ||
return OffsetSpecification::First; | ||
} | ||
|
||
let stored_offset_u = stored_offset.unwrap(); | ||
println!("restarting from stored_offset: {}", stored_offset_u); | ||
OffsetSpecification::Offset(stored_offset_u) | ||
|
||
}) | ||
.build(stream) | ||
.await | ||
.unwrap(); | ||
|
||
for i in 0..message_count { | ||
let delivery = consumer.next().await.unwrap(); | ||
{ | ||
let delivery = delivery.unwrap(); | ||
println!( | ||
"Got message: {:#?} from stream: {} with offset: {}", | ||
delivery | ||
.message() | ||
.data() | ||
.map(|data| String::from_utf8(data.to_vec()).unwrap()) | ||
.unwrap(), | ||
delivery.stream(), | ||
delivery.offset() | ||
); | ||
|
||
//store an offset | ||
if i == 10000 { | ||
let _ = consumer | ||
.store_offset(i) | ||
.await | ||
.unwrap_or_else(|e| println!("Err: {}", e)); | ||
} | ||
} | ||
} | ||
|
||
println!("Stopping consumer..."); | ||
let _ = consumer.handle().close().await; | ||
println!("consumer stopped"); | ||
Ok(()) | ||
} |
94 changes: 94 additions & 0 deletions
94
examples/single_active_consumer/single_active_consumer_super_stream.rs
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,94 @@ | ||
use futures::StreamExt; | ||
use rabbitmq_stream_client::error::StreamCreateError; | ||
use rabbitmq_stream_client::types::{ | ||
ByteCapacity, OffsetSpecification, ResponseCode, SuperStreamConsumer, | ||
}; | ||
use std::collections::HashMap; | ||
|
||
#[tokio::main] | ||
async fn main() -> Result<(), Box<dyn std::error::Error>> { | ||
use rabbitmq_stream_client::Environment; | ||
let environment = Environment::builder().build().await?; | ||
let message_count = 1000000; | ||
let super_stream = "hello-rust-super-stream"; | ||
|
||
let create_response = environment | ||
.stream_creator() | ||
.max_length(ByteCapacity::GB(5)) | ||
.create_super_stream(super_stream, 3, None) | ||
.await; | ||
|
||
if let Err(e) = create_response { | ||
if let StreamCreateError::Create { stream, status } = e { | ||
match status { | ||
// we can ignore this error because the stream already exists | ||
ResponseCode::StreamAlreadyExists => {} | ||
err => { | ||
println!("Error creating stream: {:?} {:?}", stream, err); | ||
} | ||
} | ||
} | ||
} | ||
println!( | ||
"Super stream consumer example, consuming messages from the super stream {}", | ||
super_stream | ||
); | ||
|
||
let mut super_stream_consumer: SuperStreamConsumer = environment | ||
.super_stream_consumer() | ||
// Mandatory if sac is enabled | ||
.name("consumer-group-1") | ||
.offset(OffsetSpecification::First) | ||
.enable_single_active_consumer(true) | ||
.client_provided_name("my super stream consumer for hello rust") | ||
.consumer_update(move |active, message_context| async move { | ||
let name = message_context.name(); | ||
let stream = message_context.stream(); | ||
let client = message_context.client(); | ||
|
||
println!( | ||
"single active consumer: is active: {} on stream: {} with consumer_name: {}", | ||
active, stream, name | ||
); | ||
let stored_offset = client.query_offset(name, stream.as_str()).await; | ||
|
||
if let Err(e) = stored_offset { | ||
return OffsetSpecification::First; | ||
} | ||
let stored_offset_u = stored_offset.unwrap(); | ||
println!("stored_offset_u {}", stored_offset_u.clone()); | ||
OffsetSpecification::Offset(stored_offset_u) | ||
|
||
}) | ||
.build(super_stream) | ||
.await | ||
.unwrap(); | ||
|
||
for _ in 0..message_count { | ||
let delivery = super_stream_consumer.next().await.unwrap(); | ||
{ | ||
let delivery = delivery.unwrap(); | ||
println!( | ||
"Got message: {:#?} from stream: {} with offset: {}", | ||
delivery | ||
.message() | ||
.data() | ||
.map(|data| String::from_utf8(data.to_vec()).unwrap()) | ||
.unwrap(), | ||
delivery.stream(), | ||
delivery.offset() | ||
); | ||
|
||
// Store an offset for every consumer | ||
if delivery.consumer_name().is_some() && delivery.offset() == 1000 { | ||
super_stream_consumer.client().store_offset(delivery.consumer_name().unwrap().as_str(), delivery.stream().as_str(), delivery.offset()).await; | ||
} | ||
|
||
} | ||
} | ||
|
||
println!("Stopping super stream consumer..."); | ||
let _ = super_stream_consumer.handle().close().await; | ||
println!("Super stream consumer stopped"); | ||
Ok(()) | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,88 @@ | ||
use std::io::Write; | ||
|
||
#[cfg(test)] | ||
use fake::Fake; | ||
|
||
use crate::{ | ||
codec::{Decoder, Encoder}, | ||
error::{DecodeError, EncodeError}, | ||
protocol::commands::COMMAND_CONSUMER_UPDATE, | ||
}; | ||
|
||
use super::Command; | ||
|
||
#[cfg_attr(test, derive(fake::Dummy))] | ||
#[derive(PartialEq, Eq, Debug)] | ||
pub struct ConsumerUpdateCommand { | ||
pub(crate) correlation_id: u32, | ||
subscription_id: u8, | ||
active: u8, | ||
} | ||
|
||
impl ConsumerUpdateCommand { | ||
pub fn new(correlation_id: u32, subscription_id: u8, active: u8) -> Self { | ||
Self { | ||
correlation_id, | ||
subscription_id, | ||
active, | ||
} | ||
} | ||
|
||
pub fn get_correlation_id(&self) -> u32 { | ||
self.correlation_id | ||
} | ||
|
||
pub fn is_active(&self) -> u8 { | ||
self.active | ||
} | ||
} | ||
|
||
impl Encoder for ConsumerUpdateCommand { | ||
fn encoded_size(&self) -> u32 { | ||
self.correlation_id.encoded_size() | ||
+ self.subscription_id.encoded_size() | ||
+ self.active.encoded_size() | ||
} | ||
|
||
fn encode(&self, writer: &mut impl Write) -> Result<(), EncodeError> { | ||
self.correlation_id.encode(writer)?; | ||
self.subscription_id.encode(writer)?; | ||
self.active.encode(writer)?; | ||
Ok(()) | ||
} | ||
} | ||
|
||
impl Decoder for ConsumerUpdateCommand { | ||
fn decode(input: &[u8]) -> Result<(&[u8], Self), DecodeError> { | ||
let (input, correlation_id) = u32::decode(input)?; | ||
let (input, subscription_id) = u8::decode(input)?; | ||
let (input, active) = u8::decode(input)?; | ||
|
||
Ok(( | ||
input, | ||
ConsumerUpdateCommand { | ||
correlation_id, | ||
subscription_id, | ||
active, | ||
}, | ||
)) | ||
} | ||
} | ||
|
||
impl Command for ConsumerUpdateCommand { | ||
fn key(&self) -> u16 { | ||
COMMAND_CONSUMER_UPDATE | ||
} | ||
} | ||
|
||
#[cfg(test)] | ||
mod tests { | ||
use crate::commands::tests::command_encode_decode_test; | ||
|
||
use super::ConsumerUpdateCommand; | ||
|
||
#[test] | ||
fn consumer_update_response_test() { | ||
command_encode_decode_test::<ConsumerUpdateCommand>(); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,86 @@ | ||
use std::io::Write; | ||
|
||
#[cfg(test)] | ||
use fake::Fake; | ||
|
||
use crate::{ | ||
codec::{Decoder, Encoder}, | ||
error::{DecodeError, EncodeError}, | ||
protocol::commands::COMMAND_CONSUMER_UPDATE_REQUEST, | ||
}; | ||
|
||
use crate::commands::subscribe::OffsetSpecification; | ||
|
||
use super::Command; | ||
|
||
#[cfg_attr(test, derive(fake::Dummy))] | ||
#[derive(PartialEq, Eq, Debug)] | ||
pub struct ConsumerUpdateRequestCommand { | ||
pub(crate) correlation_id: u32, | ||
response_code: u16, | ||
offset_specification: OffsetSpecification, | ||
} | ||
|
||
impl ConsumerUpdateRequestCommand { | ||
pub fn new( | ||
correlation_id: u32, | ||
response_code: u16, | ||
offset_specification: OffsetSpecification, | ||
) -> Self { | ||
Self { | ||
correlation_id, | ||
response_code, | ||
offset_specification, | ||
} | ||
} | ||
} | ||
|
||
impl Encoder for ConsumerUpdateRequestCommand { | ||
fn encoded_size(&self) -> u32 { | ||
self.correlation_id.encoded_size() | ||
+ self.response_code.encoded_size() | ||
+ self.offset_specification.encoded_size() | ||
} | ||
|
||
fn encode(&self, writer: &mut impl Write) -> Result<(), EncodeError> { | ||
self.correlation_id.encode(writer)?; | ||
self.response_code.encode(writer)?; | ||
self.offset_specification.encode(writer)?; | ||
Ok(()) | ||
} | ||
} | ||
|
||
impl Decoder for ConsumerUpdateRequestCommand { | ||
fn decode(input: &[u8]) -> Result<(&[u8], Self), DecodeError> { | ||
let (input, correlation_id) = u32::decode(input)?; | ||
let (input, response_code) = u16::decode(input)?; | ||
let (input, offset_specification) = OffsetSpecification::decode(input)?; | ||
|
||
Ok(( | ||
input, | ||
ConsumerUpdateRequestCommand { | ||
correlation_id, | ||
response_code, | ||
offset_specification, | ||
}, | ||
)) | ||
} | ||
} | ||
|
||
impl Command for ConsumerUpdateRequestCommand { | ||
fn key(&self) -> u16 { | ||
COMMAND_CONSUMER_UPDATE_REQUEST | ||
} | ||
} | ||
|
||
#[cfg(test)] | ||
mod tests { | ||
use crate::commands::tests::command_encode_decode_test; | ||
|
||
use super::ConsumerUpdateRequestCommand; | ||
|
||
#[test] | ||
fn consumer_update_request_test() { | ||
command_encode_decode_test::<ConsumerUpdateRequestCommand>(); | ||
} | ||
} |
Oops, something went wrong.