risingwave_connector/source/pulsar/enumerator/
client.rs

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
// Copyright 2024 RisingWave Labs
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
//     http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

use anyhow::anyhow;
use async_trait::async_trait;
use itertools::Itertools;
use pulsar::{Pulsar, TokioExecutor};
use risingwave_common::bail;
use serde::{Deserialize, Serialize};

use crate::error::ConnectorResult;
use crate::source::pulsar::split::PulsarSplit;
use crate::source::pulsar::topic::{parse_topic, Topic};
use crate::source::pulsar::PulsarProperties;
use crate::source::{SourceEnumeratorContextRef, SplitEnumerator};

pub struct PulsarSplitEnumerator {
    client: Pulsar<TokioExecutor>,
    topic: Topic,
    start_offset: PulsarEnumeratorOffset,
}

#[derive(Debug, Clone, Eq, PartialEq, Serialize, Deserialize, Hash)]
pub enum PulsarEnumeratorOffset {
    Earliest,
    Latest,
    MessageId(String),
    Timestamp(i64),
}

#[async_trait]
impl SplitEnumerator for PulsarSplitEnumerator {
    type Properties = PulsarProperties;
    type Split = PulsarSplit;

    async fn new(
        properties: PulsarProperties,
        _context: SourceEnumeratorContextRef,
    ) -> ConnectorResult<PulsarSplitEnumerator> {
        let pulsar = properties
            .common
            .build_client(&properties.oauth, &properties.aws_auth_props)
            .await?;
        let topic = properties.common.topic;
        let parsed_topic = parse_topic(&topic)?;

        let mut scan_start_offset = match properties
            .scan_startup_mode
            .map(|s| s.to_lowercase())
            .as_deref()
        {
            Some("earliest") => PulsarEnumeratorOffset::Earliest,
            Some("latest") => PulsarEnumeratorOffset::Latest,
            None => PulsarEnumeratorOffset::Earliest,
            _ => {
                bail!(
                    "properties `startup_mode` only supports earliest and latest or leaving it empty"
                );
            }
        };

        if let Some(s) = properties.time_offset {
            let time_offset = s.parse::<i64>().map_err(|e| anyhow!(e))?;
            scan_start_offset = PulsarEnumeratorOffset::Timestamp(time_offset)
        }

        Ok(PulsarSplitEnumerator {
            client: pulsar,
            topic: parsed_topic,
            start_offset: scan_start_offset,
        })
    }

    async fn list_splits(&mut self) -> ConnectorResult<Vec<PulsarSplit>> {
        let offset = self.start_offset.clone();
        // MessageId is only used when recovering from a State
        assert!(!matches!(offset, PulsarEnumeratorOffset::MessageId(_)));

        let topic_partitions = self
            .client
            .lookup_partitioned_topic_number(&self.topic.to_string())
            .await
            .map_err(|e| anyhow!(e))?;

        let splits = if topic_partitions > 0 {
            // partitioned topic
            (0..topic_partitions as i32)
                .map(|p| PulsarSplit {
                    topic: self.topic.sub_topic(p).unwrap(),
                    start_offset: offset.clone(),
                })
                .collect_vec()
        } else {
            // non partitioned topic
            vec![PulsarSplit {
                topic: self.topic.clone(),
                start_offset: offset.clone(),
            }]
        };

        Ok(splits)
    }
}