Avoid topic pollution by prefixing with test name.

This commit is contained in:
David Blewett 2024-01-11 15:30:10 -05:00
parent e347f9aece
commit 321c04078a
8 changed files with 49 additions and 43 deletions

View File

@ -32,7 +32,7 @@ fn create_admin_client() -> AdminClient<DefaultClientContext> {
async fn create_consumer_group(consumer_group_name: &str) {
let admin_client = create_admin_client();
let topic_name = &rand_test_topic();
let topic_name = &rand_test_topic(consumer_group_name);
let consumer: BaseConsumer = create_config()
.set("group.id", consumer_group_name.clone())
.create()
@ -124,8 +124,8 @@ async fn test_topics() {
// Verify that topics are created as specified, and that they can later
// be deleted.
{
let name1 = rand_test_topic();
let name2 = rand_test_topic();
let name1 = rand_test_topic("test_topics");
let name2 = rand_test_topic("test_topics");
// Test both the builder API and the literal construction.
let topic1 =
@ -254,7 +254,7 @@ async fn test_topics() {
// Verify that incorrect replication configurations are ignored when
// creating partitions.
{
let name = rand_test_topic();
let name = rand_test_topic("test_topics");
let topic = NewTopic::new(&name, 1, TopicReplication::Fixed(1));
let res = admin_client
@ -291,7 +291,7 @@ async fn test_topics() {
// Verify that deleting a non-existent topic fails.
{
let name = rand_test_topic();
let name = rand_test_topic("test_topics");
let res = admin_client
.delete_topics(&[&name], &opts)
.await
@ -305,8 +305,8 @@ async fn test_topics() {
// Verify that mixed-success operations properly report the successful and
// failing operators.
{
let name1 = rand_test_topic();
let name2 = rand_test_topic();
let name1 = rand_test_topic("test_topics");
let name2 = rand_test_topic("test_topics");
let topic1 = NewTopic::new(&name1, 1, TopicReplication::Fixed(1));
let topic2 = NewTopic::new(&name2, 1, TopicReplication::Fixed(1));

View File

@ -70,7 +70,7 @@ async fn test_produce_consume_base() {
let _r = env_logger::try_init();
let start_time = current_time_millis();
let topic_name = rand_test_topic();
let topic_name = rand_test_topic("test_produce_consume_base");
let message_map = populate_topic(&topic_name, 100, &value_fn, &key_fn, None, None).await;
let consumer = create_stream_consumer(&rand_test_group(), None);
consumer.subscribe(&[topic_name.as_str()]).unwrap();
@ -105,7 +105,7 @@ async fn test_produce_consume_base() {
async fn test_produce_consume_base_concurrent() {
let _r = env_logger::try_init();
let topic_name = rand_test_topic();
let topic_name = rand_test_topic("test_produce_consume_base_concurrent");
populate_topic(&topic_name, 100, &value_fn, &key_fn, None, None).await;
let consumer = Arc::new(create_stream_consumer(&rand_test_group(), None));
@ -135,7 +135,7 @@ async fn test_produce_consume_base_concurrent() {
async fn test_produce_consume_base_assign() {
let _r = env_logger::try_init();
let topic_name = rand_test_topic();
let topic_name = rand_test_topic("test_produce_consume_base_assign");
populate_topic(&topic_name, 10, &value_fn, &key_fn, Some(0), None).await;
populate_topic(&topic_name, 10, &value_fn, &key_fn, Some(1), None).await;
populate_topic(&topic_name, 10, &value_fn, &key_fn, Some(2), None).await;
@ -170,7 +170,7 @@ async fn test_produce_consume_base_assign() {
async fn test_produce_consume_base_unassign() {
let _r = env_logger::try_init();
let topic_name = rand_test_topic();
let topic_name = rand_test_topic("test_produce_consume_base_unassign");
populate_topic(&topic_name, 10, &value_fn, &key_fn, Some(0), None).await;
populate_topic(&topic_name, 10, &value_fn, &key_fn, Some(1), None).await;
populate_topic(&topic_name, 10, &value_fn, &key_fn, Some(2), None).await;
@ -195,7 +195,7 @@ async fn test_produce_consume_base_unassign() {
async fn test_produce_consume_base_incremental_assign_and_unassign() {
let _r = env_logger::try_init();
let topic_name = rand_test_topic();
let topic_name = rand_test_topic("test_produce_consume_base_incremental_assign_and_unassign");
populate_topic(&topic_name, 10, &value_fn, &key_fn, Some(0), None).await;
populate_topic(&topic_name, 10, &value_fn, &key_fn, Some(1), None).await;
populate_topic(&topic_name, 10, &value_fn, &key_fn, Some(2), None).await;
@ -236,7 +236,7 @@ async fn test_produce_consume_base_incremental_assign_and_unassign() {
async fn test_produce_consume_with_timestamp() {
let _r = env_logger::try_init();
let topic_name = rand_test_topic();
let topic_name = rand_test_topic("test_produce_consume_with_timestamp");
let message_map =
populate_topic(&topic_name, 100, &value_fn, &key_fn, Some(0), Some(1111)).await;
let consumer = create_stream_consumer(&rand_test_group(), None);
@ -277,7 +277,7 @@ async fn test_produce_consume_with_timestamp() {
async fn test_consumer_commit_message() {
let _r = env_logger::try_init();
let topic_name = rand_test_topic();
let topic_name = rand_test_topic("test_consumer_commit_message");
populate_topic(&topic_name, 10, &value_fn, &key_fn, Some(0), None).await;
populate_topic(&topic_name, 11, &value_fn, &key_fn, Some(1), None).await;
populate_topic(&topic_name, 12, &value_fn, &key_fn, Some(2), None).await;
@ -355,7 +355,7 @@ async fn test_consumer_commit_message() {
async fn test_consumer_store_offset_commit() {
let _r = env_logger::try_init();
let topic_name = rand_test_topic();
let topic_name = rand_test_topic("test_consumer_store_offset_commit");
populate_topic(&topic_name, 10, &value_fn, &key_fn, Some(0), None).await;
populate_topic(&topic_name, 11, &value_fn, &key_fn, Some(1), None).await;
populate_topic(&topic_name, 12, &value_fn, &key_fn, Some(2), None).await;
@ -440,7 +440,7 @@ async fn test_consumer_store_offset_commit() {
async fn test_consumer_commit_metadata() -> Result<(), Box<dyn Error>> {
let _ = env_logger::try_init();
let topic_name = rand_test_topic();
let topic_name = rand_test_topic("test_consumer_commit_metadata");
let group_name = rand_test_group();
populate_topic(&topic_name, 10, &value_fn, &key_fn, None, None).await;
@ -495,7 +495,7 @@ async fn test_consumer_commit_metadata() -> Result<(), Box<dyn Error>> {
async fn test_consume_partition_order() {
let _r = env_logger::try_init();
let topic_name = rand_test_topic();
let topic_name = rand_test_topic("test_consume_partition_order");
populate_topic(&topic_name, 4, &value_fn, &key_fn, Some(0), None).await;
populate_topic(&topic_name, 4, &value_fn, &key_fn, Some(1), None).await;
populate_topic(&topic_name, 4, &value_fn, &key_fn, Some(2), None).await;

View File

@ -30,7 +30,7 @@ fn future_producer(config_overrides: HashMap<&str, &str>) -> FutureProducer<Defa
#[tokio::test]
async fn test_future_producer_send() {
let producer = future_producer(HashMap::new());
let topic_name = rand_test_topic();
let topic_name = rand_test_topic("test_future_producer_send");
let results: FuturesUnordered<_> = (0..10)
.map(|_| {
@ -60,7 +60,7 @@ async fn test_future_producer_send_full() {
config.insert("message.timeout.ms", "5000");
config.insert("queue.buffering.max.messages", "1");
let producer = &future_producer(config);
let topic_name = &rand_test_topic();
let topic_name = &rand_test_topic("test_future_producer_send_full");
// Fill up the queue.
producer

View File

@ -31,7 +31,7 @@ fn create_base_consumer(
async fn test_produce_consume_seek() {
let _r = env_logger::try_init();
let topic_name = rand_test_topic();
let topic_name = rand_test_topic("test_produce_consume_seek");
populate_topic(&topic_name, 5, &value_fn, &key_fn, Some(0), None).await;
let consumer = create_base_consumer(&rand_test_group(), None);
consumer.subscribe(&[topic_name.as_str()]).unwrap();
@ -96,7 +96,7 @@ async fn test_produce_consume_seek() {
async fn test_produce_consume_seek_partitions() {
let _r = env_logger::try_init();
let topic_name = rand_test_topic();
let topic_name = rand_test_topic("test_produce_consume_seek_partitions");
populate_topic(&topic_name, 30, &value_fn, &key_fn, None, None).await;
let consumer = create_base_consumer(&rand_test_group(), None);
@ -158,7 +158,7 @@ async fn test_produce_consume_iter() {
let _r = env_logger::try_init();
let start_time = current_time_millis();
let topic_name = rand_test_topic();
let topic_name = rand_test_topic("test_produce_consume_iter");
let message_map = populate_topic(&topic_name, 100, &value_fn, &key_fn, None, None).await;
let consumer = create_base_consumer(&rand_test_group(), None);
consumer.subscribe(&[topic_name.as_str()]).unwrap();
@ -196,7 +196,7 @@ async fn test_pause_resume_consumer_iter() {
let _r = env_logger::try_init();
let topic_name = rand_test_topic();
let topic_name = rand_test_topic("test_pause_resume_consumer_iter");
populate_topic(
&topic_name,
MESSAGE_COUNT,
@ -237,7 +237,7 @@ async fn test_pause_resume_consumer_iter() {
async fn test_consume_partition_order() {
let _r = env_logger::try_init();
let topic_name = rand_test_topic();
let topic_name = rand_test_topic("test_consume_partition_order");
populate_topic(&topic_name, 4, &value_fn, &key_fn, Some(0), None).await;
populate_topic(&topic_name, 4, &value_fn, &key_fn, Some(1), None).await;
populate_topic(&topic_name, 4, &value_fn, &key_fn, Some(2), None).await;
@ -357,7 +357,7 @@ async fn test_consume_partition_order() {
async fn test_produce_consume_message_queue_nonempty_callback() {
let _r = env_logger::try_init();
let topic_name = rand_test_topic();
let topic_name = rand_test_topic("test_produce_consume_message_queue_nonempty_callback");
create_topic(&topic_name, 1).await;

View File

@ -191,7 +191,7 @@ where
#[test]
fn test_base_producer_queue_full() {
let producer = base_producer(hashmap! { "queue.buffering.max.messages" => "10" });
let topic_name = rand_test_topic();
let topic_name = rand_test_topic("test_base_producer_queue_full");
let results = (0..30)
.map(|id| {
@ -235,7 +235,7 @@ fn test_base_producer_timeout() {
"bootstrap.servers" => "1.2.3.4"
},
);
let topic_name = rand_test_topic();
let topic_name = rand_test_topic("test_base_producer_timeout");
let results_count = (0..10)
.map(|id| {
@ -346,7 +346,7 @@ fn test_base_producer_headers() {
ids: ids_set.clone(),
};
let producer = base_producer_with_context(context, HashMap::new());
let topic_name = rand_test_topic();
let topic_name = rand_test_topic("test_base_producer_headers");
let results_count = (0..10)
.map(|id| {
@ -387,7 +387,7 @@ fn test_base_producer_headers() {
fn test_threaded_producer_send() {
let context = CollectingContext::new();
let producer = threaded_producer_with_context(context.clone(), HashMap::new());
let topic_name = rand_test_topic();
let topic_name = rand_test_topic("test_threaded_producer_send");
let results_count = (0..10)
.map(|id| {
@ -431,7 +431,7 @@ fn test_base_producer_opaque_arc() -> Result<(), Box<dyn Error>> {
let shared_count = Arc::new(Mutex::new(0));
let context = OpaqueArcContext {};
let producer = base_producer_with_context(context, HashMap::new());
let topic_name = rand_test_topic();
let topic_name = rand_test_topic("test_base_producer_opaque_arc");
let results_count = (0..10)
.map(|_| {
@ -482,7 +482,13 @@ fn test_register_custom_partitioner_linger_non_zero_key_null() {
let producer = base_producer_with_context(context.clone(), config_overrides);
producer
.send(BaseRecord::<(), str, usize>::with_opaque_to(&rand_test_topic(), 0).payload(""))
.send(
BaseRecord::<(), str, usize>::with_opaque_to(
&rand_test_topic("test_register_custom_partitioner_linger_non_zero_key_null"),
0,
)
.payload(""),
)
.unwrap();
producer.flush(Duration::from_secs(10)).unwrap();
@ -499,7 +505,7 @@ fn test_register_custom_partitioner_linger_non_zero_key_null() {
fn test_custom_partitioner_base_producer() {
let context = CollectingContext::new_with_custom_partitioner(FixedPartitioner::new(2));
let producer = base_producer_with_context(context.clone(), HashMap::new());
let topic_name = rand_test_topic();
let topic_name = rand_test_topic("test_custom_partitioner_base_producer");
let results_count = (0..10)
.map(|id| {
@ -527,7 +533,7 @@ fn test_custom_partitioner_base_producer() {
fn test_custom_partitioner_threaded_producer() {
let context = CollectingContext::new_with_custom_partitioner(FixedPartitioner::new(2));
let producer = threaded_producer_with_context(context.clone(), HashMap::new());
let topic_name = rand_test_topic();
let topic_name = rand_test_topic("test_custom_partitioner_threaded_producer");
let results_count = (0..10)
.map(|id| {

View File

@ -31,7 +31,7 @@ fn create_consumer(group_id: &str) -> StreamConsumer {
async fn test_metadata() {
let _r = env_logger::try_init();
let topic_name = rand_test_topic();
let topic_name = rand_test_topic("test_metadata");
populate_topic(&topic_name, 1, &value_fn, &key_fn, Some(0), None).await;
populate_topic(&topic_name, 1, &value_fn, &key_fn, Some(1), None).await;
populate_topic(&topic_name, 1, &value_fn, &key_fn, Some(2), None).await;
@ -92,7 +92,7 @@ async fn test_metadata() {
async fn test_subscription() {
let _r = env_logger::try_init();
let topic_name = rand_test_topic();
let topic_name = rand_test_topic("test_subscription");
populate_topic(&topic_name, 10, &value_fn, &key_fn, None, None).await;
let consumer = create_consumer(&rand_test_group());
consumer.subscribe(&[topic_name.as_str()]).unwrap();
@ -109,7 +109,7 @@ async fn test_subscription() {
async fn test_group_membership() {
let _r = env_logger::try_init();
let topic_name = rand_test_topic();
let topic_name = rand_test_topic("test_group_membership");
let group_name = rand_test_group();
populate_topic(&topic_name, 1, &value_fn, &key_fn, Some(0), None).await;
populate_topic(&topic_name, 1, &value_fn, &key_fn, Some(1), None).await;

View File

@ -64,8 +64,8 @@ fn count_records(topic: &str, iso: IsolationLevel) -> Result<usize, KafkaError>
#[tokio::test]
async fn test_transaction_abort() -> Result<(), Box<dyn Error>> {
let consume_topic = rand_test_topic();
let produce_topic = rand_test_topic();
let consume_topic = rand_test_topic("test_transaction_abort");
let produce_topic = rand_test_topic("test_transaction_abort");
populate_topic(&consume_topic, 30, &value_fn, &key_fn, Some(0), None).await;
@ -132,8 +132,8 @@ async fn test_transaction_abort() -> Result<(), Box<dyn Error>> {
#[tokio::test]
async fn test_transaction_commit() -> Result<(), Box<dyn Error>> {
let consume_topic = rand_test_topic();
let produce_topic = rand_test_topic();
let consume_topic = rand_test_topic("test_transaction_commit");
let produce_topic = rand_test_topic("test_transaction_commit");
populate_topic(&consume_topic, 30, &value_fn, &key_fn, Some(0), None).await;

View File

@ -17,12 +17,12 @@ use rdkafka::producer::{FutureProducer, FutureRecord};
use rdkafka::statistics::Statistics;
use rdkafka::TopicPartitionList;
pub fn rand_test_topic() -> String {
pub fn rand_test_topic(test_name: &str) -> String {
let id = rand::thread_rng()
.gen_ascii_chars()
.take(10)
.collect::<String>();
format!("__test_{}", id)
format!("__{}_{}", test_name, id)
}
pub fn rand_test_group() -> String {
@ -170,7 +170,7 @@ mod tests {
#[tokio::test]
async fn test_populate_topic() {
let topic_name = rand_test_topic();
let topic_name = rand_test_topic("test_populate_topic");
let message_map = populate_topic(&topic_name, 100, &value_fn, &key_fn, Some(0), None).await;
let total_messages = message_map