1#![allow(missing_docs)]
24#![allow(clippy::needless_borrow)]
25
26use std::cell::RefCell;
27use std::collections::{BTreeMap, VecDeque};
28use std::hash::{Hash, Hasher};
29use std::rc::Rc;
30use std::sync::Arc;
31use std::time::Duration;
32
33use differential_dataflow::lattice::Lattice;
34use differential_dataflow::{AsCollection, Hashable, VecCollection};
35use futures::stream::StreamExt;
36use mz_ore::cast::CastFrom;
37use mz_ore::collections::CollectionExt;
38use mz_ore::now::NowFn;
39use mz_persist_client::cache::PersistClientCache;
40use mz_repr::{Diff, GlobalId, RelationDesc, Row};
41use mz_storage_types::configuration::StorageConfiguration;
42use mz_storage_types::controller::CollectionMetadata;
43use mz_storage_types::errors::DataflowError;
44use mz_storage_types::sources::{SourceConnection, SourceExport, SourceTimestamp};
45use mz_timely_util::antichain::AntichainExt;
46use mz_timely_util::builder_async::{OperatorBuilder as AsyncOperatorBuilder, PressOnDropButton};
47use mz_timely_util::capture::PusherCapture;
48use mz_timely_util::operator::ConcatenateFlatten;
49use mz_timely_util::reclock::reclock;
50use timely::PartialOrder;
51use timely::container::CapacityContainerBuilder;
52use timely::dataflow::channels::pact::Pipeline;
53use timely::dataflow::operators::capture::capture::Capture;
54use timely::dataflow::operators::core::Map as _;
55use timely::dataflow::operators::generic::OutputBuilder;
56use timely::dataflow::operators::generic::builder_rc::OperatorBuilder as OperatorBuilderRc;
57use timely::dataflow::operators::vec::Broadcast;
58use timely::dataflow::operators::{CapabilitySet, Inspect, Leave};
59use timely::dataflow::scopes::Child;
60use timely::dataflow::{Scope, StreamVec};
61use timely::order::TotalOrder;
62use timely::progress::frontier::MutableAntichain;
63use timely::progress::{Antichain, Timestamp};
64use tokio::sync::{Semaphore, watch};
65use tokio_stream::wrappers::WatchStream;
66use tracing::trace;
67
68use crate::healthcheck::{HealthStatusMessage, HealthStatusUpdate};
69use crate::metrics::StorageMetrics;
70use crate::metrics::source::SourceMetrics;
71use crate::source::reclock::ReclockOperator;
72use crate::source::types::{Probe, SourceMessage, SourceOutput, SourceRender, StackedCollection};
73use crate::statistics::SourceStatistics;
74
75#[derive(Clone)]
78pub struct RawSourceCreationConfig {
79 pub name: String,
81 pub id: GlobalId,
83 pub source_exports: BTreeMap<GlobalId, SourceExport<CollectionMetadata>>,
85 pub worker_id: usize,
87 pub worker_count: usize,
89 pub timestamp_interval: Duration,
92 pub now_fn: NowFn,
94 pub metrics: StorageMetrics,
96 pub as_of: Antichain<mz_repr::Timestamp>,
98 pub resume_uppers: BTreeMap<GlobalId, Antichain<mz_repr::Timestamp>>,
101 pub source_resume_uppers: BTreeMap<GlobalId, Vec<Row>>,
108 pub persist_clients: Arc<PersistClientCache>,
110 pub statistics: BTreeMap<GlobalId, SourceStatistics>,
112 pub shared_remap_upper: Rc<RefCell<Antichain<mz_repr::Timestamp>>>,
114 pub config: StorageConfiguration,
116 pub remap_collection_id: GlobalId,
118 pub remap_metadata: CollectionMetadata,
120 pub busy_signal: Arc<Semaphore>,
123}
124
125#[derive(Clone)]
128pub struct SourceExportCreationConfig {
129 pub id: GlobalId,
131 pub worker_id: usize,
133 pub metrics: StorageMetrics,
135 pub source_statistics: SourceStatistics,
137}
138
139impl RawSourceCreationConfig {
140 pub fn responsible_worker<P: Hash>(&self, partition: P) -> usize {
142 let mut h = std::hash::DefaultHasher::default();
143 (self.id, partition).hash(&mut h);
144 let key = usize::cast_from(h.finish());
145 key % self.worker_count
146 }
147
148 pub fn responsible_for<P: Hash>(&self, partition: P) -> bool {
150 self.responsible_worker(partition) == self.worker_id
151 }
152}
153
154pub fn create_raw_source<'g, G: Scope<Timestamp = ()>, C>(
167 scope: &mut Child<'g, G, mz_repr::Timestamp>,
168 storage_state: &crate::storage_state::StorageState,
169 committed_upper: StreamVec<Child<'g, G, mz_repr::Timestamp>, ()>,
170 config: &RawSourceCreationConfig,
171 source_connection: C,
172 start_signal: impl std::future::Future<Output = ()> + 'static,
173) -> (
174 BTreeMap<
175 GlobalId,
176 VecCollection<
177 Child<'g, G, mz_repr::Timestamp>,
178 Result<SourceOutput<C::Time>, DataflowError>,
179 Diff,
180 >,
181 >,
182 StreamVec<G, HealthStatusMessage>,
183 Vec<PressOnDropButton>,
184)
185where
186 C: SourceConnection + SourceRender + Clone + 'static,
187{
188 let worker_id = config.worker_id;
189 let id = config.id;
190
191 let mut tokens = vec![];
192
193 let (probed_upper_tx, probed_upper_rx) = watch::channel(None);
194
195 let source_metrics = Arc::new(config.metrics.get_source_metrics(id, worker_id));
196
197 let timestamp_desc = source_connection.timestamp_desc();
198
199 let (remap_collection, remap_token) = remap_operator(
200 scope,
201 storage_state,
202 config.clone(),
203 probed_upper_rx,
204 timestamp_desc,
205 );
206 let remap_collection = remap_collection.inner.broadcast().as_collection();
208 tokens.push(remap_token);
209
210 let committed_upper = reclock_committed_upper(
211 remap_collection.clone(),
212 config.as_of.clone(),
213 committed_upper,
214 id,
215 Arc::clone(&source_metrics),
216 );
217
218 let mut reclocked_exports = BTreeMap::new();
219
220 let reclocked_exports2 = &mut reclocked_exports;
221 let (health, source_tokens) = scope.parent.scoped("SourceTimeDomain", move |scope| {
222 let (exports, health_stream, source_tokens) = source_render_operator(
223 scope,
224 config,
225 source_connection,
226 probed_upper_tx,
227 committed_upper,
228 start_signal,
229 );
230
231 for (id, export) in exports {
232 let (reclock_pusher, reclocked) =
233 reclock(remap_collection.clone(), config.as_of.clone());
234 export
235 .inner
236 .map(move |(result, from_time, diff)| {
237 let result = match result {
238 Ok(msg) => Ok(SourceOutput {
239 key: msg.key.clone(),
240 value: msg.value.clone(),
241 metadata: msg.metadata.clone(),
242 from_time: from_time.clone(),
243 }),
244 Err(err) => Err(err.clone()),
245 };
246 (result, from_time.clone(), *diff)
247 })
248 .capture_into(PusherCapture(reclock_pusher));
249 reclocked_exports2.insert(id, reclocked);
250 }
251
252 (health_stream.leave(), source_tokens)
253 });
254
255 tokens.extend(source_tokens);
256
257 (reclocked_exports, health, tokens)
258}
259
260fn source_render_operator<G, C>(
263 scope: &mut G,
264 config: &RawSourceCreationConfig,
265 source_connection: C,
266 probed_upper_tx: watch::Sender<Option<Probe<C::Time>>>,
267 resume_uppers: impl futures::Stream<Item = Antichain<C::Time>> + 'static,
268 start_signal: impl std::future::Future<Output = ()> + 'static,
269) -> (
270 BTreeMap<GlobalId, StackedCollection<G, Result<SourceMessage, DataflowError>>>,
271 StreamVec<G, HealthStatusMessage>,
272 Vec<PressOnDropButton>,
273)
274where
275 G: Scope<Timestamp = C::Time>,
276 C: SourceRender + 'static,
277{
278 let source_id = config.id;
279 let worker_id = config.worker_id;
280
281 let resume_uppers = resume_uppers.inspect(move |upper| {
282 let upper = upper.pretty();
283 trace!(%upper, "timely-{worker_id} source({source_id}) received resume upper");
284 });
285
286 let (exports, health, probe_stream, tokens) =
287 source_connection.render(scope, config, resume_uppers, start_signal);
288
289 let mut export_collections = BTreeMap::new();
290
291 let source_metrics = config.metrics.get_source_metrics(config.id, worker_id);
292
293 let resume_upper = Antichain::from_iter(
295 config
296 .resume_uppers
297 .values()
298 .flat_map(|f| f.iter().cloned()),
299 );
300 source_metrics
301 .resume_upper
302 .set(mz_persist_client::metrics::encode_ts_metric(&resume_upper));
303
304 let mut health_streams = vec![];
305
306 for (id, export) in exports {
307 let name = format!("SourceGenericStats({})", id);
308 let mut builder = OperatorBuilderRc::new(name, scope.clone());
309
310 let (health_output, derived_health) = builder.new_output();
311 let mut health_output =
312 OutputBuilder::<_, CapacityContainerBuilder<_>>::from(health_output);
313 health_streams.push(derived_health);
314
315 let (output, new_export) = builder.new_output();
316 let mut output = OutputBuilder::<_, CapacityContainerBuilder<_>>::from(output);
317
318 let mut input = builder.new_input(export.inner, Pipeline);
319 export_collections.insert(id, new_export.as_collection());
320
321 let bytes_read_counter = config.metrics.source_defs.bytes_read.clone();
322 let source_statistics = config
323 .statistics
324 .get(&id)
325 .expect("statistics initialized")
326 .clone();
327
328 builder.build(move |mut caps| {
329 let mut health_cap = Some(caps.remove(0));
330
331 move |frontiers| {
332 let mut last_status = None;
333 let mut health_output = health_output.activate();
334
335 if frontiers[0].is_empty() {
336 health_cap = None;
337 return;
338 }
339 let health_cap = health_cap.as_mut().unwrap();
340
341 input.for_each(|cap, data| {
342 for (message, _, _) in data.iter() {
343 match message {
344 Ok(message) => {
345 source_statistics.inc_messages_received_by(1);
346 let key_len = u64::cast_from(message.key.byte_len());
347 let value_len = u64::cast_from(message.value.byte_len());
348 bytes_read_counter.inc_by(key_len + value_len);
349 source_statistics.inc_bytes_received_by(key_len + value_len);
350 }
351 Err(error) => {
352 let update = HealthStatusUpdate::stalled(
356 error.to_string(),
357 Some(
358 "retracting the errored value may resume the source"
359 .to_string(),
360 ),
361 );
362 let status = HealthStatusMessage {
363 id: Some(id),
364 namespace: C::STATUS_NAMESPACE.clone(),
365 update,
366 };
367 if last_status.as_ref() != Some(&status) {
368 last_status = Some(status.clone());
369 health_output.session(&health_cap).give(status);
370 }
371 }
372 }
373 }
374 let mut output = output.activate();
375 output.session(&cap).give_container(data);
376 });
377 }
378 });
379 }
380
381 probe_stream.broadcast().inspect(move |probe| {
385 let _ = probed_upper_tx.send(Some(probe.clone()));
387 });
388
389 (
390 export_collections,
391 health.concatenate_flatten::<_, CapacityContainerBuilder<_>>(health_streams),
392 tokens,
393 )
394}
395
396fn remap_operator<G, FromTime>(
402 scope: &G,
403 storage_state: &crate::storage_state::StorageState,
404 config: RawSourceCreationConfig,
405 mut probed_upper: watch::Receiver<Option<Probe<FromTime>>>,
406 remap_relation_desc: RelationDesc,
407) -> (VecCollection<G, FromTime, Diff>, PressOnDropButton)
408where
409 G: Scope<Timestamp = mz_repr::Timestamp>,
410 FromTime: SourceTimestamp,
411{
412 let RawSourceCreationConfig {
413 name,
414 id,
415 source_exports: _,
416 worker_id,
417 worker_count,
418 timestamp_interval: _,
419 remap_metadata,
420 as_of,
421 resume_uppers: _,
422 source_resume_uppers: _,
423 metrics: _,
424 now_fn,
425 persist_clients,
426 statistics: _,
427 shared_remap_upper,
428 config: _,
429 remap_collection_id,
430 busy_signal: _,
431 } = config;
432
433 let read_only_rx = storage_state.read_only_rx.clone();
434 let error_handler = storage_state.error_handler("remap_operator", id);
435
436 let chosen_worker = usize::cast_from(id.hashed() % u64::cast_from(worker_count));
437 let active_worker = chosen_worker == worker_id;
438
439 let operator_name = format!("remap({})", id);
440 let mut remap_op = AsyncOperatorBuilder::new(operator_name, scope.clone());
441 let (remap_output, remap_stream) = remap_op.new_output::<CapacityContainerBuilder<_>>();
442
443 let button = remap_op.build(move |capabilities| async move {
444 if !active_worker {
445 shared_remap_upper.borrow_mut().clear();
448 return;
449 }
450
451 let mut cap_set = CapabilitySet::from_elem(capabilities.into_element());
452
453 let remap_handle = crate::source::reclock::compat::PersistHandle::<FromTime, _>::new(
454 Arc::clone(&persist_clients),
455 read_only_rx,
456 remap_metadata.clone(),
457 as_of.clone(),
458 shared_remap_upper,
459 id,
460 "remap",
461 worker_id,
462 worker_count,
463 remap_relation_desc,
464 remap_collection_id,
465 )
466 .await;
467
468 let remap_handle = match remap_handle {
469 Ok(handle) => handle,
470 Err(e) => {
471 error_handler
472 .report_and_stop(
473 e.context(format!("Failed to create remap handle for source {name}")),
474 )
475 .await
476 }
477 };
478
479 let (mut timestamper, mut initial_batch) = ReclockOperator::new(remap_handle).await;
480
481 trace!(
484 "timely-{worker_id} remap({id}) emitting remap snapshot: trace_updates={:?}",
485 &initial_batch.updates
486 );
487
488 let cap = cap_set.delayed(cap_set.first().unwrap());
489 remap_output.give_container(&cap, &mut initial_batch.updates);
490 drop(cap);
491 cap_set.downgrade(initial_batch.upper);
492
493 let mut prev_probe_ts: Option<mz_repr::Timestamp> = None;
494
495 while !cap_set.is_empty() {
496 let new_probe = probed_upper
498 .wait_for(|new_probe| match (prev_probe_ts, new_probe) {
499 (None, Some(_)) => true,
500 (Some(prev_ts), Some(new)) => prev_ts < new.probe_ts,
501 _ => false,
502 })
503 .await
504 .map(|probe| (*probe).clone())
505 .unwrap_or_else(|_| {
506 Some(Probe {
507 probe_ts: now_fn().into(),
508 upstream_frontier: Antichain::new(),
509 })
510 });
511
512 let probe = new_probe.expect("known to be Some");
513 prev_probe_ts = Some(probe.probe_ts);
514
515 let binding_ts = probe.probe_ts;
516 let cur_source_upper = probe.upstream_frontier;
517
518 let new_into_upper = Antichain::from_elem(binding_ts.step_forward());
519
520 let mut remap_trace_batch = timestamper
521 .mint(binding_ts, new_into_upper, cur_source_upper.borrow())
522 .await;
523
524 trace!(
525 "timely-{worker_id} remap({id}) minted new bindings: \
526 updates={:?} \
527 source_upper={} \
528 trace_upper={}",
529 &remap_trace_batch.updates,
530 cur_source_upper.pretty(),
531 remap_trace_batch.upper.pretty()
532 );
533
534 let cap = cap_set.delayed(cap_set.first().unwrap());
535 remap_output.give_container(&cap, &mut remap_trace_batch.updates);
536 cap_set.downgrade(remap_trace_batch.upper);
537 }
538 });
539
540 (remap_stream.as_collection(), button.press_on_drop())
541}
542
543fn reclock_committed_upper<G, FromTime>(
547 bindings: VecCollection<G, FromTime, Diff>,
548 as_of: Antichain<G::Timestamp>,
549 committed_upper: StreamVec<G, ()>,
550 id: GlobalId,
551 metrics: Arc<SourceMetrics>,
552) -> impl futures::stream::Stream<Item = Antichain<FromTime>> + 'static
553where
554 G: Scope,
555 G::Timestamp: Lattice + TotalOrder,
556 FromTime: SourceTimestamp,
557{
558 let (tx, rx) = watch::channel(Antichain::from_elem(FromTime::minimum()));
559 let scope = bindings.scope().clone();
560
561 let name = format!("ReclockCommitUpper({id})");
562 let mut builder = OperatorBuilderRc::new(name, scope);
563
564 let mut bindings = builder.new_input(bindings.inner.clone(), Pipeline);
565 let _ = builder.new_input(committed_upper.clone(), Pipeline);
566
567 builder.build(move |_| {
568 use timely::progress::ChangeBatch;
570 let mut accepted_times: ChangeBatch<(G::Timestamp, FromTime)> = ChangeBatch::new();
571 let mut upper = Antichain::from_elem(Timestamp::minimum());
573 let mut ready_times = VecDeque::new();
575 let mut source_upper = MutableAntichain::new();
576
577 move |frontiers| {
578 bindings.for_each(|_, data| {
580 accepted_times.extend(data.drain(..).map(|(from, mut into, diff)| {
581 into.advance_by(as_of.borrow());
582 ((into, from), diff.into_inner())
583 }));
584 });
585 let new_upper = frontiers[0].frontier();
587 if PartialOrder::less_than(&upper.borrow(), &new_upper) {
588 upper = new_upper.to_owned();
589 let mut pending_times = std::mem::take(&mut accepted_times).into_inner();
592 pending_times.sort_unstable_by(|a, b| a.0.cmp(&b.0));
594 for ((into, from), diff) in pending_times.drain(..) {
595 if !upper.less_equal(&into) {
596 ready_times.push_back((from, into, diff));
597 } else {
598 accepted_times.update((into, from), diff);
599 }
600 }
601 }
602
603 if as_of.iter().all(|t| !upper.less_equal(t)) {
605 let committed_upper = frontiers[1].frontier();
606 if as_of.iter().all(|t| !committed_upper.less_equal(t)) {
607 let reclocked_upper = match committed_upper.as_option() {
648 Some(t_next) => {
649 let idx = ready_times.partition_point(|(_, t, _)| t < t_next);
650 let updates = ready_times
651 .drain(0..idx)
652 .map(|(from_time, _, diff)| (from_time, diff));
653 source_upper.update_iter(updates);
654 source_upper.frontier().to_owned()
657 }
658 None => Antichain::new(),
659 };
660 tx.send_replace(reclocked_upper);
661 }
662 }
663
664 metrics
665 .commit_upper_accepted_times
666 .set(u64::cast_from(accepted_times.len()));
667 metrics
668 .commit_upper_ready_times
669 .set(u64::cast_from(ready_times.len()));
670 }
671 });
672
673 WatchStream::from_changes(rx)
674}