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
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086
1087
1088
1089
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145
1146
1147
1148
1149
1150
1151
1152
1153
1154
1155
1156
1157
1158
1159
1160
1161
1162
1163
1164
1165
1166
1167
1168
1169
1170
1171
1172
1173
1174
1175
1176
1177
1178
1179
1180
1181
1182
1183
1184
1185
1186
1187
1188
1189
1190
1191
1192
1193
1194
1195
1196
1197
1198
1199
1200
1201
1202
1203
1204
1205
1206
1207
1208
1209
1210
1211
1212
1213
1214
1215
1216
1217
1218
1219
1220
1221
1222
1223
1224
1225
1226
1227
1228
1229
1230
1231
1232
1233
1234
1235
1236
1237
1238
1239
1240
1241
1242
1243
1244
1245
1246
1247
1248
1249
1250
1251
1252
1253
1254
1255
1256
1257
1258
1259
1260
1261
1262
1263
1264
1265
1266
1267
1268
1269
1270
1271
1272
1273
1274
1275
1276
1277
1278
1279
1280
1281
1282
1283
1284
1285
1286
1287
1288
1289
1290
1291
1292
1293
1294
1295
1296
1297
1298
1299
1300
1301
1302
1303
1304
1305
1306
1307
1308
1309
1310
1311
1312
1313
1314
1315
1316
1317
1318
1319
1320
1321
1322
1323
1324
1325
1326
1327
1328
1329
1330
1331
1332
1333
1334
1335
1336
1337
1338
1339
1340
1341
1342
1343
1344
1345
1346
1347
1348
1349
1350
1351
1352
1353
1354
1355
1356
1357
1358
1359
1360
1361
1362
1363
1364
1365
1366
1367
1368
1369
1370
1371
1372
1373
1374
1375
1376
1377
1378
1379
1380
1381
1382
1383
1384
1385
1386
1387
1388
1389
1390
1391
1392
1393
1394
1395
1396
1397
1398
1399
1400
1401
1402
1403
1404
1405
1406
1407
1408
1409
1410
1411
1412
1413
1414
1415
1416
1417
1418
1419
1420
1421
1422
1423
1424
1425
1426
1427
1428
1429
1430
1431
1432
1433
1434
1435
1436
1437
1438
1439
1440
1441
1442
1443
1444
1445
1446
1447
1448
1449
1450
1451
1452
1453
1454
1455
1456
1457
1458
1459
1460
1461
1462
1463
1464
1465
1466
1467
1468
1469
1470
1471
1472
1473
1474
1475
1476
1477
1478
1479
1480
1481
1482
1483
1484
1485
1486
1487
1488
1489
1490
1491
1492
1493
1494
1495
1496
1497
1498
1499
1500
1501
1502
1503
1504
1505
1506
1507
1508
1509
1510
1511
1512
1513
1514
1515
1516
1517
1518
1519
1520
1521
1522
1523
1524
1525
1526
1527
1528
1529
1530
1531
1532
1533
1534
1535
1536
1537
1538
1539
1540
1541
1542
1543
1544
1545
1546
1547
1548
1549
1550
1551
1552
1553
1554
1555
1556
1557
1558
1559
1560
1561
1562
1563
1564
1565
1566
1567
1568
1569
1570
1571
1572
1573
1574
1575
1576
1577
1578
1579
1580
1581
1582
1583
1584
1585
1586
1587
1588
1589
1590
1591
1592
1593
1594
1595
1596
1597
1598
1599
1600
1601
1602
1603
1604
1605
1606
1607
1608
1609
1610
1611
1612
1613
1614
1615
1616
1617
1618
1619
1620
1621
1622
1623
1624
1625
1626
1627
1628
1629
1630
1631
1632
1633
1634
1635
1636
1637
1638
1639
1640
1641
1642
1643
1644
1645
1646
1647
1648
1649
1650
1651
1652
1653
1654
1655
1656
1657
1658
1659
1660
1661
1662
1663
1664
1665
1666
1667
1668
1669
1670
1671
1672
1673
1674
1675
1676
1677
1678
1679
1680
1681
1682
1683
1684
1685
1686
1687
1688
1689
1690
1691
1692
1693
1694
1695
1696
1697
1698
1699
1700
1701
1702
1703
1704
1705
1706
1707
1708
1709
1710
1711
1712
1713
1714
1715
1716
1717
1718
1719
1720
1721
1722
1723
1724
1725
1726
1727
1728
1729
1730
1731
1732
1733
1734
1735
1736
1737
1738
1739
1740
1741
1742
1743
1744
1745
1746
1747
1748
1749
1750
1751
1752
1753
1754
1755
1756
1757
1758
1759
1760
1761
1762
1763
1764
1765
1766
1767
1768
1769
1770
1771
1772
1773
1774
1775
1776
1777
1778
1779
1780
1781
1782
1783
1784
1785
1786
1787
1788
1789
1790
1791
1792
1793
1794
1795
1796
1797
1798
1799
1800
1801
1802
1803
1804
1805
1806
1807
1808
1809
1810
1811
1812
1813
1814
1815
1816
1817
1818
1819
1820
1821
//! Message publication
use std::{
    collections::{btree_map::Entry, BTreeMap, HashMap, VecDeque},
    io::Write,
    pin::Pin,
    sync::{
        atomic::{AtomicU64, Ordering},
        Arc,
    },
    time::{Duration, SystemTime, UNIX_EPOCH},
};

use futures::{
    channel::{mpsc, oneshot},
    future::{self, try_join_all, Either},
    task::{Context, Poll},
    Future, SinkExt, StreamExt,
};
use rand::Rng;

use crate::{
    client::SerializeMessage,
    compression::Compression,
    connection::{Connection, SerialId},
    error::{ConnectionError, ProducerError},
    executor::Executor,
    message::{
        proto::{self, CommandSendReceipt, EncryptionKeys, Schema},
        BatchedMessage,
    },
    proto::CommandSuccess,
    retry_op::retry_create_producer,
    routing_policy::RoutingPolicy,
    BrokerAddress, Error, Pulsar,
};

type ProducerId = u64;
type ProducerName = String;

/// returned by [Producer::send]
///
/// it contains a channel on which we can await to get the message receipt.
/// Depending on the producer's configuration (batching, flow control, etc)and
/// the server's load, the send receipt could come much later after sending it
pub struct SendFuture(pub(crate) oneshot::Receiver<Result<CommandSendReceipt, Error>>);

impl Future for SendFuture {
    type Output = Result<CommandSendReceipt, Error>;

    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Self::Output> {
        match Pin::new(&mut self.0).poll(cx) {
            Poll::Ready(Ok(r)) => Poll::Ready(r),
            Poll::Ready(Err(_)) => Poll::Ready(Err(ProducerError::Custom(
                "producer unexpectedly disconnected".into(),
            )
            .into())),
            Poll::Pending => Poll::Pending,
        }
    }
}

/// message data that will be sent on a topic
///
/// generated from the [SerializeMessage] trait or [MessageBuilder]
///
/// this is actually a subset of the fields of a message, because batching,
/// compression and encryption should be handled by the producer
#[derive(Debug, Clone, Default)]
pub struct Message {
    /// serialized data
    pub payload: Vec<u8>,
    /// user defined properties
    pub properties: HashMap<String, String>,
    /// key to decide partition for the message
    pub partition_key: ::std::option::Option<String>,
    /// key to decide partition for the message
    pub ordering_key: ::std::option::Option<Vec<u8>>,
    /// Override namespace's replication
    pub replicate_to: ::std::vec::Vec<String>,
    /// the timestamp that this event occurs. it is typically set by applications.
    /// if this field is omitted, `publish_time` can be used for the purpose of `event_time`.
    pub event_time: ::std::option::Option<u64>,
    /// current version of the schema
    pub schema_version: ::std::option::Option<Vec<u8>>,
    /// UTC Unix timestamp in milliseconds, time at which the message should be
    /// delivered to consumers
    pub deliver_at_time: ::std::option::Option<i64>,
}

/// internal message type carrying options that must be defined
/// by the producer
#[derive(Debug, Clone, Default)]
pub(crate) struct ProducerMessage {
    pub payload: Vec<u8>,
    pub properties: HashMap<String, String>,
    ///key to decide partition for the msg
    pub partition_key: ::std::option::Option<String>,
    ///key to decide partition for the msg
    pub ordering_key: ::std::option::Option<Vec<u8>>,
    /// Override namespace's replication
    pub replicate_to: ::std::vec::Vec<String>,
    pub compression: ::std::option::Option<i32>,
    pub uncompressed_size: ::std::option::Option<u32>,
    /// Removed below checksum field from Metadata as
    /// it should be part of send-command which keeps checksum of header + payload
    ///optional sfixed64 checksum = 10;
    ///differentiate single and batch message metadata
    pub num_messages_in_batch: ::std::option::Option<i32>,
    pub event_time: ::std::option::Option<u64>,
    /// Contains encryption key name, encrypted key and metadata to describe the key
    pub encryption_keys: ::std::vec::Vec<EncryptionKeys>,
    /// Algorithm used to encrypt data key
    pub encryption_algo: ::std::option::Option<String>,
    /// Additional parameters required by encryption
    pub encryption_param: ::std::option::Option<Vec<u8>>,
    pub schema_version: ::std::option::Option<Vec<u8>>,
    /// UTC Unix timestamp in milliseconds, time at which the message should be
    /// delivered to consumers
    pub deliver_at_time: ::std::option::Option<i64>,
}

impl From<Message> for ProducerMessage {
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    fn from(m: Message) -> Self {
        ProducerMessage {
            payload: m.payload,
            properties: m.properties,
            partition_key: m.partition_key,
            ordering_key: m.ordering_key,
            replicate_to: m.replicate_to,
            event_time: m.event_time,
            schema_version: m.schema_version,
            deliver_at_time: m.deliver_at_time,
            ..Default::default()
        }
    }
}

/// Configuration options for producers
#[derive(Clone, Default)]
pub struct ProducerOptions {
    /// end to end message encryption (not implemented yet)
    pub encrypted: Option<bool>,
    /// user defined properties added to all messages
    pub metadata: BTreeMap<String, String>,
    /// schema used to encode this producer's messages
    pub schema: Option<Schema>,
    /// batch message size
    pub batch_size: Option<u32>,
    /// batch size in bytes treshold (only relevant when batch_size active).
    /// batch is sent when batch size in bytes is reached
    pub batch_byte_size: Option<usize>,
    /// the batch will be sent if this timeout is reached after the 1st message is added into the
    /// batch even if it does not reach the size or byte size limit.
    pub batch_timeout: Option<Duration>,
    /// algorithm used to compress the messages
    pub compression: Option<Compression>,
    /// producer access mode: shared = 0, exclusive = 1, waitforexclusive =2,
    /// exclusivewithoutfencing =3
    pub access_mode: Option<i32>,
    /// Whether to block if the internal pending queue, whose size is configured by
    /// [`crate::client::PulsarBuilder::with_outbound_channel_size`] is full, when awaiting
    /// [`Producer::send_non_blocking`]. (default: false)
    pub block_queue_if_full: bool,
    pub routing_policy: Option<RoutingPolicy>,
}

impl ProducerOptions {
    fn enabled_batching(&self) -> bool {
        match self.batch_size {
            Some(batch_size) => batch_size > 1,
            None => self.batch_byte_size.is_some() || self.batch_timeout.is_some(),
        }
    }
}

/// Wrapper structure that manges multiple producers at once, creating them as needed
/// ```rust,no_run
/// use pulsar::{Pulsar, TokioExecutor};
///
/// # async fn test() -> Result<(), pulsar::Error> {
/// # let addr = "pulsar://127.0.0.1:6650";
/// # let topic = "topic";
/// # let message = "data".to_owned();
/// let pulsar: Pulsar<_> = Pulsar::builder(addr, TokioExecutor).build().await?;
/// let mut producer = pulsar.producer().with_name("name").build_multi_topic();
/// let send_1 = producer.send_non_blocking(topic, &message).await?;
/// let send_2 = producer.send_non_blocking(topic, &message).await?;
/// send_1.await?;
/// send_2.await?;
/// # Ok(())
/// # }
/// ```
pub struct MultiTopicProducer<Exe: Executor> {
    client: Pulsar<Exe>,
    producers: BTreeMap<String, Producer<Exe>>,
    options: ProducerOptions,
    name: Option<String>,
}

impl<Exe: Executor> MultiTopicProducer<Exe> {
    /// producer options
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub fn options(&self) -> &ProducerOptions {
        &self.options
    }

    /// list topics currently handled by this producer
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub fn topics(&self) -> Vec<String> {
        self.producers.keys().cloned().collect()
    }

    /// stops the producer
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub async fn close_producer<S: Into<String>>(&mut self, topic: S) -> Result<(), Error> {
        let partitions = self.client.lookup_partitioned_topic(topic).await?;
        for (topic, _) in partitions {
            self.producers.remove(&topic);
        }
        Ok(())
    }

    /// sends one message on a topic
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    #[deprecated = "instead use send_non_blocking"]
    pub async fn send<T: SerializeMessage + Sized, S: Into<String>>(
        &mut self,
        topic: S,
        message: T,
    ) -> Result<SendFuture, Error> {
        let fut = self.send_non_blocking(topic, message).await?;
        let (tx, rx) = oneshot::channel();
        let _ = tx.send(fut.await);
        Ok(SendFuture(rx))
    }

    /// sends one message on a topic
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub async fn send_non_blocking<T: SerializeMessage + Sized, S: Into<String>>(
        &mut self,
        topic: S,
        message: T,
    ) -> Result<SendFuture, Error> {
        let message = T::serialize_message(message)?;
        let topic = topic.into();
        let producer = match self.producers.entry(topic) {
            Entry::Vacant(entry) => {
                let mut builder = self
                    .client
                    .producer()
                    .with_topic(entry.key())
                    .with_options(self.options.clone());
                if let Some(name) = &self.name {
                    builder = builder.with_name(name);
                }
                let producer = builder.build().await?;
                entry.insert(producer)
            }
            Entry::Occupied(entry) => entry.into_mut(),
        };

        producer.send_non_blocking(message).await
    }

    /// sends a list of messages on a topic
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    #[deprecated = "instead use send_all_non_blocking"]
    pub async fn send_all<'a, 'b, T, S, I>(
        &mut self,
        topic: S,
        messages: I,
    ) -> Result<Vec<SendFuture>, Error>
    where
        'b: 'a,
        T: 'b + SerializeMessage + Sized,
        I: IntoIterator<Item = T>,
        S: Into<String>,
    {
        let topic: String = topic.into();
        let mut futs = vec![];
        for message in messages {
            #[allow(deprecated)]
            let fut = self.send(&topic, message).await?;
            futs.push(fut);
        }
        Ok(futs)
    }

    /// sends a list of messages on a topic
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub async fn send_all_non_blocking<'a, 'b, T, S, I>(
        &mut self,
        topic: S,
        messages: I,
    ) -> Result<Vec<SendFuture>, Error>
    where
        'b: 'a,
        T: 'b + SerializeMessage + Sized,
        I: IntoIterator<Item = T>,
        S: Into<String>,
    {
        let topic = topic.into();
        let mut sends = Vec::new();
        for msg in messages {
            sends.push(self.send_non_blocking(&topic, msg).await);
        }
        // TODO determine whether to keep this approach or go with the partial send, but more mem
        // friendly lazy approach. serialize all messages before sending to avoid a partial
        // send
        if sends.iter().all(|s| s.is_ok()) {
            Ok(sends.into_iter().map(|s| s.unwrap()).collect())
        } else {
            Err(ProducerError::PartialSend(sends).into())
        }
    }
}

/// a producer for a single topic
pub struct Producer<Exe: Executor> {
    inner: ProducerInner<Exe>,
}

impl<Exe: Executor> Producer<Exe> {
    /// creates a producer builder from a client instance
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub fn builder(pulsar: &Pulsar<Exe>) -> ProducerBuilder<Exe> {
        ProducerBuilder::new(pulsar)
    }

    /// this producer's topic
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub fn topic(&self) -> &str {
        match &self.inner {
            ProducerInner::Single(p) => p.topic(),
            ProducerInner::Partitioned(p) => &p.topic,
        }
    }

    /// list of partitions for this producer's topic
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub fn partitions(&self) -> Option<Vec<String>> {
        match &self.inner {
            ProducerInner::Single(_) => None,
            ProducerInner::Partitioned(p) => {
                Some(p.producers.iter().map(|p| p.topic().to_owned()).collect())
            }
        }
    }

    /// configuration options
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub fn options(&self) -> &ProducerOptions {
        match &self.inner {
            ProducerInner::Single(p) => p.options(),
            ProducerInner::Partitioned(p) => &p.options,
        }
    }

    /// creates a message builder
    ///
    /// the created message will ber sent by this producer in [MessageBuilder::send]
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub fn create_message<'a>(&'a mut self) -> MessageBuilder<'a, (), Exe> {
        MessageBuilder::new(self)
    }

    /// test that the broker connections are still valid
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub async fn check_connection(&self) -> Result<(), Error> {
        match &self.inner {
            ProducerInner::Single(p) => p.check_connection().await,
            ProducerInner::Partitioned(p) => {
                try_join_all(p.producers.iter().map(|p| p.check_connection()))
                    .await
                    .map(drop)
            }
        }
    }

    /// Sends a message
    ///
    /// this function returns a `SendFuture` because the receipt can come long after
    /// this function was called, for various reasons:
    /// - the message was sent successfully but Pulsar did not send the receipt yet
    /// - the producer is batching messages, so this function must return immediately, and the
    ///   receipt will come when the batched messages are actually sent
    ///
    /// If [`ProducerOptions::block_queue_if_full`] is false (by default) and the internal pending
    /// queue is full, which means the send rate is too fast,
    /// [`crate::error::ConnectionError::SlowDown`] will be returned. You should handle the error
    /// like:
    ///
    /// ```rust,no_run
    /// use pulsar::error::{ConnectionError, Error, ProducerError};
    ///
    /// # async fn run(mut producer: pulsar::Producer<pulsar::TokioExecutor>) -> Result<(), pulsar::Error> {
    /// match producer.send_non_blocking("msg").await {
    ///     Ok(future) => { /* handle the send future */ }
    ///     Err(Error::Producer(ProducerError::Connection(ConnectionError::SlowDown))) => {
    ///         /* wait for a while and resent */
    ///     }
    ///     Err(e) => { /* handle other errors */ }
    /// }
    /// # Ok(())
    /// # }
    /// ```
    ///
    /// Usage:
    ///
    /// ```rust,no_run
    /// # async fn run(mut producer: pulsar::Producer<pulsar::TokioExecutor>) -> Result<(), pulsar::Error> {
    /// let f1 = producer.send_non_blocking("hello").await?;
    /// let f2 = producer.send_non_blocking("world").await?;
    /// let receipt1 = f1.await?;
    /// let receipt2 = f2.await?;
    /// # Ok(())
    /// # }
    /// ```
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub async fn send_non_blocking<T: SerializeMessage + Sized>(
        &mut self,
        message: T,
    ) -> Result<SendFuture, Error> {
        let serialized_message = T::serialize_message(message)?;
        match &mut self.inner {
            ProducerInner::Single(p) => p.send(serialized_message).await,
            ProducerInner::Partitioned(p) => {
                p.choose_partition(&serialized_message)
                    .send(serialized_message)
                    .await
            }
        }
    }

    /// Sends a message
    ///
    /// this function is similar to send_non_blocking then waits the returned `SendFuture`
    /// for the receipt.
    ///
    /// It returns the returned receipt in another `SendFuture` to be backward compatible.
    ///
    /// It is deprecated, and users should instread use send_non_blocking. Users should await the
    /// returned `SendFuture` if blocking is needed.
    ///
    /// Usage:
    ///
    /// ```rust,no_run
    /// # async fn run(mut producer: pulsar::Producer<pulsar::TokioExecutor>) -> Result<(), pulsar::Error> {
    /// let f1 = producer.send_non_blocking("hello").await?;
    /// let f2 = producer.send_non_blocking("world").await?;
    /// let receipt1 = f1.await?;
    /// let receipt2 = f2.await?;
    /// # Ok(())
    /// # }
    /// ```
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    #[deprecated = "instead use send_non_blocking"]
    pub async fn send<T: SerializeMessage + Sized>(
        &mut self,
        message: T,
    ) -> Result<SendFuture, Error> {
        let fut = self.send_non_blocking(message).await?;
        let (tx, rx) = oneshot::channel();
        let _ = tx.send(fut.await);
        Ok(SendFuture(rx))
    }

    /// sends a list of messages
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub async fn send_all<T, I>(&mut self, messages: I) -> Result<Vec<SendFuture>, Error>
    where
        T: SerializeMessage,
        I: IntoIterator<Item = T>,
    {
        let mut sends = Vec::new();
        for message in messages {
            let serialized_message = T::serialize_message(message)?;
            let producer = match &mut self.inner {
                ProducerInner::Single(p) => p,
                ProducerInner::Partitioned(p) => p.choose_partition(&serialized_message),
            };

            sends.push(producer.send(serialized_message).await);
        }
        if sends.iter().all(|s| s.is_ok()) {
            Ok(sends.into_iter().map(|s| s.unwrap()).collect())
        } else {
            Err(ProducerError::PartialSend(sends).into())
        }
    }

    /// sends the current batch of messages
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub async fn send_batch(&mut self) -> Result<(), Error> {
        match &mut self.inner {
            ProducerInner::Single(p) => p.send_batch().await,
            ProducerInner::Partitioned(p) => {
                try_join_all(p.producers.iter_mut().map(|p| p.send_batch()))
                    .await
                    .map(drop)
            }
        }
    }

    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub async fn close(&mut self) -> Result<(), Error> {
        match &mut self.inner {
            ProducerInner::Single(producer) => producer.close().await,
            ProducerInner::Partitioned(p) => {
                try_join_all(p.producers.iter_mut().map(|p| p.close()))
                    .await
                    .map(drop)
            }
        }
    }
}

enum ProducerInner<Exe: Executor> {
    Single(TopicProducer<Exe>),
    Partitioned(PartitionedProducer<Exe>),
}

struct PartitionedProducer<Exe: Executor> {
    // Guaranteed to be non-empty
    producers: Vec<TopicProducer<Exe>>,
    last_used_producer_index: usize,
    topic: String,
    options: ProducerOptions,
}

impl<Exe: Executor> PartitionedProducer<Exe> {
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    fn get_next_round_robin_producer(&mut self) -> &mut TopicProducer<Exe> {
        let amount_of_producers = self.producers.len();
        self.last_used_producer_index += 1;
        if self.last_used_producer_index >= amount_of_producers {
            self.last_used_producer_index = 0;
        }
        self.producers
            .get_mut(self.last_used_producer_index)
            .unwrap()
    }

    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub fn choose_partition(&mut self, message: &Message) -> &mut TopicProducer<Exe> {
        match &self.options.routing_policy {
            Some(RoutingPolicy::RoundRobin) => {
                // If the message has a partition key, use it
                if let Some(partition_key) = &message.partition_key {
                    let index = RoutingPolicy::compute_partition_index_for_key(
                        partition_key,
                        self.producers.len(),
                    );
                    return self.producers.get_mut(index).unwrap();
                }
                // If not, use round robin
                self.get_next_round_robin_producer()
            }
            Some(RoutingPolicy::Single) => self
                .producers
                .get_mut(self.last_used_producer_index)
                .unwrap(),
            Some(RoutingPolicy::Custom(policy)) => {
                let amount_of_producers = self.producers.len();
                self.producers
                    .get_mut(policy.route(message, amount_of_producers))
                    .unwrap()
            }
            None => self.get_next_round_robin_producer(),
        }
    }
}

/// a producer is used to publish messages on a topic
struct TopicProducer<Exe: Executor> {
    client: Pulsar<Exe>,
    connection: Arc<Connection<Exe>>,
    id: ProducerId,
    name: ProducerName,
    topic: String,
    batch: Option<Batch>,
    sequence_id: SerialId,
    compression: Option<Compression>,
    options: ProducerOptions,
    /// Schema version returned by the broker in `CommandProducerSuccess`.
    ///
    /// * **Non-batched path** — used as a default when the message does not
    ///   already have a `schema_version` set (`send_raw`).  Updated in-place
    ///   on reconnection via `send_message`.
    /// * **Batched path** — a clone is passed by value to the spawned
    ///   `message_send_loop`, which owns its own independent copy.  That copy
    ///   is updated on reconnection within the loop; this field is **not**
    ///   kept in sync and may go stale for batched producers.
    ///
    /// Per-message `schema_version` is not supported by the Pulsar protocol
    /// for batched messages (`SingleMessageMetadata` has no such field).
    schema_version: Option<Vec<u8>>,
}

impl<Exe: Executor> TopicProducer<Exe> {
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub(crate) async fn new<S: Into<String>>(
        client: Pulsar<Exe>,
        addr: BrokerAddress,
        topic: S,
        name: Option<String>,
        options: ProducerOptions,
    ) -> Result<Self, Error> {
        static PRODUCER_ID_GENERATOR: AtomicU64 = AtomicU64::new(0);

        let topic = topic.into();
        let producer_id = PRODUCER_ID_GENERATOR.fetch_add(1, Ordering::SeqCst);
        let sequence_id = SerialId::new();

        let topic = topic.clone();
        let compression = options.compression.clone();
        let mut connection = client.manager.get_connection(&addr).await?;

        let (producer_name, schema_version) = retry_create_producer(
            &client,
            &mut connection,
            addr,
            &topic,
            producer_id,
            name,
            &options,
        )
        .await?;

        if !options.enabled_batching() {
            return Ok(TopicProducer {
                client,
                connection,
                id: producer_id,
                name: producer_name,
                topic,
                sequence_id,
                compression,
                options,
                batch: None,
                schema_version,
            });
        }
        let executor = client.executor.clone();
        let batch_storage = BatchStorage {
            max_size: options.batch_size,
            max_byte_size: options.batch_byte_size,
            timeout: options.batch_timeout,
            size: 0,
            storage: match options.batch_size {
                Some(batch_size) => VecDeque::with_capacity(batch_size as usize),
                None => VecDeque::new(),
            },
        };
        // the message should be received quickly, so a small buffer is okay
        let (msg_sender, msg_receiver) = mpsc::channel::<BatchItem>(10);
        let executor_clone = executor.clone();
        let (batch_sender, batch_receiver) = mpsc::channel::<Vec<BatchItem>>(1);
        let (close_sender, close_receiver) =
            oneshot::channel::<Result<CommandSuccess, ConnectionError>>();

        let _ = executor.spawn(Box::pin(batch_process_loop(
            producer_id,
            batch_storage,
            msg_receiver,
            batch_sender,
            executor_clone,
        )));
        let _ = executor.spawn(Box::pin(message_send_loop(
            batch_receiver,
            close_sender,
            client.clone(),
            connection.clone(),
            topic.clone(),
            producer_id,
            producer_name.clone(),
            sequence_id.clone(),
            options.clone(),
            schema_version.clone(),
        )));

        Ok(TopicProducer {
            client,
            connection,
            id: producer_id,
            name: producer_name,
            topic,
            batch: Some(Batch {
                msg_sender,
                close_receiver,
            }),
            sequence_id,
            compression,
            options,
            schema_version,
        })
    }

    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    fn topic(&self) -> &str {
        &self.topic
    }

    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    fn options(&self) -> &ProducerOptions {
        &self.options
    }

    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    async fn check_connection(&self) -> Result<(), Error> {
        self.connection.sender().send_ping().await?;
        Ok(())
    }

    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    async fn send(&mut self, message: Message) -> Result<SendFuture, Error> {
        self.send_raw(message.into()).await
    }

    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    async fn send_batch(&mut self) -> Result<(), Error> {
        match &mut self.batch.as_mut().map(|batch| &mut batch.msg_sender) {
            Some(msg_sender) => {
                let (tx, rx) = oneshot::channel::<()>();
                let item = BatchItem::Flush(tx);
                let _ = msg_sender.send(item).await;
                let _ = rx.await; // ignore any error
                Ok(())
            }
            None if self.options.enabled_batching() => Err(ProducerError::Closed.into()),
            _ => Err(ProducerError::Custom("not a batching producer".to_string()).into()),
        }
    }

    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub(crate) async fn send_raw(
        &mut self,
        mut message: ProducerMessage,
    ) -> Result<SendFuture, Error> {
        let (tx, rx) = oneshot::channel();
        match &mut self.batch.as_mut().map(|batch| &mut batch.msg_sender) {
            Some(msg_sender) => {
                // Per-message schema_version is not supported by the Pulsar
                // protocol for batched messages. The schema_version is set
                // on the batch envelope in message_send_loop instead. Any
                // user-provided schema_version on individual messages is
                // structurally dropped here.
                let properties = message
                    .properties
                    .into_iter()
                    .map(|(key, value)| proto::KeyValue { key, value })
                    .collect();
                let batched = BatchedMessage {
                    metadata: proto::SingleMessageMetadata {
                        properties,
                        partition_key: message.partition_key,
                        ordering_key: message.ordering_key,
                        payload_size: message.payload.len() as i32,
                        event_time: message.event_time,
                        ..Default::default()
                    },
                    payload: message.payload,
                };
                let item = BatchItem::SingleMessage(tx, batched);
                msg_sender.send(item).await.map_err(|e| {
                    Error::Producer(ProducerError::Custom(format!(
                        "failed to send message to batch_process_loop: {e}"
                    )))
                })?;
            }
            None if self.options.enabled_batching() => {
                return Err(ProducerError::Closed.into());
            }
            _ => {
                // If the user didn't set a schema_version on the message,
                // use the one returned by the broker in
                // CommandProducerSuccess.
                if message.schema_version.is_none() {
                    message.schema_version = self.schema_version.clone();
                }
                let compressed_message = compress_message(message, &self.compression)?;
                let fut = send_message(
                    &self.client,
                    &self.topic,
                    &mut self.connection,
                    compressed_message,
                    self.id,
                    &self.name,
                    &self.sequence_id,
                    &self.options,
                    &mut self.schema_version,
                )
                .await?;
                self.client
                    .executor
                    .spawn(Box::pin(async move {
                        let _ = tx.send(fut.await);
                    }))
                    .map_err(|_| Error::Executor)?;
            }
        };
        Ok(SendFuture(rx))
    }

    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    async fn close(&mut self) -> Result<(), Error> {
        match self.batch.take() {
            None => {
                self.connection.sender().close_producer(self.id).await?;
            }
            Some(mut batch) if self.options.enabled_batching() => {
                batch.msg_sender.close_channel();
                let close_receiver = &mut batch.close_receiver;
                return match close_receiver.await {
                    Ok(Ok(_)) => Ok(()),
                    Ok(Err(e)) => Err(Error::Producer(ProducerError::Connection(e))),
                    Err(_) => Err(Error::Producer(ProducerError::Closed)),
                };
            }
            _ => {
                warn!(
                    "close called multiple times on producer {} for topic {}",
                    self.id, self.topic
                );
            }
        };
        Ok(())
    }
}

#[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
fn compress_message(
    mut message: ProducerMessage,
    compression: &Option<Compression>,
) -> Result<ProducerMessage, Error> {
    let compressed_message = match compression {
        None | Some(Compression::None) => message,
        #[cfg(feature = "lz4")]
        Some(Compression::Lz4(compression)) => {
            let compressed_payload: Vec<u8> =
                lz4::block::compress(&message.payload[..], Some(compression.mode), false)
                    .map_err(ProducerError::Io)?;

            message.uncompressed_size = Some(message.payload.len() as u32);
            message.payload = compressed_payload;
            message.compression = Some(proto::CompressionType::Lz4.into());
            message
        }
        #[cfg(feature = "flate2")]
        Some(Compression::Zlib(compression)) => {
            let mut e = flate2::write::ZlibEncoder::new(Vec::new(), compression.level);
            e.write_all(&message.payload[..])
                .map_err(ProducerError::Io)?;
            let compressed_payload = e.finish().map_err(ProducerError::Io)?;

            message.uncompressed_size = Some(message.payload.len() as u32);
            message.payload = compressed_payload;
            message.compression = Some(proto::CompressionType::Zlib.into());
            message
        }
        #[cfg(feature = "zstd")]
        Some(Compression::Zstd(compression)) => {
            let compressed_payload = zstd::encode_all(&message.payload[..], compression.level)
                .map_err(ProducerError::Io)?;
            message.uncompressed_size = Some(message.payload.len() as u32);
            message.payload = compressed_payload;
            message.compression = Some(proto::CompressionType::Zstd.into());
            message
        }
        #[cfg(feature = "snap")]
        Some(Compression::Snappy(..)) => {
            let mut compressed_payload = Vec::new();
            {
                let mut encoder = snap::write::FrameEncoder::new(&mut compressed_payload);
                encoder
                    .write_all(&message.payload[..])
                    .map_err(ProducerError::Io)?;
                encoder.flush().map_err(ProducerError::Io)?;
            }

            message.uncompressed_size = Some(message.payload.len() as u32);
            message.payload = compressed_payload;
            message.compression = Some(proto::CompressionType::Snappy.into());
            message
        }
    };
    Ok(compressed_message)
}

#[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
async fn send_message<Exe>(
    client: &Pulsar<Exe>,
    topic: &String,
    connection: &mut Arc<Connection<Exe>>,
    message: ProducerMessage,
    producer_id: ProducerId,
    producer_name: &ProducerName,
    sequence_id: &SerialId,
    options: &ProducerOptions,
    schema_version: &mut Option<Vec<u8>>,
) -> Result<impl Future<Output = Result<CommandSendReceipt, Error>>, Error>
where
    Exe: Executor,
{
    loop {
        // If a previous send timed out waiting for a receipt, the connection
        // is poisoned (error flag set) but the underlying TCP channel may
        // still be open.  Detect this early and fall through to reconnection
        // instead of sending into a black hole that will time out again.
        if !connection.is_valid() {
            warn!(
                "send_message: connection {} is no longer valid, reconnecting producer for topic: {}",
                connection.id(),
                topic
            );
            // fall through to reconnection below
        } else {
            match connection
                .sender()
                .send(
                    producer_id,
                    producer_name.clone(),
                    sequence_id.get(),
                    message.clone(),
                    options.block_queue_if_full,
                )
                .await
            {
                Ok(fut) => {
                    let fut = async move {
                        let res = fut.await;
                        res.map_err(|e| {
                            error!("wait send receipt got error: {:?}", e);
                            Error::Producer(ProducerError::Connection(e))
                        })
                    };
                    return Ok(fut);
                }
                Err(ConnectionError::Disconnected) => {}
                Err(ConnectionError::Io(e)) => {
                    if e.kind() != std::io::ErrorKind::TimedOut {
                        error!("send_message got io error: {:?}", e);
                        return Err(ProducerError::Connection(ConnectionError::Io(e)).into());
                    }
                }
                Err(e) => {
                    error!("send_message got error: {:?}", e);
                    return Err(ProducerError::Connection(e).into());
                }
            }
        }

        error!(
            "send_message: connection {} disconnected, reconnecting producer for topic: {}",
            connection.id(),
            &topic
        );

        if let Err(e) = connection.sender().close_producer(producer_id).await {
            error!(
                "could not close producer {:?}({}) for topic {}: {:?}",
                producer_name, producer_id, &topic, e
            );
        }

        let broker_address = client.lookup_topic(topic).await?;

        let (_producer_name, new_schema_version) = retry_create_producer(
            client,
            connection,
            broker_address,
            topic,
            producer_id,
            Some(producer_name.clone()),
            options,
        )
        .await?;

        // Update the producer-level schema_version for future messages.
        // The in-flight message keeps its original schema_version: it was
        // set before entering send_message and matches the schema used to
        // serialize its payload.
        *schema_version = new_schema_version;
    }
}

impl<Exe: Executor> std::ops::Drop for TopicProducer<Exe> {
    fn drop(&mut self) {
        let conn = self.connection.clone();
        let id = self.id;
        let name = self.name.clone();
        let topic = self.topic.clone();
        if let Some(mut batch) = self.batch.take() {
            batch.msg_sender.close_channel();
        }
        let _ = self.client.executor.spawn(Box::pin(async move {
            if let Err(e) = conn.sender().close_producer(id).await {
                error!(
                    "could not close producer {:?}({}) for topic {}: {:?}",
                    name, id, topic, e
                );
            }
        }));
    }
}

/// Helper structure to prepare a producer
///
/// generated from [Pulsar::producer]
#[derive(Clone)]
pub struct ProducerBuilder<Exe: Executor> {
    pulsar: Pulsar<Exe>,
    topic: Option<String>,
    name: Option<String>,
    producer_options: Option<ProducerOptions>,
}

impl<Exe: Executor> ProducerBuilder<Exe> {
    /// creates a new ProducerBuilder from a client
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub fn new(pulsar: &Pulsar<Exe>) -> Self {
        ProducerBuilder {
            pulsar: pulsar.clone(),
            topic: None,
            name: None,
            producer_options: None,
        }
    }

    /// sets the producer's topic
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub fn with_topic<S: Into<String>>(mut self, topic: S) -> Self {
        self.topic = Some(topic.into());
        self
    }

    /// sets the producer's name
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub fn with_name<S: Into<String>>(mut self, name: S) -> Self {
        self.name = Some(name.into());
        self
    }

    /// configuration options
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub fn with_options(mut self, options: ProducerOptions) -> Self {
        self.producer_options = Some(options);
        self
    }

    /// creates a new producer
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub async fn build(self) -> Result<Producer<Exe>, Error> {
        let ProducerBuilder {
            pulsar,
            topic,
            name,
            producer_options,
        } = self;
        let topic = topic.ok_or_else(|| Error::Custom("topic not set".to_string()))?;
        let options = producer_options.unwrap_or_default();

        let mut producers: Vec<TopicProducer<Exe>> = try_join_all(
            pulsar
                .lookup_partitioned_topic(&topic)
                .await?
                .into_iter()
                .map(|(topic, addr)| {
                    let name = name.clone();
                    let options = options.clone();
                    let pulsar = pulsar.clone();
                    async move {
                        let producer =
                            TopicProducer::new(pulsar, addr, topic, name, options).await?;
                        Ok::<TopicProducer<Exe>, Error>(producer)
                    }
                }),
        )
        .await?;

        // sort by partition id
        producers.sort_by_key(|prod| prod.id);

        let producer = match producers.len() {
            0 => {
                return Err(Error::Custom(format!(
                    "Unexpected error: Partition lookup returned no topics for {topic}"
                )));
            }
            1 => ProducerInner::Single(producers.into_iter().next().unwrap()),
            len => ProducerInner::Partitioned(PartitionedProducer {
                producers,
                last_used_producer_index: rand::thread_rng().gen_range(0..len),
                topic,
                options,
            }),
        };

        Ok(Producer { inner: producer })
    }

    /// creates a new [MultiTopicProducer]
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub fn build_multi_topic(self) -> MultiTopicProducer<Exe> {
        MultiTopicProducer {
            client: self.pulsar,
            producers: Default::default(),
            options: self.producer_options.unwrap_or_default(),
            name: self.name,
        }
    }
}

struct BatchStorage {
    max_size: Option<u32>,
    max_byte_size: Option<usize>,
    timeout: Option<Duration>,
    size: usize,
    storage: VecDeque<BatchItem>,
}

impl BatchStorage {
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub fn push_back(&mut self, item: BatchItem) {
        if let BatchItem::SingleMessage(_, batched_msg) = &item {
            self.size += batched_msg.metadata.payload_size as usize;
        }
        self.storage.push_back(item);
    }

    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub fn get_messages(&mut self) -> Vec<BatchItem> {
        self.size = 0;
        self.storage.drain(..).collect()
    }

    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub fn ready_to_flush(&self) -> bool {
        if let Some(max_size) = self.max_size {
            if self.storage.len() >= max_size as usize {
                return true;
            }
        }
        if let Some(max_byte_size) = self.max_byte_size {
            if self.size >= max_byte_size {
                return true;
            }
        }
        matches!(self.storage.back(), Some(BatchItem::Flush(_)))
    }
}

enum BatchItem {
    SingleMessage(
        oneshot::Sender<Result<CommandSendReceipt, Error>>,
        BatchedMessage,
    ),
    Flush(oneshot::Sender<()>),
}

struct Batch {
    // sends a message or trigger a flush
    msg_sender: mpsc::Sender<BatchItem>,
    // receives the notification when `bath_process_loop` is closed
    close_receiver: oneshot::Receiver<Result<CommandSuccess, ConnectionError>>,
}

#[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
async fn batch_process_loop(
    producer_id: ProducerId,
    mut batch_storage: BatchStorage,
    mut msg_receiver: mpsc::Receiver<BatchItem>,
    mut batch_sender: mpsc::Sender<Vec<BatchItem>>,
    executor: impl Executor,
) {
    let mut recv_future = msg_receiver.next();
    let mut timer_future: Pin<Box<dyn Future<Output = ()> + Send + 'static>> =
        Box::pin(future::pending());

    let flush = async |batch_sender: &mut mpsc::Sender<Vec<BatchItem>>,
                       messages: Vec<BatchItem>| {
        if !messages.is_empty() {
            let _ = batch_sender.send(messages).await;
        }
    };

    loop {
        match future::select(recv_future, timer_future).await {
            Either::Left((Some(batch_item), previous_timer_future)) => {
                batch_storage.push_back(batch_item);
                if batch_storage.ready_to_flush() {
                    flush(&mut batch_sender, batch_storage.get_messages()).await;
                    timer_future = Box::pin(future::pending());
                } else {
                    timer_future = match batch_storage.timeout {
                        Some(timeout) if batch_storage.storage.len() == 1 => {
                            Box::pin(executor.delay(timeout))
                        }
                        _ => previous_timer_future,
                    };
                }
                recv_future = msg_receiver.next();
            }
            Either::Left((None, _)) => {
                let count = batch_storage.storage.len();
                if count > 0 {
                    warn!("producer {}'s batch_process_loop exits when there are {} messages not flushed",
                        producer_id, count);
                    for item in batch_storage.get_messages() {
                        if let BatchItem::SingleMessage(tx, _) = item {
                            let _ = tx.send(Err(Error::Producer(ProducerError::Closed)));
                        }
                    }
                } else {
                    info!("producer {producer_id}'s batch_process_loop: channel closed, exiting");
                }
                break;
            }
            Either::Right((_, previous_recv_future)) => {
                if batch_storage.timeout.is_some() {
                    flush(&mut batch_sender, batch_storage.get_messages()).await;
                }
                timer_future = Box::pin(future::pending());
                recv_future = previous_recv_future;
            }
        }
    }
}

#[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
async fn message_send_loop<Exe>(
    mut msg_receiver: mpsc::Receiver<Vec<BatchItem>>,
    close_sender: oneshot::Sender<Result<CommandSuccess, ConnectionError>>,
    client: Pulsar<Exe>,
    mut connection: Arc<Connection<Exe>>,
    topic: String,
    producer_id: ProducerId,
    producer_name: ProducerName,
    sequence_id: SerialId,
    options: ProducerOptions,
    mut schema_version: Option<Vec<u8>>,
) where
    Exe: Executor,
{
    loop {
        match msg_receiver.next().await {
            Some(mut batch_items) => {
                if batch_items.is_empty() {
                    error!(
                        "producer {}'s message_send_loop received an empty batch unexpectedly",
                        producer_id
                    );
                    continue;
                }
                let mut payload: Vec<u8> = Vec::new();
                let mut receipts = Vec::new();

                let flush_tx = {
                    if let Some(BatchItem::Flush(_)) = batch_items.last() {
                        if let BatchItem::Flush(tx) = batch_items.pop().unwrap() {
                            Some(tx)
                        } else {
                            unreachable!()
                        }
                    } else {
                        None
                    }
                };
                let counter = batch_items.len();
                for item in batch_items {
                    if let BatchItem::SingleMessage(tx, batched_msg) = item {
                        receipts.push(tx);
                        batched_msg.serialize(&mut payload);
                    } else {
                        error!(
                            "producer {}'s message_send_loop received a Flush item unexpectedly",
                            producer_id
                        );
                    }
                }
                if counter == 0 {
                    if let Some(flush_tx) = flush_tx {
                        let _ = flush_tx.send(());
                    }
                    continue;
                }

                let message = ProducerMessage {
                    payload,
                    num_messages_in_batch: Some(counter as i32),
                    schema_version: schema_version.clone(),
                    ..Default::default()
                };

                trace!("sending a batched message of size {}", counter);

                let send = async || {
                    let compressed_message = compress_message(message, &options.compression)?;
                    send_message(
                        &client,
                        &topic,
                        &mut connection,
                        compressed_message,
                        producer_id,
                        &producer_name,
                        &sequence_id,
                        &options,
                        &mut schema_version,
                    )
                    .await?
                    .await
                };
                match send().await {
                    Ok(receipt) => {
                        for (batch_index, tx) in receipts.into_iter().enumerate() {
                            let mut receipt = receipt.clone();
                            if let Some(msg_id) = &mut receipt.message_id {
                                msg_id.batch_index = Some(batch_index as i32);
                                msg_id.batch_size = Some(counter as i32);
                            }
                            let _ = tx.send(Ok(receipt));
                        }
                        if let Some(flush_tx) = flush_tx {
                            let _ = flush_tx.send(());
                        }
                    }
                    Err(e) => {
                        let error = Arc::new(e);
                        for tx in receipts {
                            let _ =
                                tx.send(Err(Error::Producer(ProducerError::Batch(error.clone()))));
                        }
                    }
                };
            }
            None => {
                debug!("producer {producer_id} message_send_loop: channel closed, exiting");
                let close_result = connection.sender().close_producer(producer_id).await;
                let _ = close_sender.send(close_result).inspect_err(|e| {
                    warn!(
                        "{producer_id} could not notify the message_send_loop is closed: {:?}, the producer might be dropped without closing",
                        e
                    );
                });
                break;
            }
        }
    }
}

/// Helper structure to prepare a message
///
/// generated with [Producer::create_message]
pub struct MessageBuilder<'a, T, Exe: Executor> {
    producer: &'a mut Producer<Exe>,
    properties: HashMap<String, String>,
    partition_key: Option<String>,
    ordering_key: Option<Vec<u8>>,
    deliver_at_time: Option<i64>,
    event_time: Option<u64>,
    content: T,
}

impl<'a, Exe: Executor> MessageBuilder<'a, (), Exe> {
    /// creates a message builder from an existing producer
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub fn new(producer: &'a mut Producer<Exe>) -> Self {
        MessageBuilder {
            producer,
            properties: HashMap::new(),
            partition_key: None,
            ordering_key: None,
            deliver_at_time: None,
            event_time: None,
            content: (),
        }
    }
}

impl<'a, T, Exe: Executor> MessageBuilder<'a, T, Exe> {
    /// sets the message's content
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub fn with_content<C>(self, content: C) -> MessageBuilder<'a, C, Exe> {
        MessageBuilder {
            producer: self.producer,
            properties: self.properties,
            partition_key: self.partition_key,
            ordering_key: self.ordering_key,
            deliver_at_time: self.deliver_at_time,
            event_time: self.event_time,
            content,
        }
    }

    /// sets the message's partition key
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub fn with_partition_key<S: Into<String>>(mut self, partition_key: S) -> Self {
        self.partition_key = Some(partition_key.into());
        self
    }

    /// sets the message's ordering key for key_shared subscription
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub fn with_ordering_key<S: Into<Vec<u8>>>(mut self, ordering_key: S) -> Self {
        self.ordering_key = Some(ordering_key.into());
        self
    }

    /// sets the message's partition key
    ///
    /// this is the same as `with_partition_key`, this method is added for
    /// more consistency with other clients
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub fn with_key<S: Into<String>>(mut self, partition_key: S) -> Self {
        self.partition_key = Some(partition_key.into());
        self
    }

    /// sets a user defined property
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub fn with_property<S1: Into<String>, S2: Into<String>>(mut self, key: S1, value: S2) -> Self {
        self.properties.insert(key.into(), value.into());
        self
    }

    /// delivers the message at this date
    /// Note: The delayed and scheduled message attributes are only applied to shared subscription.
    /// With other subscription types, the messages will still be delivered immediately.
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub fn deliver_at(mut self, date: SystemTime) -> Result<Self, std::time::SystemTimeError> {
        self.deliver_at_time = Some(date.duration_since(UNIX_EPOCH)?.as_millis() as i64);
        Ok(self)
    }

    /// delays message deliver with this duration
    /// Note: The delayed and scheduled message attributes are only applied to shared subscription.
    /// With other subscription types, the messages will still be delivered immediately.
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub fn delay(mut self, delay: Duration) -> Result<Self, std::time::SystemTimeError> {
        let date = SystemTime::now() + delay;
        self.deliver_at_time = Some(date.duration_since(UNIX_EPOCH)?.as_millis() as i64);
        Ok(self)
    }

    // set the event time for a given message
    // By default, messages don't have an event time associated, while the publish
    // time will be be always present.
    // Set the event time to explicitly declare the time
    // that the event "happened", as opposed to when the message is being published.
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub fn event_time(mut self, event_time: u64) -> Self {
        self.event_time = Some(event_time);
        self
    }
}

impl<T: SerializeMessage + Sized, Exe: Executor> MessageBuilder<'_, T, Exe> {
    /// sends the message through the producer that created it
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    #[deprecated = "instead use send_non_blocking"]
    pub async fn send(self) -> Result<SendFuture, Error> {
        let fut = self.send_non_blocking().await?;
        let (tx, rx) = oneshot::channel();
        let _ = tx.send(fut.await);
        Ok(SendFuture(rx))
    }

    /// sends the message through the producer that created it
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub async fn send_non_blocking(self) -> Result<SendFuture, Error> {
        let MessageBuilder {
            producer,
            properties,
            partition_key,
            ordering_key,
            content,
            deliver_at_time,
            event_time,
        } = self;

        let mut message = T::serialize_message(content)?;
        message.properties = properties;
        message.partition_key = partition_key;
        message.ordering_key = ordering_key;
        message.event_time = event_time;
        message.deliver_at_time = deliver_at_time;
        producer.send_non_blocking(message).await
    }
}

#[cfg(test)]
mod tests {
    use futures::executor::block_on;
    use log::LevelFilter;

    use super::*;
    use crate::{
        routing_policy::CustomRoutingPolicy, test_utils, tests::TEST_LOGGER, TokioExecutor,
    };

    #[test]
    fn send_future_errors_when_sender_dropped() {
        let (tx, rx) = futures::channel::oneshot::channel::<Result<CommandSendReceipt, Error>>();
        // Drop the sender immediately to simulate an unexpected disconnect:
        drop(tx);

        let fut = SendFuture(rx);
        let err = block_on(fut).expect_err("expected an error when sender is dropped");

        // It should be mapped to a ProducerError::Custom inside Error::Producer
        match err {
            Error::Producer(ProducerError::Custom(msg)) => {
                assert!(
                    msg.contains("unexpectedly disconnected"),
                    "unexpected error message: {msg}"
                );
            }
            other => panic!("unexpected error variant: {:?}", other),
        }
    }

    #[test]
    fn message_converts_into_producer_message() {
        let mut props = HashMap::new();
        props.insert("a".to_string(), "1".to_string());
        props.insert("b".to_string(), "2".to_string());

        let m = Message {
            payload: b"hello".to_vec(),
            properties: props.clone(),
            partition_key: Some("key".into()),
            ordering_key: Some(vec![1, 2, 3]),
            replicate_to: vec!["r1".into(), "r2".into()],
            event_time: Some(42),
            schema_version: Some(vec![9, 9]),
            deliver_at_time: Some(123456789),
        };

        let pm: ProducerMessage = m.clone().into();

        assert_eq!(pm.payload, m.payload);
        assert_eq!(pm.properties, m.properties);
        assert_eq!(pm.partition_key, m.partition_key);
        assert_eq!(pm.ordering_key, m.ordering_key);
        assert_eq!(pm.replicate_to, m.replicate_to);
        assert_eq!(pm.event_time, m.event_time);
        assert_eq!(pm.schema_version, m.schema_version);
        assert_eq!(pm.deliver_at_time, m.deliver_at_time);

        // And defaults that the producer fills later:
        assert!(pm.num_messages_in_batch.is_none());
        assert!(pm.compression.is_none());
        assert!(pm.uncompressed_size.is_none());
    }

    #[tokio::test]
    async fn block_if_queue_full() {
        let _result = log::set_logger(&TEST_LOGGER);
        log::set_max_level(LevelFilter::Debug);
        let pulsar: Pulsar<_> = Pulsar::builder("pulsar://127.0.0.1:6650", TokioExecutor)
            .with_outbound_channel_size(3)
            .build()
            .await
            .unwrap();
        let mut producer = pulsar
            .producer()
            .with_topic(format!("block_queue_if_full_{}", rand::random::<u16>()))
            .build()
            .await
            .unwrap();
        let mut send_results = Vec::with_capacity(10);
        for i in 0..10 {
            send_results.push(producer.send_non_blocking(format!("msg-{i}")).await);
        }
        let mut failed_indexes = vec![];
        for (i, result) in send_results.into_iter().enumerate() {
            match result {
                Ok(_) => {}
                Err(Error::Producer(ProducerError::Connection(ConnectionError::SlowDown))) => {
                    failed_indexes.push(i);
                }
                Err(e) => panic!("failed to send {}: {}", i, e),
            }
        }
        info!("Messages failed due to SlowDown: {:?}", &failed_indexes);
        assert!(!failed_indexes.is_empty());

        let mut producer = pulsar
            .producer()
            .with_topic(format!("block_queue_if_full_{}", rand::random::<u16>()))
            .with_options(ProducerOptions {
                block_queue_if_full: true,
                ..Default::default()
            })
            .build()
            .await
            .unwrap();
        let mut send_results = Vec::with_capacity(10);
        for i in 0..10 {
            send_results.push(producer.send_non_blocking(format!("msg-{i}")).await);
        }
        for (i, result) in send_results.into_iter().enumerate() {
            match result {
                Ok(_) => {}
                Err(e) => panic!("failed to send {}: {}", i, e),
            }
        }
    }

    #[tokio::test]
    async fn move_producer_to_spawned_task() {
        let _result = log::set_logger(&TEST_LOGGER);
        log::set_max_level(LevelFilter::Debug);
        let pulsar: Pulsar<_> = Pulsar::builder("pulsar://127.0.0.1:6650", TokioExecutor)
            .with_outbound_channel_size(3)
            .build()
            .await
            .unwrap();
        let mut producer = pulsar
            .producer()
            .with_topic(format!("topic_{}", rand::random::<u16>()))
            .build()
            .await
            .unwrap();
        let (sender, receiver) = oneshot::channel();
        let _ = pulsar.executor.spawn(Box::pin(async move {
            sender.send(producer.close().await).unwrap();
        }));
        assert!(receiver.await.is_ok());
    }

    #[tokio::test]
    async fn test_round_robin_routing_policy() {
        let _result = log::set_logger(&TEST_LOGGER);
        log::set_max_level(LevelFilter::Debug);
        let pulsar: Pulsar<_> = Pulsar::builder("pulsar://127.0.0.1:6650", TokioExecutor)
            .build()
            .await
            .unwrap();
        let topic = format!("topic_{}", rand::random::<u16>());
        let options = ProducerOptions {
            routing_policy: Some(RoutingPolicy::RoundRobin),
            ..Default::default()
        };
        let partition_count = 3;
        test_utils::create_partitioned_topic("public", "default", &topic, partition_count).await;

        let mut producer = pulsar
            .producer()
            .with_topic(topic)
            .with_options(options)
            .build()
            .await
            .unwrap();

        // test round robin without key
        let message = "test".to_string();
        let mut producer_id = 0;
        for _ in 1..100 {
            let send_receipt = producer
                .send_non_blocking(&message)
                .await
                .unwrap()
                .await
                .unwrap();

            assert!(send_receipt.producer_id != producer_id);
            producer_id = send_receipt.producer_id;
        }

        // test round robin with key
        let key = "test";
        let message = Message {
            payload: "test".to_string().into(),
            partition_key: Some(key.to_string()),
            ..Default::default()
        };
        let CommandSendReceipt { producer_id, .. } = producer
            .send_non_blocking(message)
            .await
            .unwrap()
            .await
            .unwrap();
        for _ in 1..100 {
            let message = Message {
                payload: "test".to_string().into(),
                partition_key: Some(key.to_string()),
                ..Default::default()
            };

            let send_receipt = producer
                .send_non_blocking(message)
                .await
                .unwrap()
                .await
                .unwrap();

            assert!(send_receipt.producer_id == producer_id);
        }
    }

    #[tokio::test]
    async fn test_single_routing_policy() {
        let _result = log::set_logger(&TEST_LOGGER);
        log::set_max_level(LevelFilter::Debug);
        let pulsar: Pulsar<_> = Pulsar::builder("pulsar://127.0.0.1:6650", TokioExecutor)
            .build()
            .await
            .unwrap();
        let topic = format!("topic_{}", rand::random::<u16>());
        let options = ProducerOptions {
            routing_policy: Some(RoutingPolicy::Single),
            ..Default::default()
        };
        let partition_count = 3;
        test_utils::create_partitioned_topic("public", "default", &topic, partition_count).await;

        let mut producer = pulsar
            .producer()
            .with_topic(topic)
            .with_options(options)
            .build()
            .await
            .unwrap();

        let key = "test";
        let message = Message {
            payload: "test".to_string().into(),
            partition_key: Some(key.to_string()),
            ..Default::default()
        };

        let CommandSendReceipt { producer_id, .. } = producer
            .send_non_blocking(message)
            .await
            .unwrap()
            .await
            .unwrap();
        for _ in 1..100 {
            let message = Message {
                payload: "test".to_string().into(),
                partition_key: Some(key.to_string()),
                ..Default::default()
            };

            let send_receipt = producer
                .send_non_blocking(message)
                .await
                .unwrap()
                .await
                .unwrap();

            assert!(send_receipt.producer_id == producer_id);
        }
    }

    struct TestCustomRoutingPolicy {}

    impl CustomRoutingPolicy for TestCustomRoutingPolicy {
        fn route(&self, _message: &Message, _num_producers: usize) -> usize {
            1
        }
    }

    #[tokio::test]
    async fn test_custom_routing_policy() {
        let _result = log::set_logger(&TEST_LOGGER);
        log::set_max_level(LevelFilter::Debug);
        let pulsar: Pulsar<_> = Pulsar::builder("pulsar://127.0.0.1:6650", TokioExecutor)
            .build()
            .await
            .unwrap();
        let topic = format!("topic_{}", rand::random::<u16>());
        let options = ProducerOptions {
            routing_policy: Some(RoutingPolicy::Custom(Arc::new(TestCustomRoutingPolicy {}))),
            ..Default::default()
        };
        let partition_count = 3;
        test_utils::create_partitioned_topic("public", "default", &topic, partition_count).await;

        let mut producer = pulsar
            .producer()
            .with_topic(topic)
            .with_options(options)
            .build()
            .await
            .unwrap();

        let key = "test";
        let message = Message {
            payload: "test".to_string().into(),
            partition_key: Some(key.to_string()),
            ..Default::default()
        };

        let CommandSendReceipt { producer_id, .. } = producer
            .send_non_blocking(message)
            .await
            .unwrap()
            .await
            .unwrap();
        for _ in 1..100 {
            let message = Message {
                payload: "test".to_string().into(),
                partition_key: Some(key.to_string()),
                ..Default::default()
            };

            let send_receipt = producer
                .send_non_blocking(message)
                .await
                .unwrap()
                .await
                .unwrap();

            assert!(send_receipt.producer_id == producer_id);
        }
    }
}