pulsar 6.7.2

Rust client for Apache Pulsar
Documentation
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
//! # Pure Rust async await client for Apache Pulsar
//!
//! This is a pure Rust client for Apache Pulsar that does not depend on the
//! C++ Pulsar library. It provides an async/await based API, compatible with
//! [Tokio](https://tokio.rs/) and [async-std](https://async.rs/).
//!
//! Features:
//! - URL based (`pulsar://` and `pulsar+ssl://`) connections with DNS lookup
//! - multi topic consumers (based on a regex)
//! - TLS connection
//! - configurable executor (Tokio or async-std)
//! - automatic reconnection with exponential back off
//! - message batching
//! - compression with LZ4, zlib, zstd or Snappy (can be deactivated with Cargo features)
//!
//! ## Examples
//!
//! Copy this into your project's Cargo.toml:
//!
//! ```toml
//! [dependencies]
//! env_logger = "0.9"
//! pulsar = "4.1.1"
//! serde = { version = "1.0", features = ["derive"] }
//! serde_json = "1.0"
//! tokio = { version = "1.0", features = ["macros", "rt-multi-thread"] }
//! log = "0.4.6"
//! futures = "0.3"
//! ```
//!
//! ### Producing
//! ```rust,no_run
//! use pulsar::{
//!     message::proto, producer, Error as PulsarError, Pulsar, SerializeMessage, TokioExecutor,
//! };
//! use serde::{Deserialize, Serialize};
//!
//! #[derive(Serialize, Deserialize)]
//! struct TestData {
//!     data: String,
//! }
//!
//! impl SerializeMessage for TestData {
//!     fn serialize_message(input: Self) -> Result<producer::Message, PulsarError> {
//!         let payload =
//!             serde_json::to_vec(&input).map_err(|e| PulsarError::Custom(e.to_string()))?;
//!         Ok(producer::Message {
//!             payload,
//!             ..Default::default()
//!         })
//!     }
//! }
//!
//! #[tokio::main]
//! async fn main() -> Result<(), pulsar::Error> {
//!     env_logger::init();
//!
//!     let addr = "pulsar://127.0.0.1:6650";
//!     let pulsar: Pulsar<_> = Pulsar::builder(addr, TokioExecutor).build().await?;
//!     let mut producer = pulsar
//!         .producer()
//!         .with_topic("non-persistent://public/default/test")
//!         .with_name("my producer")
//!         .with_options(producer::ProducerOptions {
//!             schema: Some(proto::Schema {
//!                 r#type: proto::schema::Type::String as i32,
//!                 ..Default::default()
//!             }),
//!             ..Default::default()
//!         })
//!         .build()
//!         .await?;
//!
//!     let mut counter = 0usize;
//!     loop {
//!         producer
//!             .send(TestData {
//!                 data: "data".to_string(),
//!             })
//!             .await?;
//!
//!         counter += 1;
//!         println!("{} messages", counter);
//!         tokio::time::sleep(std::time::Duration::from_millis(2000)).await;
//!     }
//! }
//! ```
//!
//! ### Consuming
//! ```rust,no_run
//! use futures::TryStreamExt;
//! use pulsar::{
//!     message::{proto::command_subscribe::SubType, Payload},
//!     Consumer, DeserializeMessage, Pulsar, TokioExecutor,
//! };
//! use serde::{Deserialize, Serialize};
//!
//! #[derive(Serialize, Deserialize)]
//! struct TestData {
//!     data: String,
//! }
//!
//! impl DeserializeMessage for TestData {
//!     type Output = Result<TestData, serde_json::Error>;
//!
//!     fn deserialize_message(payload: &Payload) -> Self::Output {
//!         serde_json::from_slice(&payload.data)
//!     }
//! }
//!
//! #[tokio::main]
//! async fn main() -> Result<(), pulsar::Error> {
//!     env_logger::init();
//!
//!     let addr = "pulsar://127.0.0.1:6650";
//!     let pulsar: Pulsar<_> = Pulsar::builder(addr, TokioExecutor).build().await?;
//!
//!     let mut consumer: Consumer<TestData, _> = pulsar
//!         .consumer()
//!         .with_topic("test")
//!         .with_consumer_name("test_consumer")
//!         .with_subscription_type(SubType::Exclusive)
//!         .with_subscription("test_subscription")
//!         .build()
//!         .await?;
//!
//!     let mut counter = 0usize;
//!     while let Some(msg) = consumer.try_next().await? {
//!         consumer.ack(&msg).await?;
//!         let data = match msg.deserialize() {
//!             Ok(data) => data,
//!             Err(e) => {
//!                 log::error!("could not deserialize message: {:?}", e);
//!                 break;
//!             }
//!         };
//!
//!         if data.data.as_str() != "data" {
//!             log::error!("Unexpected payload: {}", &data.data);
//!             break;
//!         }
//!         counter += 1;
//!         log::info!("got {} messages", counter);
//!     }
//!
//!     Ok(())
//! }
//! ```

#![recursion_limit = "256"]
#![allow(clippy::too_many_arguments)]
#![allow(clippy::large_enum_variant)]
extern crate futures;
#[macro_use]
extern crate log;
extern crate nom;
extern crate prost_derive;

#[cfg(test)]
#[macro_use]
extern crate serde;

#[cfg(all(feature = "tokio-rustls-runtime-aws-lc-rs", feature = "tokio-runtime"))]
compile_error!("You have selected both features \"tokio-rustls-runtime-aws-lc-rs\" and \"tokio-runtime\" which are exclusive, please choose one of them");
#[cfg(all(feature = "tokio-rustls-runtime-ring", feature = "tokio-runtime"))]
compile_error!("You have selected both features \"tokio-rustls-runtime-ring\" and \"tokio-runtime\" which are exclusive, please choose one of them");
#[cfg(all(
    feature = "tokio-rustls-runtime-aws-lc-rs",
    feature = "tokio-rustls-runtime-ring"
))]
compile_error!("You have selected both features \"tokio-rustls-runtime-aws-lc-rs\" and \"tokio-rustls-runtime-ring\" which are exclusive, please choose one of them");
#[cfg(all(
    feature = "async-std-rustls-runtime-aws-lc-rs",
    feature = "async-std-runtime"
))]
compile_error!("You have selected both features \"async-std-rustls-runtime-aws-lc-rs\" and \"async-std-runtime\" which are exclusive, please choose one of them");
#[cfg(all(
    feature = "async-std-rustls-runtime-ring",
    feature = "async-std-runtime"
))]
compile_error!("You have selected both features \"async-std-rustls-runtime-ring\" and \"async-std-runtime\" which are exclusive, please choose one of them");
#[cfg(all(
    feature = "async-std-rustls-runtime-aws-lc-rs",
    feature = "async-std-rustls-runtime-ring"
))]
compile_error!("You have selected both features \"async-std-rustls-runtime-aws-lc-rs\" and \"async-std-rustls-runtime-ring\" which are exclusive, please choose one of them");

pub use client::{DeserializeMessage, Pulsar, PulsarBuilder, SerializeMessage};
pub use connection::Authentication;
pub use connection_manager::{
    BrokerAddress, ConnectionRetryOptions, OperationRetryOptions, TlsOptions,
};
pub use consumer::{Consumer, ConsumerBuilder, ConsumerOptions};
pub use error::Error;
#[cfg(any(
    feature = "async-std-runtime",
    feature = "async-std-rustls-runtime-aws-lc-rs",
    feature = "async-std-rustls-runtime-ring"
))]
pub use executor::AsyncStdExecutor;
pub use executor::Executor;
#[cfg(any(
    feature = "tokio-runtime",
    feature = "tokio-rustls-runtime-aws-lc-rs",
    feature = "tokio-rustls-runtime-ring"
))]
pub use executor::TokioExecutor;
pub use message::{
    proto::{self, command_subscribe::SubType, CommandSendReceipt},
    Payload,
};
pub use producer::{MultiTopicProducer, Producer, ProducerOptions};

pub mod authentication;
mod client;
pub mod compression;
mod connection;
mod connection_manager;
pub mod consumer;
pub mod error;
pub mod executor;
pub mod message;
pub mod producer;
pub mod reader;
mod retry_op;
pub mod routing_policy;
mod service_discovery;
mod test_utils;

#[cfg(all(
    any(
        feature = "tokio-rustls-runtime-aws-lc-rs",
        feature = "tokio-rustls-runtime-ring",
        feature = "async-std-rustls-runtime-aws-lc-rs",
        feature = "async-std-rustls-runtime-ring"
    ),
    not(any(feature = "tokio-runtime", feature = "async-std-runtime"))
))]
pub(crate) type Certificate = rustls::pki_types::CertificateDer<'static>;
#[cfg(any(feature = "tokio-runtime", feature = "async-std-runtime"))]
pub(crate) type Certificate = native_tls::Certificate;

#[cfg(test)]
mod tests {
    use std::{
        collections::BTreeSet,
        time::{Duration, Instant},
    };

    use assert_matches::assert_matches;
    use futures::{future::try_join_all, StreamExt};
    use log::{Metadata, Record};
    use serde_json::Value;
    #[cfg(any(
        feature = "tokio-runtime",
        feature = "tokio-rustls-runtime-aws-lc-rs",
        feature = "tokio-rustls-runtime-ring"
    ))]
    use tokio::time::timeout;

    use super::*;
    #[cfg(any(
        feature = "tokio-runtime",
        feature = "tokio-rustls-runtime-aws-lc-rs",
        feature = "tokio-rustls-runtime-ring"
    ))]
    use crate::{
        client::SerializeMessage,
        consumer::{InitialPosition, Message},
        error::ProducerError,
        message::{proto::command_subscribe::SubType, Payload},
        producer::SendFuture,
        proto::MessageIdData,
        Error as PulsarError,
    };

    #[derive(Debug, Serialize, Deserialize)]
    struct TestData {
        pub id: u64,
        pub data: String,
    }

    impl SerializeMessage for &TestData {
        fn serialize_message(input: Self) -> Result<producer::Message, PulsarError> {
            let payload =
                serde_json::to_vec(input).map_err(|e| PulsarError::Custom(e.to_string()))?;
            Ok(producer::Message {
                payload,
                ..Default::default()
            })
        }
    }

    impl DeserializeMessage for TestData {
        type Output = Result<TestData, serde_json::Error>;

        fn deserialize_message(payload: &Payload) -> Self::Output {
            serde_json::from_slice(&payload.data)
        }
    }

    pub struct SimpleLogger {
        pub tag: &'static str,
    }
    impl log::Log for SimpleLogger {
        fn enabled(&self, _metadata: &Metadata) -> bool {
            //metadata.level() <= Level::Info
            true
        }

        fn log(&self, record: &Record) {
            if self.enabled(record.metadata()) {
                println!(
                    "{} {} {}\t{}\t{}",
                    chrono::Utc::now(),
                    self.tag,
                    record.level(),
                    record.module_path().unwrap(),
                    record.args()
                );
            }
        }
        fn flush(&self) {}
    }

    pub static TEST_LOGGER: SimpleLogger = SimpleLogger { tag: "" };

    #[tokio::test]
    #[cfg(any(
        feature = "tokio-runtime",
        feature = "tokio-rustls-runtime-aws-lc-rs",
        feature = "tokio-rustls-runtime-ring"
    ))]
    async fn round_trip() {
        let pulsar = test_utils::new_pulsar().await;

        // random topic to better allow multiple test runs while debugging
        let topic = format!("test_{}", rand::random::<u16>());

        let mut producer = pulsar.producer().with_topic(&topic).build().await.unwrap();
        info!("producer created");

        let message_ids: BTreeSet<u64> = (0..100).collect();

        info!("will send message");
        let mut sends = Vec::new();
        for &id in &message_ids {
            let message = TestData {
                data: "data".to_string(),
                id,
            };
            sends.push(producer.send_non_blocking(&message).await.unwrap());
        }
        try_join_all(sends).await.unwrap();

        info!("sent");

        let mut consumer: Consumer<TestData, _> = pulsar
            .consumer()
            .with_topic(&topic)
            .with_consumer_name("test_consumer")
            .with_subscription_type(SubType::Exclusive)
            .with_subscription("test_subscription")
            .with_options(ConsumerOptions {
                initial_position: InitialPosition::Earliest,
                ..Default::default()
            })
            .build()
            .await
            .unwrap();

        info!("consumer created");

        let topics = consumer.topics();
        debug!("consumer connected to {:?}", topics);
        assert_eq!(topics.len(), 1);
        assert!(topics[0].ends_with(&topic));

        let mut received = BTreeSet::new();
        while let Ok(Some(msg)) = timeout(Duration::from_secs(10), consumer.next()).await {
            let msg: Message<TestData> = msg.unwrap();
            info!("id: {:?}", msg.message_id());
            received.insert(msg.deserialize().unwrap().id);
            consumer.ack(&msg).await.unwrap();
            if received.len() == message_ids.len() {
                break;
            }
        }
        assert_eq!(received.len(), message_ids.len());
        assert_eq!(received, message_ids);
    }

    #[tokio::test]
    #[cfg(any(
        feature = "tokio-runtime",
        feature = "tokio-rustls-runtime-aws-lc-rs",
        feature = "tokio-rustls-runtime-ring"
    ))]
    async fn unsized_data() {
        let pulsar = test_utils::new_pulsar().await;
        let test_id: u16 = rand::random();

        // test &str
        {
            let topic = format!("test_unsized_data_str_{test_id}");
            let send_data = "some unsized data";

            pulsar
                .send(&topic, send_data.to_string())
                .await
                .unwrap()
                .await
                .unwrap();

            let mut consumer = pulsar
                .consumer()
                .with_topic(&topic)
                .with_subscription_type(SubType::Exclusive)
                .with_subscription("test_subscription")
                .with_options(ConsumerOptions {
                    initial_position: InitialPosition::Earliest,
                    ..Default::default()
                })
                .build::<String>()
                .await
                .unwrap();

            let msg = timeout(Duration::from_secs(1), consumer.next())
                .await
                .unwrap()
                .unwrap()
                .unwrap();
            consumer.ack(&msg).await.unwrap();

            let data = msg.deserialize().unwrap();
            if data.as_str() != send_data {
                panic!("Unexpected payload in &str test: {}", &data);
            }
        }

        // test &[u8]
        {
            let topic = format!("test_unsized_data_bytes_{test_id}");
            let send_data: &[u8] = &[0, 1, 2, 3];

            pulsar
                .send(&topic, send_data.to_vec())
                .await
                .unwrap()
                .await
                .unwrap();

            let mut consumer = pulsar
                .consumer()
                .with_topic(&topic)
                .with_subscription_type(SubType::Exclusive)
                .with_subscription("test_subscription")
                .with_options(ConsumerOptions {
                    initial_position: InitialPosition::Earliest,
                    ..Default::default()
                })
                .build::<Vec<u8>>()
                .await
                .unwrap();

            let msg: Message<Vec<u8>> = timeout(Duration::from_secs(1), consumer.next())
                .await
                .unwrap()
                .unwrap()
                .unwrap();
            consumer.ack(&msg).await.unwrap();
            let data = msg.deserialize();
            if data.as_slice() != send_data {
                panic!("Unexpected payload in &[u8] test: {:?}", &data);
            }
        }
    }

    #[tokio::test]
    #[cfg(any(
        feature = "tokio-runtime",
        feature = "tokio-rustls-runtime-aws-lc-rs",
        feature = "tokio-rustls-runtime-ring"
    ))]
    async fn redelivery() {
        let pulsar = test_utils::new_pulsar().await;
        let topic = format!("test_redelivery_{}", rand::random::<u16>());
        pulsar
            .send(&topic, String::from("data"))
            .await
            .unwrap()
            .await
            .unwrap();

        let mut consumer: Consumer<String, _> = pulsar
            .consumer()
            .with_topic(topic)
            .with_unacked_message_resend_delay(Some(Duration::from_millis(100)))
            .with_options(ConsumerOptions {
                initial_position: InitialPosition::Earliest,
                ..Default::default()
            })
            .build()
            .await
            .unwrap();

        let _first_receipt = timeout(Duration::from_secs(2), consumer.next())
            .await
            .unwrap()
            .unwrap()
            .unwrap();
        let first_received = Instant::now();
        let second_receipt = timeout(Duration::from_secs(2), consumer.next())
            .await
            .unwrap()
            .unwrap()
            .unwrap();
        let redelivery = first_received.elapsed();
        consumer.ack(&second_receipt).await.unwrap();

        assert!(redelivery < Duration::from_secs(1));
    }

    const EMPTY_VALUES: Vec<String> = vec![];

    #[tokio::test]
    #[cfg(any(
        feature = "tokio-runtime",
        feature = "tokio-rustls-runtime-aws-lc-rs",
        feature = "tokio-rustls-runtime-ring"
    ))]
    async fn batching() {
        use assert_matches::assert_matches;

        let pulsar = test_utils::new_pulsar().await;
        let topic = format!("test_batching_{}", rand::random::<u16>());
        let mut consumer: Consumer<String, _> =
            pulsar.consumer().with_topic(&topic).build().await.unwrap();

        let to_strings = |v: Vec<&str>| v.iter().map(|s| s.to_string()).collect::<Vec<String>>();

        // Case 1: batching with size enabled
        let mut producer =
            create_batched_producer(pulsar.clone(), &topic, Some(5), None, None).await;
        let mut send_futures = send_messages(&mut producer, vec!["0", "1", "2", "3"]).await;
        assert_eq!(receive_messages(&mut consumer, 1, 500).await, EMPTY_VALUES);

        send_futures.append(send_messages(&mut producer, vec!["4"]).await.as_mut());
        assert_eq!(
            receive_messages(&mut consumer, 5, 500).await,
            to_strings(vec!["0", "1", "2", "3", "4"])
        );
        assert_eq!(
            get_send_msg_ids(send_futures).await,
            vec![(0, 0, 5), (0, 1, 5), (0, 2, 5), (0, 3, 5), (0, 4, 5)]
        );

        // Case 2: batching with byte size enabled
        let mut producer =
            create_batched_producer(pulsar.clone(), &topic, None, Some(500), None).await;
        let mut send_futures = send_messages(&mut producer, vec!["first"]).await;
        assert_eq!(receive_messages(&mut consumer, 1, 500).await, EMPTY_VALUES);

        let value_with_large_size = "a".repeat(500);
        send_futures.push(
            producer
                .send_non_blocking(value_with_large_size.clone())
                .await
                .unwrap(),
        );
        assert_eq!(
            receive_messages(&mut consumer, 2, 500).await,
            vec!["first".to_string(), value_with_large_size.clone()]
        );
        assert_eq!(
            get_send_msg_ids(send_futures).await,
            vec![(1, 0, 2), (1, 1, 2)]
        );

        // Case 3: batching with timeout enabled
        let mut producer = create_batched_producer(
            pulsar.clone(),
            &topic,
            None,
            None,
            Some(Duration::from_secs(1)),
        )
        .await;
        let send_futures = send_messages(&mut producer, vec!["a"]).await;
        assert_eq!(receive_messages(&mut consumer, 1, 700).await, EMPTY_VALUES);
        assert_eq!(
            receive_messages(&mut consumer, 1, 700).await,
            vec!["a".to_string()]
        );
        assert_eq!(get_send_msg_ids(send_futures).await, vec![(2, 0, 1)]);

        pulsar.executor.delay(Duration::from_millis(800)).await;
        // If the previous timer was not reset after flushing, the batched messages will be flushed
        // again 200ms later. Here we verify the batch timer will only be scheduled after the 1st
        // message is sent.
        let send_futures = send_messages(&mut producer, vec!["b"]).await;
        assert_eq!(receive_messages(&mut consumer, 1, 300).await, EMPTY_VALUES);
        assert_eq!(
            receive_messages(&mut consumer, 1, 800).await,
            vec!["b".to_string()]
        );
        assert_eq!(get_send_msg_ids(send_futures).await, vec![(3, 0, 1)]);

        // Case 4: batching with multiple limitations
        let mut producer = create_batched_producer(
            pulsar.clone(),
            &topic,
            Some(3),
            Some(500),
            Some(Duration::from_secs(1)),
        )
        .await;
        // size limit reached
        let send_futures = send_messages(&mut producer, vec!["a", "b", "c"]).await;
        assert_eq!(
            receive_messages(&mut consumer, 3, 500).await,
            to_strings(vec!["a", "b", "c"])
        );
        assert_eq!(
            get_send_msg_ids(send_futures).await,
            vec![(4, 0, 3), (4, 1, 3), (4, 2, 3)]
        );
        // byte size limit reached
        let send_receipt = producer
            .send_non_blocking(value_with_large_size.clone())
            .await
            .unwrap();
        assert_eq!(
            receive_messages(&mut consumer, 3, 500).await,
            vec![value_with_large_size.clone()]
        );
        assert_eq!(get_send_msg_ids(vec![send_receipt]).await, vec![(5, 0, 1)]);
        // timeout reached
        let send_futures = send_messages(&mut producer, vec!["d", "e"]).await;
        assert_eq!(
            receive_messages(&mut consumer, 2, 1300).await,
            to_strings(vec!["d", "e"])
        );
        assert_eq!(
            get_send_msg_ids(send_futures).await,
            vec![(6, 0, 2), (6, 1, 2)]
        );

        // send operations after close will fail
        let _ = producer.close().await;
        let error = producer.send_non_blocking("msg").await.err().unwrap();
        assert_matches!(error, PulsarError::Producer(ProducerError::Closed));
    }

    #[tokio::test]
    #[cfg(any(
        feature = "tokio-runtime",
        feature = "tokio-rustls-runtime-aws-lc-rs",
        feature = "tokio-rustls-runtime-ring"
    ))]
    async fn flush() {
        let pulsar = test_utils::new_pulsar().await;
        let topic = format!("test_flush_{}", rand::random::<u16>());
        let mut consumer: Consumer<String, _> =
            pulsar.consumer().with_topic(&topic).build().await.unwrap();
        let mut producer =
            create_batched_producer(pulsar.clone(), &topic, Some(100), None, None).await;
        let send_futures = send_messages(&mut producer, vec!["0", "1", "2"]).await;
        assert_eq!(receive_messages(&mut consumer, 1, 500).await, EMPTY_VALUES);

        producer.send_batch().await.unwrap();
        // The send futures should be all completed after flush
        let msg_ids: Vec<(u64, i32, i32)> = send_futures
            .into_iter()
            .map(|mut future| {
                future
                    .0
                    .try_recv()
                    .unwrap()
                    .unwrap()
                    .unwrap()
                    .message_id
                    .unwrap()
            })
            .map(|msg_id| msg_id_to_tuple(&msg_id))
            .collect();
        assert_eq!(msg_ids, vec![(0, 0, 3), (0, 1, 3), (0, 2, 3)]);

        // Flush 0 messages should be ok
        producer.send_batch().await.unwrap();

        assert!(!is_publishers_empty(&topic).await);

        let send_futures = send_messages(&mut producer, vec!["3", "4"]).await;
        producer.close().await.unwrap();
        // After the CloseProducer RPC is done, the producer should be removed from the publishers
        // list in topic stats
        assert!(is_publishers_empty(&topic).await);

        for send_future in send_futures {
            let error = send_future.await.err().unwrap();
            assert_matches!(error, PulsarError::Producer(ProducerError::Closed));
        }

        let error = producer.send_batch().await.err().unwrap();
        assert_matches!(error, PulsarError::Producer(ProducerError::Closed));
    }

    async fn create_batched_producer<Exe>(
        pulsar: Pulsar<Exe>,
        topic: &str,
        batch_size: Option<u32>,
        batch_byte_size: Option<usize>,
        batch_timeout: Option<Duration>,
    ) -> Producer<Exe>
    where
        Exe: Executor,
    {
        pulsar
            .producer()
            .with_topic(topic)
            .with_options(ProducerOptions {
                batch_size,
                batch_byte_size,
                batch_timeout,
                ..Default::default()
            })
            .build()
            .await
            .unwrap()
    }

    async fn get_send_msg_ids(send_futures: Vec<SendFuture>) -> Vec<(u64, i32, i32)> {
        timeout(Duration::from_millis(100), try_join_all(send_futures))
            .await
            .unwrap()
            .unwrap()
            .into_iter()
            .map(|receipt| {
                let msg_id = receipt.message_id.unwrap();
                msg_id_to_tuple(&msg_id)
            })
            .collect()
    }

    fn msg_id_to_tuple(msg_id: &MessageIdData) -> (u64, i32, i32) {
        (
            msg_id.entry_id,
            msg_id.batch_index.unwrap_or(-1),
            msg_id.batch_size.unwrap_or(-1),
        )
    }

    async fn send_messages(
        producer: &mut Producer<impl Executor>,
        values: Vec<&str>,
    ) -> Vec<SendFuture> {
        let mut send_receipts = Vec::new();
        for v in values {
            send_receipts.push(producer.send_non_blocking(v.to_string()).await.unwrap());
        }
        send_receipts
    }

    async fn receive_messages(
        consumer: &mut Consumer<String, impl Executor>,
        max_num_messages: usize,
        receive_timeout_ms: u64,
    ) -> Vec<String> {
        let mut actual_values = Vec::new();
        loop {
            match timeout(Duration::from_millis(receive_timeout_ms), consumer.next()).await {
                Ok(Some(msg)) => {
                    let msg = msg.unwrap();
                    consumer.ack(&msg).await.unwrap();
                    let data = msg.deserialize().unwrap();
                    actual_values.push(data);
                    if actual_values.len() >= max_num_messages {
                        break;
                    }
                }
                Ok(None) => break,
                Err(e) => {
                    info!("timed out waiting for messages: {}", e);
                    break;
                }
            }
        }
        actual_values
    }

    async fn is_publishers_empty(topic: &str) -> bool {
        let stats_url =
            format!("http://127.0.0.1:8080/admin/v2/persistent/public/default/{topic}/stats");
        let response = reqwest::get(stats_url).await.unwrap();
        assert!(response.status().is_success());
        let json_value: Value = response.json().await.unwrap();
        if let Some(publishers) = json_value.get("publishers") {
            let publishers = publishers.as_array().unwrap();
            publishers.is_empty()
        } else {
            panic!("No publishers in the stats");
        }
    }

    #[tokio::test]
    async fn flush_on_partitioned_topic() {
        let pulsar = test_utils::new_pulsar().await;
        let topic = format!("test_flush_on_part_topic_{}", rand::random::<u16>());

        const NUM_PARTITIONS: u32 = 2;
        test_utils::create_partitioned_topic("public", "default", &topic, NUM_PARTITIONS).await;
        let mut producer =
            create_batched_producer(pulsar.clone(), &topic, Some(2), None, None).await;
        let send_future = producer.send_non_blocking("msg").await.unwrap();
        producer.send_batch().await.unwrap();

        let msg_id = send_future.await.unwrap().message_id.unwrap();
        for i in 0..NUM_PARTITIONS {
            let mut reader = pulsar
                .reader()
                .with_topic(format!("{}-partition-{}", topic, i))
                .build::<String>()
                .await
                .unwrap();
            let last_msg_ids = reader.get_last_message_id().await.unwrap();
            let last_msg_id = last_msg_ids.first().unwrap();
            if last_msg_id.ledger_id != u64::MAX && last_msg_id.entry_id != u64::MAX {
                assert_eq!(last_msg_id.ledger_id, msg_id.ledger_id);
                assert_eq!(last_msg_id.entry_id, msg_id.entry_id);
            }
        }
    }
}