risingwave_meta/stream/source_manager/
worker.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
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
// Copyright 2025 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.

#[cfg(not(debug_assertions))]
use risingwave_connector::error::ConnectorError;
use risingwave_connector::source::AnySplitEnumerator;

use super::*;

const MAX_FAIL_CNT: u32 = 10;
const DEFAULT_SOURCE_TICK_TIMEOUT: Duration = Duration::from_secs(10);

// The key used to load `SplitImpl` directly from source properties.
// When this key is present, the enumerator will only return the given ones
// instead of fetching them from the external source.
// Only valid in debug builds - will return an error in release builds.
const DEBUG_SPLITS_KEY: &str = "debug_splits";

pub struct SharedSplitMap {
    pub splits: Option<BTreeMap<SplitId, SplitImpl>>,
}

type SharedSplitMapRef = Arc<Mutex<SharedSplitMap>>;

/// `ConnectorSourceWorker` keeps fetching the latest split metadata from the external source service ([`Self::tick`]),
/// and maintains it in `current_splits`.
pub struct ConnectorSourceWorker {
    source_id: SourceId,
    source_name: String,
    current_splits: SharedSplitMapRef,
    // XXX: box or arc?
    enumerator: Box<dyn AnySplitEnumerator>,
    period: Duration,
    metrics: Arc<MetaMetrics>,
    connector_properties: ConnectorProperties,
    fail_cnt: u32,
    source_is_up: LabelGuardedIntGauge<2>,

    debug_splits: Option<Vec<SplitImpl>>,
}

fn extract_prop_from_existing_source(source: &Source) -> ConnectorResult<ConnectorProperties> {
    let options_with_secret =
        WithOptionsSecResolved::new(source.with_properties.clone(), source.secret_refs.clone());
    let mut properties = ConnectorProperties::extract(options_with_secret, false)?;
    properties.init_from_pb_source(source);
    Ok(properties)
}
fn extract_prop_from_new_source(source: &Source) -> ConnectorResult<ConnectorProperties> {
    let options_with_secret = WithOptionsSecResolved::new(
        {
            let mut with_properties = source.with_properties.clone();
            let _removed = with_properties.remove(DEBUG_SPLITS_KEY);

            #[cfg(not(debug_assertions))]
            {
                if _removed.is_some() {
                    return Err(ConnectorError::from(anyhow::anyhow!(
                        "`debug_splits` is not allowed in release mode"
                    )));
                }
            }

            with_properties
        },
        source.secret_refs.clone(),
    );
    let mut properties = ConnectorProperties::extract(options_with_secret, true)?;
    properties.init_from_pb_source(source);
    Ok(properties)
}

/// Used to create a new [`ConnectorSourceWorkerHandle`] for a new source.
///
/// It will call [`ConnectorSourceWorker::tick()`] to fetch split metadata once before returning.
pub async fn create_source_worker(
    source: &Source,
    metrics: Arc<MetaMetrics>,
) -> MetaResult<ConnectorSourceWorkerHandle> {
    tracing::info!("spawning new watcher for source {}", source.id);

    let splits = Arc::new(Mutex::new(SharedSplitMap { splits: None }));
    let current_splits_ref = splits.clone();

    let connector_properties = extract_prop_from_new_source(source)?;
    let enable_scale_in = connector_properties.enable_drop_split();
    let enable_adaptive_splits = connector_properties.enable_adaptive_splits();
    let (command_tx, command_rx) = tokio::sync::mpsc::unbounded_channel();
    let handle = {
        let mut worker = ConnectorSourceWorker::create(
            source,
            connector_properties,
            DEFAULT_SOURCE_WORKER_TICK_INTERVAL,
            current_splits_ref.clone(),
            metrics,
        )
        .await?;

        // if fail to fetch meta info, will refuse to create source

        // todo: make the timeout configurable, longer than `properties.sync.call.timeout`
        // in kafka
        tokio::time::timeout(DEFAULT_SOURCE_TICK_TIMEOUT, worker.tick())
            .await
            .ok()
            .with_context(|| {
                format!(
                    "failed to fetch meta info for source {}, timeout {:?}",
                    source.id, DEFAULT_SOURCE_TICK_TIMEOUT
                )
            })??;

        tokio::spawn(async move { worker.run(command_rx).await })
    };
    Ok(ConnectorSourceWorkerHandle {
        handle,
        command_tx,
        splits,
        enable_drop_split: enable_scale_in,
        enable_adaptive_splits,
    })
}

/// Used on startup ([`SourceManager::new`]). Failed sources will not block meta startup.
pub fn create_source_worker_async(
    source: Source,
    managed_sources: &mut HashMap<SourceId, ConnectorSourceWorkerHandle>,
    metrics: Arc<MetaMetrics>,
) -> MetaResult<()> {
    tracing::info!("spawning new watcher for source {}", source.id);

    let splits = Arc::new(Mutex::new(SharedSplitMap { splits: None }));
    let current_splits_ref = splits.clone();
    let source_id = source.id;

    let connector_properties = extract_prop_from_existing_source(&source)?;

    let enable_drop_split = connector_properties.enable_drop_split();
    let enable_adaptive_splits = connector_properties.enable_adaptive_splits();
    let (command_tx, command_rx) = tokio::sync::mpsc::unbounded_channel();
    let handle = tokio::spawn(async move {
        let mut ticker = time::interval(DEFAULT_SOURCE_WORKER_TICK_INTERVAL);
        ticker.set_missed_tick_behavior(MissedTickBehavior::Skip);

        let mut worker = loop {
            ticker.tick().await;

            match ConnectorSourceWorker::create(
                &source,
                connector_properties.clone(),
                DEFAULT_SOURCE_WORKER_TICK_INTERVAL,
                current_splits_ref.clone(),
                metrics.clone(),
            )
            .await
            {
                Ok(worker) => {
                    break worker;
                }
                Err(e) => {
                    tracing::warn!(error = %e.as_report(), "failed to create source worker");
                }
            }
        };

        worker.run(command_rx).await
    });

    managed_sources.insert(
        source_id as SourceId,
        ConnectorSourceWorkerHandle {
            handle,
            command_tx,
            splits,
            enable_drop_split,
            enable_adaptive_splits,
        },
    );
    Ok(())
}

const DEFAULT_SOURCE_WORKER_TICK_INTERVAL: Duration = Duration::from_secs(30);

impl ConnectorSourceWorker {
    /// Recreate the `SplitEnumerator` to establish a new connection to the external source service.
    async fn refresh(&mut self) -> MetaResult<()> {
        let enumerator = self
            .connector_properties
            .clone()
            .create_split_enumerator(Arc::new(SourceEnumeratorContext {
                metrics: self.metrics.source_enumerator_metrics.clone(),
                info: SourceEnumeratorInfo {
                    source_id: self.source_id as u32,
                },
            }))
            .await
            .context("failed to create SplitEnumerator")?;
        self.enumerator = enumerator;
        self.fail_cnt = 0;
        tracing::info!("refreshed source enumerator: {}", self.source_name);
        Ok(())
    }

    /// On creation, connection to the external source service will be established, but `splits`
    /// will not be updated until `tick` is called.
    pub async fn create(
        source: &Source,
        connector_properties: ConnectorProperties,
        period: Duration,
        splits: Arc<Mutex<SharedSplitMap>>,
        metrics: Arc<MetaMetrics>,
    ) -> MetaResult<Self> {
        let enumerator = connector_properties
            .clone()
            .create_split_enumerator(Arc::new(SourceEnumeratorContext {
                metrics: metrics.source_enumerator_metrics.clone(),
                info: SourceEnumeratorInfo {
                    source_id: source.id,
                },
            }))
            .await
            .context("failed to create SplitEnumerator")?;

        let source_is_up = metrics
            .source_is_up
            .with_guarded_label_values(&[source.id.to_string().as_str(), &source.name]);

        Ok(Self {
            source_id: source.id as SourceId,
            source_name: source.name.clone(),
            current_splits: splits,
            enumerator,
            period,
            metrics,
            connector_properties,
            fail_cnt: 0,
            source_is_up,
            debug_splits: {
                let debug_splits = source.with_properties.get(DEBUG_SPLITS_KEY);
                #[cfg(not(debug_assertions))]
                {
                    if debug_splits.is_some() {
                        return Err(ConnectorError::from(anyhow::anyhow!(
                            "`debug_splits` is not allowed in release mode"
                        ))
                        .into());
                    }
                    None
                }

                #[cfg(debug_assertions)]
                {
                    use risingwave_common::types::JsonbVal;
                    if let Some(debug_splits) = debug_splits {
                        let mut splits = Vec::new();
                        let debug_splits_value =
                            jsonbb::serde_json::from_str::<serde_json::Value>(debug_splits)
                                .context("failed to parse split impl")?;
                        for split_impl_value in debug_splits_value.as_array().unwrap() {
                            splits.push(SplitImpl::restore_from_json(JsonbVal::from(
                                split_impl_value.clone(),
                            ))?);
                        }
                        Some(splits)
                    } else {
                        None
                    }
                }
            },
        })
    }

    pub async fn run(&mut self, mut command_rx: UnboundedReceiver<SourceWorkerCommand>) {
        let mut interval = time::interval(self.period);
        interval.set_missed_tick_behavior(MissedTickBehavior::Skip);
        loop {
            select! {
                biased;
                cmd = command_rx.borrow_mut().recv() => {
                    if let Some(cmd) = cmd {
                        match cmd {
                            SourceWorkerCommand::Tick(tx) => {
                                let _ = tx.send(self.tick().await);
                            }
                            SourceWorkerCommand::DropFragments(fragment_ids) => {
                                if let Err(e) = self.drop_fragments(fragment_ids).await {
                                    // when error happens, we just log it and ignore
                                    tracing::warn!(error = %e.as_report(), "error happened when drop fragment");
                                }
                            }
                            SourceWorkerCommand::FinishBackfill(fragment_ids) => {
                                if let Err(e) = self.finish_backfill(fragment_ids).await {
                                    // when error happens, we just log it and ignore
                                    tracing::warn!(error = %e.as_report(), "error happened when finish backfill");
                                }
                            }
                            SourceWorkerCommand::Terminate => {
                                return;
                            }
                        }
                    }
                }
                _ = interval.tick() => {
                    if self.fail_cnt > MAX_FAIL_CNT {
                        if let Err(e) = self.refresh().await {
                            tracing::error!(error = %e.as_report(), "error happened when refresh from connector source worker");
                        }
                    }
                    if let Err(e) = self.tick().await {
                        tracing::error!(error = %e.as_report(), "error happened when tick from connector source worker");
                    }
                }
            }
        }
    }

    /// Uses [`risingwave_connector::source::SplitEnumerator`] to fetch the latest split metadata from the external source service.
    async fn tick(&mut self) -> MetaResult<()> {
        let source_is_up = |res: i64| {
            self.source_is_up.set(res);
        };

        let splits = {
            if let Some(debug_splits) = &self.debug_splits {
                debug_splits.clone()
            } else {
                self.enumerator.list_splits().await.inspect_err(|_| {
                    source_is_up(0);
                    self.fail_cnt += 1;
                })?
            }
        };

        source_is_up(1);
        self.fail_cnt = 0;
        let mut current_splits = self.current_splits.lock().await;
        current_splits.splits.replace(
            splits
                .into_iter()
                .map(|split| (split.id(), split))
                .collect(),
        );

        Ok(())
    }

    async fn drop_fragments(&mut self, fragment_ids: Vec<FragmentId>) -> MetaResult<()> {
        self.enumerator.on_drop_fragments(fragment_ids).await?;
        Ok(())
    }

    async fn finish_backfill(&mut self, fragment_ids: Vec<FragmentId>) -> MetaResult<()> {
        self.enumerator.on_finish_backfill(fragment_ids).await?;
        Ok(())
    }
}

/// Handle for a running [`ConnectorSourceWorker`].
pub struct ConnectorSourceWorkerHandle {
    #[expect(dead_code)]
    handle: JoinHandle<()>,
    command_tx: UnboundedSender<SourceWorkerCommand>,
    pub splits: SharedSplitMapRef,
    pub enable_drop_split: bool,
    pub enable_adaptive_splits: bool,
}

impl ConnectorSourceWorkerHandle {
    pub fn get_enable_adaptive_splits(&self) -> bool {
        self.enable_adaptive_splits
    }

    pub async fn discovered_splits(
        &self,
        source_id: SourceId,
        actors: &HashSet<ActorId>,
    ) -> MetaResult<BTreeMap<Arc<str>, SplitImpl>> {
        // XXX: when is this None? Can we remove the Option?
        let Some(mut discovered_splits) = self.splits.lock().await.splits.clone() else {
            tracing::info!(
                "The discover loop for source {} is not ready yet; we'll wait for the next run",
                source_id
            );
            return Ok(BTreeMap::new());
        };
        if discovered_splits.is_empty() {
            tracing::warn!("No splits discovered for source {}", source_id);
        }

        if self.enable_adaptive_splits {
            // Connector supporting adaptive splits returns just one split, and we need to make the number of splits equal to the number of actors in this fragment.
            // Because we Risingwave consume the splits statelessly and we do not need to keep the id internally, we always use actor_id as split_id.
            // And prev splits record should be dropped via CN.

            debug_assert!(self.enable_drop_split);
            debug_assert!(discovered_splits.len() == 1);
            discovered_splits =
                fill_adaptive_split(discovered_splits.values().next().unwrap(), actors)?;
        }

        Ok(discovered_splits)
    }

    fn send_command(&self, command: SourceWorkerCommand) -> MetaResult<()> {
        let cmd_str = format!("{:?}", command);
        self.command_tx
            .send(command)
            .with_context(|| format!("failed to send {cmd_str} command to source worker"))?;
        Ok(())
    }

    /// Force [`ConnectorSourceWorker::tick()`] to be called.
    pub async fn force_tick(&self) -> MetaResult<()> {
        let (tx, rx) = oneshot::channel();
        self.send_command(SourceWorkerCommand::Tick(tx))?;
        rx.await
            .context("failed to receive tick command response from source worker")?
            .context("source worker tick failed")?;
        Ok(())
    }

    pub fn drop_fragments(&self, fragment_ids: Vec<FragmentId>) {
        tracing::debug!("drop_fragments: {:?}", fragment_ids);
        if let Err(e) = self.send_command(SourceWorkerCommand::DropFragments(fragment_ids)) {
            // ignore drop fragment error, just log it
            tracing::warn!(error = %e.as_report(), "failed to drop fragments");
        }
    }

    pub fn finish_backfill(&self, fragment_ids: Vec<FragmentId>) {
        tracing::debug!("finish_backfill: {:?}", fragment_ids);
        if let Err(e) = self.send_command(SourceWorkerCommand::FinishBackfill(fragment_ids)) {
            // ignore error, just log it
            tracing::warn!(error = %e.as_report(), "failed to finish backfill");
        }
    }

    pub fn terminate(&self, dropped_fragments: Option<BTreeSet<FragmentId>>) {
        tracing::debug!("terminate: {:?}", dropped_fragments);
        if let Some(dropped_fragments) = dropped_fragments {
            self.drop_fragments(dropped_fragments.into_iter().collect());
        }
        if let Err(e) = self.send_command(SourceWorkerCommand::Terminate) {
            // ignore terminate error, just log it
            tracing::warn!(error = %e.as_report(), "failed to terminate source worker");
        }
    }
}

#[derive(educe::Educe)]
#[educe(Debug)]
pub enum SourceWorkerCommand {
    /// Sync command to force [`ConnectorSourceWorker::tick()`] to be called.
    Tick(#[educe(Debug(ignore))] oneshot::Sender<MetaResult<()>>),
    /// Async command to drop a fragment.
    DropFragments(Vec<FragmentId>),
    /// Async command to finish backfill.
    FinishBackfill(Vec<FragmentId>),
    /// Terminate the worker task.
    Terminate,
}