1use std::cmp::Ordering;
13use std::collections::{BTreeMap, BTreeSet};
14use std::fmt::Debug;
15use std::future::Future;
16use std::sync::Arc;
17
18use differential_dataflow::difference::Monoid;
19use differential_dataflow::lattice::Lattice;
20use futures::Stream;
21use mz_ore::instrument;
22use mz_ore::task::AbortOnDropHandle;
23use mz_persist_client::cfg::USE_CRITICAL_SINCE_TXN;
24use mz_persist_client::critical::SinceHandle;
25use mz_persist_client::read::{Cursor, LazyPartStats, ListenEvent, ReadHandle, Since, Subscribe};
26use mz_persist_client::stats::{SnapshotPartsStats, SnapshotStats};
27use mz_persist_client::write::WriteHandle;
28use mz_persist_client::{Diagnostics, PersistClient, ShardId};
29use mz_persist_types::txn::{TxnsCodec, TxnsEntry};
30use mz_persist_types::{Codec, Codec64, StepForward};
31use timely::order::TotalOrder;
32use timely::progress::{Antichain, Timestamp};
33use tokio::sync::{mpsc, oneshot};
34use tracing::{debug, warn};
35use uuid::Uuid;
36
37use crate::TxnsCodecDefault;
38use crate::txn_cache::{TxnsCache, TxnsCacheState};
39
40#[derive(Debug)]
47#[cfg_attr(test, derive(PartialEq))]
48pub struct DataSnapshot<T> {
49 pub(crate) data_id: ShardId,
51 pub(crate) latest_write: Option<T>,
54 pub(crate) as_of: T,
56 pub(crate) empty_to: T,
59}
60
61impl<T: Timestamp + Lattice + TotalOrder + Codec64 + Sync> DataSnapshot<T> {
62 #[instrument(level = "debug", fields(shard = %self.data_id, ts = ?self.as_of, empty_to = ?self.empty_to))]
65 pub(crate) async fn unblock_read<K, V, D>(&self, mut data_write: WriteHandle<K, V, T, D>)
66 where
67 K: Debug + Codec,
68 V: Debug + Codec,
69 D: Debug + Monoid + Ord + Codec64 + Send + Sync,
70 {
71 debug!(
72 "unblock_read latest_write={:?} as_of={:?} for {:.9}",
73 self.latest_write,
74 self.as_of,
75 self.data_id.to_string()
76 );
77 if let Some(latest_write) = self.latest_write.as_ref() {
79 let () = data_write
80 .wait_for_upper_past(&Antichain::from_elem(latest_write.clone()))
81 .await;
82 }
83
84 let Some(mut data_upper) = data_write.shared_upper().into_option() else {
96 debug!(
99 "CaA data snapshot {:.9} shard finalized",
100 self.data_id.to_string(),
101 );
102 return;
103 };
104
105 while data_upper < self.empty_to {
109 if let Some(latest_write) = self.latest_write.as_ref() {
113 assert!(latest_write < &data_upper);
114 }
115 assert!(self.as_of < self.empty_to);
116 let res = crate::small_caa(
117 || format!("data {:.9} unblock reads", self.data_id.to_string()),
118 &mut data_write,
119 &[],
120 data_upper.clone(),
121 self.empty_to.clone(),
122 )
123 .await;
124 match res {
125 Ok(()) => {
126 data_write.expire().await;
130 break;
131 }
132 Err(new_data_upper) => {
133 data_upper = new_data_upper;
134 continue;
135 }
136 }
137 }
138 }
139
140 pub async fn snapshot_and_fetch<K, V, D>(
142 &self,
143 data_read: &mut ReadHandle<K, V, T, D>,
144 ) -> Result<Vec<((K, V), T, D)>, Since<T>>
145 where
146 K: Debug + Codec + Ord,
147 V: Debug + Codec + Ord,
148 D: Debug + Monoid + Ord + Codec64 + Send + Sync,
149 {
150 let data_write = WriteHandle::from_read(data_read, "unblock_read");
151 self.unblock_read(data_write).await;
152 data_read
153 .snapshot_and_fetch(Antichain::from_elem(self.as_of.clone()))
154 .await
155 }
156
157 pub async fn snapshot_cursor<K, V, D>(
159 &self,
160 data_read: &mut ReadHandle<K, V, T, D>,
161 should_fetch_part: impl for<'a> Fn(Option<&'a LazyPartStats>) -> bool,
162 ) -> Result<Cursor<K, V, T, D>, Since<T>>
163 where
164 K: Debug + Codec + Ord,
165 V: Debug + Codec + Ord,
166 D: Debug + Monoid + Ord + Codec64 + Send + Sync,
167 {
168 let data_write = WriteHandle::from_read(data_read, "unblock_read");
169 self.unblock_read(data_write).await;
170 data_read
171 .snapshot_cursor(Antichain::from_elem(self.as_of.clone()), should_fetch_part)
172 .await
173 }
174
175 pub async fn snapshot_and_stream<K, V, D>(
177 &self,
178 data_read: &mut ReadHandle<K, V, T, D>,
179 ) -> Result<impl Stream<Item = ((K, V), T, D)> + use<K, V, T, D>, Since<T>>
180 where
181 K: Debug + Codec + Ord,
182 V: Debug + Codec + Ord,
183 D: Debug + Monoid + Ord + Codec64 + Send + Sync,
184 {
185 let data_write = WriteHandle::from_read(data_read, "unblock_read");
186 self.unblock_read(data_write).await;
187 data_read
188 .snapshot_and_stream(Antichain::from_elem(self.as_of.clone()))
189 .await
190 }
191
192 pub fn snapshot_stats_from_critical<K, V, D>(
194 &self,
195 data_since: &SinceHandle<K, V, T, D>,
196 ) -> impl Future<Output = Result<SnapshotStats, Since<T>>> + Send + 'static
197 where
198 K: Debug + Codec + Ord,
199 V: Debug + Codec + Ord,
200 D: Monoid + Codec64 + Send + Sync,
201 {
202 let as_of = self.latest_write.clone().map(Antichain::from_elem);
219 data_since.snapshot_stats(as_of)
220 }
221
222 pub fn snapshot_stats_from_leased<K, V, D>(
224 &self,
225 data_since: &ReadHandle<K, V, T, D>,
226 ) -> impl Future<Output = Result<SnapshotStats, Since<T>>> + Send + 'static
227 where
228 K: Debug + Codec + Ord,
229 V: Debug + Codec + Ord,
230 D: Ord + Monoid + Codec64 + Send + Sync,
231 {
232 let as_of = self.latest_write.clone().map(Antichain::from_elem);
249 data_since.snapshot_stats(as_of)
250 }
251
252 pub async fn snapshot_parts_stats<K, V, D>(
254 &self,
255 data_read: &ReadHandle<K, V, T, D>,
256 ) -> Result<SnapshotPartsStats, Since<T>>
257 where
258 K: Debug + Codec + Ord,
259 V: Debug + Codec + Ord,
260 D: Debug + Monoid + Ord + Codec64 + Send + Sync,
261 {
262 let data_write = WriteHandle::from_read(data_read, "unblock_read");
263 self.unblock_read(data_write).await;
264 data_read
265 .snapshot_parts_stats(Antichain::from_elem(self.as_of.clone()))
266 .await
267 }
268
269 pub(crate) fn validate(&self) -> Result<(), String> {
270 if let Some(latest_write) = self.latest_write.as_ref() {
271 if !(latest_write <= &self.as_of) {
272 return Err(format!(
273 "latest_write {:?} not <= as_of {:?}",
274 self.latest_write, self.as_of
275 ));
276 }
277 }
278 if !(self.as_of < self.empty_to) {
279 return Err(format!(
280 "as_of {:?} not < empty_to {:?}",
281 self.as_of, self.empty_to
282 ));
283 }
284 Ok(())
285 }
286}
287
288#[derive(Debug)]
292#[cfg_attr(any(test, debug_assertions), derive(PartialEq))]
293pub enum DataListenNext<T> {
294 ReadDataTo(T),
297 EmitLogicalProgress(T),
301 WaitForTxnsProgress,
305}
306
307#[derive(Debug, Clone, serde::Serialize, serde::Deserialize)]
315pub struct DataRemapEntry<T> {
316 pub physical_upper: T,
318 pub logical_upper: T,
321}
322
323#[derive(Debug)]
325pub(crate) struct DataSubscribe<T> {
326 pub(crate) data_id: ShardId,
328 pub(crate) snapshot: Option<DataSnapshot<T>>,
331 pub(crate) remap: DataRemapEntry<T>,
333}
334
335#[derive(Debug)]
337pub struct DataSubscription<T> {
338 subscribe: DataSubscribe<T>,
340 tx: mpsc::UnboundedSender<DataRemapEntry<T>>,
342}
343
344#[derive(Debug, Clone)]
346pub struct TxnsRead<T> {
347 txns_id: ShardId,
348 tx: mpsc::UnboundedSender<TxnsReadCmd<T>>,
349 _read_task: Arc<AbortOnDropHandle<()>>,
350 _subscribe_task: Arc<AbortOnDropHandle<()>>,
351}
352
353impl<T: Timestamp + Lattice + Codec64 + Sync> TxnsRead<T> {
354 pub async fn start<C>(client: PersistClient, txns_id: ShardId) -> Self
356 where
357 T: TotalOrder + StepForward,
358 C: TxnsCodec + 'static,
359 {
360 let (tx, rx) = mpsc::unbounded_channel();
361
362 let (mut subscribe_task, cache) =
363 TxnsSubscribeTask::<T, C>::open(&client, txns_id, None, tx.clone()).await;
364
365 let mut task = TxnsReadTask {
366 rx,
367 cache,
368 pending_waits_by_ts: BTreeSet::new(),
369 pending_waits_by_id: BTreeMap::new(),
370 data_subscriptions: Vec::new(),
371 };
372
373 let read_task =
374 mz_ore::task::spawn(|| "txn-wal::read_task", async move { task.run().await });
375
376 let subscribe_task = mz_ore::task::spawn(|| "txn-wal::subscribe_task", async move {
377 subscribe_task.run().await
378 });
379
380 TxnsRead {
381 txns_id,
382 tx,
383 _read_task: Arc::new(read_task.abort_on_drop()),
384 _subscribe_task: Arc::new(subscribe_task.abort_on_drop()),
385 }
386 }
387
388 pub fn txns_id(&self) -> &ShardId {
390 &self.txns_id
391 }
392
393 pub async fn data_snapshot(&self, data_id: ShardId, as_of: T) -> DataSnapshot<T> {
395 self.send(|tx| TxnsReadCmd::DataSnapshot { data_id, as_of, tx })
396 .await
397 }
398
399 pub(crate) async fn data_subscribe<K, V, D>(
403 &self,
404 data_id: ShardId,
405 as_of: T,
406 unblock: WriteHandle<K, V, T, D>,
407 ) -> mpsc::UnboundedReceiver<DataRemapEntry<T>>
408 where
409 K: Debug + Codec,
410 V: Debug + Codec,
411 T: Timestamp + Lattice + TotalOrder + StepForward + Codec64,
412 D: Debug + Monoid + Ord + Codec64 + Send + Sync,
413 {
414 let (snapshot, rest) = self
415 .send(|tx| TxnsReadCmd::DataSubscribe { data_id, as_of, tx })
416 .await;
417 if let Some(snapshot) = snapshot {
418 snapshot.unblock_read(unblock).await;
419 }
420 rest
421 }
422
423 pub async fn update_ge(&self, ts: T) {
425 let wait = WaitTs::GreaterEqual(ts);
426 self.update(wait).await
427 }
428
429 pub async fn update_gt(&self, ts: T) {
431 let wait = WaitTs::GreaterThan(ts);
432 self.update(wait).await
433 }
434
435 async fn update(&self, wait: WaitTs<T>) {
436 let id = Uuid::new_v4();
437 let res = self.send(|tx| TxnsReadCmd::Wait {
438 id: id.clone(),
439 ts: wait,
440 tx,
441 });
442
443 let mut cancel_guard = CancelWaitOnDrop {
446 id,
447 tx: Some(self.tx.clone()),
448 };
449
450 let res = res.await;
451
452 cancel_guard.complete();
454
455 res
456 }
457
458 async fn send<R: std::fmt::Debug>(
459 &self,
460 cmd: impl FnOnce(oneshot::Sender<R>) -> TxnsReadCmd<T>,
461 ) -> R {
462 let (tx, rx) = oneshot::channel();
463 let req = cmd(tx);
464 let () = self.tx.send(req).expect("task unexpectedly shut down");
465 rx.await.expect("task unexpectedly shut down")
466 }
467}
468
469struct CancelWaitOnDrop<T> {
472 id: Uuid,
473 tx: Option<mpsc::UnboundedSender<TxnsReadCmd<T>>>,
474}
475
476impl<T> CancelWaitOnDrop<T> {
477 pub fn complete(&mut self) {
480 self.tx.take();
481 }
482}
483
484impl<T> Drop for CancelWaitOnDrop<T> {
485 fn drop(&mut self) {
486 let tx = match self.tx.take() {
487 Some(tx) => tx,
488 None => {
489 return;
491 }
492 };
493
494 let _ = tx.send(TxnsReadCmd::CancelWait {
495 id: self.id.clone(),
496 });
497 }
498}
499
500#[derive(Debug)]
501enum TxnsReadCmd<T> {
502 Updates {
503 entries: Vec<(TxnsEntry, T, i64)>,
504 frontier: T,
505 },
506 DataSnapshot {
507 data_id: ShardId,
508 as_of: T,
509 tx: oneshot::Sender<DataSnapshot<T>>,
510 },
511 DataSubscribe {
512 data_id: ShardId,
513 as_of: T,
514 tx: oneshot::Sender<(
515 Option<DataSnapshot<T>>,
516 mpsc::UnboundedReceiver<DataRemapEntry<T>>,
517 )>,
518 },
519 Wait {
520 id: Uuid,
521 ts: WaitTs<T>,
522 tx: oneshot::Sender<()>,
523 },
524 CancelWait {
525 id: Uuid,
526 },
527}
528
529#[derive(Debug, PartialEq, Eq, Clone)]
530enum WaitTs<T> {
531 GreaterEqual(T),
532 GreaterThan(T),
533}
534
535impl<T: Ord> Ord for WaitTs<T> {
543 fn cmp(&self, other: &Self) -> std::cmp::Ordering {
544 let self_ts = match self {
545 WaitTs::GreaterEqual(ts) => ts,
546 WaitTs::GreaterThan(ts) => ts,
547 };
548 let other_ts = match other {
549 WaitTs::GreaterEqual(ts) => ts,
550 WaitTs::GreaterThan(ts) => ts,
551 };
552
553 if self_ts < other_ts {
554 Ordering::Less
555 } else if *self_ts > *other_ts {
556 Ordering::Greater
557 } else if matches!(self, WaitTs::GreaterEqual(_)) && matches!(other, WaitTs::GreaterThan(_))
558 {
559 Ordering::Less
560 } else if matches!(self, WaitTs::GreaterThan(_)) && matches!(other, WaitTs::GreaterEqual(_))
561 {
562 Ordering::Greater
563 } else {
564 Ordering::Equal
565 }
566 }
567}
568
569impl<T: Ord> PartialOrd for WaitTs<T> {
570 fn partial_cmp(&self, other: &Self) -> Option<Ordering> {
571 Some(self.cmp(other))
572 }
573}
574
575impl<T: Timestamp + Lattice> WaitTs<T> {
576 fn is_ready(&self, frontier: &T) -> bool {
578 match &self {
579 WaitTs::GreaterEqual(ts) => {
580 if frontier >= ts {
581 return true;
582 }
583 }
584 WaitTs::GreaterThan(ts) => {
585 if frontier > ts {
586 return true;
587 }
588 }
589 };
590
591 false
592 }
593}
594
595#[derive(Debug)]
596struct TxnsReadTask<T> {
597 rx: mpsc::UnboundedReceiver<TxnsReadCmd<T>>,
598 cache: TxnsCacheState<T>,
599 pending_waits_by_ts: BTreeSet<(WaitTs<T>, Uuid)>,
600 pending_waits_by_id: BTreeMap<Uuid, PendingWait<T>>,
601 data_subscriptions: Vec<DataSubscription<T>>,
602}
603
604#[derive(Debug)]
607struct PendingWait<T> {
608 ts: WaitTs<T>,
609 tx: Option<oneshot::Sender<()>>,
610}
611
612impl<T: Timestamp + Lattice + Codec64> PendingWait<T> {
613 fn maybe_complete(&mut self, frontier: &T) -> bool {
618 if self.tx.is_none() {
619 return true;
621 }
622
623 if self.ts.is_ready(frontier) {
624 let _ = self.tx.take().expect("known to exist").send(());
625 return true;
626 }
627
628 if let Some(tx) = self.tx.as_ref() {
629 if tx.is_closed() {
630 self.tx.take();
632 return true;
633 }
634 }
635
636 false
637 }
638}
639
640impl<T> TxnsReadTask<T>
641where
642 T: Timestamp + Lattice + TotalOrder + StepForward + Codec64 + Sync,
643{
644 async fn run(&mut self) {
645 while let Some(cmd) = self.rx.recv().await {
646 match cmd {
647 TxnsReadCmd::Updates { entries, frontier } => {
648 tracing::trace!(
649 "updates from subscribe task at ({:?}): {:?}",
650 frontier,
651 entries
652 );
653
654 self.cache.push_entries(entries.clone(), frontier.clone());
655
656 self.data_subscriptions
657 .retain(|subscription| !subscription.tx.is_closed());
658 for subscription in &mut self.data_subscriptions {
659 Self::update_subscription(subscription, &self.cache);
660 }
661
662 loop {
666 let first_wait = self.pending_waits_by_ts.first();
667
668 let (wait_ts, id) = match first_wait {
669 Some(wait) => wait,
670 None => break,
671 };
672
673 let completed = wait_ts.is_ready(&frontier);
674
675 if completed {
676 let mut wait = self
677 .pending_waits_by_id
678 .remove(id)
679 .expect("wait must be in map");
680
681 let really_completed = wait.maybe_complete(&frontier);
682 assert!(really_completed);
683
684 self.pending_waits_by_ts.pop_first();
685 } else {
686 break;
689 }
690 }
691 }
692 TxnsReadCmd::DataSnapshot { data_id, as_of, tx } => {
693 let res = self.cache.data_snapshot(data_id, as_of.clone());
694 let _ = tx.send(res);
695 }
696 TxnsReadCmd::DataSubscribe { data_id, as_of, tx } => {
697 let mut subscribe = self.cache.data_subscribe(data_id, as_of.clone());
698 let snapshot = subscribe.snapshot.take();
699 let (sub_tx, sub_rx) = mpsc::unbounded_channel();
700 sub_tx
702 .send(subscribe.remap.clone())
703 .expect("receiver still in scope");
704 let mut subscription = DataSubscription {
705 subscribe,
706 tx: sub_tx,
707 };
708 Self::update_subscription(&mut subscription, &self.cache);
710 self.data_subscriptions.push(subscription);
711 let _ = tx.send((snapshot, sub_rx));
712 }
713 TxnsReadCmd::Wait { id, ts, tx } => {
714 let mut pending_wait = PendingWait { ts, tx: Some(tx) };
715 let completed = pending_wait.maybe_complete(&self.cache.progress_exclusive);
716 if !completed {
717 let wait_ts = pending_wait.ts.clone();
718 self.pending_waits_by_ts.insert((wait_ts, id.clone()));
719 self.pending_waits_by_id.insert(id, pending_wait);
720 }
721 }
722 TxnsReadCmd::CancelWait { id } => {
723 if let Some(pending_wait) = self.pending_waits_by_id.remove(&id) {
728 self.pending_waits_by_ts.remove(&(pending_wait.ts, id));
729 }
730 }
731 }
732 }
733 warn!("TxnsReadTask shutting down");
734 }
735
736 fn update_subscription(subscription: &mut DataSubscription<T>, cache: &TxnsCacheState<T>) {
737 loop {
738 match cache.data_listen_next(
739 &subscription.subscribe.data_id,
740 &subscription.subscribe.remap.logical_upper,
741 ) {
742 DataListenNext::ReadDataTo(new_upper) => {
744 subscription.subscribe.remap.physical_upper = new_upper.clone();
746 subscription.subscribe.remap.logical_upper = new_upper.clone();
747 }
748 DataListenNext::EmitLogicalProgress(new_progress) => {
751 assert!(subscription.subscribe.remap.physical_upper < new_progress);
752 assert!(subscription.subscribe.remap.logical_upper < new_progress);
753
754 subscription.subscribe.remap.logical_upper = new_progress.clone();
755 }
756 DataListenNext::WaitForTxnsProgress => break,
759 };
760 let _ = subscription.tx.send(subscription.subscribe.remap.clone());
762 }
763 assert_eq!(
764 cache.progress_exclusive, subscription.subscribe.remap.logical_upper,
765 "we should update the subscription up to the current progress_exclusive"
766 );
767 }
768}
769
770#[derive(Debug)]
773struct TxnsSubscribeTask<T, C: TxnsCodec = TxnsCodecDefault> {
774 txns_subscribe: Subscribe<C::Key, C::Val, T, i64>,
775
776 buf: Vec<(TxnsEntry, T, i64)>,
779
780 tx: mpsc::UnboundedSender<TxnsReadCmd<T>>,
782
783 only_data_id: Option<ShardId>,
790}
791
792impl<T, C> TxnsSubscribeTask<T, C>
793where
794 T: Timestamp + Lattice + TotalOrder + StepForward + Codec64 + Sync,
795 C: TxnsCodec,
796{
797 pub async fn open(
809 client: &PersistClient,
810 txns_id: ShardId,
811 only_data_id: Option<ShardId>,
812 tx: mpsc::UnboundedSender<TxnsReadCmd<T>>,
813 ) -> (Self, TxnsCacheState<T>) {
814 let (txns_key_schema, txns_val_schema) = C::schemas();
815 let txns_read: ReadHandle<<C as TxnsCodec>::Key, <C as TxnsCodec>::Val, T, i64> = client
816 .open_leased_reader(
817 txns_id,
818 Arc::new(txns_key_schema),
819 Arc::new(txns_val_schema),
820 Diagnostics {
821 shard_name: "txns".to_owned(),
822 handle_purpose: "read txns".to_owned(),
823 },
824 USE_CRITICAL_SINCE_TXN.get(client.dyncfgs()),
825 )
826 .await
827 .expect("txns schema shouldn't change");
828 let (state, txns_subscribe) = TxnsCacheState::init::<C>(only_data_id, txns_read).await;
829 let subscribe_task = TxnsSubscribeTask {
830 txns_subscribe,
831 buf: Vec::new(),
832 tx,
833 only_data_id,
834 };
835
836 (subscribe_task, state)
837 }
838
839 async fn run(&mut self) {
840 loop {
841 let events = self.txns_subscribe.next(None).await;
842 for event in events {
843 match event {
844 ListenEvent::Progress(frontier) => {
845 let frontier_ts = frontier
846 .into_option()
847 .expect("nothing should close the txns shard");
848 let entries = std::mem::take(&mut self.buf);
849 let res = self.tx.send(TxnsReadCmd::Updates {
850 entries,
851 frontier: frontier_ts,
852 });
853 if let Err(e) = res {
854 warn!("TxnsSubscribeTask shutting down: {}", e);
855 return;
856 }
857 }
858 ListenEvent::Updates(parts) => {
859 TxnsCache::<T, C>::fetch_parts(
860 self.only_data_id.clone(),
861 &mut self.txns_subscribe,
862 parts,
863 &mut self.buf,
864 )
865 .await;
866 }
867 };
868 }
869 }
870 }
871}
872
873#[cfg(test)]
874mod tests {
875 use super::WaitTs;
876
877 #[mz_ore::test]
878 fn wait_ts_ord() {
879 let mut waits = vec![
880 WaitTs::GreaterThan(3),
881 WaitTs::GreaterThan(2),
882 WaitTs::GreaterEqual(2),
883 WaitTs::GreaterThan(1),
884 ];
885
886 waits.sort();
887
888 let expected = vec![
889 WaitTs::GreaterThan(1),
890 WaitTs::GreaterEqual(2),
891 WaitTs::GreaterThan(2),
892 WaitTs::GreaterThan(3),
893 ];
894
895 assert_eq!(waits, expected);
896 }
897}