risingwave_connector/source/
mod.rs

1// Copyright 2025 RisingWave Labs
2//
3// Licensed under the Apache License, Version 2.0 (the "License");
4// you may not use this file except in compliance with the License.
5// You may obtain a copy of the License at
6//
7//     http://www.apache.org/licenses/LICENSE-2.0
8//
9// Unless required by applicable law or agreed to in writing, software
10// distributed under the License is distributed on an "AS IS" BASIS,
11// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12// See the License for the specific language governing permissions and
13// limitations under the License.
14
15pub mod prelude {
16    // import all split enumerators
17    pub use crate::source::datagen::DatagenSplitEnumerator;
18    pub use crate::source::filesystem::LegacyS3SplitEnumerator;
19    pub use crate::source::filesystem::opendal_source::OpendalEnumerator;
20    pub use crate::source::google_pubsub::PubsubSplitEnumerator as GooglePubsubSplitEnumerator;
21    pub use crate::source::iceberg::IcebergSplitEnumerator;
22    pub use crate::source::kafka::KafkaSplitEnumerator;
23    pub use crate::source::kinesis::KinesisSplitEnumerator;
24    pub use crate::source::mqtt::MqttSplitEnumerator;
25    pub use crate::source::nats::NatsSplitEnumerator;
26    pub use crate::source::nexmark::NexmarkSplitEnumerator;
27    pub use crate::source::pulsar::PulsarSplitEnumerator;
28    pub use crate::source::test_source::TestSourceSplitEnumerator as TestSplitEnumerator;
29    pub type AzblobSplitEnumerator =
30        OpendalEnumerator<crate::source::filesystem::opendal_source::OpendalAzblob>;
31    pub type GcsSplitEnumerator =
32        OpendalEnumerator<crate::source::filesystem::opendal_source::OpendalGcs>;
33    pub type OpendalS3SplitEnumerator =
34        OpendalEnumerator<crate::source::filesystem::opendal_source::OpendalS3>;
35    pub type PosixFsSplitEnumerator =
36        OpendalEnumerator<crate::source::filesystem::opendal_source::OpendalPosixFs>;
37    pub use crate::source::cdc::enumerator::DebeziumSplitEnumerator;
38    pub use crate::source::filesystem::opendal_source::BatchPosixFsEnumerator as BatchPosixFsSplitEnumerator;
39    pub type CitusCdcSplitEnumerator = DebeziumSplitEnumerator<crate::source::cdc::Citus>;
40    pub type MongodbCdcSplitEnumerator = DebeziumSplitEnumerator<crate::source::cdc::Mongodb>;
41    pub type PostgresCdcSplitEnumerator = DebeziumSplitEnumerator<crate::source::cdc::Postgres>;
42    pub type MysqlCdcSplitEnumerator = DebeziumSplitEnumerator<crate::source::cdc::Mysql>;
43    pub type SqlServerCdcSplitEnumerator = DebeziumSplitEnumerator<crate::source::cdc::SqlServer>;
44}
45
46pub mod base;
47pub mod batch;
48pub mod cdc;
49pub mod data_gen_util;
50pub mod datagen;
51pub mod filesystem;
52pub mod google_pubsub;
53pub mod kafka;
54pub mod kinesis;
55pub mod monitor;
56pub mod mqtt;
57pub mod nats;
58pub mod nexmark;
59pub mod pulsar;
60
61mod util;
62use std::future::IntoFuture;
63
64pub use base::{UPSTREAM_SOURCE_KEY, WEBHOOK_CONNECTOR, *};
65pub use batch::BatchSourceSplitImpl;
66pub(crate) use common::*;
67use google_cloud_pubsub::subscription::Subscription;
68pub use google_pubsub::GOOGLE_PUBSUB_CONNECTOR;
69pub use kafka::KAFKA_CONNECTOR;
70pub use kinesis::KINESIS_CONNECTOR;
71pub use mqtt::MQTT_CONNECTOR;
72pub use nats::NATS_CONNECTOR;
73mod common;
74pub mod iceberg;
75mod manager;
76pub mod reader;
77pub mod test_source;
78
79use async_nats::jetstream::consumer::AckPolicy as JetStreamAckPolicy;
80use async_nats::jetstream::context::Context as JetStreamContext;
81pub use manager::{SourceColumnDesc, SourceColumnType};
82use risingwave_common::array::{Array, ArrayRef};
83use risingwave_common::row::OwnedRow;
84use thiserror_ext::AsReport;
85pub use util::fill_adaptive_split;
86
87pub use crate::source::filesystem::LEGACY_S3_CONNECTOR;
88pub use crate::source::filesystem::opendal_source::{
89    AZBLOB_CONNECTOR, BATCH_POSIX_FS_CONNECTOR, GCS_CONNECTOR, OPENDAL_S3_CONNECTOR,
90    POSIX_FS_CONNECTOR,
91};
92pub use crate::source::nexmark::NEXMARK_CONNECTOR;
93pub use crate::source::pulsar::PULSAR_CONNECTOR;
94
95pub fn should_copy_to_format_encode_options(key: &str, connector: &str) -> bool {
96    const PREFIXES: &[&str] = &[
97        "schema.registry",
98        "schema.location",
99        "message",
100        "key.message",
101        "without_header",
102        "delimiter",
103        // AwsAuthProps
104        "region",
105        "endpoint_url",
106        "access_key",
107        "secret_key",
108        "session_token",
109        "arn",
110        "external_id",
111        "profile",
112    ];
113    PREFIXES.iter().any(|prefix| key.starts_with(prefix))
114        || (key == "endpoint" && !connector.eq_ignore_ascii_case(KINESIS_CONNECTOR))
115}
116
117/// Tasks executed by `WaitCheckpointWorker`
118pub enum WaitCheckpointTask {
119    CommitCdcOffset(Option<(SplitId, String)>),
120    AckPubsubMessage(Subscription, Vec<ArrayRef>),
121    AckNatsJetStream(JetStreamContext, Vec<ArrayRef>, JetStreamAckPolicy),
122}
123
124impl WaitCheckpointTask {
125    pub async fn run(self) {
126        self.run_with_on_commit_success(|_source_id, _offset| {
127            // Default implementation: no action on commit success
128        })
129        .await;
130    }
131
132    pub async fn run_with_on_commit_success<F>(self, mut on_commit_success: F)
133    where
134        F: FnMut(u64, &str),
135    {
136        use std::str::FromStr;
137        match self {
138            WaitCheckpointTask::CommitCdcOffset(updated_offset) => {
139                if let Some((split_id, offset)) = updated_offset {
140                    let source_id: u64 = u64::from_str(split_id.as_ref()).unwrap();
141                    // notify cdc connector to commit offset
142                    match cdc::jni_source::commit_cdc_offset(source_id, offset.clone()) {
143                        Ok(()) => {
144                            // Execute callback after successful commit
145                            on_commit_success(source_id, &offset);
146                        }
147                        Err(e) => {
148                            tracing::error!(
149                                error = %e.as_report(),
150                                "source#{source_id}: failed to commit cdc offset: {offset}.",
151                            )
152                        }
153                    }
154                }
155            }
156            WaitCheckpointTask::AckPubsubMessage(subscription, ack_id_arrs) => {
157                async fn ack(subscription: &Subscription, ack_ids: Vec<String>) {
158                    tracing::trace!("acking pubsub messages {:?}", ack_ids);
159                    match subscription.ack(ack_ids).await {
160                        Ok(()) => {}
161                        Err(e) => {
162                            tracing::error!(
163                                error = %e.as_report(),
164                                "failed to ack pubsub messages",
165                            )
166                        }
167                    }
168                }
169                const MAX_ACK_BATCH_SIZE: usize = 1000;
170                let mut ack_ids: Vec<String> = vec![];
171                for arr in ack_id_arrs {
172                    for ack_id in arr.as_utf8().iter().flatten() {
173                        ack_ids.push(ack_id.to_owned());
174                        if ack_ids.len() >= MAX_ACK_BATCH_SIZE {
175                            ack(&subscription, std::mem::take(&mut ack_ids)).await;
176                        }
177                    }
178                }
179                ack(&subscription, ack_ids).await;
180            }
181            WaitCheckpointTask::AckNatsJetStream(
182                ref context,
183                reply_subjects_arrs,
184                ref ack_policy,
185            ) => {
186                async fn ack(context: &JetStreamContext, reply_subject: String) {
187                    match context.publish(reply_subject.clone(), "+ACK".into()).await {
188                        Err(e) => {
189                            tracing::error!(error = %e.as_report(), subject = ?reply_subject, "failed to ack NATS JetStream message");
190                        }
191                        Ok(ack_future) => {
192                            let _ = ack_future.into_future().await;
193                        }
194                    }
195                }
196
197                let reply_subjects = reply_subjects_arrs
198                    .iter()
199                    .flat_map(|arr| {
200                        arr.as_utf8()
201                            .iter()
202                            .flatten()
203                            .map(|s| s.to_owned())
204                            .collect::<Vec<String>>()
205                    })
206                    .collect::<Vec<String>>();
207
208                match ack_policy {
209                    JetStreamAckPolicy::None => (),
210                    JetStreamAckPolicy::Explicit => {
211                        for reply_subject in reply_subjects {
212                            if reply_subject.is_empty() {
213                                continue;
214                            }
215                            ack(context, reply_subject).await;
216                        }
217                    }
218                    JetStreamAckPolicy::All => {
219                        if let Some(reply_subject) = reply_subjects.last() {
220                            ack(context, reply_subject.clone()).await;
221                        }
222                    }
223                }
224            }
225        }
226    }
227}
228
229#[derive(Clone, Debug, PartialEq)]
230pub struct CdcTableSnapshotSplitCommon<T: Clone> {
231    pub split_id: i64,
232    pub left_bound_inclusive: T,
233    pub right_bound_exclusive: T,
234}
235
236pub type CdcTableSnapshotSplit = CdcTableSnapshotSplitCommon<OwnedRow>;
237pub type CdcTableSnapshotSplitRaw = CdcTableSnapshotSplitCommon<Vec<u8>>;