@@ -12,7 +12,8 @@ use rdkafka::client::DefaultClientContext;
1212use rdkafka:: consumer:: { BaseConsumer , CommitMode , Consumer , DefaultConsumerContext } ;
1313use rdkafka:: error:: { KafkaError , RDKafkaErrorCode } ;
1414use rdkafka:: metadata:: Metadata ;
15- use rdkafka:: { ClientConfig , TopicPartitionList } ;
15+ use rdkafka:: producer:: { FutureProducer , FutureRecord , Producer } ;
16+ use rdkafka:: { ClientConfig , Offset , TopicPartitionList } ;
1617
1718use crate :: utils:: * ;
1819
@@ -356,6 +357,114 @@ async fn test_topics() {
356357 }
357358}
358359
360+ /// Test the admin client's delete records functionality.
361+ #[ tokio:: test]
362+ async fn test_delete_records ( ) {
363+ let producer = create_config ( ) . create :: < FutureProducer < _ > > ( ) . unwrap ( ) ;
364+ let admin_client = create_admin_client ( ) ;
365+ let timeout = Some ( Duration :: from_secs ( 1 ) ) ;
366+ let opts = AdminOptions :: new ( ) . operation_timeout ( timeout) ;
367+ let topic = rand_test_topic ( "test_delete_records" ) ;
368+ let make_record = || FutureRecord :: < str , str > :: to ( & topic) . payload ( "data" ) ;
369+
370+ // Create a topic with a single partition.
371+ admin_client
372+ . create_topics (
373+ & [ NewTopic :: new ( & topic, 1 , TopicReplication :: Fixed ( 1 ) ) ] ,
374+ & opts,
375+ )
376+ . await
377+ . expect ( "topic creation failed" ) ;
378+
379+ // Ensure that the topic begins with low and high water marks of 0.
380+ let ( lo, hi) = producer
381+ . client ( )
382+ . fetch_watermarks ( & topic, 0 , timeout)
383+ . unwrap ( ) ;
384+ assert_eq ! ( lo, 0 ) ;
385+ assert_eq ! ( hi, 0 ) ;
386+
387+ // Produce five messages to the topic.
388+ for _ in 0 ..5 {
389+ producer. send ( make_record ( ) , timeout) . await . unwrap ( ) ;
390+ }
391+
392+ // Ensure that the high water mark has advanced to 5.
393+ let ( lo, hi) = producer
394+ . client ( )
395+ . fetch_watermarks ( & topic, 0 , timeout)
396+ . unwrap ( ) ;
397+ assert_eq ! ( lo, 0 ) ;
398+ assert_eq ! ( hi, 5 ) ;
399+
400+ // Delete the record at offset 0.
401+ let mut tpl = TopicPartitionList :: new ( ) ;
402+ tpl. add_partition_offset ( & topic, 0 , Offset :: Offset ( 1 ) )
403+ . unwrap ( ) ;
404+ let res_tpl = admin_client. delete_records ( & tpl, & opts) . await . unwrap ( ) ;
405+ assert_eq ! ( res_tpl. count( ) , 1 ) ;
406+ assert_eq ! ( res_tpl. elements( ) [ 0 ] . topic( ) , topic) ;
407+ assert_eq ! ( res_tpl. elements( ) [ 0 ] . partition( ) , 0 ) ;
408+ assert_eq ! ( res_tpl. elements( ) [ 0 ] . offset( ) , Offset :: Offset ( 1 ) ) ;
409+ assert_eq ! ( res_tpl. elements( ) [ 0 ] . error( ) , Ok ( ( ) ) ) ;
410+
411+ // Ensure that the low water mark has advanced to 1.
412+ let ( lo, hi) = producer
413+ . client ( )
414+ . fetch_watermarks ( & topic, 0 , timeout)
415+ . unwrap ( ) ;
416+ assert_eq ! ( lo, 1 ) ;
417+ assert_eq ! ( hi, 5 ) ;
418+
419+ // Delete the record at offset 1 and also include an invalid partition in
420+ // the request. The invalid partition should not cause the request to fail,
421+ // but we should be able to see the per-partition error in the returned
422+ // topic partition list.
423+ let mut tpl = TopicPartitionList :: new ( ) ;
424+ tpl. add_partition_offset ( & topic, 0 , Offset :: Offset ( 2 ) )
425+ . unwrap ( ) ;
426+ tpl. add_partition_offset ( & topic, 1 , Offset :: Offset ( 1 ) )
427+ . unwrap ( ) ;
428+ let res_tpl = admin_client. delete_records ( & tpl, & opts) . await . unwrap ( ) ;
429+ assert_eq ! ( res_tpl. count( ) , 2 ) ;
430+ assert_eq ! ( res_tpl. elements( ) [ 0 ] . topic( ) , topic) ;
431+ assert_eq ! ( res_tpl. elements( ) [ 0 ] . partition( ) , 0 ) ;
432+ assert_eq ! ( res_tpl. elements( ) [ 0 ] . offset( ) , Offset :: Offset ( 2 ) ) ;
433+ assert_eq ! ( res_tpl. elements( ) [ 0 ] . error( ) , Ok ( ( ) ) ) ;
434+ assert_eq ! ( res_tpl. elements( ) [ 1 ] . topic( ) , topic) ;
435+ assert_eq ! ( res_tpl. elements( ) [ 1 ] . partition( ) , 1 ) ;
436+ assert_eq ! (
437+ res_tpl. elements( ) [ 1 ] . error( ) ,
438+ Err ( KafkaError :: OffsetFetch ( RDKafkaErrorCode :: UnknownPartition ) )
439+ ) ;
440+
441+ // Ensure that the low water mark has advanced to 2.
442+ let ( lo, hi) = producer
443+ . client ( )
444+ . fetch_watermarks ( & topic, 0 , timeout)
445+ . unwrap ( ) ;
446+ assert_eq ! ( lo, 2 ) ;
447+ assert_eq ! ( hi, 5 ) ;
448+
449+ // Delete all records up to offset 5.
450+ let mut tpl = TopicPartitionList :: new ( ) ;
451+ tpl. add_partition_offset ( & topic, 0 , Offset :: End ) . unwrap ( ) ;
452+ let res_tpl = admin_client. delete_records ( & tpl, & opts) . await . unwrap ( ) ;
453+ assert_eq ! ( res_tpl. count( ) , 1 ) ;
454+ assert_eq ! ( res_tpl. elements( ) [ 0 ] . topic( ) , topic) ;
455+ assert_eq ! ( res_tpl. elements( ) [ 0 ] . partition( ) , 0 ) ;
456+ assert_eq ! ( res_tpl. elements( ) [ 0 ] . offset( ) , Offset :: Offset ( 5 ) ) ;
457+ assert_eq ! ( res_tpl. elements( ) [ 0 ] . error( ) , Ok ( ( ) ) ) ;
458+
459+ // Ensure that the low water mark has advanced to 5.
460+ let ( lo, hi) = producer
461+ . client ( )
462+ . fetch_watermarks ( & topic, 0 , timeout)
463+ . unwrap ( ) ;
464+ assert_eq ! ( lo, 5 ) ;
465+ assert_eq ! ( hi, 5 ) ;
466+ }
467+
359468#[ tokio:: test]
360469async fn test_configs ( ) {
361470 let admin_client = create_admin_client ( ) ;
0 commit comments