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
use std::{string::FromUtf8Error, sync::Arc};

use futures::{
    channel::{mpsc, oneshot},
    future::{select, Either},
    lock::Mutex,
    pin_mut, StreamExt,
};

use crate::{
    connection::Authentication,
    connection_manager::{
        BrokerAddress, ConnectionManager, ConnectionRetryOptions, OperationRetryOptions, TlsOptions,
    },
    consumer::{ConsumerBuilder, ConsumerOptions, InitialPosition},
    error::{ConnectionError, Error},
    executor::Executor,
    message::{
        proto::{self, CommandSendReceipt},
        Payload,
    },
    producer::{self, ProducerBuilder, SendFuture},
    service_discovery::ServiceDiscovery,
};

/// Helper trait for consumer deserialization
pub trait DeserializeMessage {
    /// type produced from the message
    type Output: Sized;
    /// deserialize method that will be called by the consumer
    fn deserialize_message(payload: &Payload) -> Self::Output;
}

impl DeserializeMessage for Vec<u8> {
    type Output = Self;

    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    fn deserialize_message(payload: &Payload) -> Self::Output {
        payload.data.to_vec()
    }
}

impl DeserializeMessage for String {
    type Output = Result<String, FromUtf8Error>;

    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    fn deserialize_message(payload: &Payload) -> Self::Output {
        String::from_utf8(payload.data.to_vec())
    }
}

/// Helper trait for message serialization
pub trait SerializeMessage {
    /// serialize method that will be called by the producer
    fn serialize_message(input: Self) -> Result<producer::Message, Error>;
}

impl SerializeMessage for producer::Message {
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    fn serialize_message(input: Self) -> Result<producer::Message, Error> {
        Ok(input)
    }
}

impl SerializeMessage for () {
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    fn serialize_message(_input: Self) -> Result<producer::Message, Error> {
        Ok(producer::Message {
            ..Default::default()
        })
    }
}

impl<'a> SerializeMessage for &'a [u8] {
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    fn serialize_message(input: Self) -> Result<producer::Message, Error> {
        Ok(producer::Message {
            payload: input.to_vec(),
            ..Default::default()
        })
    }
}

impl SerializeMessage for Vec<u8> {
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    fn serialize_message(input: Self) -> Result<producer::Message, Error> {
        Ok(producer::Message {
            payload: input,
            ..Default::default()
        })
    }
}

impl SerializeMessage for String {
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    fn serialize_message(input: Self) -> Result<producer::Message, Error> {
        let payload = input.into_bytes();
        Ok(producer::Message {
            payload,
            ..Default::default()
        })
    }
}

impl<'a> SerializeMessage for &'a String {
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    fn serialize_message(input: Self) -> Result<producer::Message, Error> {
        let payload = input.as_bytes().to_vec();
        Ok(producer::Message {
            payload,
            ..Default::default()
        })
    }
}

impl<'a> SerializeMessage for &'a str {
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    fn serialize_message(input: Self) -> Result<producer::Message, Error> {
        let payload = input.as_bytes().to_vec();
        Ok(producer::Message {
            payload,
            ..Default::default()
        })
    }
}

/// Pulsar client
///
/// This is the starting point of this API, used to create connections, producers and consumers
///
/// While methods are provided to create the client, producers and consumers directly,
/// the builders should be used for more clarity:
///
/// ```rust,no_run
/// use pulsar::{Pulsar, TokioExecutor};
///
/// # async fn run(auth: pulsar::Authentication, retry: pulsar::ConnectionRetryOptions) -> Result<(), pulsar::Error> {
/// let addr = "pulsar://127.0.0.1:6650";
/// // you can indicate which executor you use as the return type of client creation
/// let pulsar: Pulsar<_> = Pulsar::builder(addr, TokioExecutor)
///     .with_auth(auth)
///     .with_connection_retry_options(retry)
///     .build()
///     .await?;
///
/// let mut producer = pulsar
///     .producer()
///     .with_topic("non-persistent://public/default/test")
///     .with_name("my producer")
///     .build()
///     .await?;
/// # Ok(())
/// # }
/// ```
#[derive(Clone)]
pub struct Pulsar<Exe: Executor> {
    pub(crate) manager: Arc<ConnectionManager<Exe>>,
    service_discovery: Arc<ServiceDiscovery<Exe>>,
    // this field is an Option to avoid a cyclic dependency between Pulsar
    // and run_producer: the run_producer loop needs a client to create
    // a multitopic producer, this producer stores internally a copy
    // of the Pulsar struct. So even if we drop the main Pulsar instance,
    // the run_producer loop still lives because it contains a copy of
    // the sender it waits on.
    // o,solve this, we create a client without this sender, use it in
    // run_producer, then fill in the producer field afterwards in the
    // main Pulsar instance
    producer: Option<mpsc::UnboundedSender<SendMessage>>,
    pub(crate) operation_retry_options: OperationRetryOptions,
    pub(crate) executor: Arc<Exe>,
}

impl<Exe: Executor> Pulsar<Exe> {
    /// creates a new client
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub(crate) async fn new<S: Into<String>>(
        url: S,
        auth: Option<Arc<Mutex<Box<dyn crate::authentication::Authentication>>>>,
        connection_retry_parameters: Option<ConnectionRetryOptions>,
        operation_retry_parameters: Option<OperationRetryOptions>,
        tls_options: Option<TlsOptions>,
        executor: Exe,
    ) -> Result<Self, Error> {
        let url: String = url.into();
        let executor = Arc::new(executor);
        let operation_retry_options = operation_retry_parameters.unwrap_or_default();
        let manager = ConnectionManager::new(
            url,
            auth,
            connection_retry_parameters,
            operation_retry_options.clone(),
            tls_options,
            executor.clone(),
        )
        .await?;
        let manager = Arc::new(manager);

        // set up a regular connection check
        let weak_manager = Arc::downgrade(&manager);
        let mut interval = executor.interval(std::time::Duration::from_secs(60));
        let res = executor.spawn(Box::pin(async move {
            while let Some(()) = interval.next().await {
                if let Some(strong_manager) = weak_manager.upgrade() {
                    strong_manager.check_connections().await;
                } else {
                    // if all the strong references to the manager were dropped,
                    // we can stop the task
                    break;
                }
            }
        }));
        if res.is_err() {
            error!("the executor could not spawn the check connection task");
            return Err(crate::error::ConnectionError::Shutdown.into());
        }

        let service_discovery = Arc::new(ServiceDiscovery::with_manager(manager.clone()));
        let (producer, producer_rx) = mpsc::unbounded();

        let mut client = Pulsar {
            manager,
            service_discovery,
            producer: None,
            operation_retry_options,
            executor,
        };

        let _ = client
            .executor
            .spawn(Box::pin(run_producer(client.clone(), producer_rx)));
        client.producer = Some(producer);
        Ok(client)
    }

    /// creates a new client builder
    ///
    /// ```rust,no_run
    /// use pulsar::{Pulsar, TokioExecutor};
    ///
    /// # async fn run() -> Result<(), pulsar::Error> {
    /// let addr = "pulsar://127.0.0.1:6650";
    /// // you can indicate which executor you use as the return type of client creation
    /// let pulsar: Pulsar<_> = Pulsar::builder(addr, TokioExecutor).build().await?;
    /// # Ok(())
    /// # }
    /// ```
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub fn builder<S: Into<String>>(url: S, executor: Exe) -> PulsarBuilder<Exe> {
        PulsarBuilder {
            url: url.into(),
            auth_provider: None,
            connection_retry_options: None,
            operation_retry_options: None,
            tls_options: None,
            executor,
        }
    }

    /// creates a consumer builder
    ///
    /// ```rust,no_run
    /// use pulsar::{SubType, Consumer};
    ///
    /// # async fn run(pulsar: pulsar::Pulsar<pulsar::TokioExecutor>) -> Result<(), pulsar::Error> {
    /// # type TestData = String;
    /// let mut consumer: Consumer<TestData, _> = pulsar
    ///     .consumer()
    ///     .with_topic("non-persistent://public/default/test")
    ///     .with_consumer_name("test_consumer")
    ///     .with_subscription_type(SubType::Exclusive)
    ///     .with_subscription("test_subscription")
    ///     .build()
    ///     .await?;
    /// # Ok(())
    /// # }
    /// ```
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub fn consumer(&self) -> ConsumerBuilder<Exe> {
        ConsumerBuilder::new(self)
    }

    /// creates a producer builder
    ///
    /// ```rust,no_run
    /// # async fn run(pulsar: pulsar::Pulsar<pulsar::TokioExecutor>) -> Result<(), pulsar::Error> {
    /// let mut producer = pulsar
    ///     .producer()
    ///     .with_topic("non-persistent://public/default/test")
    ///     .with_name("my producer")
    ///     .build()
    ///     .await?;
    /// # Ok(())
    /// # }
    /// ```
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub fn producer(&self) -> ProducerBuilder<Exe> {
        ProducerBuilder::new(self)
    }

    /// creates a reader builder
    /// ```rust, no_run
    /// use pulsar::reader::Reader;
    ///
    /// # async fn run(pulsar: pulsar::Pulsar<pulsar::TokioExecutor>) -> Result<(), pulsar::Error> {
    /// # type TestData = String;
    /// let mut reader: Reader<TestData, _> = pulsar
    ///     .reader()
    ///     .with_topic("non-persistent://public/default/test")
    ///     .with_consumer_name("my_reader")
    ///     .into_reader()
    ///     .await?;
    /// # Ok(())
    /// # }
    /// ```
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub fn reader(&self) -> ConsumerBuilder<Exe> {
        // this makes it exactly the same like the consumer() method though
        ConsumerBuilder::new(self).with_options(
            ConsumerOptions::default()
                .durable(false)
                .with_initial_position(InitialPosition::Latest),
        )
    }

    /// gets the address of a broker handling the topic
    ///
    /// ```rust,no_run
    /// # async fn run(pulsar: pulsar::Pulsar<pulsar::TokioExecutor>) -> Result<(), pulsar::Error> {
    /// let broker_address = pulsar.lookup_topic("persistent://public/default/test").await?;
    /// # Ok(())
    /// # }
    /// ```
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub async fn lookup_topic<S: Into<String>>(&self, topic: S) -> Result<BrokerAddress, Error> {
        self.service_discovery
            .lookup_topic(topic)
            .await
            .map_err(|e| e.into())
    }

    /// gets the number of partitions for a partitioned topic
    ///
    /// ```rust,no_run
    /// # async fn run(pulsar: pulsar::Pulsar<pulsar::TokioExecutor>) -> Result<(), pulsar::Error> {
    /// let nb = pulsar.lookup_partitioned_topic_number("persistent://public/default/test").await?;
    /// # Ok(())
    /// # }
    /// ```
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub async fn lookup_partitioned_topic_number<S: Into<String>>(
        &self,
        topic: S,
    ) -> Result<u32, Error> {
        self.service_discovery
            .lookup_partitioned_topic_number(topic)
            .await
            .map_err(|e| e.into())
    }

    /// gets the address of brokers handling the topic's partitions. If the topic is not
    /// a partitioned topic, result will be a single element containing the topic and address
    /// of the non-partitioned topic provided.
    ///
    /// ```rust,no_run
    /// # async fn run(pulsar: pulsar::Pulsar<pulsar::TokioExecutor>) -> Result<(), pulsar::Error> {
    /// let broker_addresses = pulsar.lookup_partitioned_topic("persistent://public/default/test").await?;
    /// # Ok(())
    /// # }
    /// ```
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub async fn lookup_partitioned_topic<S: Into<String>>(
        &self,
        topic: S,
    ) -> Result<Vec<(String, BrokerAddress)>, Error> {
        self.service_discovery
            .lookup_partitioned_topic(topic)
            .await
            .map_err(|e| e.into())
    }

    /// gets the list of topics from a namespace
    ///
    /// ```rust,no_run
    /// use pulsar::message::proto::command_get_topics_of_namespace::Mode;
    ///
    /// # async fn run(pulsar: pulsar::Pulsar<pulsar::TokioExecutor>) -> Result<(), pulsar::Error> {
    /// let topics = pulsar.get_topics_of_namespace("public/default".to_string(), Mode::Persistent).await?;
    /// # Ok(())
    /// # }
    /// ```
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub async fn get_topics_of_namespace(
        &self,
        namespace: String,
        mode: proto::command_get_topics_of_namespace::Mode,
    ) -> Result<Vec<String>, Error> {
        let conn = self.manager.get_base_connection().await?;
        let topics = conn
            .sender()
            .get_topics_of_namespace(namespace, mode)
            .await?;
        Ok(topics.topics)
    }

    /// Sends a message on a topic.
    ///
    /// This function will lazily initialize and re-use producers as needed. For better
    /// control over producers, creating and using a `Producer` is recommended.
    ///
    /// ```rust,no_run
    /// use pulsar::message::proto::command_get_topics_of_namespace::Mode;
    ///
    /// # async fn run(pulsar: pulsar::Pulsar<pulsar::TokioExecutor>) -> Result<(), pulsar::Error> {
    /// let topics = pulsar.send("persistent://public/default/test", "hello world!").await?;
    /// # Ok(())
    /// # }
    /// ```
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub async fn send<S: Into<String>, M: SerializeMessage + Sized>(
        &self,
        topic: S,
        message: M,
    ) -> Result<SendFuture, Error> {
        let message = M::serialize_message(message)?;
        self.send_raw(message, topic).await
    }

    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    async fn send_raw<S: Into<String>>(
        &self,
        message: producer::Message,
        topic: S,
    ) -> Result<SendFuture, Error> {
        let (resolver, future) = oneshot::channel();
        self.producer
            .as_ref()
            .expect("a client without the producer channel should only be used internally")
            .unbounded_send(SendMessage {
                topic: topic.into(),
                message,
                resolver,
            })
            .map_err(|_| Error::Custom("producer unexpectedly disconnected".into()))?;
        Ok(SendFuture(future))
    }
}

/// Helper structure to generate a [Pulsar] client
pub struct PulsarBuilder<Exe: Executor> {
    url: String,
    auth_provider: Option<Box<dyn crate::authentication::Authentication>>,
    connection_retry_options: Option<ConnectionRetryOptions>,
    operation_retry_options: Option<OperationRetryOptions>,
    tls_options: Option<TlsOptions>,
    executor: Exe,
}

impl<Exe: Executor> PulsarBuilder<Exe> {
    /// Authentication parameters (JWT, Biscuit, etc)
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub fn with_auth(self, auth: Authentication) -> Self {
        self.with_auth_provider(Box::new(auth))
    }

    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub fn with_auth_provider(
        mut self,
        auth: Box<dyn crate::authentication::Authentication>,
    ) -> Self {
        self.auth_provider = Some(auth);
        self
    }

    /// Exponential back off parameters for automatic reconnection
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub fn with_connection_retry_options(
        mut self,
        connection_retry_options: ConnectionRetryOptions,
    ) -> Self {
        self.connection_retry_options = Some(connection_retry_options);
        self
    }

    /// Retry parameters for Pulsar operations
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub fn with_operation_retry_options(
        mut self,
        operation_retry_options: OperationRetryOptions,
    ) -> Self {
        self.operation_retry_options = Some(operation_retry_options);
        self
    }

    /// add a custom certificate chain to authenticate the server in TLS connections
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub fn with_certificate_chain(mut self, certificate_chain: Vec<u8>) -> Self {
        match &mut self.tls_options {
            Some(tls) => tls.certificate_chain = Some(certificate_chain),
            None => {
                self.tls_options = Some(TlsOptions {
                    certificate_chain: Some(certificate_chain),
                    ..Default::default()
                })
            }
        }
        self
    }

    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub fn with_allow_insecure_connection(mut self, allow: bool) -> Self {
        match &mut self.tls_options {
            Some(tls) => tls.allow_insecure_connection = allow,
            None => {
                self.tls_options = Some(TlsOptions {
                    allow_insecure_connection: allow,
                    ..Default::default()
                })
            }
        }
        self
    }

    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub fn with_tls_hostname_verification_enabled(mut self, enabled: bool) -> Self {
        match &mut self.tls_options {
            Some(tls) => tls.tls_hostname_verification_enabled = enabled,
            None => {
                self.tls_options = Some(TlsOptions {
                    tls_hostname_verification_enabled: enabled,
                    ..Default::default()
                })
            }
        }
        self
    }

    /// add a custom certificate chain from a file to authenticate the server in TLS connections
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub fn with_certificate_chain_file<P: AsRef<std::path::Path>>(
        self,
        path: P,
    ) -> Result<Self, std::io::Error> {
        use std::io::Read;

        let mut file = std::fs::File::open(path)?;
        let mut v = vec![];
        file.read_to_end(&mut v)?;

        Ok(self.with_certificate_chain(v))
    }

    /// creates the Pulsar client and connects it
    #[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
    pub async fn build(self) -> Result<Pulsar<Exe>, Error> {
        let PulsarBuilder {
            url,
            auth_provider,
            connection_retry_options,
            operation_retry_options,
            tls_options,
            executor,
        } = self;

        Pulsar::new(
            url,
            auth_provider.map(|p| Arc::new(Mutex::new(p))),
            connection_retry_options,
            operation_retry_options,
            tls_options,
            executor,
        )
        .await
    }
}

struct SendMessage {
    topic: String,
    message: producer::Message,
    resolver: oneshot::Sender<Result<CommandSendReceipt, Error>>,
}

#[cfg_attr(feature = "telemetry", tracing::instrument(skip_all))]
async fn run_producer<Exe: Executor>(
    client: Pulsar<Exe>,
    mut messages: mpsc::UnboundedReceiver<SendMessage>,
) {
    let mut producer = client.producer().build_multi_topic();
    while let Some(SendMessage {
        topic,
        message: payload,
        resolver,
    }) = messages.next().await
    {
        match producer.send_non_blocking(topic, payload).await {
            Ok(send_f) => {
                let delay_f = client
                    .executor
                    .delay(client.operation_retry_options.operation_timeout);

                let _ = client.executor.spawn(Box::pin(async move {
                    pin_mut!(delay_f);
                    match select(send_f, delay_f).await {
                        Either::Left((res, _)) => {
                            let _ = resolver.send(res);
                        }
                        Either::Right(_) => {
                            let _ = resolver.send(Err(Error::from(ConnectionError::Io(
                                std::io::Error::new(
                                    std::io::ErrorKind::TimedOut,
                                    "client sink timed out when sending message to the Pulsar server",
                                ),
                            ))));
                        }
                    }
                }));
            }
            Err(e) => {
                let _ = resolver.send(Err(e));
            }
        }
    }
}