risingwave_connector/source/kafka/
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
15use std::collections::HashMap;
16
17use serde::Deserialize;
18use serde_with::{DisplayFromStr, serde_as};
19
20use crate::connector_common::{AwsAuthProps, KafkaConnectionProps, KafkaPrivateLinkCommon};
21use crate::enforce_secret::EnforceSecret;
22use crate::error::ConnectorResult;
23
24mod client_context;
25pub mod enumerator;
26pub mod private_link;
27pub mod source;
28pub mod split;
29pub mod stats;
30
31pub use client_context::*;
32pub use enumerator::*;
33pub use source::*;
34pub use split::*;
35use with_options::WithOptions;
36
37use crate::connector_common::{KafkaCommon, RdKafkaPropertiesCommon};
38use crate::source::SourceProperties;
39
40pub const KAFKA_CONNECTOR: &str = "kafka";
41pub const KAFKA_PROPS_BROKER_KEY: &str = "properties.bootstrap.server";
42pub const KAFKA_PROPS_BROKER_KEY_ALIAS: &str = "kafka.brokers";
43pub const PRIVATELINK_CONNECTION: &str = "privatelink";
44
45/// Properties for the rdkafka library. Leave a field as `None` to use the default value.
46/// These properties are not intended to be exposed to users in the majority of cases.
47///
48/// See also <https://github.com/confluentinc/librdkafka/blob/master/CONFIGURATION.md>
49#[serde_as]
50#[derive(Clone, Debug, Deserialize, Default, WithOptions)]
51pub struct RdKafkaPropertiesConsumer {
52    /// Minimum number of messages per topic+partition librdkafka tries to maintain in the local
53    /// consumer queue.
54    #[serde(rename = "properties.queued.min.messages")]
55    #[serde_as(as = "Option<DisplayFromStr>")]
56    #[with_option(allow_alter_on_fly)]
57    pub queued_min_messages: Option<usize>,
58
59    #[serde(rename = "properties.queued.max.messages.kbytes")]
60    #[serde_as(as = "Option<DisplayFromStr>")]
61    #[with_option(allow_alter_on_fly)]
62    pub queued_max_messages_kbytes: Option<usize>,
63
64    /// Maximum time the broker may wait to fill the Fetch response with `fetch.min.`bytes of
65    /// messages.
66    #[serde(rename = "properties.fetch.wait.max.ms")]
67    #[serde_as(as = "Option<DisplayFromStr>")]
68    #[with_option(allow_alter_on_fly)]
69    pub fetch_wait_max_ms: Option<usize>,
70
71    /// Minimum number of bytes the broker responds with. If fetch.wait.max.ms expires the accumulated data will be sent to the client regardless of this setting.
72    #[serde(rename = "properties.fetch.min.bytes")]
73    #[serde_as(as = "Option<DisplayFromStr>")]
74    pub fetch_min_bytes: Option<usize>,
75
76    /// Initial maximum number of bytes per topic+partition to request when fetching messages from the broker. If the client encounters a message larger than this value it will gradually try to increase it until the entire message can be fetched.
77    #[serde(rename = "properties.fetch.message.max.bytes")]
78    #[serde_as(as = "Option<DisplayFromStr>")]
79    pub fetch_message_max_bytes: Option<usize>,
80
81    /// How long to postpone the next fetch request for a topic+partition in case the current fetch
82    /// queue thresholds (`queued.min.messages` or `queued.max.messages.kbytes`) have been
83    /// exceeded. This property may need to be decreased if the queue thresholds are set low
84    /// and the application is experiencing long (~1s) delays between messages. Low values may
85    /// increase CPU utilization.
86    #[serde(rename = "properties.fetch.queue.backoff.ms")]
87    #[serde_as(as = "Option<DisplayFromStr>")]
88    #[with_option(allow_alter_on_fly)]
89    pub fetch_queue_backoff_ms: Option<usize>,
90
91    /// Maximum amount of data the broker shall return for a Fetch request. Messages are fetched in
92    /// batches by the consumer and if the first message batch in the first non-empty partition of
93    /// the Fetch request is larger than this value, then the message batch will still be returned
94    /// to ensure the consumer can make progress. The maximum message batch size accepted by the
95    /// broker is defined via `message.max.bytes` (broker config) or `max.message.bytes` (broker
96    /// topic config). `fetch.max.bytes` is automatically adjusted upwards to be at least
97    /// `message.max.bytes` (consumer config).
98    #[serde(rename = "properties.fetch.max.bytes")]
99    #[serde_as(as = "Option<DisplayFromStr>")]
100    #[with_option(allow_alter_on_fly)]
101    pub fetch_max_bytes: Option<usize>,
102
103    /// Whether to automatically and periodically commit offsets in the background.
104    ///
105    /// Note that RisingWave does NOT rely on committed offsets. Committing offset is only for exposing the
106    /// progress for monitoring. Setting this to false can avoid creating consumer groups.
107    ///
108    /// default: true
109    #[serde(rename = "properties.enable.auto.commit")]
110    #[serde_as(as = "Option<DisplayFromStr>")]
111    #[with_option(allow_alter_on_fly)]
112    pub enable_auto_commit: Option<bool>,
113}
114
115#[derive(Clone, Debug, Deserialize, WithOptions)]
116pub struct KafkaProperties {
117    /// This parameter is not intended to be exposed to users.
118    /// This parameter specifies only for one parallelism. The parallelism of kafka source
119    /// is equal to the parallelism passed into compute nodes. So users need to calculate
120    /// how many bytes will be consumed in total across all the parallelism by themselves.
121    #[serde(rename = "bytes.per.second", alias = "kafka.bytes.per.second")]
122    pub bytes_per_second: Option<String>,
123
124    /// This parameter is not intended to be exposed to users.
125    /// This parameter specifies only for one parallelism. The parallelism of kafka source
126    /// is equal to the parallelism passed into compute nodes. So users need to calculate
127    /// how many messages will be consumed in total across all the parallelism by themselves.
128    #[serde(rename = "max.num.messages", alias = "kafka.max.num.messages")]
129    pub max_num_messages: Option<String>,
130
131    #[serde(rename = "scan.startup.mode", alias = "kafka.scan.startup.mode")]
132    pub scan_startup_mode: Option<String>,
133
134    #[serde(
135        rename = "scan.startup.timestamp.millis",
136        alias = "kafka.time.offset",
137        alias = "scan.startup.timestamp_millis" // keep for compatibility
138    )]
139    pub time_offset: Option<String>,
140
141    /// Specify a custom consumer group id prefix for the source.
142    /// Defaults to `rw-consumer`.
143    ///
144    /// Notes:
145    /// - Each job (materialized view) will have a separated consumer group and
146    ///   contains a generated suffix in the group id.
147    ///   The consumer group will be `{group_id_prefix}-{fragment_id}`.
148    /// - The consumer group is solely for monintoring progress in some external
149    ///   Kafka tools, and for authorization. RisingWave does not rely on committed
150    ///   offsets, and does not join the consumer group. It just reports offsets
151    ///   to the group.
152    #[serde(rename = "group.id.prefix")]
153    #[with_option(allow_alter_on_fly)]
154    pub group_id_prefix: Option<String>,
155
156    /// This parameter is used to tell `KafkaSplitReader` to produce `UpsertMessage`s, which
157    /// combine both key and value fields of the Kafka message.
158    /// TODO: Currently, `Option<bool>` can not be parsed here.
159    #[serde(rename = "upsert")]
160    pub upsert: Option<String>,
161
162    #[serde(flatten)]
163    pub common: KafkaCommon,
164
165    #[serde(flatten)]
166    pub connection: KafkaConnectionProps,
167
168    #[serde(flatten)]
169    pub rdkafka_properties_common: RdKafkaPropertiesCommon,
170
171    #[serde(flatten)]
172    pub rdkafka_properties_consumer: RdKafkaPropertiesConsumer,
173
174    #[serde(flatten)]
175    pub privatelink_common: KafkaPrivateLinkCommon,
176
177    #[serde(flatten)]
178    pub aws_auth_props: AwsAuthProps,
179
180    #[serde(flatten)]
181    pub unknown_fields: HashMap<String, String>,
182}
183
184impl EnforceSecret for KafkaProperties {
185    fn enforce_secret<'a>(prop_iter: impl Iterator<Item = &'a str>) -> ConnectorResult<()> {
186        for prop in prop_iter {
187            KafkaConnectionProps::enforce_one(prop)?;
188            AwsAuthProps::enforce_one(prop)?;
189        }
190        Ok(())
191    }
192}
193
194impl SourceProperties for KafkaProperties {
195    type Split = KafkaSplit;
196    type SplitEnumerator = KafkaSplitEnumerator;
197    type SplitReader = KafkaSplitReader;
198
199    const SOURCE_NAME: &'static str = KAFKA_CONNECTOR;
200}
201
202impl crate::source::UnknownFields for KafkaProperties {
203    fn unknown_fields(&self) -> HashMap<String, String> {
204        self.unknown_fields.clone()
205    }
206}
207
208impl KafkaProperties {
209    pub fn set_client(&self, c: &mut rdkafka::ClientConfig) {
210        self.rdkafka_properties_common.set_client(c);
211        self.rdkafka_properties_consumer.set_client(c);
212    }
213
214    pub fn group_id(&self, fragment_id: u32) -> String {
215        format!(
216            "{}-{}",
217            self.group_id_prefix.as_deref().unwrap_or("rw-consumer"),
218            fragment_id
219        )
220    }
221}
222
223const KAFKA_ISOLATION_LEVEL: &str = "read_committed";
224
225impl RdKafkaPropertiesConsumer {
226    pub fn set_client(&self, c: &mut rdkafka::ClientConfig) {
227        if let Some(v) = &self.queued_min_messages {
228            c.set("queued.min.messages", v.to_string());
229        }
230        if let Some(v) = &self.queued_max_messages_kbytes {
231            c.set("queued.max.messages.kbytes", v.to_string());
232        }
233        if let Some(v) = &self.fetch_wait_max_ms {
234            c.set("fetch.wait.max.ms", v.to_string());
235        }
236        if let Some(v) = &self.fetch_min_bytes {
237            c.set("fetch.min.bytes", v.to_string());
238        }
239        if let Some(v) = &self.fetch_message_max_bytes {
240            c.set("fetch.message.max.bytes", v.to_string());
241        }
242        if let Some(v) = &self.fetch_queue_backoff_ms {
243            c.set("fetch.queue.backoff.ms", v.to_string());
244        }
245        if let Some(v) = &self.fetch_max_bytes {
246            c.set("fetch.max.bytes", v.to_string());
247        }
248        if let Some(v) = &self.enable_auto_commit {
249            c.set("enable.auto.commit", v.to_string());
250        }
251    }
252}
253
254#[cfg(test)]
255mod test {
256    use std::collections::BTreeMap;
257
258    use maplit::btreemap;
259
260    use super::*;
261
262    #[test]
263    fn test_parse_config_consumer_common() {
264        let config: BTreeMap<String, String> = btreemap! {
265            // common
266            "properties.bootstrap.server".to_owned() => "127.0.0.1:9092".to_owned(),
267            "topic".to_owned() => "test".to_owned(),
268            // kafka props
269            "scan.startup.mode".to_owned() => "earliest".to_owned(),
270            // RdKafkaPropertiesCommon
271            "properties.message.max.bytes".to_owned() => "12345".to_owned(),
272            "properties.receive.message.max.bytes".to_owned() => "54321".to_owned(),
273            // RdKafkaPropertiesConsumer
274            "properties.queued.min.messages".to_owned() => "114514".to_owned(),
275            "properties.queued.max.messages.kbytes".to_owned() => "114514".to_owned(),
276            "properties.fetch.wait.max.ms".to_owned() => "114514".to_owned(),
277            "properties.fetch.max.bytes".to_owned() => "114514".to_owned(),
278            "properties.enable.auto.commit".to_owned() => "true".to_owned(),
279            "properties.fetch.queue.backoff.ms".to_owned() => "114514".to_owned(),
280            // PrivateLink
281            "broker.rewrite.endpoints".to_owned() => "{\"broker1\": \"10.0.0.1:8001\"}".to_owned(),
282        };
283
284        let props: KafkaProperties =
285            serde_json::from_value(serde_json::to_value(config).unwrap()).unwrap();
286
287        assert_eq!(props.scan_startup_mode, Some("earliest".to_owned()));
288        assert_eq!(
289            props.rdkafka_properties_common.receive_message_max_bytes,
290            Some(54321)
291        );
292        assert_eq!(
293            props.rdkafka_properties_common.message_max_bytes,
294            Some(12345)
295        );
296        assert_eq!(
297            props.rdkafka_properties_consumer.queued_min_messages,
298            Some(114514)
299        );
300        assert_eq!(
301            props.rdkafka_properties_consumer.queued_max_messages_kbytes,
302            Some(114514)
303        );
304        assert_eq!(
305            props.rdkafka_properties_consumer.fetch_wait_max_ms,
306            Some(114514)
307        );
308        assert_eq!(
309            props.rdkafka_properties_consumer.fetch_max_bytes,
310            Some(114514)
311        );
312        assert_eq!(
313            props.rdkafka_properties_consumer.enable_auto_commit,
314            Some(true)
315        );
316        assert_eq!(
317            props.rdkafka_properties_consumer.fetch_queue_backoff_ms,
318            Some(114514)
319        );
320        let hashmap: BTreeMap<String, String> = btreemap! {
321            "broker1".to_owned() => "10.0.0.1:8001".to_owned()
322        };
323        assert_eq!(props.privatelink_common.broker_rewrite_map, Some(hashmap));
324    }
325}