1use std::any::Any;
13use std::collections::btree_map;
14use std::collections::{BTreeMap, BTreeSet};
15use std::fmt::{Debug, Display};
16use std::str::FromStr;
17use std::sync::{Arc, Mutex};
18use std::time::Duration;
19
20use async_trait::async_trait;
21use chrono::{DateTime, DurationRound, TimeDelta, Utc};
22use derivative::Derivative;
23use differential_dataflow::lattice::Lattice;
24use futures::FutureExt;
25use futures::StreamExt;
26use itertools::Itertools;
27use mz_build_info::BuildInfo;
28use mz_cluster_client::client::ClusterReplicaLocation;
29use mz_cluster_client::metrics::{ControllerMetrics, WallclockLagMetrics};
30use mz_cluster_client::{ReplicaId, WallclockLagFn};
31use mz_controller_types::dyncfgs::{
32 ENABLE_0DT_DEPLOYMENT_SOURCES, WALLCLOCK_LAG_RECORDING_INTERVAL,
33};
34use mz_ore::collections::CollectionExt;
35use mz_ore::metrics::MetricsRegistry;
36use mz_ore::now::{EpochMillis, NowFn};
37use mz_ore::task::AbortOnDropHandle;
38use mz_ore::{assert_none, halt, instrument, soft_panic_or_log};
39use mz_persist_client::batch::ProtoBatch;
40use mz_persist_client::cache::PersistClientCache;
41use mz_persist_client::cfg::USE_CRITICAL_SINCE_SNAPSHOT;
42use mz_persist_client::read::ReadHandle;
43use mz_persist_client::schema::CaESchema;
44use mz_persist_client::write::WriteHandle;
45use mz_persist_client::{Diagnostics, PersistClient, PersistLocation, ShardId};
46use mz_persist_types::Codec64;
47use mz_persist_types::codec_impls::UnitSchema;
48use mz_repr::adt::timestamp::CheckedTimestamp;
49use mz_repr::{Datum, Diff, GlobalId, RelationDesc, RelationVersion, Row, TimestampManipulation};
50use mz_storage_client::client::{
51 AppendOnlyUpdate, RunIngestionCommand, RunOneshotIngestion, RunSinkCommand, Status,
52 StatusUpdate, StorageCommand, StorageResponse, TableData,
53};
54use mz_storage_client::controller::{
55 BoxFuture, CollectionDescription, DataSource, ExportDescription, ExportState,
56 IntrospectionType, MonotonicAppender, PersistEpoch, Response, StorageController,
57 StorageMetadata, StorageTxn, StorageWriteOp, WallclockLag, WallclockLagHistogramPeriod,
58};
59use mz_storage_client::healthcheck::{
60 MZ_AWS_PRIVATELINK_CONNECTION_STATUS_HISTORY_DESC, MZ_SINK_STATUS_HISTORY_DESC,
61 MZ_SOURCE_STATUS_HISTORY_DESC, REPLICA_STATUS_HISTORY_DESC,
62};
63use mz_storage_client::metrics::StorageControllerMetrics;
64use mz_storage_client::statistics::{
65 ControllerSinkStatistics, ControllerSourceStatistics, WebhookStatistics,
66};
67use mz_storage_client::storage_collections::StorageCollections;
68use mz_storage_types::configuration::StorageConfiguration;
69use mz_storage_types::connections::ConnectionContext;
70use mz_storage_types::connections::inline::InlinedConnection;
71use mz_storage_types::controller::{AlterError, CollectionMetadata, StorageError, TxnsCodecRow};
72use mz_storage_types::errors::CollectionMissing;
73use mz_storage_types::instances::StorageInstanceId;
74use mz_storage_types::oneshot_sources::{OneshotIngestionRequest, OneshotResultCallback};
75use mz_storage_types::parameters::StorageParameters;
76use mz_storage_types::read_holds::ReadHold;
77use mz_storage_types::read_policy::ReadPolicy;
78use mz_storage_types::sinks::{StorageSinkConnection, StorageSinkDesc};
79use mz_storage_types::sources::{
80 GenericSourceConnection, IngestionDescription, SourceConnection, SourceData, SourceDesc,
81 SourceExport, SourceExportDataConfig,
82};
83use mz_storage_types::{AlterCompatible, StorageDiff, dyncfgs};
84use mz_txn_wal::metrics::Metrics as TxnMetrics;
85use mz_txn_wal::txn_read::TxnsRead;
86use mz_txn_wal::txns::TxnsHandle;
87use timely::order::{PartialOrder, TotalOrder};
88use timely::progress::Timestamp as TimelyTimestamp;
89use timely::progress::frontier::MutableAntichain;
90use timely::progress::{Antichain, ChangeBatch, Timestamp};
91use tokio::sync::watch::{Sender, channel};
92use tokio::sync::{mpsc, oneshot};
93use tokio::time::MissedTickBehavior;
94use tokio::time::error::Elapsed;
95use tracing::{debug, info, warn};
96
97use crate::collection_mgmt::{
98 AppendOnlyIntrospectionConfig, CollectionManagerKind, DifferentialIntrospectionConfig,
99};
100use crate::instance::{Instance, ReplicaConfig};
101
102mod collection_mgmt;
103mod history;
104mod instance;
105mod persist_handles;
106mod rtr;
107mod statistics;
108
109#[derive(Derivative)]
110#[derivative(Debug)]
111struct PendingOneshotIngestion {
112 #[derivative(Debug = "ignore")]
114 result_tx: OneshotResultCallback<ProtoBatch>,
115 cluster_id: StorageInstanceId,
117}
118
119impl PendingOneshotIngestion {
120 pub(crate) fn cancel(self) {
124 (self.result_tx)(vec![Err("canceled".to_string())])
125 }
126}
127
128#[derive(Derivative)]
130#[derivative(Debug)]
131pub struct Controller<T: Timestamp + Lattice + Codec64 + From<EpochMillis> + TimestampManipulation>
132{
133 build_info: &'static BuildInfo,
135 now: NowFn,
137
138 read_only: bool,
144
145 pub(crate) collections: BTreeMap<GlobalId, CollectionState<T>>,
150
151 dropped_objects: BTreeMap<GlobalId, BTreeSet<ReplicaId>>,
160
161 pub(crate) persist_table_worker: persist_handles::PersistTableWriteWorker<T>,
163 txns_read: TxnsRead<T>,
165 txns_metrics: Arc<TxnMetrics>,
166 stashed_responses: Vec<(Option<ReplicaId>, StorageResponse<T>)>,
167 #[derivative(Debug = "ignore")]
169 pending_table_handle_drops_tx: mpsc::UnboundedSender<GlobalId>,
170 #[derivative(Debug = "ignore")]
172 pending_table_handle_drops_rx: mpsc::UnboundedReceiver<GlobalId>,
173 #[derivative(Debug = "ignore")]
175 pending_oneshot_ingestions: BTreeMap<uuid::Uuid, PendingOneshotIngestion>,
176
177 pub(crate) collection_manager: collection_mgmt::CollectionManager<T>,
179
180 pub(crate) introspection_ids: BTreeMap<IntrospectionType, GlobalId>,
182 introspection_tokens: Arc<Mutex<BTreeMap<GlobalId, Box<dyn Any + Send + Sync>>>>,
187
188 source_statistics: Arc<Mutex<statistics::SourceStatistics>>,
193 sink_statistics: Arc<Mutex<BTreeMap<(GlobalId, Option<ReplicaId>), ControllerSinkStatistics>>>,
196 statistics_interval_sender: Sender<Duration>,
198
199 instances: BTreeMap<StorageInstanceId, Instance<T>>,
201 initialized: bool,
203 config: StorageConfiguration,
205 persist_location: PersistLocation,
207 persist: Arc<PersistClientCache>,
209 metrics: StorageControllerMetrics,
211 recorded_frontiers: BTreeMap<GlobalId, (Antichain<T>, Antichain<T>)>,
214 recorded_replica_frontiers: BTreeMap<(GlobalId, ReplicaId), Antichain<T>>,
217
218 #[derivative(Debug = "ignore")]
220 wallclock_lag: WallclockLagFn<T>,
221 wallclock_lag_last_recorded: DateTime<Utc>,
223
224 storage_collections: Arc<dyn StorageCollections<Timestamp = T> + Send + Sync>,
226 migrated_storage_collections: BTreeSet<GlobalId>,
228
229 maintenance_ticker: tokio::time::Interval,
231 maintenance_scheduled: bool,
233
234 instance_response_tx: mpsc::UnboundedSender<(Option<ReplicaId>, StorageResponse<T>)>,
236 instance_response_rx: mpsc::UnboundedReceiver<(Option<ReplicaId>, StorageResponse<T>)>,
238
239 persist_warm_task: Option<AbortOnDropHandle<Box<dyn Debug + Send>>>,
241}
242
243fn warm_persist_state_in_background(
248 client: PersistClient,
249 shard_ids: impl Iterator<Item = ShardId> + Send + 'static,
250) -> mz_ore::task::JoinHandle<Box<dyn Debug + Send>> {
251 const MAX_CONCURRENT_WARMS: usize = 16;
253 let logic = async move {
254 let fetchers: Vec<_> = tokio_stream::iter(shard_ids)
255 .map(|shard_id| {
256 let client = client.clone();
257 async move {
258 client
259 .create_batch_fetcher::<SourceData, (), mz_repr::Timestamp, StorageDiff>(
260 shard_id,
261 Arc::new(RelationDesc::empty()),
262 Arc::new(UnitSchema),
263 true,
264 Diagnostics::from_purpose("warm persist load state"),
265 )
266 .await
267 }
268 })
269 .buffer_unordered(MAX_CONCURRENT_WARMS)
270 .collect()
271 .await;
272 let fetchers: Box<dyn Debug + Send> = Box::new(fetchers);
273 fetchers
274 };
275 mz_ore::task::spawn(|| "warm_persist_load_state", logic)
276}
277
278#[async_trait(?Send)]
279impl<T> StorageController for Controller<T>
280where
281 T: Timestamp
282 + Lattice
283 + TotalOrder
284 + Codec64
285 + From<EpochMillis>
286 + TimestampManipulation
287 + Into<Datum<'static>>
288 + Display,
289{
290 type Timestamp = T;
291
292 fn initialization_complete(&mut self) {
293 self.reconcile_dangling_statistics();
294 self.initialized = true;
295
296 for instance in self.instances.values_mut() {
297 instance.send(StorageCommand::InitializationComplete);
298 }
299 }
300
301 fn update_parameters(&mut self, config_params: StorageParameters) {
302 self.storage_collections
303 .update_parameters(config_params.clone());
304
305 self.persist.cfg().apply_from(&config_params.dyncfg_updates);
308
309 for instance in self.instances.values_mut() {
310 let params = Box::new(config_params.clone());
311 instance.send(StorageCommand::UpdateConfiguration(params));
312 }
313 self.config.update(config_params);
314 self.statistics_interval_sender
315 .send_replace(self.config.parameters.statistics_interval);
316 self.collection_manager.update_user_batch_duration(
317 self.config
318 .parameters
319 .user_storage_managed_collections_batch_duration,
320 );
321 }
322
323 fn config(&self) -> &StorageConfiguration {
325 &self.config
326 }
327
328 fn collection_metadata(&self, id: GlobalId) -> Result<CollectionMetadata, CollectionMissing> {
329 self.storage_collections.collection_metadata(id)
330 }
331
332 fn collection_hydrated(
333 &self,
334 collection_id: GlobalId,
335 ) -> Result<bool, StorageError<Self::Timestamp>> {
336 let collection = self.collection(collection_id)?;
337
338 let instance_id = match &collection.data_source {
339 DataSource::Ingestion(ingestion_description) => ingestion_description.instance_id,
340 DataSource::IngestionExport { ingestion_id, .. } => {
341 let ingestion_state = self.collections.get(ingestion_id).expect("known to exist");
342
343 let instance_id = match &ingestion_state.data_source {
344 DataSource::Ingestion(ingestion_desc) => ingestion_desc.instance_id,
345 _ => unreachable!("SourceExport must only refer to primary source"),
346 };
347
348 instance_id
349 }
350 _ => return Ok(true),
351 };
352
353 let instance = self.instances.get(&instance_id).ok_or_else(|| {
354 StorageError::IngestionInstanceMissing {
355 storage_instance_id: instance_id,
356 ingestion_id: collection_id,
357 }
358 })?;
359
360 if instance.replica_ids().next().is_none() {
361 return Ok(true);
364 }
365
366 match &collection.extra_state {
367 CollectionStateExtra::Ingestion(ingestion_state) => {
368 Ok(ingestion_state.hydrated_on.len() >= 1)
370 }
371 CollectionStateExtra::Export(_) => {
372 Ok(true)
377 }
378 CollectionStateExtra::None => {
379 Ok(true)
383 }
384 }
385 }
386
387 #[mz_ore::instrument(level = "debug")]
388 fn collections_hydrated_on_replicas(
389 &self,
390 target_replica_ids: Option<Vec<ReplicaId>>,
391 target_cluster_id: &StorageInstanceId,
392 exclude_collections: &BTreeSet<GlobalId>,
393 ) -> Result<bool, StorageError<Self::Timestamp>> {
394 if target_replica_ids.as_ref().is_some_and(|v| v.is_empty()) {
397 return Ok(true);
398 }
399
400 let target_replicas: Option<BTreeSet<ReplicaId>> =
403 target_replica_ids.map(|ids| ids.into_iter().collect());
404
405 let mut all_hydrated = true;
406 for (collection_id, collection_state) in self.collections.iter() {
407 if collection_id.is_transient() || exclude_collections.contains(collection_id) {
408 continue;
409 }
410 let hydrated = match &collection_state.extra_state {
411 CollectionStateExtra::Ingestion(state) => {
412 if &state.instance_id != target_cluster_id {
413 continue;
414 }
415 match &target_replicas {
416 Some(target_replicas) => !state.hydrated_on.is_disjoint(target_replicas),
417 None => {
418 state.hydrated_on.len() >= 1
421 }
422 }
423 }
424 CollectionStateExtra::Export(_) => {
425 true
430 }
431 CollectionStateExtra::None => {
432 true
436 }
437 };
438 if !hydrated {
439 tracing::info!(%collection_id, "collection is not hydrated on any replica");
440 all_hydrated = false;
441 }
444 }
445 Ok(all_hydrated)
446 }
447
448 fn collection_frontiers(
449 &self,
450 id: GlobalId,
451 ) -> Result<(Antichain<Self::Timestamp>, Antichain<Self::Timestamp>), CollectionMissing> {
452 let frontiers = self.storage_collections.collection_frontiers(id)?;
453 Ok((frontiers.implied_capability, frontiers.write_frontier))
454 }
455
456 fn collections_frontiers(
457 &self,
458 mut ids: Vec<GlobalId>,
459 ) -> Result<Vec<(GlobalId, Antichain<T>, Antichain<T>)>, CollectionMissing> {
460 let mut result = vec![];
461 ids.retain(|&id| match self.export(id) {
466 Ok(export) => {
467 result.push((
468 id,
469 export.input_hold().since().clone(),
470 export.write_frontier.clone(),
471 ));
472 false
473 }
474 Err(_) => true,
475 });
476 result.extend(
477 self.storage_collections
478 .collections_frontiers(ids)?
479 .into_iter()
480 .map(|frontiers| {
481 (
482 frontiers.id,
483 frontiers.implied_capability,
484 frontiers.write_frontier,
485 )
486 }),
487 );
488
489 Ok(result)
490 }
491
492 fn active_collection_metadatas(&self) -> Vec<(GlobalId, CollectionMetadata)> {
493 self.storage_collections.active_collection_metadatas()
494 }
495
496 fn active_ingestion_exports(
497 &self,
498 instance_id: StorageInstanceId,
499 ) -> Box<dyn Iterator<Item = &GlobalId> + '_> {
500 let active_storage_collections: BTreeMap<_, _> = self
501 .storage_collections
502 .active_collection_frontiers()
503 .into_iter()
504 .map(|c| (c.id, c))
505 .collect();
506
507 let active_exports = self.instances[&instance_id]
508 .active_ingestion_exports()
509 .filter(move |id| {
510 let frontiers = active_storage_collections.get(id);
511 match frontiers {
512 Some(frontiers) => !frontiers.write_frontier.is_empty(),
513 None => {
514 false
516 }
517 }
518 });
519
520 Box::new(active_exports)
521 }
522
523 fn check_exists(&self, id: GlobalId) -> Result<(), StorageError<Self::Timestamp>> {
524 self.storage_collections.check_exists(id)
525 }
526
527 fn create_instance(&mut self, id: StorageInstanceId, workload_class: Option<String>) {
528 let metrics = self.metrics.for_instance(id);
529 let mut instance = Instance::new(
530 workload_class,
531 metrics,
532 self.now.clone(),
533 self.instance_response_tx.clone(),
534 );
535 if self.initialized {
536 instance.send(StorageCommand::InitializationComplete);
537 }
538 if !self.read_only {
539 instance.send(StorageCommand::AllowWrites);
540 }
541
542 let params = Box::new(self.config.parameters.clone());
543 instance.send(StorageCommand::UpdateConfiguration(params));
544
545 let old_instance = self.instances.insert(id, instance);
546 assert_none!(old_instance, "storage instance {id} already exists");
547 }
548
549 fn drop_instance(&mut self, id: StorageInstanceId) {
550 let instance = self.instances.remove(&id);
551 assert!(instance.is_some(), "storage instance {id} does not exist");
552 }
553
554 fn update_instance_workload_class(
555 &mut self,
556 id: StorageInstanceId,
557 workload_class: Option<String>,
558 ) {
559 let instance = self
560 .instances
561 .get_mut(&id)
562 .unwrap_or_else(|| panic!("instance {id} does not exist"));
563
564 instance.workload_class = workload_class;
565 }
566
567 fn connect_replica(
568 &mut self,
569 instance_id: StorageInstanceId,
570 replica_id: ReplicaId,
571 location: ClusterReplicaLocation,
572 ) {
573 let instance = self
574 .instances
575 .get_mut(&instance_id)
576 .unwrap_or_else(|| panic!("instance {instance_id} does not exist"));
577
578 let config = ReplicaConfig {
579 build_info: self.build_info,
580 location,
581 grpc_client: self.config.parameters.grpc_client.clone(),
582 };
583 instance.add_replica(replica_id, config);
584 }
585
586 fn drop_replica(&mut self, instance_id: StorageInstanceId, replica_id: ReplicaId) {
587 let instance = self
588 .instances
589 .get_mut(&instance_id)
590 .unwrap_or_else(|| panic!("instance {instance_id} does not exist"));
591
592 let status_now = mz_ore::now::to_datetime((self.now)());
593 let mut source_status_updates = vec![];
594 let mut sink_status_updates = vec![];
595
596 let make_update = |id, object_type| StatusUpdate {
597 id,
598 status: Status::Paused,
599 timestamp: status_now,
600 error: None,
601 hints: BTreeSet::from([format!(
602 "The replica running this {object_type} has been dropped"
603 )]),
604 namespaced_errors: Default::default(),
605 replica_id: Some(replica_id),
606 };
607
608 for ingestion_id in instance.active_ingestions() {
609 if let Some(active_replicas) = self.dropped_objects.get_mut(ingestion_id) {
610 active_replicas.remove(&replica_id);
611 if active_replicas.is_empty() {
612 self.dropped_objects.remove(ingestion_id);
613 }
614 }
615
616 let ingestion = self
617 .collections
618 .get_mut(ingestion_id)
619 .expect("instance contains unknown ingestion");
620
621 let ingestion_description = match &ingestion.data_source {
622 DataSource::Ingestion(ingestion_description) => ingestion_description.clone(),
623 _ => panic!(
624 "unexpected data source for ingestion: {:?}",
625 ingestion.data_source
626 ),
627 };
628
629 let old_style_ingestion = *ingestion_id != ingestion_description.remap_collection_id;
630 let subsource_ids = ingestion_description.collection_ids().filter(|id| {
631 let should_discard =
636 old_style_ingestion && id == &ingestion_description.remap_collection_id;
637 !should_discard
638 });
639 for id in subsource_ids {
640 source_status_updates.push(make_update(id, "source"));
641 }
642 }
643
644 for id in instance.active_exports() {
645 if let Some(active_replicas) = self.dropped_objects.get_mut(id) {
646 active_replicas.remove(&replica_id);
647 if active_replicas.is_empty() {
648 self.dropped_objects.remove(id);
649 }
650 }
651
652 sink_status_updates.push(make_update(*id, "sink"));
653 }
654
655 instance.drop_replica(replica_id);
656
657 if !self.read_only {
658 if !source_status_updates.is_empty() {
659 self.append_status_introspection_updates(
660 IntrospectionType::SourceStatusHistory,
661 source_status_updates,
662 );
663 }
664 if !sink_status_updates.is_empty() {
665 self.append_status_introspection_updates(
666 IntrospectionType::SinkStatusHistory,
667 sink_status_updates,
668 );
669 }
670 }
671 }
672
673 async fn evolve_nullability_for_bootstrap(
674 &mut self,
675 storage_metadata: &StorageMetadata,
676 collections: Vec<(GlobalId, RelationDesc)>,
677 ) -> Result<(), StorageError<Self::Timestamp>> {
678 let persist_client = self
679 .persist
680 .open(self.persist_location.clone())
681 .await
682 .unwrap();
683
684 for (global_id, relation_desc) in collections {
685 let shard_id = storage_metadata.get_collection_shard(global_id)?;
686 let diagnostics = Diagnostics {
687 shard_name: global_id.to_string(),
688 handle_purpose: "evolve nullability for bootstrap".to_string(),
689 };
690 let latest_schema = persist_client
691 .latest_schema::<SourceData, (), T, StorageDiff>(shard_id, diagnostics)
692 .await
693 .expect("invalid persist usage");
694 let Some((schema_id, current_schema, _)) = latest_schema else {
695 tracing::debug!(?global_id, "no schema registered");
696 continue;
697 };
698 tracing::debug!(?global_id, ?current_schema, new_schema = ?relation_desc, "migrating schema");
699
700 let diagnostics = Diagnostics {
701 shard_name: global_id.to_string(),
702 handle_purpose: "evolve nullability for bootstrap".to_string(),
703 };
704 let evolve_result = persist_client
705 .compare_and_evolve_schema::<SourceData, (), T, StorageDiff>(
706 shard_id,
707 schema_id,
708 &relation_desc,
709 &UnitSchema,
710 diagnostics,
711 )
712 .await
713 .expect("invalid persist usage");
714 match evolve_result {
715 CaESchema::Ok(_) => (),
716 CaESchema::ExpectedMismatch {
717 schema_id,
718 key,
719 val: _,
720 } => {
721 return Err(StorageError::PersistSchemaEvolveRace {
722 global_id,
723 shard_id,
724 schema_id,
725 relation_desc: key,
726 });
727 }
728 CaESchema::Incompatible => {
729 return Err(StorageError::PersistInvalidSchemaEvolve {
730 global_id,
731 shard_id,
732 });
733 }
734 };
735 }
736
737 Ok(())
738 }
739
740 #[instrument(name = "storage::create_collections")]
759 async fn create_collections_for_bootstrap(
760 &mut self,
761 storage_metadata: &StorageMetadata,
762 register_ts: Option<Self::Timestamp>,
763 mut collections: Vec<(GlobalId, CollectionDescription<Self::Timestamp>)>,
764 migrated_storage_collections: &BTreeSet<GlobalId>,
765 ) -> Result<(), StorageError<Self::Timestamp>> {
766 self.migrated_storage_collections
767 .extend(migrated_storage_collections.iter().cloned());
768
769 self.storage_collections
770 .create_collections_for_bootstrap(
771 storage_metadata,
772 register_ts.clone(),
773 collections.clone(),
774 migrated_storage_collections,
775 )
776 .await?;
777
778 drop(self.persist_warm_task.take());
781
782 collections.sort_by_key(|(id, _)| *id);
787 collections.dedup();
788 for pos in 1..collections.len() {
789 if collections[pos - 1].0 == collections[pos].0 {
790 return Err(StorageError::CollectionIdReused(collections[pos].0));
791 }
792 }
793
794 let enriched_with_metadata = collections
796 .into_iter()
797 .map(|(id, description)| {
798 let data_shard = storage_metadata.get_collection_shard::<T>(id)?;
799
800 let txns_shard = description
803 .data_source
804 .in_txns()
805 .then(|| *self.txns_read.txns_id());
806
807 let metadata = CollectionMetadata {
808 persist_location: self.persist_location.clone(),
809 data_shard,
810 relation_desc: description.desc.clone(),
811 txns_shard,
812 };
813
814 Ok((id, description, metadata))
815 })
816 .collect_vec();
817
818 let persist_client = self
820 .persist
821 .open(self.persist_location.clone())
822 .await
823 .unwrap();
824 let persist_client = &persist_client;
825
826 use futures::stream::{StreamExt, TryStreamExt};
829 let this = &*self;
830 let mut to_register: Vec<_> = futures::stream::iter(enriched_with_metadata)
831 .map(|data: Result<_, StorageError<Self::Timestamp>>| {
832 async move {
833 let (id, description, metadata) = data?;
834
835 debug!("mapping GlobalId={} to shard ({})", id, metadata.data_shard);
838
839 let write = this
840 .open_data_handles(
841 &id,
842 metadata.data_shard,
843 metadata.relation_desc.clone(),
844 persist_client,
845 )
846 .await;
847
848 Ok::<_, StorageError<T>>((id, description, write, metadata))
849 }
850 })
851 .buffer_unordered(50)
853 .try_collect()
866 .await?;
867
868 let mut to_execute = BTreeSet::new();
871 let mut new_collections = BTreeSet::new();
876 let mut table_registers = Vec::with_capacity(to_register.len());
877
878 to_register.sort_by_key(|(id, ..)| *id);
880
881 let (tables_to_register, collections_to_register): (Vec<_>, Vec<_>) = to_register
887 .into_iter()
888 .partition(|(_id, desc, ..)| desc.data_source == DataSource::Table);
889 let to_register = tables_to_register
890 .into_iter()
891 .rev()
892 .chain(collections_to_register.into_iter());
893
894 let mut new_source_statistic_entries = BTreeSet::new();
898 let mut new_webhook_statistic_entries = BTreeSet::new();
899 let mut new_sink_statistic_entries = BTreeSet::new();
900
901 for (id, description, write, metadata) in to_register {
902 let is_in_txns = |id, metadata: &CollectionMetadata| {
903 metadata.txns_shard.is_some()
904 && !(self.read_only && migrated_storage_collections.contains(&id))
905 };
906
907 to_execute.insert(id);
908 new_collections.insert(id);
909
910 let write_frontier = write.upper();
911
912 let storage_dependencies = self.determine_collection_dependencies(id, &description)?;
914
915 let dependency_read_holds = self
916 .storage_collections
917 .acquire_read_holds(storage_dependencies)
918 .expect("can acquire read holds");
919
920 let mut dependency_since = Antichain::from_elem(T::minimum());
921 for read_hold in dependency_read_holds.iter() {
922 dependency_since.join_assign(read_hold.since());
923 }
924
925 let data_source = description.data_source;
926
927 if !dependency_read_holds.is_empty()
936 && !is_in_txns(id, &metadata)
937 && !matches!(&data_source, DataSource::Sink { .. })
938 {
939 if dependency_since.is_empty() {
945 halt!(
946 "dependency since frontier is empty while dependent upper \
947 is not empty (dependent id={id}, write_frontier={:?}, dependency_read_holds={:?}), \
948 this indicates concurrent deletion of a collection",
949 write_frontier,
950 dependency_read_holds,
951 );
952 }
953
954 if description.primary.is_none() {
972 mz_ore::soft_assert_or_log!(
973 write_frontier.elements() == &[T::minimum()]
974 || write_frontier.is_empty()
975 || PartialOrder::less_than(&dependency_since, write_frontier),
976 "dependency since has advanced past dependent ({id}) upper \n
977 dependent ({id}): upper {:?} \n
978 dependency since {:?} \n
979 dependency read holds: {:?}",
980 write_frontier,
981 dependency_since,
982 dependency_read_holds,
983 );
984 }
985 }
986
987 let mut extra_state = CollectionStateExtra::None;
989 let mut maybe_instance_id = None;
990 match &data_source {
991 DataSource::Introspection(typ) => {
992 debug!(
993 ?data_source, meta = ?metadata,
994 "registering {id} with persist monotonic worker",
995 );
996 self.register_introspection_collection(
1002 id,
1003 *typ,
1004 write,
1005 persist_client.clone(),
1006 )?;
1007 }
1008 DataSource::Webhook => {
1009 debug!(
1010 ?data_source, meta = ?metadata,
1011 "registering {id} with persist monotonic worker",
1012 );
1013 new_source_statistic_entries.insert(id);
1014 new_webhook_statistic_entries.insert(id);
1017 self.collection_manager
1023 .register_append_only_collection(id, write, false, None);
1024 }
1025 DataSource::IngestionExport {
1026 ingestion_id,
1027 details,
1028 data_config,
1029 } => {
1030 debug!(
1031 ?data_source, meta = ?metadata,
1032 "not registering {id} with a controller persist worker",
1033 );
1034 let ingestion_state = self
1036 .collections
1037 .get_mut(ingestion_id)
1038 .expect("known to exist");
1039
1040 let instance_id = match &mut ingestion_state.data_source {
1041 DataSource::Ingestion(ingestion_desc) => {
1042 ingestion_desc.source_exports.insert(
1043 id,
1044 SourceExport {
1045 storage_metadata: (),
1046 details: details.clone(),
1047 data_config: data_config.clone(),
1048 },
1049 );
1050
1051 ingestion_desc.instance_id
1056 }
1057 _ => unreachable!(
1058 "SourceExport must only refer to primary sources that already exist"
1059 ),
1060 };
1061
1062 to_execute.remove(&id);
1064 to_execute.insert(*ingestion_id);
1065
1066 let ingestion_state = IngestionState {
1067 read_capabilities: MutableAntichain::from(dependency_since.clone()),
1068 dependency_read_holds,
1069 derived_since: dependency_since,
1070 write_frontier: Antichain::from_elem(Self::Timestamp::minimum()),
1071 hold_policy: ReadPolicy::step_back(),
1072 instance_id,
1073 hydrated_on: BTreeSet::new(),
1074 };
1075
1076 extra_state = CollectionStateExtra::Ingestion(ingestion_state);
1077 maybe_instance_id = Some(instance_id);
1078
1079 new_source_statistic_entries.insert(id);
1080 }
1081 DataSource::Table => {
1082 debug!(
1083 ?data_source, meta = ?metadata,
1084 "registering {id} with persist table worker",
1085 );
1086 table_registers.push((id, write));
1087 }
1088 DataSource::Progress | DataSource::Other => {
1089 debug!(
1090 ?data_source, meta = ?metadata,
1091 "not registering {id} with a controller persist worker",
1092 );
1093 }
1094 DataSource::Ingestion(ingestion_desc) => {
1095 debug!(
1096 ?data_source, meta = ?metadata,
1097 "not registering {id} with a controller persist worker",
1098 );
1099
1100 let mut dependency_since = Antichain::from_elem(T::minimum());
1101 for read_hold in dependency_read_holds.iter() {
1102 dependency_since.join_assign(read_hold.since());
1103 }
1104
1105 let ingestion_state = IngestionState {
1106 read_capabilities: MutableAntichain::from(dependency_since.clone()),
1107 dependency_read_holds,
1108 derived_since: dependency_since,
1109 write_frontier: Antichain::from_elem(Self::Timestamp::minimum()),
1110 hold_policy: ReadPolicy::step_back(),
1111 instance_id: ingestion_desc.instance_id,
1112 hydrated_on: BTreeSet::new(),
1113 };
1114
1115 extra_state = CollectionStateExtra::Ingestion(ingestion_state);
1116 maybe_instance_id = Some(ingestion_desc.instance_id);
1117
1118 new_source_statistic_entries.insert(id);
1119 }
1120 DataSource::Sink { desc } => {
1121 let mut dependency_since = Antichain::from_elem(T::minimum());
1122 for read_hold in dependency_read_holds.iter() {
1123 dependency_since.join_assign(read_hold.since());
1124 }
1125
1126 let [self_hold, read_hold] =
1127 dependency_read_holds.try_into().expect("two holds");
1128
1129 let state = ExportState::new(
1130 desc.instance_id,
1131 read_hold,
1132 self_hold,
1133 write_frontier.clone(),
1134 ReadPolicy::step_back(),
1135 );
1136 maybe_instance_id = Some(state.cluster_id);
1137 extra_state = CollectionStateExtra::Export(state);
1138
1139 new_sink_statistic_entries.insert(id);
1140 }
1141 }
1142
1143 let wallclock_lag_metrics = self.metrics.wallclock_lag_metrics(id, maybe_instance_id);
1144 let collection_state =
1145 CollectionState::new(data_source, metadata, extra_state, wallclock_lag_metrics);
1146
1147 self.collections.insert(id, collection_state);
1148 }
1149
1150 {
1151 let mut source_statistics = self.source_statistics.lock().expect("poisoned");
1152
1153 for id in new_webhook_statistic_entries {
1156 source_statistics.webhook_statistics.entry(id).or_default();
1157 }
1158
1159 }
1163
1164 if !table_registers.is_empty() {
1166 let register_ts = register_ts
1167 .expect("caller should have provided a register_ts when creating a table");
1168
1169 if self.read_only {
1170 table_registers
1180 .retain(|(id, _write_handle)| migrated_storage_collections.contains(id));
1181
1182 self.persist_table_worker
1183 .register(register_ts, table_registers)
1184 .await
1185 .expect("table worker unexpectedly shut down");
1186 } else {
1187 self.persist_table_worker
1188 .register(register_ts, table_registers)
1189 .await
1190 .expect("table worker unexpectedly shut down");
1191 }
1192 }
1193
1194 self.append_shard_mappings(new_collections.into_iter(), Diff::ONE);
1195
1196 for id in to_execute {
1198 match &self.collection(id)?.data_source {
1199 DataSource::Ingestion(ingestion) => {
1200 if !self.read_only
1201 || (ENABLE_0DT_DEPLOYMENT_SOURCES.get(self.config.config_set())
1202 && ingestion.desc.connection.supports_read_only())
1203 {
1204 self.run_ingestion(id)?;
1205 }
1206 }
1207 DataSource::IngestionExport { .. } => unreachable!(
1208 "ingestion exports do not execute directly, but instead schedule their source to be re-executed"
1209 ),
1210 DataSource::Introspection(_)
1211 | DataSource::Webhook
1212 | DataSource::Table
1213 | DataSource::Progress
1214 | DataSource::Other => {}
1215 DataSource::Sink { .. } => {
1216 if !self.read_only {
1217 self.run_export(id)?;
1218 }
1219 }
1220 };
1221 }
1222
1223 Ok(())
1224 }
1225
1226 fn check_alter_ingestion_source_desc(
1227 &mut self,
1228 ingestion_id: GlobalId,
1229 source_desc: &SourceDesc,
1230 ) -> Result<(), StorageError<Self::Timestamp>> {
1231 let source_collection = self.collection(ingestion_id)?;
1232 let data_source = &source_collection.data_source;
1233 match &data_source {
1234 DataSource::Ingestion(cur_ingestion) => {
1235 cur_ingestion
1236 .desc
1237 .alter_compatible(ingestion_id, source_desc)?;
1238 }
1239 o => {
1240 tracing::info!(
1241 "{ingestion_id} inalterable because its data source is {:?} and not an ingestion",
1242 o
1243 );
1244 Err(AlterError { id: ingestion_id })?
1245 }
1246 }
1247
1248 Ok(())
1249 }
1250
1251 async fn alter_ingestion_connections(
1252 &mut self,
1253 source_connections: BTreeMap<GlobalId, GenericSourceConnection<InlinedConnection>>,
1254 ) -> Result<(), StorageError<Self::Timestamp>> {
1255 let mut ingestions_to_run = BTreeSet::new();
1256
1257 for (id, conn) in source_connections {
1258 let collection = self
1259 .collections
1260 .get_mut(&id)
1261 .ok_or_else(|| StorageError::IdentifierMissing(id))?;
1262
1263 match &mut collection.data_source {
1264 DataSource::Ingestion(ingestion) => {
1265 if ingestion.desc.connection != conn {
1268 tracing::info!(from = ?ingestion.desc.connection, to = ?conn, "alter_ingestion_connections, updating");
1269 ingestion.desc.connection = conn;
1270 ingestions_to_run.insert(id);
1271 } else {
1272 tracing::warn!(
1273 "update_source_connection called on {id} but the \
1274 connection was the same"
1275 );
1276 }
1277 }
1278 o => {
1279 tracing::warn!("update_source_connection called on {:?}", o);
1280 Err(StorageError::IdentifierInvalid(id))?;
1281 }
1282 }
1283 }
1284
1285 for id in ingestions_to_run {
1286 self.run_ingestion(id)?;
1287 }
1288 Ok(())
1289 }
1290
1291 async fn alter_ingestion_export_data_configs(
1292 &mut self,
1293 source_exports: BTreeMap<GlobalId, SourceExportDataConfig>,
1294 ) -> Result<(), StorageError<Self::Timestamp>> {
1295 let mut ingestions_to_run = BTreeSet::new();
1296
1297 for (source_export_id, new_data_config) in source_exports {
1298 let source_export_collection = self
1301 .collections
1302 .get_mut(&source_export_id)
1303 .ok_or_else(|| StorageError::IdentifierMissing(source_export_id))?;
1304 let ingestion_id = match &mut source_export_collection.data_source {
1305 DataSource::IngestionExport {
1306 ingestion_id,
1307 details: _,
1308 data_config,
1309 } => {
1310 *data_config = new_data_config.clone();
1311 *ingestion_id
1312 }
1313 o => {
1314 tracing::warn!("alter_ingestion_export_data_configs called on {:?}", o);
1315 Err(StorageError::IdentifierInvalid(source_export_id))?
1316 }
1317 };
1318 let ingestion_collection = self
1321 .collections
1322 .get_mut(&ingestion_id)
1323 .ok_or_else(|| StorageError::IdentifierMissing(ingestion_id))?;
1324
1325 match &mut ingestion_collection.data_source {
1326 DataSource::Ingestion(ingestion_desc) => {
1327 let source_export = ingestion_desc
1328 .source_exports
1329 .get_mut(&source_export_id)
1330 .ok_or_else(|| StorageError::IdentifierMissing(source_export_id))?;
1331
1332 if source_export.data_config != new_data_config {
1335 tracing::info!(?source_export_id, from = ?source_export.data_config, to = ?new_data_config, "alter_ingestion_export_data_configs, updating");
1336 source_export.data_config = new_data_config;
1337
1338 ingestions_to_run.insert(ingestion_id);
1339 } else {
1340 tracing::warn!(
1341 "alter_ingestion_export_data_configs called on \
1342 export {source_export_id} of {ingestion_id} but \
1343 the data config was the same"
1344 );
1345 }
1346 }
1347 o => {
1348 tracing::warn!("alter_ingestion_export_data_configs called on {:?}", o);
1349 Err(StorageError::IdentifierInvalid(ingestion_id))?
1350 }
1351 }
1352 }
1353
1354 for id in ingestions_to_run {
1355 self.run_ingestion(id)?;
1356 }
1357 Ok(())
1358 }
1359
1360 async fn alter_table_desc(
1361 &mut self,
1362 existing_collection: GlobalId,
1363 new_collection: GlobalId,
1364 new_desc: RelationDesc,
1365 expected_version: RelationVersion,
1366 register_ts: Self::Timestamp,
1367 ) -> Result<(), StorageError<Self::Timestamp>> {
1368 let data_shard = {
1369 let Controller {
1370 collections,
1371 storage_collections,
1372 ..
1373 } = self;
1374
1375 let existing = collections
1376 .get(&existing_collection)
1377 .ok_or(StorageError::IdentifierMissing(existing_collection))?;
1378 if existing.data_source != DataSource::Table {
1379 return Err(StorageError::IdentifierInvalid(existing_collection));
1380 }
1381
1382 storage_collections
1384 .alter_table_desc(
1385 existing_collection,
1386 new_collection,
1387 new_desc.clone(),
1388 expected_version,
1389 )
1390 .await?;
1391
1392 existing.collection_metadata.data_shard.clone()
1393 };
1394
1395 let persist_client = self
1396 .persist
1397 .open(self.persist_location.clone())
1398 .await
1399 .expect("invalid persist location");
1400 let write_handle = self
1401 .open_data_handles(
1402 &existing_collection,
1403 data_shard,
1404 new_desc.clone(),
1405 &persist_client,
1406 )
1407 .await;
1408
1409 let collection_meta = CollectionMetadata {
1410 persist_location: self.persist_location.clone(),
1411 data_shard,
1412 relation_desc: new_desc.clone(),
1413 txns_shard: Some(self.txns_read.txns_id().clone()),
1415 };
1416 let wallclock_lag_metrics = self.metrics.wallclock_lag_metrics(new_collection, None);
1418 let collection_state = CollectionState::new(
1419 DataSource::Table,
1420 collection_meta,
1421 CollectionStateExtra::None,
1422 wallclock_lag_metrics,
1423 );
1424
1425 self.collections.insert(new_collection, collection_state);
1428
1429 self.persist_table_worker
1430 .register(register_ts, vec![(new_collection, write_handle)])
1431 .await
1432 .expect("table worker unexpectedly shut down");
1433
1434 self.append_shard_mappings([new_collection].into_iter(), Diff::ONE);
1435
1436 Ok(())
1437 }
1438
1439 fn export(
1440 &self,
1441 id: GlobalId,
1442 ) -> Result<&ExportState<Self::Timestamp>, StorageError<Self::Timestamp>> {
1443 self.collections
1444 .get(&id)
1445 .and_then(|c| match &c.extra_state {
1446 CollectionStateExtra::Export(state) => Some(state),
1447 _ => None,
1448 })
1449 .ok_or(StorageError::IdentifierMissing(id))
1450 }
1451
1452 fn export_mut(
1453 &mut self,
1454 id: GlobalId,
1455 ) -> Result<&mut ExportState<Self::Timestamp>, StorageError<Self::Timestamp>> {
1456 self.collections
1457 .get_mut(&id)
1458 .and_then(|c| match &mut c.extra_state {
1459 CollectionStateExtra::Export(state) => Some(state),
1460 _ => None,
1461 })
1462 .ok_or(StorageError::IdentifierMissing(id))
1463 }
1464
1465 async fn create_oneshot_ingestion(
1467 &mut self,
1468 ingestion_id: uuid::Uuid,
1469 collection_id: GlobalId,
1470 instance_id: StorageInstanceId,
1471 request: OneshotIngestionRequest,
1472 result_tx: OneshotResultCallback<ProtoBatch>,
1473 ) -> Result<(), StorageError<Self::Timestamp>> {
1474 let collection_meta = self
1475 .collections
1476 .get(&collection_id)
1477 .ok_or_else(|| StorageError::IdentifierMissing(collection_id))?
1478 .collection_metadata
1479 .clone();
1480 let instance = self.instances.get_mut(&instance_id).ok_or_else(|| {
1481 StorageError::Generic(anyhow::anyhow!("missing cluster {instance_id}"))
1483 })?;
1484 let oneshot_cmd = RunOneshotIngestion {
1485 ingestion_id,
1486 collection_id,
1487 collection_meta,
1488 request,
1489 };
1490
1491 if !self.read_only {
1492 instance.send(StorageCommand::RunOneshotIngestion(Box::new(oneshot_cmd)));
1493 let pending = PendingOneshotIngestion {
1494 result_tx,
1495 cluster_id: instance_id,
1496 };
1497 let novel = self
1498 .pending_oneshot_ingestions
1499 .insert(ingestion_id, pending);
1500 assert_none!(novel);
1501 Ok(())
1502 } else {
1503 Err(StorageError::ReadOnly)
1504 }
1505 }
1506
1507 fn cancel_oneshot_ingestion(
1508 &mut self,
1509 ingestion_id: uuid::Uuid,
1510 ) -> Result<(), StorageError<Self::Timestamp>> {
1511 if self.read_only {
1512 return Err(StorageError::ReadOnly);
1513 }
1514
1515 let pending = self
1516 .pending_oneshot_ingestions
1517 .remove(&ingestion_id)
1518 .ok_or_else(|| {
1519 StorageError::Generic(anyhow::anyhow!("missing oneshot ingestion {ingestion_id}"))
1521 })?;
1522
1523 match self.instances.get_mut(&pending.cluster_id) {
1524 Some(instance) => {
1525 instance.send(StorageCommand::CancelOneshotIngestion(ingestion_id));
1526 }
1527 None => {
1528 mz_ore::soft_panic_or_log!(
1529 "canceling oneshot ingestion on non-existent cluster, ingestion {:?}, instance {}",
1530 ingestion_id,
1531 pending.cluster_id,
1532 );
1533 }
1534 }
1535 pending.cancel();
1537
1538 Ok(())
1539 }
1540
1541 async fn alter_export(
1542 &mut self,
1543 id: GlobalId,
1544 new_description: ExportDescription<Self::Timestamp>,
1545 ) -> Result<(), StorageError<Self::Timestamp>> {
1546 let from_id = new_description.sink.from;
1547
1548 let desired_read_holds = vec![from_id.clone(), id.clone()];
1551 let [input_hold, self_hold] = self
1552 .storage_collections
1553 .acquire_read_holds(desired_read_holds)
1554 .expect("missing dependency")
1555 .try_into()
1556 .expect("expected number of holds");
1557 let from_storage_metadata = self.storage_collections.collection_metadata(from_id)?;
1558 let to_storage_metadata = self.storage_collections.collection_metadata(id)?;
1559
1560 let cur_export = self.export_mut(id)?;
1562 let input_readable = cur_export
1563 .write_frontier
1564 .iter()
1565 .all(|t| input_hold.since().less_than(t));
1566 if !input_readable {
1567 return Err(StorageError::ReadBeforeSince(from_id));
1568 }
1569
1570 let new_export = ExportState {
1571 read_capabilities: cur_export.read_capabilities.clone(),
1572 cluster_id: new_description.instance_id,
1573 derived_since: cur_export.derived_since.clone(),
1574 read_holds: [input_hold, self_hold],
1575 read_policy: cur_export.read_policy.clone(),
1576 write_frontier: cur_export.write_frontier.clone(),
1577 };
1578 *cur_export = new_export;
1579
1580 let with_snapshot = new_description.sink.with_snapshot
1587 && !PartialOrder::less_than(&new_description.sink.as_of, &cur_export.write_frontier);
1588
1589 let cmd = RunSinkCommand {
1590 id,
1591 description: StorageSinkDesc {
1592 from: from_id,
1593 from_desc: new_description.sink.from_desc,
1594 connection: new_description.sink.connection,
1595 envelope: new_description.sink.envelope,
1596 as_of: new_description.sink.as_of,
1597 version: new_description.sink.version,
1598 from_storage_metadata,
1599 with_snapshot,
1600 to_storage_metadata,
1601 commit_interval: new_description.sink.commit_interval,
1602 },
1603 };
1604
1605 let instance = self
1607 .instances
1608 .get_mut(&new_description.instance_id)
1609 .ok_or_else(|| StorageError::ExportInstanceMissing {
1610 storage_instance_id: new_description.instance_id,
1611 export_id: id,
1612 })?;
1613
1614 instance.send(StorageCommand::RunSink(Box::new(cmd)));
1615 Ok(())
1616 }
1617
1618 async fn alter_export_connections(
1620 &mut self,
1621 exports: BTreeMap<GlobalId, StorageSinkConnection>,
1622 ) -> Result<(), StorageError<Self::Timestamp>> {
1623 let mut updates_by_instance =
1624 BTreeMap::<StorageInstanceId, Vec<(RunSinkCommand<T>, ExportDescription<T>)>>::new();
1625
1626 for (id, connection) in exports {
1627 let (mut new_export_description, as_of): (ExportDescription<Self::Timestamp>, _) = {
1635 let export = &self.collections[&id];
1636 let DataSource::Sink { desc } = &export.data_source else {
1637 panic!("export exists")
1638 };
1639 let CollectionStateExtra::Export(state) = &export.extra_state else {
1640 panic!("export exists")
1641 };
1642 let export_description = desc.clone();
1643 let as_of = state.input_hold().since().clone();
1644
1645 (export_description, as_of)
1646 };
1647 let current_sink = new_export_description.sink.clone();
1648
1649 new_export_description.sink.connection = connection;
1650
1651 current_sink.alter_compatible(id, &new_export_description.sink)?;
1653
1654 let from_storage_metadata = self
1655 .storage_collections
1656 .collection_metadata(new_export_description.sink.from)?;
1657 let to_storage_metadata = self.storage_collections.collection_metadata(id)?;
1658
1659 let cmd = RunSinkCommand {
1660 id,
1661 description: StorageSinkDesc {
1662 from: new_export_description.sink.from,
1663 from_desc: new_export_description.sink.from_desc.clone(),
1664 connection: new_export_description.sink.connection.clone(),
1665 envelope: new_export_description.sink.envelope,
1666 with_snapshot: new_export_description.sink.with_snapshot,
1667 version: new_export_description.sink.version,
1668 as_of: as_of.to_owned(),
1679 from_storage_metadata,
1680 to_storage_metadata,
1681 commit_interval: new_export_description.sink.commit_interval,
1682 },
1683 };
1684
1685 let update = updates_by_instance
1686 .entry(new_export_description.instance_id)
1687 .or_default();
1688 update.push((cmd, new_export_description));
1689 }
1690
1691 for (instance_id, updates) in updates_by_instance {
1692 let mut export_updates = BTreeMap::new();
1693 let mut cmds = Vec::with_capacity(updates.len());
1694
1695 for (cmd, export_state) in updates {
1696 export_updates.insert(cmd.id, export_state);
1697 cmds.push(cmd);
1698 }
1699
1700 let instance = self.instances.get_mut(&instance_id).ok_or_else(|| {
1702 StorageError::ExportInstanceMissing {
1703 storage_instance_id: instance_id,
1704 export_id: *export_updates
1705 .keys()
1706 .next()
1707 .expect("set of exports not empty"),
1708 }
1709 })?;
1710
1711 for cmd in cmds {
1712 instance.send(StorageCommand::RunSink(Box::new(cmd)));
1713 }
1714
1715 for (id, new_export_description) in export_updates {
1717 let Some(state) = self.collections.get_mut(&id) else {
1718 panic!("export known to exist")
1719 };
1720 let DataSource::Sink { desc } = &mut state.data_source else {
1721 panic!("export known to exist")
1722 };
1723 *desc = new_export_description;
1724 }
1725 }
1726
1727 Ok(())
1728 }
1729
1730 fn drop_tables(
1745 &mut self,
1746 storage_metadata: &StorageMetadata,
1747 identifiers: Vec<GlobalId>,
1748 ts: Self::Timestamp,
1749 ) -> Result<(), StorageError<Self::Timestamp>> {
1750 let (table_write_ids, data_source_ids): (Vec<_>, Vec<_>) = identifiers
1752 .into_iter()
1753 .partition(|id| match self.collections[id].data_source {
1754 DataSource::Table => true,
1755 DataSource::IngestionExport { .. } | DataSource::Webhook => false,
1756 _ => panic!("identifier is not a table: {}", id),
1757 });
1758
1759 if table_write_ids.len() > 0 {
1761 let drop_notif = self
1762 .persist_table_worker
1763 .drop_handles(table_write_ids.clone(), ts);
1764 let tx = self.pending_table_handle_drops_tx.clone();
1765 mz_ore::task::spawn(|| "table-cleanup".to_string(), async move {
1766 drop_notif.await;
1767 for identifier in table_write_ids {
1768 let _ = tx.send(identifier);
1769 }
1770 });
1771 }
1772
1773 if data_source_ids.len() > 0 {
1775 self.validate_collection_ids(data_source_ids.iter().cloned())?;
1776 self.drop_sources_unvalidated(storage_metadata, data_source_ids)?;
1777 }
1778
1779 Ok(())
1780 }
1781
1782 fn drop_sources(
1783 &mut self,
1784 storage_metadata: &StorageMetadata,
1785 identifiers: Vec<GlobalId>,
1786 ) -> Result<(), StorageError<Self::Timestamp>> {
1787 self.validate_collection_ids(identifiers.iter().cloned())?;
1788 self.drop_sources_unvalidated(storage_metadata, identifiers)
1789 }
1790
1791 fn drop_sources_unvalidated(
1792 &mut self,
1793 storage_metadata: &StorageMetadata,
1794 ids: Vec<GlobalId>,
1795 ) -> Result<(), StorageError<Self::Timestamp>> {
1796 let mut ingestions_to_execute = BTreeSet::new();
1799 let mut ingestions_to_drop = BTreeSet::new();
1800 let mut source_statistics_to_drop = Vec::new();
1801
1802 let mut collections_to_drop = Vec::new();
1806
1807 for id in ids.iter() {
1808 let metadata = storage_metadata.get_collection_shard::<T>(*id);
1809 mz_ore::soft_assert_or_log!(
1810 matches!(metadata, Err(StorageError::IdentifierMissing(_))),
1811 "dropping {id}, but drop was not synchronized with storage \
1812 controller via `synchronize_collections`"
1813 );
1814
1815 let collection_state = self.collections.get(id);
1816
1817 if let Some(collection_state) = collection_state {
1818 match collection_state.data_source {
1819 DataSource::Webhook => {
1820 let fut = self.collection_manager.unregister_collection(*id);
1823 mz_ore::task::spawn(|| format!("storage-webhook-cleanup-{id}"), fut);
1824
1825 collections_to_drop.push(*id);
1826 source_statistics_to_drop.push(*id);
1827 }
1828 DataSource::Ingestion(_) => {
1829 ingestions_to_drop.insert(*id);
1830 source_statistics_to_drop.push(*id);
1831 }
1832 DataSource::IngestionExport { ingestion_id, .. } => {
1833 ingestions_to_execute.insert(ingestion_id);
1840
1841 let ingestion_state = match self.collections.get_mut(&ingestion_id) {
1843 Some(ingestion_collection) => ingestion_collection,
1844 None => {
1846 tracing::error!(
1847 "primary source {ingestion_id} seemingly dropped before subsource {id}"
1848 );
1849 continue;
1850 }
1851 };
1852
1853 match &mut ingestion_state.data_source {
1854 DataSource::Ingestion(ingestion_desc) => {
1855 let removed = ingestion_desc.source_exports.remove(id);
1856 mz_ore::soft_assert_or_log!(
1857 removed.is_some(),
1858 "dropped subsource {id} already removed from source exports"
1859 );
1860 }
1861 _ => unreachable!(
1862 "SourceExport must only refer to primary sources that already exist"
1863 ),
1864 };
1865
1866 ingestions_to_drop.insert(*id);
1870 source_statistics_to_drop.push(*id);
1871 }
1872 DataSource::Progress | DataSource::Table | DataSource::Other => {
1873 collections_to_drop.push(*id);
1874 }
1875 DataSource::Introspection(_) | DataSource::Sink { .. } => {
1876 soft_panic_or_log!(
1879 "drop_sources called on a {:?} (id={id}))",
1880 collection_state.data_source,
1881 );
1882 }
1883 }
1884 }
1885 }
1886
1887 ingestions_to_execute.retain(|id| !ingestions_to_drop.contains(id));
1889 for ingestion_id in ingestions_to_execute {
1890 self.run_ingestion(ingestion_id)?;
1891 }
1892
1893 let ingestion_policies = ingestions_to_drop
1900 .iter()
1901 .map(|id| (*id, ReadPolicy::ValidFrom(Antichain::new())))
1902 .collect();
1903
1904 tracing::debug!(
1905 ?ingestion_policies,
1906 "dropping sources by setting read hold policies"
1907 );
1908 self.set_hold_policies(ingestion_policies);
1909
1910 let shards_to_update: BTreeSet<_> = ingestions_to_drop
1912 .iter()
1913 .chain(collections_to_drop.iter())
1914 .cloned()
1915 .collect();
1916 self.append_shard_mappings(shards_to_update.into_iter(), Diff::MINUS_ONE);
1917
1918 let status_now = mz_ore::now::to_datetime((self.now)());
1919 let mut status_updates = vec![];
1920 for id in ingestions_to_drop.iter() {
1921 status_updates.push(StatusUpdate::new(*id, status_now, Status::Dropped));
1922 }
1923
1924 if !self.read_only {
1925 self.append_status_introspection_updates(
1926 IntrospectionType::SourceStatusHistory,
1927 status_updates,
1928 );
1929 }
1930
1931 {
1932 let mut source_statistics = self.source_statistics.lock().expect("poisoned");
1933 for id in source_statistics_to_drop {
1934 source_statistics
1935 .source_statistics
1936 .retain(|(stats_id, _), _| stats_id != &id);
1937 source_statistics
1938 .webhook_statistics
1939 .retain(|stats_id, _| stats_id != &id);
1940 }
1941 }
1942
1943 for id in ingestions_to_drop.iter().chain(collections_to_drop.iter()) {
1945 tracing::info!(%id, "dropping collection state");
1946 let collection = self
1947 .collections
1948 .remove(id)
1949 .expect("list populated after checking that self.collections contains it");
1950
1951 let instance = match &collection.extra_state {
1952 CollectionStateExtra::Ingestion(ingestion) => Some(ingestion.instance_id),
1953 CollectionStateExtra::Export(export) => Some(export.cluster_id()),
1954 CollectionStateExtra::None => None,
1955 }
1956 .and_then(|i| self.instances.get(&i));
1957
1958 if let Some(instance) = instance {
1962 let active_replicas = instance.get_active_replicas_for_object(id);
1963 if !active_replicas.is_empty() {
1964 match &collection.data_source {
1971 DataSource::Ingestion(ingestion_desc) => {
1972 if *id != ingestion_desc.remap_collection_id {
1973 self.dropped_objects.insert(
1974 ingestion_desc.remap_collection_id,
1975 active_replicas.clone(),
1976 );
1977 }
1978 }
1979 _ => {}
1980 }
1981
1982 self.dropped_objects.insert(*id, active_replicas);
1983 }
1984 }
1985 }
1986
1987 self.storage_collections
1989 .drop_collections_unvalidated(storage_metadata, ids);
1990
1991 Ok(())
1992 }
1993
1994 fn drop_sinks(
1996 &mut self,
1997 storage_metadata: &StorageMetadata,
1998 identifiers: Vec<GlobalId>,
1999 ) -> Result<(), StorageError<Self::Timestamp>> {
2000 self.validate_export_ids(identifiers.iter().cloned())?;
2001 self.drop_sinks_unvalidated(storage_metadata, identifiers);
2002 Ok(())
2003 }
2004
2005 fn drop_sinks_unvalidated(
2006 &mut self,
2007 storage_metadata: &StorageMetadata,
2008 mut sinks_to_drop: Vec<GlobalId>,
2009 ) {
2010 sinks_to_drop.retain(|id| self.export(*id).is_ok());
2012
2013 let drop_policy = sinks_to_drop
2020 .iter()
2021 .map(|id| (*id, ReadPolicy::ValidFrom(Antichain::new())))
2022 .collect();
2023
2024 tracing::debug!(
2025 ?drop_policy,
2026 "dropping sources by setting read hold policies"
2027 );
2028 self.set_hold_policies(drop_policy);
2029
2030 let status_now = mz_ore::now::to_datetime((self.now)());
2037
2038 let mut status_updates = vec![];
2040 {
2041 let mut sink_statistics = self.sink_statistics.lock().expect("poisoned");
2042 for id in sinks_to_drop.iter() {
2043 status_updates.push(StatusUpdate::new(*id, status_now, Status::Dropped));
2044 sink_statistics.retain(|(stats_id, _), _| stats_id != id);
2045 }
2046 }
2047
2048 if !self.read_only {
2049 self.append_status_introspection_updates(
2050 IntrospectionType::SinkStatusHistory,
2051 status_updates,
2052 );
2053 }
2054
2055 for id in sinks_to_drop.iter() {
2057 tracing::info!(%id, "dropping export state");
2058 let collection = self
2059 .collections
2060 .remove(id)
2061 .expect("list populated after checking that self.collections contains it");
2062
2063 let instance = match &collection.extra_state {
2064 CollectionStateExtra::Ingestion(ingestion) => Some(ingestion.instance_id),
2065 CollectionStateExtra::Export(export) => Some(export.cluster_id()),
2066 CollectionStateExtra::None => None,
2067 }
2068 .and_then(|i| self.instances.get(&i));
2069
2070 if let Some(instance) = instance {
2074 let active_replicas = instance.get_active_replicas_for_object(id);
2075 if !active_replicas.is_empty() {
2076 self.dropped_objects.insert(*id, active_replicas);
2077 }
2078 }
2079 }
2080
2081 self.storage_collections
2083 .drop_collections_unvalidated(storage_metadata, sinks_to_drop);
2084 }
2085
2086 #[instrument(level = "debug")]
2087 fn append_table(
2088 &mut self,
2089 write_ts: Self::Timestamp,
2090 advance_to: Self::Timestamp,
2091 commands: Vec<(GlobalId, Vec<TableData>)>,
2092 ) -> Result<
2093 tokio::sync::oneshot::Receiver<Result<(), StorageError<Self::Timestamp>>>,
2094 StorageError<Self::Timestamp>,
2095 > {
2096 if self.read_only {
2097 if !commands
2100 .iter()
2101 .all(|(id, _)| id.is_system() && self.migrated_storage_collections.contains(id))
2102 {
2103 return Err(StorageError::ReadOnly);
2104 }
2105 }
2106
2107 for (id, updates) in commands.iter() {
2109 if !updates.is_empty() {
2110 if !write_ts.less_than(&advance_to) {
2111 return Err(StorageError::UpdateBeyondUpper(*id));
2112 }
2113 }
2114 }
2115
2116 Ok(self
2117 .persist_table_worker
2118 .append(write_ts, advance_to, commands))
2119 }
2120
2121 fn monotonic_appender(
2122 &self,
2123 id: GlobalId,
2124 ) -> Result<MonotonicAppender<Self::Timestamp>, StorageError<Self::Timestamp>> {
2125 self.collection_manager.monotonic_appender(id)
2126 }
2127
2128 fn webhook_statistics(
2129 &self,
2130 id: GlobalId,
2131 ) -> Result<Arc<WebhookStatistics>, StorageError<Self::Timestamp>> {
2132 let source_statistics = self.source_statistics.lock().expect("poisoned");
2134 source_statistics
2135 .webhook_statistics
2136 .get(&id)
2137 .cloned()
2138 .ok_or(StorageError::IdentifierMissing(id))
2139 }
2140
2141 async fn ready(&mut self) {
2142 if self.maintenance_scheduled {
2143 return;
2144 }
2145
2146 if !self.pending_table_handle_drops_rx.is_empty() {
2147 return;
2148 }
2149
2150 tokio::select! {
2151 Some(m) = self.instance_response_rx.recv() => {
2152 self.stashed_responses.push(m);
2153 while let Ok(m) = self.instance_response_rx.try_recv() {
2154 self.stashed_responses.push(m);
2155 }
2156 }
2157 _ = self.maintenance_ticker.tick() => {
2158 self.maintenance_scheduled = true;
2159 },
2160 };
2161 }
2162
2163 #[instrument(level = "debug")]
2164 fn process(
2165 &mut self,
2166 storage_metadata: &StorageMetadata,
2167 ) -> Result<Option<Response<T>>, anyhow::Error> {
2168 if self.maintenance_scheduled {
2170 self.maintain();
2171 self.maintenance_scheduled = false;
2172 }
2173
2174 for instance in self.instances.values_mut() {
2175 instance.rehydrate_failed_replicas();
2176 }
2177
2178 let mut status_updates = vec![];
2179 let mut updated_frontiers = BTreeMap::new();
2180
2181 let stashed_responses = std::mem::take(&mut self.stashed_responses);
2183 for resp in stashed_responses {
2184 match resp {
2185 (_replica_id, StorageResponse::FrontierUpper(id, upper)) => {
2186 self.update_write_frontier(id, &upper);
2187 updated_frontiers.insert(id, upper);
2188 }
2189 (replica_id, StorageResponse::DroppedId(id)) => {
2190 let replica_id = replica_id.expect("DroppedId from unknown replica");
2191 if let Some(remaining_replicas) = self.dropped_objects.get_mut(&id) {
2192 remaining_replicas.remove(&replica_id);
2193 if remaining_replicas.is_empty() {
2194 self.dropped_objects.remove(&id);
2195 }
2196 } else {
2197 soft_panic_or_log!("unexpected DroppedId for {id}");
2198 }
2199 }
2200 (replica_id, StorageResponse::StatisticsUpdates(source_stats, sink_stats)) => {
2201 {
2203 let replica_id = if let Some(replica_id) = replica_id {
2210 replica_id
2211 } else {
2212 tracing::error!(
2213 ?source_stats,
2214 "missing replica_id for source statistics update"
2215 );
2216 continue;
2217 };
2218
2219 let mut shared_stats = self.source_statistics.lock().expect("poisoned");
2220
2221 for stat in source_stats {
2222 let collection_id = stat.id.clone();
2223
2224 if self.collection(collection_id).is_err() {
2225 continue;
2228 }
2229
2230 let entry = shared_stats
2231 .source_statistics
2232 .entry((stat.id, Some(replica_id)));
2233
2234 match entry {
2235 btree_map::Entry::Vacant(vacant_entry) => {
2236 let mut stats = ControllerSourceStatistics::new(
2237 collection_id,
2238 Some(replica_id),
2239 );
2240 stats.incorporate(stat);
2241 vacant_entry.insert(stats);
2242 }
2243 btree_map::Entry::Occupied(mut occupied_entry) => {
2244 occupied_entry.get_mut().incorporate(stat);
2245 }
2246 }
2247 }
2248 }
2249
2250 {
2251 let replica_id = if let Some(replica_id) = replica_id {
2262 replica_id
2263 } else {
2264 tracing::error!(
2265 ?sink_stats,
2266 "missing replica_id for sink statistics update"
2267 );
2268 continue;
2269 };
2270
2271 let mut shared_stats = self.sink_statistics.lock().expect("poisoned");
2272
2273 for stat in sink_stats {
2274 let collection_id = stat.id.clone();
2275
2276 if self.collection(collection_id).is_err() {
2277 continue;
2280 }
2281
2282 let entry = shared_stats.entry((stat.id, Some(replica_id)));
2283
2284 match entry {
2285 btree_map::Entry::Vacant(vacant_entry) => {
2286 let mut stats =
2287 ControllerSinkStatistics::new(collection_id, replica_id);
2288 stats.incorporate(stat);
2289 vacant_entry.insert(stats);
2290 }
2291 btree_map::Entry::Occupied(mut occupied_entry) => {
2292 occupied_entry.get_mut().incorporate(stat);
2293 }
2294 }
2295 }
2296 }
2297 }
2298 (replica_id, StorageResponse::StatusUpdate(mut status_update)) => {
2299 match status_update.status {
2315 Status::Running => {
2316 let collection = self.collections.get_mut(&status_update.id);
2317 match collection {
2318 Some(collection) => {
2319 match collection.extra_state {
2320 CollectionStateExtra::Ingestion(
2321 ref mut ingestion_state,
2322 ) => {
2323 if ingestion_state.hydrated_on.is_empty() {
2324 tracing::debug!(ingestion_id = %status_update.id, "ingestion is hydrated");
2325 }
2326 ingestion_state.hydrated_on.insert(replica_id.expect(
2327 "replica id should be present for status running",
2328 ));
2329 }
2330 CollectionStateExtra::Export(_) => {
2331 }
2333 CollectionStateExtra::None => {
2334 }
2336 }
2337 }
2338 None => (), }
2341 }
2342 Status::Paused => {
2343 let collection = self.collections.get_mut(&status_update.id);
2344 match collection {
2345 Some(collection) => {
2346 match collection.extra_state {
2347 CollectionStateExtra::Ingestion(
2348 ref mut ingestion_state,
2349 ) => {
2350 tracing::debug!(ingestion_id = %status_update.id, "ingestion is now paused");
2357 ingestion_state.hydrated_on.clear();
2358 }
2359 CollectionStateExtra::Export(_) => {
2360 }
2362 CollectionStateExtra::None => {
2363 }
2365 }
2366 }
2367 None => (), }
2370 }
2371 _ => (),
2372 }
2373
2374 if let Some(id) = replica_id {
2376 status_update.replica_id = Some(id);
2377 }
2378 status_updates.push(status_update);
2379 }
2380 (_replica_id, StorageResponse::StagedBatches(batches)) => {
2381 for (ingestion_id, batches) in batches {
2382 match self.pending_oneshot_ingestions.remove(&ingestion_id) {
2383 Some(pending) => {
2384 if let Some(instance) = self.instances.get_mut(&pending.cluster_id)
2387 {
2388 instance
2389 .send(StorageCommand::CancelOneshotIngestion(ingestion_id));
2390 }
2391 (pending.result_tx)(batches)
2393 }
2394 None => {
2395 }
2398 }
2399 }
2400 }
2401 }
2402 }
2403
2404 self.record_status_updates(status_updates);
2405
2406 let mut dropped_table_ids = Vec::new();
2408 while let Ok(dropped_id) = self.pending_table_handle_drops_rx.try_recv() {
2409 dropped_table_ids.push(dropped_id);
2410 }
2411 if !dropped_table_ids.is_empty() {
2412 self.drop_sources(storage_metadata, dropped_table_ids)?;
2413 }
2414
2415 if updated_frontiers.is_empty() {
2416 Ok(None)
2417 } else {
2418 Ok(Some(Response::FrontierUpdates(
2419 updated_frontiers.into_iter().collect(),
2420 )))
2421 }
2422 }
2423
2424 async fn inspect_persist_state(
2425 &self,
2426 id: GlobalId,
2427 ) -> Result<serde_json::Value, anyhow::Error> {
2428 let collection = &self.storage_collections.collection_metadata(id)?;
2429 let client = self
2430 .persist
2431 .open(collection.persist_location.clone())
2432 .await?;
2433 let shard_state = client
2434 .inspect_shard::<Self::Timestamp>(&collection.data_shard)
2435 .await?;
2436 let json_state = serde_json::to_value(shard_state)?;
2437 Ok(json_state)
2438 }
2439
2440 fn append_introspection_updates(
2441 &mut self,
2442 type_: IntrospectionType,
2443 updates: Vec<(Row, Diff)>,
2444 ) {
2445 let id = self.introspection_ids[&type_];
2446 let updates = updates.into_iter().map(|update| update.into()).collect();
2447 self.collection_manager.blind_write(id, updates);
2448 }
2449
2450 fn append_status_introspection_updates(
2451 &mut self,
2452 type_: IntrospectionType,
2453 updates: Vec<StatusUpdate>,
2454 ) {
2455 let id = self.introspection_ids[&type_];
2456 let updates: Vec<_> = updates.into_iter().map(|update| update.into()).collect();
2457 if !updates.is_empty() {
2458 self.collection_manager.blind_write(id, updates);
2459 }
2460 }
2461
2462 fn update_introspection_collection(&mut self, type_: IntrospectionType, op: StorageWriteOp) {
2463 let id = self.introspection_ids[&type_];
2464 self.collection_manager.differential_write(id, op);
2465 }
2466
2467 fn append_only_introspection_tx(
2468 &self,
2469 type_: IntrospectionType,
2470 ) -> mpsc::UnboundedSender<(
2471 Vec<AppendOnlyUpdate>,
2472 oneshot::Sender<Result<(), StorageError<Self::Timestamp>>>,
2473 )> {
2474 let id = self.introspection_ids[&type_];
2475 self.collection_manager.append_only_write_sender(id)
2476 }
2477
2478 fn differential_introspection_tx(
2479 &self,
2480 type_: IntrospectionType,
2481 ) -> mpsc::UnboundedSender<(
2482 StorageWriteOp,
2483 oneshot::Sender<Result<(), StorageError<Self::Timestamp>>>,
2484 )> {
2485 let id = self.introspection_ids[&type_];
2486 self.collection_manager.differential_write_sender(id)
2487 }
2488
2489 async fn real_time_recent_timestamp(
2490 &self,
2491 timestamp_objects: BTreeSet<GlobalId>,
2492 timeout: Duration,
2493 ) -> Result<
2494 BoxFuture<Result<Self::Timestamp, StorageError<Self::Timestamp>>>,
2495 StorageError<Self::Timestamp>,
2496 > {
2497 use mz_storage_types::sources::GenericSourceConnection;
2498
2499 let mut rtr_futures = BTreeMap::new();
2500
2501 for id in timestamp_objects.into_iter().filter(GlobalId::is_user) {
2503 let collection = match self.collection(id) {
2504 Ok(c) => c,
2505 Err(_) => continue,
2507 };
2508
2509 let (source_conn, remap_id) = match &collection.data_source {
2510 DataSource::Ingestion(IngestionDescription {
2511 desc: SourceDesc { connection, .. },
2512 remap_collection_id,
2513 ..
2514 }) => match connection {
2515 GenericSourceConnection::Kafka(_)
2516 | GenericSourceConnection::Postgres(_)
2517 | GenericSourceConnection::MySql(_)
2518 | GenericSourceConnection::SqlServer(_) => {
2519 (connection.clone(), *remap_collection_id)
2520 }
2521
2522 GenericSourceConnection::LoadGenerator(_) => continue,
2527 },
2528 _ => {
2530 continue;
2531 }
2532 };
2533
2534 let config = self.config().clone();
2536
2537 let read_handle = self.read_handle_for_snapshot(remap_id).await?;
2545
2546 let remap_read_hold = self
2549 .storage_collections
2550 .acquire_read_holds(vec![remap_id])
2551 .map_err(|_e| StorageError::ReadBeforeSince(remap_id))?
2552 .expect_element(|| "known to be exactly one");
2553
2554 let remap_as_of = remap_read_hold
2555 .since()
2556 .to_owned()
2557 .into_option()
2558 .ok_or(StorageError::ReadBeforeSince(remap_id))?;
2559
2560 rtr_futures.insert(
2561 id,
2562 tokio::time::timeout(timeout, async move {
2563 use mz_storage_types::sources::SourceConnection as _;
2564
2565 let as_of = Antichain::from_elem(remap_as_of);
2568 let remap_subscribe = read_handle
2569 .subscribe(as_of.clone())
2570 .await
2571 .map_err(|_| StorageError::ReadBeforeSince(remap_id))?;
2572
2573 tracing::debug!(?id, type_ = source_conn.name(), upstream = ?source_conn.external_reference(), "fetching real time recency");
2574
2575 let result = rtr::real_time_recency_ts(source_conn, id, config, as_of, remap_subscribe)
2576 .await.map_err(|e| {
2577 tracing::debug!(?id, "real time recency error: {:?}", e);
2578 e
2579 });
2580
2581 drop(remap_read_hold);
2583
2584 result
2585 }),
2586 );
2587 }
2588
2589 Ok(Box::pin(async move {
2590 let (ids, futs): (Vec<_>, Vec<_>) = rtr_futures.into_iter().unzip();
2591 ids.into_iter()
2592 .zip_eq(futures::future::join_all(futs).await)
2593 .try_fold(T::minimum(), |curr, (id, per_source_res)| {
2594 let new =
2595 per_source_res.map_err(|_e: Elapsed| StorageError::RtrTimeout(id))??;
2596 Ok::<_, StorageError<Self::Timestamp>>(std::cmp::max(curr, new))
2597 })
2598 }))
2599 }
2600
2601 fn dump(&self) -> Result<serde_json::Value, anyhow::Error> {
2602 let Self {
2604 build_info: _,
2605 now: _,
2606 read_only,
2607 collections,
2608 dropped_objects,
2609 persist_table_worker: _,
2610 txns_read: _,
2611 txns_metrics: _,
2612 stashed_responses,
2613 pending_table_handle_drops_tx: _,
2614 pending_table_handle_drops_rx: _,
2615 pending_oneshot_ingestions,
2616 collection_manager: _,
2617 introspection_ids,
2618 introspection_tokens: _,
2619 source_statistics: _,
2620 sink_statistics: _,
2621 statistics_interval_sender: _,
2622 instances,
2623 initialized,
2624 config,
2625 persist_location,
2626 persist: _,
2627 metrics: _,
2628 recorded_frontiers,
2629 recorded_replica_frontiers,
2630 wallclock_lag: _,
2631 wallclock_lag_last_recorded,
2632 storage_collections: _,
2633 migrated_storage_collections,
2634 maintenance_ticker: _,
2635 maintenance_scheduled,
2636 instance_response_tx: _,
2637 instance_response_rx: _,
2638 persist_warm_task: _,
2639 } = self;
2640
2641 let collections: BTreeMap<_, _> = collections
2642 .iter()
2643 .map(|(id, c)| (id.to_string(), format!("{c:?}")))
2644 .collect();
2645 let dropped_objects: BTreeMap<_, _> = dropped_objects
2646 .iter()
2647 .map(|(id, rs)| (id.to_string(), format!("{rs:?}")))
2648 .collect();
2649 let stashed_responses: Vec<_> =
2650 stashed_responses.iter().map(|r| format!("{r:?}")).collect();
2651 let pending_oneshot_ingestions: BTreeMap<_, _> = pending_oneshot_ingestions
2652 .iter()
2653 .map(|(uuid, i)| (uuid.to_string(), format!("{i:?}")))
2654 .collect();
2655 let introspection_ids: BTreeMap<_, _> = introspection_ids
2656 .iter()
2657 .map(|(typ, id)| (format!("{typ:?}"), id.to_string()))
2658 .collect();
2659 let instances: BTreeMap<_, _> = instances
2660 .iter()
2661 .map(|(id, i)| (id.to_string(), format!("{i:?}")))
2662 .collect();
2663 let recorded_frontiers: BTreeMap<_, _> = recorded_frontiers
2664 .iter()
2665 .map(|(id, fs)| (id.to_string(), format!("{fs:?}")))
2666 .collect();
2667 let recorded_replica_frontiers: Vec<_> = recorded_replica_frontiers
2668 .iter()
2669 .map(|((gid, rid), f)| (gid.to_string(), rid.to_string(), format!("{f:?}")))
2670 .collect();
2671 let migrated_storage_collections: Vec<_> = migrated_storage_collections
2672 .iter()
2673 .map(|id| id.to_string())
2674 .collect();
2675
2676 Ok(serde_json::json!({
2677 "read_only": read_only,
2678 "collections": collections,
2679 "dropped_objects": dropped_objects,
2680 "stashed_responses": stashed_responses,
2681 "pending_oneshot_ingestions": pending_oneshot_ingestions,
2682 "introspection_ids": introspection_ids,
2683 "instances": instances,
2684 "initialized": initialized,
2685 "config": format!("{config:?}"),
2686 "persist_location": format!("{persist_location:?}"),
2687 "recorded_frontiers": recorded_frontiers,
2688 "recorded_replica_frontiers": recorded_replica_frontiers,
2689 "wallclock_lag_last_recorded": format!("{wallclock_lag_last_recorded:?}"),
2690 "migrated_storage_collections": migrated_storage_collections,
2691 "maintenance_scheduled": maintenance_scheduled,
2692 }))
2693 }
2694}
2695
2696pub fn prepare_initialization<T>(txn: &mut dyn StorageTxn<T>) -> Result<(), StorageError<T>> {
2703 if txn.get_txn_wal_shard().is_none() {
2704 let txns_id = ShardId::new();
2705 txn.write_txn_wal_shard(txns_id)?;
2706 }
2707
2708 Ok(())
2709}
2710
2711impl<T> Controller<T>
2712where
2713 T: Timestamp
2714 + Lattice
2715 + TotalOrder
2716 + Codec64
2717 + From<EpochMillis>
2718 + TimestampManipulation
2719 + Into<Datum<'static>>,
2720 Self: StorageController<Timestamp = T>,
2721{
2722 pub async fn new(
2730 build_info: &'static BuildInfo,
2731 persist_location: PersistLocation,
2732 persist_clients: Arc<PersistClientCache>,
2733 now: NowFn,
2734 wallclock_lag: WallclockLagFn<T>,
2735 txns_metrics: Arc<TxnMetrics>,
2736 read_only: bool,
2737 metrics_registry: &MetricsRegistry,
2738 controller_metrics: ControllerMetrics,
2739 connection_context: ConnectionContext,
2740 txn: &dyn StorageTxn<T>,
2741 storage_collections: Arc<dyn StorageCollections<Timestamp = T> + Send + Sync>,
2742 ) -> Self {
2743 let txns_client = persist_clients
2744 .open(persist_location.clone())
2745 .await
2746 .expect("location should be valid");
2747
2748 let persist_warm_task = warm_persist_state_in_background(
2749 txns_client.clone(),
2750 txn.get_collection_metadata().into_values(),
2751 );
2752 let persist_warm_task = Some(persist_warm_task.abort_on_drop());
2753
2754 let txns_id = txn
2758 .get_txn_wal_shard()
2759 .expect("must call prepare initialization before creating storage controller");
2760
2761 let persist_table_worker = if read_only {
2762 let txns_write = txns_client
2763 .open_writer(
2764 txns_id,
2765 Arc::new(TxnsCodecRow::desc()),
2766 Arc::new(UnitSchema),
2767 Diagnostics {
2768 shard_name: "txns".to_owned(),
2769 handle_purpose: "follow txns upper".to_owned(),
2770 },
2771 )
2772 .await
2773 .expect("txns schema shouldn't change");
2774 persist_handles::PersistTableWriteWorker::new_read_only_mode(txns_write)
2775 } else {
2776 let mut txns = TxnsHandle::open(
2777 T::minimum(),
2778 txns_client.clone(),
2779 txns_client.dyncfgs().clone(),
2780 Arc::clone(&txns_metrics),
2781 txns_id,
2782 )
2783 .await;
2784 txns.upgrade_version().await;
2785 persist_handles::PersistTableWriteWorker::new_txns(txns)
2786 };
2787 let txns_read = TxnsRead::start::<TxnsCodecRow>(txns_client.clone(), txns_id).await;
2788
2789 let collection_manager = collection_mgmt::CollectionManager::new(read_only, now.clone());
2790
2791 let introspection_ids = BTreeMap::new();
2792 let introspection_tokens = Arc::new(Mutex::new(BTreeMap::new()));
2793
2794 let (statistics_interval_sender, _) =
2795 channel(mz_storage_types::parameters::STATISTICS_INTERVAL_DEFAULT);
2796
2797 let (pending_table_handle_drops_tx, pending_table_handle_drops_rx) =
2798 tokio::sync::mpsc::unbounded_channel();
2799
2800 let mut maintenance_ticker = tokio::time::interval(Duration::from_secs(1));
2801 maintenance_ticker.set_missed_tick_behavior(MissedTickBehavior::Skip);
2802
2803 let (instance_response_tx, instance_response_rx) = mpsc::unbounded_channel();
2804
2805 let metrics = StorageControllerMetrics::new(metrics_registry, controller_metrics);
2806
2807 let now_dt = mz_ore::now::to_datetime(now());
2808
2809 Self {
2810 build_info,
2811 collections: BTreeMap::default(),
2812 dropped_objects: Default::default(),
2813 persist_table_worker,
2814 txns_read,
2815 txns_metrics,
2816 stashed_responses: vec![],
2817 pending_table_handle_drops_tx,
2818 pending_table_handle_drops_rx,
2819 pending_oneshot_ingestions: BTreeMap::default(),
2820 collection_manager,
2821 introspection_ids,
2822 introspection_tokens,
2823 now,
2824 read_only,
2825 source_statistics: Arc::new(Mutex::new(statistics::SourceStatistics {
2826 source_statistics: BTreeMap::new(),
2827 webhook_statistics: BTreeMap::new(),
2828 })),
2829 sink_statistics: Arc::new(Mutex::new(BTreeMap::new())),
2830 statistics_interval_sender,
2831 instances: BTreeMap::new(),
2832 initialized: false,
2833 config: StorageConfiguration::new(connection_context, mz_dyncfgs::all_dyncfgs()),
2834 persist_location,
2835 persist: persist_clients,
2836 metrics,
2837 recorded_frontiers: BTreeMap::new(),
2838 recorded_replica_frontiers: BTreeMap::new(),
2839 wallclock_lag,
2840 wallclock_lag_last_recorded: now_dt,
2841 storage_collections,
2842 migrated_storage_collections: BTreeSet::new(),
2843 maintenance_ticker,
2844 maintenance_scheduled: false,
2845 instance_response_rx,
2846 instance_response_tx,
2847 persist_warm_task,
2848 }
2849 }
2850
2851 #[instrument(level = "debug")]
2859 fn set_hold_policies(&mut self, policies: Vec<(GlobalId, ReadPolicy<T>)>) {
2860 let mut read_capability_changes = BTreeMap::default();
2861
2862 for (id, policy) in policies.into_iter() {
2863 if let Some(collection) = self.collections.get_mut(&id) {
2864 let (write_frontier, derived_since, hold_policy) = match &mut collection.extra_state
2865 {
2866 CollectionStateExtra::Ingestion(ingestion) => (
2867 ingestion.write_frontier.borrow(),
2868 &mut ingestion.derived_since,
2869 &mut ingestion.hold_policy,
2870 ),
2871 CollectionStateExtra::None => {
2872 unreachable!("set_hold_policies is only called for ingestions");
2873 }
2874 CollectionStateExtra::Export(export) => (
2875 export.write_frontier.borrow(),
2876 &mut export.derived_since,
2877 &mut export.read_policy,
2878 ),
2879 };
2880
2881 let new_derived_since = policy.frontier(write_frontier);
2882 let mut update = swap_updates(derived_since, new_derived_since);
2883 if !update.is_empty() {
2884 read_capability_changes.insert(id, update);
2885 }
2886
2887 *hold_policy = policy;
2888 }
2889 }
2890
2891 if !read_capability_changes.is_empty() {
2892 self.update_hold_capabilities(&mut read_capability_changes);
2893 }
2894 }
2895
2896 #[instrument(level = "debug", fields(updates))]
2897 fn update_write_frontier(&mut self, id: GlobalId, new_upper: &Antichain<T>) {
2898 let mut read_capability_changes = BTreeMap::default();
2899
2900 if let Some(collection) = self.collections.get_mut(&id) {
2901 let (write_frontier, derived_since, hold_policy) = match &mut collection.extra_state {
2902 CollectionStateExtra::Ingestion(ingestion) => (
2903 &mut ingestion.write_frontier,
2904 &mut ingestion.derived_since,
2905 &ingestion.hold_policy,
2906 ),
2907 CollectionStateExtra::None => {
2908 if matches!(collection.data_source, DataSource::Progress) {
2909 } else {
2911 tracing::error!(
2912 ?collection,
2913 ?new_upper,
2914 "updated write frontier for collection which is not an ingestion"
2915 );
2916 }
2917 return;
2918 }
2919 CollectionStateExtra::Export(export) => (
2920 &mut export.write_frontier,
2921 &mut export.derived_since,
2922 &export.read_policy,
2923 ),
2924 };
2925
2926 if PartialOrder::less_than(write_frontier, new_upper) {
2927 write_frontier.clone_from(new_upper);
2928 }
2929
2930 let new_derived_since = hold_policy.frontier(write_frontier.borrow());
2931 let mut update = swap_updates(derived_since, new_derived_since);
2932 if !update.is_empty() {
2933 read_capability_changes.insert(id, update);
2934 }
2935 } else if self.dropped_objects.contains_key(&id) {
2936 } else {
2939 soft_panic_or_log!("spurious upper update for {id}: {new_upper:?}");
2940 }
2941
2942 if !read_capability_changes.is_empty() {
2943 self.update_hold_capabilities(&mut read_capability_changes);
2944 }
2945 }
2946
2947 #[instrument(level = "debug", fields(updates))]
2951 fn update_hold_capabilities(&mut self, updates: &mut BTreeMap<GlobalId, ChangeBatch<T>>) {
2952 let mut collections_net = BTreeMap::new();
2954
2955 while let Some(key) = updates.keys().rev().next().cloned() {
2960 let mut update = updates.remove(&key).unwrap();
2961
2962 if key.is_user() {
2963 debug!(id = %key, ?update, "update_hold_capability");
2964 }
2965
2966 if let Some(collection) = self.collections.get_mut(&key) {
2967 match &mut collection.extra_state {
2968 CollectionStateExtra::Ingestion(ingestion) => {
2969 let changes = ingestion.read_capabilities.update_iter(update.drain());
2970 update.extend(changes);
2971
2972 let (changes, frontier, _cluster_id) =
2973 collections_net.entry(key).or_insert_with(|| {
2974 (
2975 <ChangeBatch<_>>::new(),
2976 Antichain::new(),
2977 ingestion.instance_id,
2978 )
2979 });
2980
2981 changes.extend(update.drain());
2982 *frontier = ingestion.read_capabilities.frontier().to_owned();
2983 }
2984 CollectionStateExtra::None => {
2985 soft_panic_or_log!(
2987 "trying to update holds for collection {collection:?} which is not \
2988 an ingestion: {update:?}"
2989 );
2990 continue;
2991 }
2992 CollectionStateExtra::Export(export) => {
2993 let changes = export.read_capabilities.update_iter(update.drain());
2994 update.extend(changes);
2995
2996 let (changes, frontier, _cluster_id) =
2997 collections_net.entry(key).or_insert_with(|| {
2998 (<ChangeBatch<_>>::new(), Antichain::new(), export.cluster_id)
2999 });
3000
3001 changes.extend(update.drain());
3002 *frontier = export.read_capabilities.frontier().to_owned();
3003 }
3004 }
3005 } else {
3006 tracing::warn!(id = ?key, ?update, "update_hold_capabilities for unknown object");
3008 }
3009 }
3010
3011 for (key, (mut changes, frontier, cluster_id)) in collections_net {
3014 if !changes.is_empty() {
3015 if key.is_user() {
3016 debug!(id = %key, ?frontier, "downgrading ingestion read holds!");
3017 }
3018
3019 let collection = self
3020 .collections
3021 .get_mut(&key)
3022 .expect("missing collection state");
3023
3024 let read_holds = match &mut collection.extra_state {
3025 CollectionStateExtra::Ingestion(ingestion) => {
3026 ingestion.dependency_read_holds.as_mut_slice()
3027 }
3028 CollectionStateExtra::Export(export) => export.read_holds.as_mut_slice(),
3029 CollectionStateExtra::None => {
3030 soft_panic_or_log!(
3031 "trying to downgrade read holds for collection which is not an \
3032 ingestion: {collection:?}"
3033 );
3034 continue;
3035 }
3036 };
3037
3038 for read_hold in read_holds.iter_mut() {
3039 read_hold
3040 .try_downgrade(frontier.clone())
3041 .expect("we only advance the frontier");
3042 }
3043
3044 if let Some(instance) = self.instances.get_mut(&cluster_id) {
3046 instance.send(StorageCommand::AllowCompaction(key, frontier.clone()));
3047 } else {
3048 soft_panic_or_log!(
3049 "missing instance client for cluster {cluster_id} while we still have outstanding AllowCompaction command {frontier:?} for {key}"
3050 );
3051 }
3052 }
3053 }
3054 }
3055
3056 fn validate_collection_ids(
3058 &self,
3059 ids: impl Iterator<Item = GlobalId>,
3060 ) -> Result<(), StorageError<T>> {
3061 for id in ids {
3062 self.storage_collections.check_exists(id)?;
3063 }
3064 Ok(())
3065 }
3066
3067 fn validate_export_ids(
3069 &self,
3070 ids: impl Iterator<Item = GlobalId>,
3071 ) -> Result<(), StorageError<T>> {
3072 for id in ids {
3073 self.export(id)?;
3074 }
3075 Ok(())
3076 }
3077
3078 async fn open_data_handles(
3086 &self,
3087 id: &GlobalId,
3088 shard: ShardId,
3089 relation_desc: RelationDesc,
3090 persist_client: &PersistClient,
3091 ) -> WriteHandle<SourceData, (), T, StorageDiff> {
3092 let diagnostics = Diagnostics {
3093 shard_name: id.to_string(),
3094 handle_purpose: format!("controller data for {}", id),
3095 };
3096
3097 let mut write = persist_client
3098 .open_writer(
3099 shard,
3100 Arc::new(relation_desc),
3101 Arc::new(UnitSchema),
3102 diagnostics.clone(),
3103 )
3104 .await
3105 .expect("invalid persist usage");
3106
3107 write.fetch_recent_upper().await;
3116
3117 write
3118 }
3119
3120 fn register_introspection_collection(
3125 &mut self,
3126 id: GlobalId,
3127 introspection_type: IntrospectionType,
3128 write_handle: WriteHandle<SourceData, (), T, StorageDiff>,
3129 persist_client: PersistClient,
3130 ) -> Result<(), StorageError<T>> {
3131 tracing::info!(%id, ?introspection_type, "registering introspection collection");
3132
3133 let force_writable = self.read_only && self.migrated_storage_collections.contains(&id);
3137 if force_writable {
3138 assert!(id.is_system(), "unexpected non-system global id: {id:?}");
3139 info!("writing to migrated storage collection {id} in read-only mode");
3140 }
3141
3142 let prev = self.introspection_ids.insert(introspection_type, id);
3143 assert!(
3144 prev.is_none(),
3145 "cannot have multiple IDs for introspection type"
3146 );
3147
3148 let metadata = self.storage_collections.collection_metadata(id)?.clone();
3149
3150 let read_handle_fn = move || {
3151 let persist_client = persist_client.clone();
3152 let metadata = metadata.clone();
3153
3154 let fut = async move {
3155 let read_handle = persist_client
3156 .open_leased_reader::<SourceData, (), T, StorageDiff>(
3157 metadata.data_shard,
3158 Arc::new(metadata.relation_desc.clone()),
3159 Arc::new(UnitSchema),
3160 Diagnostics {
3161 shard_name: id.to_string(),
3162 handle_purpose: format!("snapshot {}", id),
3163 },
3164 USE_CRITICAL_SINCE_SNAPSHOT.get(persist_client.dyncfgs()),
3165 )
3166 .await
3167 .expect("invalid persist usage");
3168 read_handle
3169 };
3170
3171 fut.boxed()
3172 };
3173
3174 let recent_upper = write_handle.shared_upper();
3175
3176 match CollectionManagerKind::from(&introspection_type) {
3177 CollectionManagerKind::Differential => {
3182 let statistics_retention_duration =
3183 dyncfgs::STATISTICS_RETENTION_DURATION.get(self.config().config_set());
3184
3185 let introspection_config = DifferentialIntrospectionConfig {
3187 recent_upper,
3188 introspection_type,
3189 storage_collections: Arc::clone(&self.storage_collections),
3190 collection_manager: self.collection_manager.clone(),
3191 source_statistics: Arc::clone(&self.source_statistics),
3192 sink_statistics: Arc::clone(&self.sink_statistics),
3193 statistics_interval: self.config.parameters.statistics_interval.clone(),
3194 statistics_interval_receiver: self.statistics_interval_sender.subscribe(),
3195 statistics_retention_duration,
3196 metrics: self.metrics.clone(),
3197 introspection_tokens: Arc::clone(&self.introspection_tokens),
3198 };
3199 self.collection_manager.register_differential_collection(
3200 id,
3201 write_handle,
3202 read_handle_fn,
3203 force_writable,
3204 introspection_config,
3205 );
3206 }
3207 CollectionManagerKind::AppendOnly => {
3215 let introspection_config = AppendOnlyIntrospectionConfig {
3216 introspection_type,
3217 config_set: Arc::clone(self.config.config_set()),
3218 parameters: self.config.parameters.clone(),
3219 storage_collections: Arc::clone(&self.storage_collections),
3220 };
3221 self.collection_manager.register_append_only_collection(
3222 id,
3223 write_handle,
3224 force_writable,
3225 Some(introspection_config),
3226 );
3227 }
3228 }
3229
3230 Ok(())
3231 }
3232
3233 fn reconcile_dangling_statistics(&self) {
3236 self.source_statistics
3237 .lock()
3238 .expect("poisoned")
3239 .source_statistics
3240 .retain(|(k, _replica_id), _| self.storage_collections.check_exists(*k).is_ok());
3242 self.sink_statistics
3243 .lock()
3244 .expect("poisoned")
3245 .retain(|(k, _replica_id), _| self.export(*k).is_ok());
3246 }
3247
3248 #[instrument(level = "debug")]
3258 fn append_shard_mappings<I>(&self, global_ids: I, diff: Diff)
3259 where
3260 I: Iterator<Item = GlobalId>,
3261 {
3262 mz_ore::soft_assert_or_log!(
3263 diff == Diff::MINUS_ONE || diff == Diff::ONE,
3264 "use 1 for insert or -1 for delete"
3265 );
3266
3267 let id = *self
3268 .introspection_ids
3269 .get(&IntrospectionType::ShardMapping)
3270 .expect("should be registered before this call");
3271
3272 let mut updates = vec![];
3273 let mut row_buf = Row::default();
3275
3276 for global_id in global_ids {
3277 let shard_id = if let Some(collection) = self.collections.get(&global_id) {
3278 collection.collection_metadata.data_shard.clone()
3279 } else {
3280 panic!("unknown global id: {}", global_id);
3281 };
3282
3283 let mut packer = row_buf.packer();
3284 packer.push(Datum::from(global_id.to_string().as_str()));
3285 packer.push(Datum::from(shard_id.to_string().as_str()));
3286 updates.push((row_buf.clone(), diff));
3287 }
3288
3289 self.collection_manager.differential_append(id, updates);
3290 }
3291
3292 fn determine_collection_dependencies(
3294 &self,
3295 self_id: GlobalId,
3296 collection_desc: &CollectionDescription<T>,
3297 ) -> Result<Vec<GlobalId>, StorageError<T>> {
3298 let mut dependencies = Vec::new();
3299
3300 if let Some(id) = collection_desc.primary {
3301 dependencies.push(id);
3302 }
3303
3304 match &collection_desc.data_source {
3305 DataSource::Introspection(_)
3306 | DataSource::Webhook
3307 | DataSource::Table
3308 | DataSource::Progress
3309 | DataSource::Other => (),
3310 DataSource::IngestionExport { ingestion_id, .. } => {
3311 let source_collection = self.collection(*ingestion_id)?;
3314 let ingestion_remap_collection_id = match &source_collection.data_source {
3315 DataSource::Ingestion(ingestion) => ingestion.remap_collection_id,
3316 _ => unreachable!(
3317 "SourceExport must only refer to primary sources that already exist"
3318 ),
3319 };
3320
3321 dependencies.extend([self_id, ingestion_remap_collection_id]);
3327 }
3328 DataSource::Ingestion(ingestion) => {
3330 dependencies.push(self_id);
3335 if self_id != ingestion.remap_collection_id {
3336 dependencies.push(ingestion.remap_collection_id);
3337 }
3338 }
3339 DataSource::Sink { desc } => {
3340 dependencies.extend([self_id, desc.sink.from]);
3342 }
3343 };
3344
3345 Ok(dependencies)
3346 }
3347
3348 async fn read_handle_for_snapshot(
3349 &self,
3350 id: GlobalId,
3351 ) -> Result<ReadHandle<SourceData, (), T, StorageDiff>, StorageError<T>> {
3352 let metadata = self.storage_collections.collection_metadata(id)?;
3353 read_handle_for_snapshot(&self.persist, id, &metadata).await
3354 }
3355
3356 fn record_status_updates(&mut self, updates: Vec<StatusUpdate>) {
3359 if self.read_only {
3360 return;
3361 }
3362
3363 let mut sink_status_updates = vec![];
3364 let mut source_status_updates = vec![];
3365
3366 for update in updates {
3367 let id = update.id;
3368 if self.export(id).is_ok() {
3369 sink_status_updates.push(update);
3370 } else if self.storage_collections.check_exists(id).is_ok() {
3371 source_status_updates.push(update);
3372 }
3373 }
3374
3375 self.append_status_introspection_updates(
3376 IntrospectionType::SourceStatusHistory,
3377 source_status_updates,
3378 );
3379 self.append_status_introspection_updates(
3380 IntrospectionType::SinkStatusHistory,
3381 sink_status_updates,
3382 );
3383 }
3384
3385 fn collection(&self, id: GlobalId) -> Result<&CollectionState<T>, StorageError<T>> {
3386 self.collections
3387 .get(&id)
3388 .ok_or(StorageError::IdentifierMissing(id))
3389 }
3390
3391 fn run_ingestion(&mut self, id: GlobalId) -> Result<(), StorageError<T>> {
3394 tracing::info!(%id, "starting ingestion");
3395
3396 let collection = self.collection(id)?;
3397 let ingestion_description = match &collection.data_source {
3398 DataSource::Ingestion(i) => i.clone(),
3399 _ => {
3400 tracing::warn!("run_ingestion called on non-ingestion ID {}", id);
3401 Err(StorageError::IdentifierInvalid(id))?
3402 }
3403 };
3404
3405 let mut source_exports = BTreeMap::new();
3407 for (export_id, export) in ingestion_description.source_exports.clone() {
3408 let export_storage_metadata = self.collection(export_id)?.collection_metadata.clone();
3409 source_exports.insert(
3410 export_id,
3411 SourceExport {
3412 storage_metadata: export_storage_metadata,
3413 details: export.details,
3414 data_config: export.data_config,
3415 },
3416 );
3417 }
3418
3419 let remap_collection = self.collection(ingestion_description.remap_collection_id)?;
3420
3421 let description = IngestionDescription::<CollectionMetadata> {
3422 source_exports,
3423 remap_metadata: remap_collection.collection_metadata.clone(),
3424 desc: ingestion_description.desc.clone(),
3426 instance_id: ingestion_description.instance_id,
3427 remap_collection_id: ingestion_description.remap_collection_id,
3428 };
3429
3430 let storage_instance_id = description.instance_id;
3431 let instance = self
3433 .instances
3434 .get_mut(&storage_instance_id)
3435 .ok_or_else(|| StorageError::IngestionInstanceMissing {
3436 storage_instance_id,
3437 ingestion_id: id,
3438 })?;
3439
3440 let augmented_ingestion = Box::new(RunIngestionCommand { id, description });
3441 instance.send(StorageCommand::RunIngestion(augmented_ingestion));
3442
3443 Ok(())
3444 }
3445
3446 fn run_export(&mut self, id: GlobalId) -> Result<(), StorageError<T>> {
3449 let DataSource::Sink { desc: description } = &self.collections[&id].data_source else {
3450 return Err(StorageError::IdentifierMissing(id));
3451 };
3452
3453 let from_storage_metadata = self
3454 .storage_collections
3455 .collection_metadata(description.sink.from)?;
3456 let to_storage_metadata = self.storage_collections.collection_metadata(id)?;
3457
3458 let export_state = self.storage_collections.collection_frontiers(id)?;
3462 let mut as_of = description.sink.as_of.clone();
3463 as_of.join_assign(&export_state.implied_capability);
3464 let with_snapshot = description.sink.with_snapshot
3465 && !PartialOrder::less_than(&as_of, &export_state.write_frontier);
3466
3467 info!(
3468 sink_id = %id,
3469 from_id = %description.sink.from,
3470 write_frontier = ?export_state.write_frontier,
3471 ?as_of,
3472 ?with_snapshot,
3473 "run_export"
3474 );
3475
3476 let cmd = RunSinkCommand {
3477 id,
3478 description: StorageSinkDesc {
3479 from: description.sink.from,
3480 from_desc: description.sink.from_desc.clone(),
3481 connection: description.sink.connection.clone(),
3482 envelope: description.sink.envelope,
3483 as_of,
3484 version: description.sink.version,
3485 from_storage_metadata,
3486 with_snapshot,
3487 to_storage_metadata,
3488 commit_interval: description.sink.commit_interval,
3489 },
3490 };
3491
3492 let storage_instance_id = description.instance_id.clone();
3493
3494 let instance = self
3495 .instances
3496 .get_mut(&storage_instance_id)
3497 .ok_or_else(|| StorageError::ExportInstanceMissing {
3498 storage_instance_id,
3499 export_id: id,
3500 })?;
3501
3502 instance.send(StorageCommand::RunSink(Box::new(cmd)));
3503
3504 Ok(())
3505 }
3506
3507 fn update_frontier_introspection(&mut self) {
3512 let mut global_frontiers = BTreeMap::new();
3513 let mut replica_frontiers = BTreeMap::new();
3514
3515 for collection_frontiers in self.storage_collections.active_collection_frontiers() {
3516 let id = collection_frontiers.id;
3517 let since = collection_frontiers.read_capabilities;
3518 let upper = collection_frontiers.write_frontier;
3519
3520 let instance = self
3521 .collections
3522 .get(&id)
3523 .and_then(|collection_state| match &collection_state.extra_state {
3524 CollectionStateExtra::Ingestion(ingestion) => Some(ingestion.instance_id),
3525 CollectionStateExtra::Export(export) => Some(export.cluster_id()),
3526 CollectionStateExtra::None => None,
3527 })
3528 .and_then(|i| self.instances.get(&i));
3529
3530 if let Some(instance) = instance {
3531 for replica_id in instance.replica_ids() {
3532 replica_frontiers.insert((id, replica_id), upper.clone());
3533 }
3534 }
3535
3536 global_frontiers.insert(id, (since, upper));
3537 }
3538
3539 let mut global_updates = Vec::new();
3540 let mut replica_updates = Vec::new();
3541
3542 let mut push_global_update =
3543 |id: GlobalId, (since, upper): (Antichain<T>, Antichain<T>), diff: Diff| {
3544 let read_frontier = since.into_option().map_or(Datum::Null, |t| t.into());
3545 let write_frontier = upper.into_option().map_or(Datum::Null, |t| t.into());
3546 let row = Row::pack_slice(&[
3547 Datum::String(&id.to_string()),
3548 read_frontier,
3549 write_frontier,
3550 ]);
3551 global_updates.push((row, diff));
3552 };
3553
3554 let mut push_replica_update =
3555 |(id, replica_id): (GlobalId, ReplicaId), upper: Antichain<T>, diff: Diff| {
3556 let write_frontier = upper.into_option().map_or(Datum::Null, |t| t.into());
3557 let row = Row::pack_slice(&[
3558 Datum::String(&id.to_string()),
3559 Datum::String(&replica_id.to_string()),
3560 write_frontier,
3561 ]);
3562 replica_updates.push((row, diff));
3563 };
3564
3565 let mut old_global_frontiers =
3566 std::mem::replace(&mut self.recorded_frontiers, global_frontiers);
3567 for (&id, new) in &self.recorded_frontiers {
3568 match old_global_frontiers.remove(&id) {
3569 Some(old) if &old != new => {
3570 push_global_update(id, new.clone(), Diff::ONE);
3571 push_global_update(id, old, Diff::MINUS_ONE);
3572 }
3573 Some(_) => (),
3574 None => push_global_update(id, new.clone(), Diff::ONE),
3575 }
3576 }
3577 for (id, old) in old_global_frontiers {
3578 push_global_update(id, old, Diff::MINUS_ONE);
3579 }
3580
3581 let mut old_replica_frontiers =
3582 std::mem::replace(&mut self.recorded_replica_frontiers, replica_frontiers);
3583 for (&key, new) in &self.recorded_replica_frontiers {
3584 match old_replica_frontiers.remove(&key) {
3585 Some(old) if &old != new => {
3586 push_replica_update(key, new.clone(), Diff::ONE);
3587 push_replica_update(key, old, Diff::MINUS_ONE);
3588 }
3589 Some(_) => (),
3590 None => push_replica_update(key, new.clone(), Diff::ONE),
3591 }
3592 }
3593 for (key, old) in old_replica_frontiers {
3594 push_replica_update(key, old, Diff::MINUS_ONE);
3595 }
3596
3597 let id = self.introspection_ids[&IntrospectionType::Frontiers];
3598 self.collection_manager
3599 .differential_append(id, global_updates);
3600
3601 let id = self.introspection_ids[&IntrospectionType::ReplicaFrontiers];
3602 self.collection_manager
3603 .differential_append(id, replica_updates);
3604 }
3605
3606 fn refresh_wallclock_lag(&mut self) {
3625 let now_ms = (self.now)();
3626 let histogram_period =
3627 WallclockLagHistogramPeriod::from_epoch_millis(now_ms, self.config.config_set());
3628
3629 let frontier_lag = |frontier: &Antichain<T>| match frontier.as_option() {
3630 Some(ts) => (self.wallclock_lag)(ts.clone()),
3631 None => Duration::ZERO,
3632 };
3633
3634 for frontiers in self.storage_collections.active_collection_frontiers() {
3635 let id = frontiers.id;
3636 let Some(collection) = self.collections.get_mut(&id) else {
3637 continue;
3638 };
3639
3640 let collection_unreadable =
3641 PartialOrder::less_equal(&frontiers.write_frontier, &frontiers.read_capabilities);
3642 let lag = if collection_unreadable {
3643 WallclockLag::Undefined
3644 } else {
3645 let lag = frontier_lag(&frontiers.write_frontier);
3646 WallclockLag::Seconds(lag.as_secs())
3647 };
3648
3649 collection.wallclock_lag_max = collection.wallclock_lag_max.max(lag);
3650
3651 let secs = lag.unwrap_seconds_or(u64::MAX);
3654 collection.wallclock_lag_metrics.observe(secs);
3655
3656 if let Some(stash) = &mut collection.wallclock_lag_histogram_stash {
3657 let bucket = lag.map_seconds(|secs| secs.next_power_of_two());
3658
3659 let instance_id = match &collection.extra_state {
3660 CollectionStateExtra::Ingestion(i) => Some(i.instance_id),
3661 CollectionStateExtra::Export(e) => Some(e.cluster_id()),
3662 CollectionStateExtra::None => None,
3663 };
3664 let workload_class = instance_id
3665 .and_then(|id| self.instances.get(&id))
3666 .and_then(|i| i.workload_class.clone());
3667 let labels = match workload_class {
3668 Some(wc) => [("workload_class", wc.clone())].into(),
3669 None => BTreeMap::new(),
3670 };
3671
3672 let key = (histogram_period, bucket, labels);
3673 *stash.entry(key).or_default() += Diff::ONE;
3674 }
3675 }
3676
3677 self.maybe_record_wallclock_lag();
3679 }
3680
3681 fn maybe_record_wallclock_lag(&mut self) {
3689 if self.read_only {
3690 return;
3691 }
3692
3693 let duration_trunc = |datetime: DateTime<_>, interval| {
3694 let td = TimeDelta::from_std(interval).ok()?;
3695 datetime.duration_trunc(td).ok()
3696 };
3697
3698 let interval = WALLCLOCK_LAG_RECORDING_INTERVAL.get(self.config.config_set());
3699 let now_dt = mz_ore::now::to_datetime((self.now)());
3700 let now_trunc = duration_trunc(now_dt, interval).unwrap_or_else(|| {
3701 soft_panic_or_log!("excessive wallclock lag recording interval: {interval:?}");
3702 let default = WALLCLOCK_LAG_RECORDING_INTERVAL.default();
3703 duration_trunc(now_dt, *default).unwrap()
3704 });
3705 if now_trunc <= self.wallclock_lag_last_recorded {
3706 return;
3707 }
3708
3709 let now_ts: CheckedTimestamp<_> = now_trunc.try_into().expect("must fit");
3710
3711 let mut history_updates = Vec::new();
3712 let mut histogram_updates = Vec::new();
3713 let mut row_buf = Row::default();
3714 for frontiers in self.storage_collections.active_collection_frontiers() {
3715 let id = frontiers.id;
3716 let Some(collection) = self.collections.get_mut(&id) else {
3717 continue;
3718 };
3719
3720 let max_lag = std::mem::replace(&mut collection.wallclock_lag_max, WallclockLag::MIN);
3721 let row = Row::pack_slice(&[
3722 Datum::String(&id.to_string()),
3723 Datum::Null,
3724 max_lag.into_interval_datum(),
3725 Datum::TimestampTz(now_ts),
3726 ]);
3727 history_updates.push((row, Diff::ONE));
3728
3729 let Some(stash) = &mut collection.wallclock_lag_histogram_stash else {
3730 continue;
3731 };
3732
3733 for ((period, lag, labels), count) in std::mem::take(stash) {
3734 let mut packer = row_buf.packer();
3735 packer.extend([
3736 Datum::TimestampTz(period.start),
3737 Datum::TimestampTz(period.end),
3738 Datum::String(&id.to_string()),
3739 lag.into_uint64_datum(),
3740 ]);
3741 let labels = labels.iter().map(|(k, v)| (*k, Datum::String(v)));
3742 packer.push_dict(labels);
3743
3744 histogram_updates.push((row_buf.clone(), count));
3745 }
3746 }
3747
3748 if !history_updates.is_empty() {
3749 self.append_introspection_updates(
3750 IntrospectionType::WallclockLagHistory,
3751 history_updates,
3752 );
3753 }
3754 if !histogram_updates.is_empty() {
3755 self.append_introspection_updates(
3756 IntrospectionType::WallclockLagHistogram,
3757 histogram_updates,
3758 );
3759 }
3760
3761 self.wallclock_lag_last_recorded = now_trunc;
3762 }
3763
3764 fn maintain(&mut self) {
3769 self.update_frontier_introspection();
3770 self.refresh_wallclock_lag();
3771
3772 for instance in self.instances.values_mut() {
3774 instance.refresh_state_metrics();
3775 }
3776 }
3777}
3778
3779impl From<&IntrospectionType> for CollectionManagerKind {
3780 fn from(value: &IntrospectionType) -> Self {
3781 match value {
3782 IntrospectionType::ShardMapping
3783 | IntrospectionType::Frontiers
3784 | IntrospectionType::ReplicaFrontiers
3785 | IntrospectionType::StorageSourceStatistics
3786 | IntrospectionType::StorageSinkStatistics
3787 | IntrospectionType::ComputeDependencies
3788 | IntrospectionType::ComputeOperatorHydrationStatus
3789 | IntrospectionType::ComputeMaterializedViewRefreshes
3790 | IntrospectionType::ComputeErrorCounts
3791 | IntrospectionType::ComputeHydrationTimes => CollectionManagerKind::Differential,
3792
3793 IntrospectionType::SourceStatusHistory
3794 | IntrospectionType::SinkStatusHistory
3795 | IntrospectionType::PrivatelinkConnectionStatusHistory
3796 | IntrospectionType::ReplicaStatusHistory
3797 | IntrospectionType::ReplicaMetricsHistory
3798 | IntrospectionType::WallclockLagHistory
3799 | IntrospectionType::WallclockLagHistogram
3800 | IntrospectionType::PreparedStatementHistory
3801 | IntrospectionType::StatementExecutionHistory
3802 | IntrospectionType::SessionHistory
3803 | IntrospectionType::StatementLifecycleHistory
3804 | IntrospectionType::SqlText => CollectionManagerKind::AppendOnly,
3805 }
3806 }
3807}
3808
3809async fn snapshot_statistics<T>(
3815 id: GlobalId,
3816 upper: Antichain<T>,
3817 storage_collections: &Arc<dyn StorageCollections<Timestamp = T> + Send + Sync>,
3818) -> Vec<Row>
3819where
3820 T: Codec64 + From<EpochMillis> + TimestampManipulation,
3821{
3822 match upper.as_option() {
3823 Some(f) if f > &T::minimum() => {
3824 let as_of = f.step_back().unwrap();
3825
3826 let snapshot = storage_collections.snapshot(id, as_of).await.unwrap();
3827 snapshot
3828 .into_iter()
3829 .map(|(row, diff)| {
3830 assert_eq!(diff, 1);
3831 row
3832 })
3833 .collect()
3834 }
3835 _ => Vec::new(),
3838 }
3839}
3840
3841async fn read_handle_for_snapshot<T>(
3842 persist: &PersistClientCache,
3843 id: GlobalId,
3844 metadata: &CollectionMetadata,
3845) -> Result<ReadHandle<SourceData, (), T, StorageDiff>, StorageError<T>>
3846where
3847 T: Timestamp + Lattice + Codec64 + From<EpochMillis> + TimestampManipulation,
3848{
3849 let persist_client = persist
3850 .open(metadata.persist_location.clone())
3851 .await
3852 .unwrap();
3853
3854 let read_handle = persist_client
3859 .open_leased_reader::<SourceData, (), _, _>(
3860 metadata.data_shard,
3861 Arc::new(metadata.relation_desc.clone()),
3862 Arc::new(UnitSchema),
3863 Diagnostics {
3864 shard_name: id.to_string(),
3865 handle_purpose: format!("snapshot {}", id),
3866 },
3867 USE_CRITICAL_SINCE_SNAPSHOT.get(persist_client.dyncfgs()),
3868 )
3869 .await
3870 .expect("invalid persist usage");
3871 Ok(read_handle)
3872}
3873
3874#[derive(Debug)]
3876struct CollectionState<T: TimelyTimestamp> {
3877 pub data_source: DataSource<T>,
3879
3880 pub collection_metadata: CollectionMetadata,
3881
3882 pub extra_state: CollectionStateExtra<T>,
3883
3884 wallclock_lag_max: WallclockLag,
3886 wallclock_lag_histogram_stash: Option<
3893 BTreeMap<
3894 (
3895 WallclockLagHistogramPeriod,
3896 WallclockLag,
3897 BTreeMap<&'static str, String>,
3898 ),
3899 Diff,
3900 >,
3901 >,
3902 wallclock_lag_metrics: WallclockLagMetrics,
3904}
3905
3906impl<T: TimelyTimestamp> CollectionState<T> {
3907 fn new(
3908 data_source: DataSource<T>,
3909 collection_metadata: CollectionMetadata,
3910 extra_state: CollectionStateExtra<T>,
3911 wallclock_lag_metrics: WallclockLagMetrics,
3912 ) -> Self {
3913 let wallclock_lag_histogram_stash = match &data_source {
3917 DataSource::Other => None,
3918 _ => Some(Default::default()),
3919 };
3920
3921 Self {
3922 data_source,
3923 collection_metadata,
3924 extra_state,
3925 wallclock_lag_max: WallclockLag::MIN,
3926 wallclock_lag_histogram_stash,
3927 wallclock_lag_metrics,
3928 }
3929 }
3930}
3931
3932#[derive(Debug)]
3934enum CollectionStateExtra<T: TimelyTimestamp> {
3935 Ingestion(IngestionState<T>),
3936 Export(ExportState<T>),
3937 None,
3938}
3939
3940#[derive(Debug)]
3942struct IngestionState<T: TimelyTimestamp> {
3943 pub read_capabilities: MutableAntichain<T>,
3945
3946 pub derived_since: Antichain<T>,
3949
3950 pub dependency_read_holds: Vec<ReadHold<T>>,
3952
3953 pub write_frontier: Antichain<T>,
3955
3956 pub hold_policy: ReadPolicy<T>,
3963
3964 pub instance_id: StorageInstanceId,
3966
3967 pub hydrated_on: BTreeSet<ReplicaId>,
3969}
3970
3971struct StatusHistoryDesc<K> {
3976 retention_policy: StatusHistoryRetentionPolicy,
3977 extract_key: Box<dyn Fn(&[Datum]) -> K + Send>,
3978 extract_time: Box<dyn Fn(&[Datum]) -> CheckedTimestamp<DateTime<Utc>> + Send>,
3979}
3980enum StatusHistoryRetentionPolicy {
3981 LastN(usize),
3983 TimeWindow(Duration),
3985}
3986
3987fn source_status_history_desc(
3988 params: &StorageParameters,
3989) -> StatusHistoryDesc<(GlobalId, Option<ReplicaId>)> {
3990 let desc = &MZ_SOURCE_STATUS_HISTORY_DESC;
3991 let (source_id_idx, _) = desc.get_by_name(&"source_id".into()).expect("exists");
3992 let (replica_id_idx, _) = desc.get_by_name(&"replica_id".into()).expect("exists");
3993 let (time_idx, _) = desc.get_by_name(&"occurred_at".into()).expect("exists");
3994
3995 StatusHistoryDesc {
3996 retention_policy: StatusHistoryRetentionPolicy::LastN(
3997 params.keep_n_source_status_history_entries,
3998 ),
3999 extract_key: Box::new(move |datums| {
4000 (
4001 GlobalId::from_str(datums[source_id_idx].unwrap_str()).expect("GlobalId column"),
4002 if datums[replica_id_idx].is_null() {
4003 None
4004 } else {
4005 Some(
4006 ReplicaId::from_str(datums[replica_id_idx].unwrap_str())
4007 .expect("ReplicaId column"),
4008 )
4009 },
4010 )
4011 }),
4012 extract_time: Box::new(move |datums| datums[time_idx].unwrap_timestamptz()),
4013 }
4014}
4015
4016fn sink_status_history_desc(
4017 params: &StorageParameters,
4018) -> StatusHistoryDesc<(GlobalId, Option<ReplicaId>)> {
4019 let desc = &MZ_SINK_STATUS_HISTORY_DESC;
4020 let (sink_id_idx, _) = desc.get_by_name(&"sink_id".into()).expect("exists");
4021 let (replica_id_idx, _) = desc.get_by_name(&"replica_id".into()).expect("exists");
4022 let (time_idx, _) = desc.get_by_name(&"occurred_at".into()).expect("exists");
4023
4024 StatusHistoryDesc {
4025 retention_policy: StatusHistoryRetentionPolicy::LastN(
4026 params.keep_n_sink_status_history_entries,
4027 ),
4028 extract_key: Box::new(move |datums| {
4029 (
4030 GlobalId::from_str(datums[sink_id_idx].unwrap_str()).expect("GlobalId column"),
4031 if datums[replica_id_idx].is_null() {
4032 None
4033 } else {
4034 Some(
4035 ReplicaId::from_str(datums[replica_id_idx].unwrap_str())
4036 .expect("ReplicaId column"),
4037 )
4038 },
4039 )
4040 }),
4041 extract_time: Box::new(move |datums| datums[time_idx].unwrap_timestamptz()),
4042 }
4043}
4044
4045fn privatelink_status_history_desc(params: &StorageParameters) -> StatusHistoryDesc<GlobalId> {
4046 let desc = &MZ_AWS_PRIVATELINK_CONNECTION_STATUS_HISTORY_DESC;
4047 let (key_idx, _) = desc.get_by_name(&"connection_id".into()).expect("exists");
4048 let (time_idx, _) = desc.get_by_name(&"occurred_at".into()).expect("exists");
4049
4050 StatusHistoryDesc {
4051 retention_policy: StatusHistoryRetentionPolicy::LastN(
4052 params.keep_n_privatelink_status_history_entries,
4053 ),
4054 extract_key: Box::new(move |datums| {
4055 GlobalId::from_str(datums[key_idx].unwrap_str()).expect("GlobalId column")
4056 }),
4057 extract_time: Box::new(move |datums| datums[time_idx].unwrap_timestamptz()),
4058 }
4059}
4060
4061fn replica_status_history_desc(params: &StorageParameters) -> StatusHistoryDesc<(GlobalId, u64)> {
4062 let desc = &REPLICA_STATUS_HISTORY_DESC;
4063 let (replica_idx, _) = desc.get_by_name(&"replica_id".into()).expect("exists");
4064 let (process_idx, _) = desc.get_by_name(&"process_id".into()).expect("exists");
4065 let (time_idx, _) = desc.get_by_name(&"occurred_at".into()).expect("exists");
4066
4067 StatusHistoryDesc {
4068 retention_policy: StatusHistoryRetentionPolicy::TimeWindow(
4069 params.replica_status_history_retention_window,
4070 ),
4071 extract_key: Box::new(move |datums| {
4072 (
4073 GlobalId::from_str(datums[replica_idx].unwrap_str()).expect("GlobalId column"),
4074 datums[process_idx].unwrap_uint64(),
4075 )
4076 }),
4077 extract_time: Box::new(move |datums| datums[time_idx].unwrap_timestamptz()),
4078 }
4079}
4080
4081fn swap_updates<T: Timestamp>(
4083 from: &mut Antichain<T>,
4084 mut replace_with: Antichain<T>,
4085) -> ChangeBatch<T> {
4086 let mut update = ChangeBatch::new();
4087 if PartialOrder::less_equal(from, &replace_with) {
4088 update.extend(replace_with.iter().map(|time| (time.clone(), 1)));
4089 std::mem::swap(from, &mut replace_with);
4090 update.extend(replace_with.iter().map(|time| (time.clone(), -1)));
4091 }
4092 update
4093}