1#![rustfmt::skip]
19
20use std::collections::{HashMap, HashSet};
21use std::sync::LazyLock;
22use crate::error::ConnectorError;
23
24macro_rules! use_source_properties {
25 ({ $({ $variant_name:ident, $prop_name:ty, $split:ty }),* }) => {
26 $(
27 #[allow(unused_imports)]
28 pub(super) use $prop_name;
29 )*
30 };
31}
32
33mod source_properties {
34 use crate::for_all_sources;
35 use crate::source::base::SourceProperties;
36
37 for_all_sources!(use_source_properties);
38
39 macro_rules! impl_source_name_to_prop_type_name_inner {
45 ({ $({$variant:ident, $prop_name:ty, $split:ty}),* }) => {
46 pub fn source_name_to_prop_type_name(source_name: &str) -> Option<&'static str> {
47 match source_name {
48 $(
49 <$prop_name>::SOURCE_NAME => Some(std::any::type_name::<$prop_name>()),
50 )*
51 _ => None,
52 }
53 }
54 };
55 }
56
57 macro_rules! impl_source_name_to_prop_type_name {
58 () => {
59 $crate::for_all_sources! { impl_source_name_to_prop_type_name_inner }
60 };
61 }
62
63 impl_source_name_to_prop_type_name!();
64}
65
66mod sink_properties {
67 use crate::use_all_sink_configs;
68 use crate::sink::Sink;
69 use crate::sink::file_sink::fs::FsSink;
70
71 use_all_sink_configs!();
72
73 macro_rules! impl_sink_name_to_config_type_name_inner {
74 ({ $({ $variant_name:ident, $sink_type:ty, $config_type:ty }),* }) => {
75 pub fn sink_name_to_config_type_name(sink_name: &str) -> Option<&'static str> {
76 match sink_name {
77 $(
78 <$sink_type>::SINK_NAME => Some(std::any::type_name::<$config_type>()),
79 )*
80 _ => None,
81 }
82 }
83 };
84 }
85
86 macro_rules! impl_sink_name_to_config_type_name {
87 () => {
88 $crate::for_all_sinks! { impl_sink_name_to_config_type_name_inner }
89 };
90 }
91
92 impl_sink_name_to_config_type_name!();
93}
94
95pub static SOURCE_ALLOW_ALTER_ON_FLY_FIELDS: LazyLock<HashMap<String, HashSet<String>>> = LazyLock::new(|| {
97 use source_properties::*;
98 let mut map = HashMap::new();
99 map.try_insert(
101 std::any::type_name::<MysqlCdcProperties>().to_owned(),
102 ["cdc.source.wait.streaming.start.timeout".to_owned()].into_iter().collect(),
103 ).unwrap();
104 map.try_insert(
105 std::any::type_name::<PostgresCdcProperties>().to_owned(),
106 ["cdc.source.wait.streaming.start.timeout".to_owned()].into_iter().collect(),
107 ).unwrap();
108 map.try_insert(
109 std::any::type_name::<SqlServerCdcProperties>().to_owned(),
110 ["cdc.source.wait.streaming.start.timeout".to_owned()].into_iter().collect(),
111 ).unwrap();
112
113 map.try_insert(
114 std::any::type_name::<MongodbCdcProperties>().to_owned(),
115 ["cdc.source.wait.streaming.start.timeout".to_owned()].into_iter().collect(),
116 ).unwrap();
117 map.try_insert(
119 std::any::type_name::<KafkaProperties>().to_owned(),
120 [
121 "group.id.prefix".to_owned(),
122 "properties.sync.call.timeout".to_owned(),
123 "properties.security.protocol".to_owned(),
124 "properties.ssl.endpoint.identification.algorithm".to_owned(),
125 "properties.sasl.mechanism".to_owned(),
126 "properties.sasl.username".to_owned(),
127 "properties.sasl.password".to_owned(),
128 "properties.message.max.bytes".to_owned(),
129 "properties.receive.message.max.bytes".to_owned(),
130 "properties.statistics.interval.ms".to_owned(),
131 "properties.client.id".to_owned(),
132 "properties.enable.ssl.certificate.verification".to_owned(),
133 "properties.queued.min.messages".to_owned(),
134 "properties.queued.max.messages.kbytes".to_owned(),
135 "properties.fetch.wait.max.ms".to_owned(),
136 "properties.fetch.queue.backoff.ms".to_owned(),
137 "properties.fetch.max.bytes".to_owned(),
138 "properties.enable.auto.commit".to_owned(),
139 ].into_iter().collect(),
140 ).unwrap();
141 map
142});
143
144pub static SINK_ALLOW_ALTER_ON_FLY_FIELDS: LazyLock<HashMap<String, HashSet<String>>> = LazyLock::new(|| {
146 use sink_properties::*;
147 let mut map = HashMap::new();
148 map.try_insert(
150 std::any::type_name::<ClickHouseConfig>().to_owned(),
151 [
152 "commit_checkpoint_interval".to_owned(),
153 ].into_iter().collect(),
154 ).unwrap();
155 map.try_insert(
157 std::any::type_name::<DeltaLakeConfig>().to_owned(),
158 [
159 "commit_checkpoint_interval".to_owned(),
160 ].into_iter().collect(),
161 ).unwrap();
162 map.try_insert(
164 std::any::type_name::<DorisConfig>().to_owned(),
165 [
166 "doris.stream_load.http.timeout.ms".to_owned(),
167 ].into_iter().collect(),
168 ).unwrap();
169 map.try_insert(
171 std::any::type_name::<IcebergConfig>().to_owned(),
172 [
173 "commit_checkpoint_interval".to_owned(),
174 "enable_compaction".to_owned(),
175 "compaction_interval_sec".to_owned(),
176 "enable_snapshot_expiration".to_owned(),
177 "snapshot_expiration_max_age_millis".to_owned(),
178 "snapshot_expiration_retain_last".to_owned(),
179 "snapshot_expiration_clear_expired_files".to_owned(),
180 "snapshot_expiration_clear_expired_meta_data".to_owned(),
181 "compaction.max_snapshots_num".to_owned(),
182 "compaction.small_files_threshold_mb".to_owned(),
183 "compaction.delete_files_count_threshold".to_owned(),
184 "compaction.trigger_snapshot_count".to_owned(),
185 "compaction.target_file_size_mb".to_owned(),
186 "compaction.type".to_owned(),
187 ].into_iter().collect(),
188 ).unwrap();
189 map.try_insert(
191 std::any::type_name::<KafkaConfig>().to_owned(),
192 [
193 "properties.sync.call.timeout".to_owned(),
194 "properties.security.protocol".to_owned(),
195 "properties.ssl.endpoint.identification.algorithm".to_owned(),
196 "properties.sasl.mechanism".to_owned(),
197 "properties.sasl.username".to_owned(),
198 "properties.sasl.password".to_owned(),
199 "properties.message.max.bytes".to_owned(),
200 "properties.receive.message.max.bytes".to_owned(),
201 "properties.statistics.interval.ms".to_owned(),
202 "properties.client.id".to_owned(),
203 "properties.enable.ssl.certificate.verification".to_owned(),
204 "properties.allow.auto.create.topics".to_owned(),
205 "properties.queue.buffering.max.messages".to_owned(),
206 "properties.queue.buffering.max.kbytes".to_owned(),
207 "properties.queue.buffering.max.ms".to_owned(),
208 "properties.enable.idempotence".to_owned(),
209 "properties.message.send.max.retries".to_owned(),
210 "properties.retry.backoff.ms".to_owned(),
211 "properties.batch.num.messages".to_owned(),
212 "properties.batch.size".to_owned(),
213 "properties.message.timeout.ms".to_owned(),
214 "properties.max.in.flight.requests.per.connection".to_owned(),
215 "properties.request.required.acks".to_owned(),
216 ].into_iter().collect(),
217 ).unwrap();
218 map.try_insert(
220 std::any::type_name::<SnowflakeV2Config>().to_owned(),
221 [
222 "commit_checkpoint_interval".to_owned(),
223 ].into_iter().collect(),
224 ).unwrap();
225 map.try_insert(
227 std::any::type_name::<StarrocksConfig>().to_owned(),
228 [
229 "starrocks.stream_load.http.timeout.ms".to_owned(),
230 "commit_checkpoint_interval".to_owned(),
231 ].into_iter().collect(),
232 ).unwrap();
233 map
234});
235
236pub static CONNECTION_ALLOW_ALTER_ON_FLY_FIELDS: LazyLock<HashMap<String, HashSet<String>>> = LazyLock::new(|| {
238 use crate::connector_common::*;
239 let mut map = HashMap::new();
240 map.try_insert(
242 std::any::type_name::<KafkaConnection>().to_owned(),
243 [
244 "properties.security.protocol".to_owned(),
245 "properties.ssl.endpoint.identification.algorithm".to_owned(),
246 "properties.sasl.mechanism".to_owned(),
247 "properties.sasl.username".to_owned(),
248 "properties.sasl.password".to_owned(),
249 ].into_iter().collect(),
250 ).unwrap();
251 map
252});
253
254pub fn get_source_connectors_with_allow_alter_on_fly_fields() -> Vec<&'static str> {
256 SOURCE_ALLOW_ALTER_ON_FLY_FIELDS.keys().map(|s| s.as_str()).collect()
257}
258
259pub fn get_sink_connectors_with_allow_alter_on_fly_fields() -> Vec<&'static str> {
261 SINK_ALLOW_ALTER_ON_FLY_FIELDS.keys().map(|s| s.as_str()).collect()
262}
263
264pub fn get_connection_names_with_allow_alter_on_fly_fields() -> Vec<&'static str> {
266 CONNECTION_ALLOW_ALTER_ON_FLY_FIELDS.keys().map(|s| s.as_str()).collect()
267}
268
269pub fn check_source_allow_alter_on_fly_fields(
272 connector_name: &str,
273 fields: &[String],
274) -> crate::error::ConnectorResult<()> {
275 let Some(type_name) = source_properties::source_name_to_prop_type_name(connector_name) else {
277 return Err(ConnectorError::from(anyhow::anyhow!(
278 "Unknown source connector: {connector_name}"
279 )));
280 };
281 let Some(allowed_fields) = SOURCE_ALLOW_ALTER_ON_FLY_FIELDS.get(type_name) else {
282 return Err(ConnectorError::from(anyhow::anyhow!(
283 "No allow_alter_on_fly fields registered for connector: {connector_name}"
284 )));
285 };
286 for field in fields {
287 if !allowed_fields.contains(field) {
288 return Err(ConnectorError::from(anyhow::anyhow!(
289 "Field '{field}' is not allowed to be altered on the fly for connector: {connector_name}"
290 )));
291 }
292 }
293 Ok(())
294}
295
296pub fn check_connection_allow_alter_on_fly_fields(
297 connection_name: &str,
298 fields: &[String],
299) -> crate::error::ConnectorResult<()> {
300 use crate::source::connection_name_to_prop_type_name;
301
302 let Some(type_name) = connection_name_to_prop_type_name(connection_name) else {
304 return Err(ConnectorError::from(anyhow::anyhow!(
305 "Unknown connection: {connection_name}"
306 )));
307 };
308 let Some(allowed_fields) = CONNECTION_ALLOW_ALTER_ON_FLY_FIELDS.get(type_name) else {
309 return Err(ConnectorError::from(anyhow::anyhow!(
310 "No allow_alter_on_fly fields registered for connection: {connection_name}"
311 )));
312 };
313 for field in fields {
314 if !allowed_fields.contains(field) {
315 return Err(ConnectorError::from(anyhow::anyhow!(
316 "Field '{field}' is not allowed to be altered on the fly for connection: {connection_name}"
317 )));
318 }
319 }
320 Ok(())
321}
322
323pub fn check_sink_allow_alter_on_fly_fields(
326 sink_name: &str,
327 fields: &[String],
328) -> crate::error::ConnectorResult<()> {
329 let Some(type_name) = sink_properties::sink_name_to_config_type_name(sink_name) else {
331 return Err(ConnectorError::from(anyhow::anyhow!(
332 "Unknown sink connector: {sink_name}"
333 )));
334 };
335 let Some(allowed_fields) = SINK_ALLOW_ALTER_ON_FLY_FIELDS.get(type_name) else {
336 return Err(ConnectorError::from(anyhow::anyhow!(
337 "No allow_alter_on_fly fields registered for sink: {sink_name}"
338 )));
339 };
340 for field in fields {
341 if !allowed_fields.contains(field) {
342 return Err(ConnectorError::from(anyhow::anyhow!(
343 "Field '{field}' is not allowed to be altered on the fly for sink: {sink_name}"
344 )));
345 }
346 }
347 Ok(())
348}