From a8ad4ba39f6f59d0c8322a24ffbee286a372c050 Mon Sep 17 00:00:00 2001 From: Joseph Koshakow Date: Wed, 8 Jan 2025 13:15:04 -0500 Subject: [PATCH 01/50] storage: Simplify dataflow as_of computation Previously, async storage workers would calculate the as_of of a dataflow from the since of the dataflow's output's remap shards. If there was more than one distinct remap shard among the outputs, then the storage worker would panic. It's expected that the only collection that will ever have a remap shard is the ingestion collection itself. Furthermore, we are planning to remove the ingestion collection from the outputs of the dataflow (in fact there's already a feature flag that does this). If the ingestion is removed from the outputs, then no output will have a remap shard, and the as_of will always be empty. This commit simplifies the existing as_of calculation and fixes the as_of calculation when the ingestion collection is removed from the outputs. It does this by calculating the as_of directly from the ingestion's remap shard. Additionally, it asserts that if any of the outputs have a remap shard, then it must be equal to the ingestion's remap shard. Works towards resolving #MaterializeInc/database-issues/issues/8620 --- .../src/storage_state/async_storage_worker.rs | 110 +++++++++--------- 1 file changed, 52 insertions(+), 58 deletions(-) diff --git a/src/storage/src/storage_state/async_storage_worker.rs b/src/storage/src/storage_state/async_storage_worker.rs index 0ecbf7d7681ab..ef6673b418a1f 100644 --- a/src/storage/src/storage_state/async_storage_worker.rs +++ b/src/storage/src/storage_state/async_storage_worker.rs @@ -217,9 +217,55 @@ impl AsyncStorageWorker { // arbitrarily hold back collections to perform historical queries and when // the storage command protocol is updated such that these calculations are // performed by the controller and not here. - let mut as_of = Antichain::new(); let mut resume_uppers = BTreeMap::new(); - let mut seen_remap_shard = None; + + // TODO(petrosagg): The as_of of the ingestion should normally be based + // on the since frontiers of its outputs. Even though the storage + // controller makes sure to make downgrade decisions in an organized + // and ordered fashion, it then proceeds to persist them in an + // asynchronous and disorganized fashion to persist. The net effect is + // that upon restart, or upon observing the persist state like this + // function, one can see non-sensical results like the since of A be in + // advance of B even when B depends on A! This can happen because the + // downgrade of B gets reordered and lost. Here is our best attempt at + // playing detective of what the controller meant to do by blindly + // assuming that the since of the remap shard is a suitable since + // frontier without consulting the since frontier of the outputs. One + // day we will enforce order to chaos and this comment will be deleted. + let remap_shard = ingestion_description + .ingestion_metadata + .remap_shard + .expect("ingestions must have a remap shard"); + let client = persist_clients + .open( + ingestion_description + .ingestion_metadata + .persist_location + .clone(), + ) + .await + .expect("error creating persist client"); + let read_handle = client + .open_leased_reader::( + remap_shard, + Arc::new(ingestion_description.desc.connection.timestamp_desc()), + Arc::new(UnitSchema), + Diagnostics { + shard_name: ingestion_description + .remap_collection_id + .to_string(), + handle_purpose: format!("resumption data for {}", id), + }, + false, + ) + .await + .unwrap(); + let as_of = read_handle.since().clone(); + mz_ore::task::spawn(move || "deferred_expire", async move { + tokio::time::sleep(std::time::Duration::from_secs(300)).await; + read_handle.expire().await; + }); + let seen_remap_shard = remap_shard.clone(); for (id, export) in ingestion_description.source_exports.iter() { // Explicit destructuring to force a compile error when the metadata change @@ -263,63 +309,11 @@ impl AsyncStorageWorker { resume_uppers.insert(*id, upper); write_handle.expire().await; - // TODO(petrosagg): The as_of of the ingestion should normally be based - // on the since frontiers of its outputs. Even though the storage - // controller makes sure to make downgrade decisions in an organized - // and ordered fashion, it then proceeds to persist them in an - // asynchronous and disorganized fashion to persist. The net effect is - // that upon restart, or upon observing the persist state like this - // function, one can see non-sensical results like the since of A be in - // advance of B even when B depends on A! This can happen because the - // downgrade of B gets reordered and lost. Here is our best attempt at - // playing detective of what the controller meant to do by blindly - // assuming that the since of the remap shard is a suitable since - // frontier without consulting the since frontier of the outputs. One - // day we will enforce order to chaos and this comment will be deleted. if let Some(remap_shard) = remap_shard { - match seen_remap_shard.as_ref() { - None => { - let read_handle = client - .open_leased_reader::( - *remap_shard, - Arc::new( - ingestion_description - .desc - .connection - .timestamp_desc(), - ), - Arc::new(UnitSchema), - Diagnostics { - shard_name: ingestion_description - .remap_collection_id - .to_string(), - handle_purpose: format!( - "resumption data for {}", - id - ), - }, - false, - ) - .await - .unwrap(); - as_of.clone_from(read_handle.since()); - mz_ore::task::spawn( - move || "deferred_expire", - async move { - tokio::time::sleep(std::time::Duration::from_secs( - 300, - )) - .await; - read_handle.expire().await; - }, - ); - seen_remap_shard = Some(remap_shard.clone()); - } - Some(shard) => assert_eq!( - shard, remap_shard, - "ingestion with multiple remap shards" - ), - } + assert_eq!( + seen_remap_shard, *remap_shard, + "ingestion with multiple remap shards" + ); } } From b84e2904f96eca59e9a72832eabfd498f607bac6 Mon Sep 17 00:00:00 2001 From: Joseph Koshakow Date: Wed, 8 Jan 2025 14:22:43 -0500 Subject: [PATCH 02/50] storage: Always initialize ingestion statistics Previously, collection statistics were only initialized for ingestion dataflow outputs. When the `force_source_table_syntax` flag is enabled, the ingestion collection is excluded from the ingestion dataflow outputs. As a result, statistics are never created for the ingestion collection. This causes later parts of the code to panic because it is assumed that all ingestion collections have statistics initialized. This commit fixes the issue by ensuring that statistics are always initialized for ingestion collections, even if it's not included in the dataflow outputs. Works towards resolving #MaterializeInc/database-issues/issues/8620 --- src/storage/src/internal_control.rs | 2 ++ src/storage/src/storage_state.rs | 34 ++++++++++++++++++- .../src/storage_state/async_storage_worker.rs | 30 ++++++++++++++-- 3 files changed, 62 insertions(+), 4 deletions(-) diff --git a/src/storage/src/internal_control.rs b/src/storage/src/internal_control.rs index 904cf68bd5ee1..1e28b0e96f309 100644 --- a/src/storage/src/internal_control.rs +++ b/src/storage/src/internal_control.rs @@ -81,6 +81,8 @@ pub enum InternalStorageCommand { /// A frontier in the source time domain with the property that all updates not beyond it /// have already been durably ingested. source_resume_uppers: BTreeMap>, + /// The upper of the ingestion collection's shard. + ingestion_upper: Antichain, }, /// Render a sink dataflow. RunSinkDataflow( diff --git a/src/storage/src/storage_state.rs b/src/storage/src/storage_state.rs index 19049058903a7..262e6b18c8f26 100644 --- a/src/storage/src/storage_state.rs +++ b/src/storage/src/storage_state.rs @@ -96,7 +96,8 @@ use mz_storage_types::configuration::StorageConfiguration; use mz_storage_types::connections::ConnectionContext; use mz_storage_types::controller::CollectionMetadata; use mz_storage_types::sinks::{MetadataFilled, StorageSinkDesc}; -use mz_storage_types::sources::IngestionDescription; +use mz_storage_types::sources::envelope::{KeyEnvelope, NoneEnvelope}; +use mz_storage_types::sources::{IngestionDescription, SourceEnvelope}; use mz_storage_types::AlterCompatible; use mz_timely_util::builder_async::PressOnDropButton; use mz_txn_wal::operator::TxnsContext; @@ -504,6 +505,7 @@ impl<'w, A: Allocate> Worker<'w, A> { as_of, resume_uppers, source_resume_uppers, + ingestion_upper, } => { // NOTE: If we want to share the load of async processing we // have to change `handle_storage_command` and change this @@ -520,6 +522,7 @@ impl<'w, A: Allocate> Worker<'w, A> { as_of, resume_uppers, source_resume_uppers, + ingestion_upper, }, ); } @@ -623,6 +626,7 @@ impl<'w, A: Allocate> Worker<'w, A> { as_of, resume_uppers, source_resume_uppers, + ingestion_upper, } => { info!( ?as_of, @@ -632,6 +636,7 @@ impl<'w, A: Allocate> Worker<'w, A> { self.timely_worker.peers(), ); + let mut ingestion_stats_initialized = false; for (export_id, export) in ingestion_description.source_exports.iter() { let resume_upper = resume_uppers[export_id].clone(); self.storage_state.aggregated_statistics.initialize_source( @@ -649,6 +654,33 @@ impl<'w, A: Allocate> Worker<'w, A> { ) }, ); + if *export_id == ingestion_id { + ingestion_stats_initialized = true; + } + } + // TODO(database-issues#8620): Unconditionally create ingestion statistics once + // sources no longer export to primary collections and only export to explicit + // SourceExports (tables). + if !ingestion_stats_initialized { + self.storage_state.aggregated_statistics.initialize_source( + ingestion_id, + ingestion_upper.clone(), + || { + SourceStatistics::new( + ingestion_id, + self.storage_state.timely_worker_index, + &self.storage_state.metrics.source_statistics, + ingestion_id, + &ingestion_description.ingestion_metadata.data_shard, + // TODO(jkosh44) Does this envelope make sense? + SourceEnvelope::None(NoneEnvelope { + key_envelope: KeyEnvelope::None, + key_arity: 0, + }), + ingestion_upper, + ) + }, + ); } for id in ingestion_description.collection_ids() { diff --git a/src/storage/src/storage_state/async_storage_worker.rs b/src/storage/src/storage_state/async_storage_worker.rs index ef6673b418a1f..e17e2b6c17d7e 100644 --- a/src/storage/src/storage_state/async_storage_worker.rs +++ b/src/storage/src/storage_state/async_storage_worker.rs @@ -72,6 +72,8 @@ pub enum AsyncStorageWorkerResponse { /// A frontier in the source time domain with the property that all updates not beyond it /// have already been durably ingested. source_resume_uppers: BTreeMap>, + /// The upper of the ingestion collection's shard. + ingestion_upper: Antichain, }, } @@ -245,7 +247,7 @@ impl AsyncStorageWorker { ) .await .expect("error creating persist client"); - let read_handle = client + let ingestion_read_handle = client .open_leased_reader::( remap_shard, Arc::new(ingestion_description.desc.connection.timestamp_desc()), @@ -260,13 +262,34 @@ impl AsyncStorageWorker { ) .await .unwrap(); - let as_of = read_handle.since().clone(); + let as_of = ingestion_read_handle.since().clone(); mz_ore::task::spawn(move || "deferred_expire", async move { tokio::time::sleep(std::time::Duration::from_secs(300)).await; - read_handle.expire().await; + ingestion_read_handle.expire().await; }); let seen_remap_shard = remap_shard.clone(); + let mut ingestion_write_handle = client + .open_writer::( + ingestion_description.ingestion_metadata.data_shard, + Arc::new( + ingestion_description + .ingestion_metadata + .relation_desc + .clone(), + ), + Arc::new(UnitSchema), + Diagnostics { + shard_name: id.to_string(), + handle_purpose: format!("resumption data {}", id), + }, + ) + .await + .unwrap(); + let ingestion_upper = + ingestion_write_handle.fetch_recent_upper().await.clone(); + ingestion_write_handle.expire().await; + for (id, export) in ingestion_description.source_exports.iter() { // Explicit destructuring to force a compile error when the metadata change let CollectionMetadata { @@ -385,6 +408,7 @@ impl AsyncStorageWorker { as_of, resume_uppers, source_resume_uppers, + ingestion_upper, }); if let Err(_err) = res { From 785d6a736e08d9fb403ef66976557ff4a7b3906e Mon Sep 17 00:00:00 2001 From: Jan Teske Date: Mon, 21 Oct 2024 15:49:23 +0200 Subject: [PATCH 03/50] More generic sender for `ReadHold` changes This commit changes `ReadHold` to contain sender closure, rather than an `UnboundedSender` directly, for transmitting read hold changes. This provides the abililty to construct `ReadHolds` that are connected to channels with an item type different from `(GlobalId, ChangeBatch)`, performing the necessary transformation inside the closure. --- src/adapter/tests/timestamp_selection.rs | 2 +- src/compute-client/src/as_of_selection.rs | 2 +- src/compute-client/src/controller.rs | 2 +- src/compute-client/src/controller/instance.rs | 4 +- src/storage-client/src/storage_collections.rs | 2 +- src/storage-types/src/read_holds.rs | 46 +++++++++++-------- 6 files changed, 33 insertions(+), 25 deletions(-) diff --git a/src/adapter/tests/timestamp_selection.rs b/src/adapter/tests/timestamp_selection.rs index 5a330407ab71a..91340cb6f34f0 100644 --- a/src/adapter/tests/timestamp_selection.rs +++ b/src/adapter/tests/timestamp_selection.rs @@ -119,7 +119,7 @@ impl TimestampProvider for Frontiers { let mock_read_hold = |id, frontier| { let (tx, _rx) = tokio::sync::mpsc::unbounded_channel(); - ReadHold::new(id, frontier, tx) + ReadHold::with_channel(id, frontier, tx) }; for (instance_id, ids) in id_bundle.compute_ids.iter() { diff --git a/src/compute-client/src/as_of_selection.rs b/src/compute-client/src/as_of_selection.rs index 707be1b0888ca..7ecf102be6b60 100644 --- a/src/compute-client/src/as_of_selection.rs +++ b/src/compute-client/src/as_of_selection.rs @@ -944,7 +944,7 @@ mod tests { .get(&id) .ok_or(ReadHoldError::CollectionMissing(id))?; let (tx, _rx) = tokio::sync::mpsc::unbounded_channel(); - holds.push(ReadHold::new(id, read.clone(), tx)); + holds.push(ReadHold::with_channel(id, read.clone(), tx)); } Ok(holds) } diff --git a/src/compute-client/src/controller.rs b/src/compute-client/src/controller.rs index cdccba750a0ab..045d057e1a1b5 100644 --- a/src/compute-client/src/controller.rs +++ b/src/compute-client/src/controller.rs @@ -1177,7 +1177,7 @@ impl InstanceState { since }); - let hold = ReadHold::new(id, since, self.read_holds_tx.clone()); + let hold = ReadHold::with_channel(id, since, self.read_holds_tx.clone()); Ok(hold) } diff --git a/src/compute-client/src/controller/instance.rs b/src/compute-client/src/controller/instance.rs index 000650a354f36..a4e154733bcda 100644 --- a/src/compute-client/src/controller/instance.rs +++ b/src/compute-client/src/controller/instance.rs @@ -2182,8 +2182,8 @@ impl CollectionState { // Initialize collection read holds. // Note that the implied read hold was already added to the `read_capabilities` when // `shared` was created, so we only need to add the warmup read hold here. - let implied_read_hold = ReadHold::new(collection_id, since.clone(), read_holds_tx.clone()); - let warmup_read_hold = ReadHold::new(collection_id, since.clone(), read_holds_tx); + let implied_read_hold = ReadHold::with_channel(collection_id, since.clone(), read_holds_tx.clone()); + let warmup_read_hold = ReadHold::with_channel(collection_id, since.clone(), read_holds_tx); let updates = warmup_read_hold.since().iter().map(|t| (t.clone(), 1)); shared.lock_read_capabilities(|c| { diff --git a/src/storage-client/src/storage_collections.rs b/src/storage-client/src/storage_collections.rs index 1ed5c91b1ac12..808d0ea8488c4 100644 --- a/src/storage-client/src/storage_collections.rs +++ b/src/storage-client/src/storage_collections.rs @@ -1996,7 +1996,7 @@ where let acquired_holds = advanced_holds .into_iter() - .map(|(id, since)| ReadHold::new(id, since, self.holds_tx.clone())) + .map(|(id, since)| ReadHold::with_channel(id, since, self.holds_tx.clone())) .collect_vec(); trace!(?desired_holds, ?acquired_holds, "acquire_read_holds"); diff --git a/src/storage-types/src/read_holds.rs b/src/storage-types/src/read_holds.rs index b2baa5c1c7e52..06dc31ecb4f45 100644 --- a/src/storage-types/src/read_holds.rs +++ b/src/storage-types/src/read_holds.rs @@ -8,13 +8,21 @@ // by the Apache License, Version 2.0. use std::fmt::Debug; +use std::sync::Arc; use mz_repr::GlobalId; use thiserror::Error; use timely::progress::{Antichain, ChangeBatch, Timestamp as TimelyTimestamp}; use timely::PartialOrder; +use tokio::sync::mpsc::error::SendError; use tokio::sync::mpsc::UnboundedSender; +pub type ChangeTx = Arc< + dyn Fn(GlobalId, ChangeBatch) -> Result<(), SendError<(GlobalId, ChangeBatch)>> + + Send + + Sync, +>; + /// Token that represents a hold on a collection. This prevents the since of the /// collection from progressing beyond the hold. In other words, it cannot /// become true that our hold is `less_than` the since. @@ -29,7 +37,7 @@ pub struct ReadHold { since: Antichain, /// For communicating changes to this read hold back to whoever issued it. - holds_tx: UnboundedSender<(GlobalId, ChangeBatch)>, + change_tx: ChangeTx, } impl Debug for ReadHold { @@ -55,18 +63,23 @@ pub enum ReadHoldDowngradeError { } impl ReadHold { - pub fn new( - id: GlobalId, - since: Antichain, - holds_tx: UnboundedSender<(GlobalId, ChangeBatch)>, - ) -> Self { + pub fn new(id: GlobalId, since: Antichain, change_tx: ChangeTx) -> Self { Self { id, since, - holds_tx, + change_tx, } } + pub fn with_channel( + id: GlobalId, + since: Antichain, + channel_tx: UnboundedSender<(GlobalId, ChangeBatch)>, + ) -> Self { + let tx = Arc::new(move |id, changes| channel_tx.send((id, changes))); + Self::new(id, since, tx) + } + /// Returns the [GlobalId] of the collection that this [ReadHold] is for. pub fn id(&self) -> GlobalId { self.id @@ -83,24 +96,19 @@ impl ReadHold { /// Merges `other` into `self`, keeping the overall read hold. /// /// # Panics + /// /// Panics when trying to merge a [ReadHold] for a different collection - /// (different [GlobalId]) or when trying to merge a [ReadHold] from a - /// different issuer. + /// (different [GlobalId]). pub fn merge_assign(&mut self, mut other: ReadHold) { assert_eq!( self.id, other.id, "can only merge ReadHolds for the same ID" ); - assert!( - self.holds_tx.same_channel(&other.holds_tx), - "can only merge ReadHolds that come from the same issuer" - ); let mut changes = ChangeBatch::new(); - changes.extend(self.since.iter().map(|t| (t.clone(), -1))); - changes.extend(other.since.iter().map(|t| (t.clone(), -1))); + // It's very important that we clear the since of other. Otherwise, it's // Drop impl would try and drop it again, by sending another ChangeBatch // on drop. @@ -113,7 +121,7 @@ impl ReadHold { // in one go. changes.extend(self.since.iter().map(|t| (t.clone(), 1))); - match self.holds_tx.send((self.id.clone(), changes)) { + match (self.change_tx)(self.id, changes) { Ok(_) => (), Err(e) => { panic!("cannot merge ReadHold: {}", e); @@ -143,7 +151,7 @@ impl ReadHold { if !changes.is_empty() { // If the other side already hung up, that's ok. - let _ = self.holds_tx.send((self.id.clone(), changes)); + let _ = (self.change_tx)(self.id, changes); } Ok(()) @@ -170,7 +178,7 @@ impl Clone for ReadHold { if !changes.is_empty() { // We do care about sending here. If the other end hung up we don't // really have a read hold anymore. - match self.holds_tx.send((self.id.clone(), changes)) { + match (self.change_tx)(self.id.clone(), changes) { Ok(_) => (), Err(e) => { panic!("cannot clone ReadHold: {}", e); @@ -181,7 +189,7 @@ impl Clone for ReadHold { Self { id: self.id.clone(), since: self.since.clone(), - holds_tx: self.holds_tx.clone(), + change_tx: Arc::clone(&self.change_tx), } } } From 92f56f36ae5f621264888621f2c7bcf871a8e582 Mon Sep 17 00:00:00 2001 From: Jan Teske Date: Mon, 21 Oct 2024 16:20:12 +0200 Subject: [PATCH 04/50] controller/compute: sequence external read hold changes Changes to external `ReadHold`s on compute collections are now sequenced through the `command_{tx,rx}` channels of the respective `Instance` controllers. This ensures that `Instance`s observe read hold changes always after they have observed the creation of the affected collection, which wasn't the case previously. As a side effect of this change, each external compute `ReadHold` now contains a copy of an instance `Client`. These copies keep the connected `Instance`s alive, even when the `ComputeController` has already dropped them. This seems fine in that arguably an instance shouldn't shut down if somebody still has read holds on its collections. But it is a change to the previous behavior. --- src/compute-client/src/controller.rs | 29 +++++++------------ src/compute-client/src/controller/instance.rs | 24 ++++++++++----- 2 files changed, 26 insertions(+), 27 deletions(-) diff --git a/src/compute-client/src/controller.rs b/src/compute-client/src/controller.rs index 045d057e1a1b5..059767bd25088 100644 --- a/src/compute-client/src/controller.rs +++ b/src/compute-client/src/controller.rs @@ -58,7 +58,7 @@ use mz_storage_types::read_policy::ReadPolicy; use mz_storage_types::time_dependence::{TimeDependence, TimeDependenceError}; use prometheus::proto::LabelPair; use serde::{Deserialize, Serialize}; -use timely::progress::{Antichain, ChangeBatch, Timestamp}; +use timely::progress::{Antichain, Timestamp}; use timely::PartialOrder; use tokio::sync::{mpsc, oneshot}; use tokio::time::{self, MissedTickBehavior}; @@ -539,8 +539,6 @@ where logs.push((log, id, shared)); } - let (read_holds_tx, read_holds_rx) = mpsc::unbounded_channel(); - let client = instance::Client::spawn( id, self.build_info, @@ -553,11 +551,9 @@ where Arc::clone(&self.dyncfg), self.response_tx.clone(), self.introspection_tx.clone(), - read_holds_tx.clone(), - read_holds_rx, ); - let instance = InstanceState::new(client, collections, read_holds_tx); + let instance = InstanceState::new(client, collections); self.instances.insert(id, instance); self.instance_workload_classes @@ -1103,24 +1099,14 @@ struct InstanceState { client: instance::Client, replicas: BTreeSet, collections: BTreeMap>, - /// Sender for updates to collection read holds. - /// - /// Copies of this sender are given to [`ReadHold`]s that are created in - /// [`InstanceState::acquire_read_hold`]. - read_holds_tx: mpsc::UnboundedSender<(GlobalId, ChangeBatch)>, } impl InstanceState { - fn new( - client: instance::Client, - collections: BTreeMap>, - read_holds_tx: mpsc::UnboundedSender<(GlobalId, ChangeBatch)>, - ) -> Self { + fn new(client: instance::Client, collections: BTreeMap>) -> Self { Self { client, replicas: Default::default(), collections, - read_holds_tx, } } @@ -1177,7 +1163,13 @@ impl InstanceState { since }); - let hold = ReadHold::with_channel(id, since, self.read_holds_tx.clone()); + let client = self.client.clone(); + let tx = Arc::new(move |id, change| { + client.send(Box::new(move |i| i.report_read_hold_change(id, change))); + Ok(()) + }); + + let hold = ReadHold::new(id, since, tx); Ok(hold) } @@ -1196,7 +1188,6 @@ impl InstanceState { client: _, replicas, collections, - read_holds_tx: _, } = self; let instance = self.call_sync(|i| i.dump()).await?; diff --git a/src/compute-client/src/controller/instance.rs b/src/compute-client/src/controller/instance.rs index a4e154733bcda..a3285f1f8a39b 100644 --- a/src/compute-client/src/controller/instance.rs +++ b/src/compute-client/src/controller/instance.rs @@ -129,7 +129,7 @@ impl From for ReadPolicyError { pub type Command = Box) + Send>; /// A client for an [`Instance`] task. -#[derive(Debug)] +#[derive(Debug, Clone)] pub(super) struct Client { /// A sender for commands for the instance. command_tx: mpsc::UnboundedSender>, @@ -158,8 +158,6 @@ where dyncfg: Arc, response_tx: mpsc::UnboundedSender>, introspection_tx: crossbeam_channel::Sender, - read_holds_tx: mpsc::UnboundedSender<(GlobalId, ChangeBatch)>, - read_holds_rx: mpsc::UnboundedReceiver<(GlobalId, ChangeBatch)>, ) -> Self { let (command_tx, command_rx) = mpsc::unbounded_channel(); @@ -177,8 +175,6 @@ where command_rx, response_tx, introspection_tx, - read_holds_tx, - read_holds_rx, ) .run(), ); @@ -729,6 +725,17 @@ impl Instance { .expect("Cannot error if target_replica_ids is None") } + /// Report an external read hold change. + /// + /// Changes to externally held read holds are sequenced through `command_rx`, to ensure that, + /// e.g., we don't receive changes for a collection before we have seen its creation command. + #[mz_ore::instrument(level = "debug")] + pub fn report_read_hold_change(&self, id: GlobalId, changes: ChangeBatch) { + self.read_holds_tx + .send((id, changes)) + .expect("rx is held by `self`"); + } + /// Clean up collection state that is not needed anymore. /// /// Three conditions need to be true before we can remove state for a collection: @@ -872,9 +879,9 @@ where command_rx: mpsc::UnboundedReceiver>, response_tx: mpsc::UnboundedSender>, introspection_tx: crossbeam_channel::Sender, - read_holds_tx: mpsc::UnboundedSender<(GlobalId, ChangeBatch)>, - read_holds_rx: mpsc::UnboundedReceiver<(GlobalId, ChangeBatch)>, ) -> Self { + let (read_holds_tx, read_holds_rx) = mpsc::unbounded_channel(); + let mut collections = BTreeMap::new(); let mut log_sources = BTreeMap::new(); for (log, id, shared) in arranged_logs { @@ -2182,7 +2189,8 @@ impl CollectionState { // Initialize collection read holds. // Note that the implied read hold was already added to the `read_capabilities` when // `shared` was created, so we only need to add the warmup read hold here. - let implied_read_hold = ReadHold::with_channel(collection_id, since.clone(), read_holds_tx.clone()); + let implied_read_hold = + ReadHold::with_channel(collection_id, since.clone(), read_holds_tx.clone()); let warmup_read_hold = ReadHold::with_channel(collection_id, since.clone(), read_holds_tx); let updates = warmup_read_hold.since().iter().map(|t| (t.clone(), 1)); From 062dcd83b67b57c5c964cceed151d0cd63e4c48a Mon Sep 17 00:00:00 2001 From: Jan Teske Date: Fri, 20 Dec 2024 13:03:12 +0100 Subject: [PATCH 05/50] Revert "controller/compute: defer application of early read hold changes" This reverts commit 6f2214e5026661e4b31673bbeabede216ad27f80. --- src/compute-client/src/controller/instance.rs | 27 +++---------------- 1 file changed, 4 insertions(+), 23 deletions(-) diff --git a/src/compute-client/src/controller/instance.rs b/src/compute-client/src/controller/instance.rs index a3285f1f8a39b..f00da7dec6dc0 100644 --- a/src/compute-client/src/controller/instance.rs +++ b/src/compute-client/src/controller/instance.rs @@ -277,12 +277,6 @@ pub(super) struct Instance { /// /// Received updates are applied by [`Instance::apply_read_hold_changes`]. read_holds_rx: mpsc::UnboundedReceiver<(GlobalId, ChangeBatch)>, - /// Stashed read hold changes. - /// - /// Used by [`Instance::apply_read_hold_changes`] to stash read hold changes that cannot be - /// applied immediately until they can be applied. - stashed_read_hold_changes: BTreeMap>, - /// A sender for responses from replicas. replica_tx: mz_ore::channel::InstrumentedUnboundedSender, IntCounter>, /// A receiver for responses from replicas. @@ -805,7 +799,6 @@ impl Instance { wallclock_lag_last_refresh, read_holds_tx: _, read_holds_rx: _, - stashed_read_hold_changes, replica_tx: _, replica_rx: _, } = self; @@ -840,10 +833,6 @@ impl Instance { .map(|(id, epoch)| (id.to_string(), epoch)) .collect(); let wallclock_lag_last_refresh = format!("{wallclock_lag_last_refresh:?}"); - let stashed_read_hold_changes: BTreeMap<_, _> = stashed_read_hold_changes - .iter() - .map(|(id, changes)| (id.to_string(), changes)) - .collect(); let map = serde_json::Map::from_iter([ field("initialized", initialized)?, @@ -856,7 +845,6 @@ impl Instance { field("envd_epoch", envd_epoch)?, field("replica_epochs", replica_epochs)?, field("wallclock_lag_last_refresh", wallclock_lag_last_refresh)?, - field("stashed_read_hold_changes", stashed_read_hold_changes)?, ]); Ok(serde_json::Value::Object(map)) } @@ -925,7 +913,6 @@ where wallclock_lag_last_refresh: Instant::now(), read_holds_tx, read_holds_rx, - stashed_read_hold_changes: Default::default(), replica_tx, replica_rx, } @@ -1670,14 +1657,13 @@ where /// Apply collection read hold changes pending in `read_holds_rx`. fn apply_read_hold_changes(&mut self) { let mut allowed_compaction = BTreeMap::new(); - let mut stashed_changes = std::mem::take(&mut self.stashed_read_hold_changes); // It's more efficient to apply updates for greater IDs before updates for smaller IDs, // since ID order usually matches dependency order and downgrading read holds on a // collection can cause downgrades on its dependencies. So instead of processing changes as // they come in, we batch them up as much as we can and process them in reverse ID order. let mut recv_batch = || { - let mut batch = std::mem::take(&mut stashed_changes); + let mut batch = BTreeMap::<_, ChangeBatch<_>>::new(); while let Ok((id, mut update)) = self.read_holds_rx.try_recv() { batch .entry(id) @@ -1692,14 +1678,9 @@ where while let Some(batch) = recv_batch() { for (id, mut update) in batch.into_iter().rev() { let Some(collection) = self.collections.get_mut(&id) else { - // The `ComputeController` provides a sync API for creating collections and - // taking out read holds on them, without waiting for the collection to be - // created in the `Instance`. Thus we might see read hold changes for - // collections that haven't been created yet. Stash them for later application. - self.stashed_read_hold_changes - .entry(id) - .and_modify(|e| e.extend(update.drain())) - .or_insert(update); + soft_panic_or_log!( + "read hold change for absent collection (id={id}, changes={update:?})" + ); continue; }; From bbe7a98da66e7df853ab0dbe17731fa27748d5d4 Mon Sep 17 00:00:00 2001 From: Jan Teske Date: Wed, 23 Oct 2024 17:43:49 +0200 Subject: [PATCH 06/50] compute: correctly propagate read holds in shutdown check Now that some of the read holds held by `Instance` are sequencing their changes through the command channel, it is no longer guaranteed that an `apply_read_hold_changes` call also applies cascading changes on dependencies. The previous `check_empty` check expected this to be the case, so it would incorrectly fail its asserts. This is fixed by having the check also drain and apply the contents of the command channel, until no new read capability changes are produced. To clarify the semantics, `check_empty` is also renamed to `shutdown` and explicitly terminates the execution of the `Instance` task. --- src/compute-client/src/controller.rs | 42 +++++++++++-------- src/compute-client/src/controller/instance.rs | 32 ++++++++++---- 2 files changed, 49 insertions(+), 25 deletions(-) diff --git a/src/compute-client/src/controller.rs b/src/compute-client/src/controller.rs index 059767bd25088..0408785ef1de8 100644 --- a/src/compute-client/src/controller.rs +++ b/src/compute-client/src/controller.rs @@ -58,8 +58,9 @@ use mz_storage_types::read_policy::ReadPolicy; use mz_storage_types::time_dependence::{TimeDependence, TimeDependenceError}; use prometheus::proto::LabelPair; use serde::{Deserialize, Serialize}; -use timely::progress::{Antichain, Timestamp}; +use timely::progress::{Antichain, ChangeBatch, Timestamp}; use timely::PartialOrder; +use tokio::sync::mpsc::error::SendError; use tokio::sync::{mpsc, oneshot}; use tokio::time::{self, MissedTickBehavior}; use tracing::debug_span; @@ -604,7 +605,7 @@ where /// Panics if the identified `instance` still has active replicas. pub fn drop_instance(&mut self, id: ComputeInstanceId) { if let Some(instance) = self.instances.remove(&id) { - instance.call(|i| i.check_empty()); + instance.call(|i| i.shutdown()); } self.instance_workload_classes @@ -1119,12 +1120,14 @@ impl InstanceState { F: FnOnce(&mut Instance) + Send + 'static, { let otel_ctx = OpenTelemetryContext::obtain(); - self.client.send(Box::new(move |instance| { - let _span = debug_span!("instance::call").entered(); - otel_ctx.attach_as_parent(); - - f(instance) - })); + self.client + .send(Box::new(move |instance| { + let _span = debug_span!("instance::call").entered(); + otel_ctx.attach_as_parent(); + + f(instance) + })) + .expect("instance not dropped"); } pub async fn call_sync(&self, f: F) -> R @@ -1134,13 +1137,15 @@ impl InstanceState { { let (tx, rx) = oneshot::channel(); let otel_ctx = OpenTelemetryContext::obtain(); - self.client.send(Box::new(move |instance| { - let _span = debug_span!("instance::call_sync").entered(); - otel_ctx.attach_as_parent(); + self.client + .send(Box::new(move |instance| { + let _span = debug_span!("instance::call_sync").entered(); + otel_ctx.attach_as_parent(); - let result = f(instance); - let _ = tx.send(result); - })); + let result = f(instance); + let _ = tx.send(result); + })) + .expect("instance not dropped"); rx.await.expect("instance not dropped") } @@ -1164,9 +1169,12 @@ impl InstanceState { }); let client = self.client.clone(); - let tx = Arc::new(move |id, change| { - client.send(Box::new(move |i| i.report_read_hold_change(id, change))); - Ok(()) + let tx = Arc::new(move |id, change: ChangeBatch<_>| { + let cmd: instance::Command<_> = { + let change = change.clone(); + Box::new(move |i| i.report_read_hold_change(id, change)) + }; + client.send(cmd).map_err(|_| SendError((id, change))) }); let hold = ReadHold::new(id, since, tx); diff --git a/src/compute-client/src/controller/instance.rs b/src/compute-client/src/controller/instance.rs index f00da7dec6dc0..866ffed48dff7 100644 --- a/src/compute-client/src/controller/instance.rs +++ b/src/compute-client/src/controller/instance.rs @@ -45,6 +45,7 @@ use thiserror::Error; use timely::progress::frontier::MutableAntichain; use timely::progress::{Antichain, ChangeBatch, Timestamp}; use timely::PartialOrder; +use tokio::sync::mpsc::error::SendError; use tokio::sync::{mpsc, oneshot}; use uuid::Uuid; @@ -136,8 +137,8 @@ pub(super) struct Client { } impl Client { - pub fn send(&self, command: Command) { - self.command_tx.send(command).expect("instance not dropped"); + pub fn send(&self, command: Command) -> Result<(), SendError>> { + self.command_tx.send(command) } } @@ -975,20 +976,35 @@ where } } - /// Check that the current instance is empty. + /// Shut down this instance. /// - /// This method exists to help us find bugs where the client drops a compute instance that - /// still has replicas or collections installed, and later assumes that said - /// replicas/collections still exists. + /// This method runs various assertions ensuring the instance state is empty. It exists to help + /// us find bugs where the client drops a compute instance that still has replicas or + /// collections installed, and later assumes that said replicas/collections still exists. /// /// # Panics /// /// Panics if the compute instance still has active replicas. /// Panics if the compute instance still has collections installed. #[mz_ore::instrument(level = "debug")] - pub fn check_empty(&mut self) { + pub fn shutdown(&mut self) { + // Taking the `command_rx` ensures that the [`Instance::run`] loop terminates. + let (_tx, rx) = mpsc::unbounded_channel(); + let mut command_rx = std::mem::replace(&mut self.command_rx, rx); + + // Apply all outstanding read hold changes. This might cause read hold downgrades to be + // added to `command_tx`, so we need to apply those in a loop. + while !self.read_holds_rx.is_empty() { + self.apply_read_hold_changes(); + + // TODO(teskje): Make `Command` an enum and assert that all received commands are read + // hold downgrades. + while let Ok(cmd) = command_rx.try_recv() { + cmd(self); + } + } + // Collections might have been dropped but not cleaned up yet. - self.apply_read_hold_changes(); self.cleanup_collections(); let stray_replicas: Vec<_> = self.replicas.keys().collect(); From 84cdf01e35a1018f4e535c4da67251436c0e825b Mon Sep 17 00:00:00 2001 From: Jan Teske Date: Fri, 20 Dec 2024 13:31:07 +0100 Subject: [PATCH 07/50] controller/compute: send all read hold changes through command_tx This commit changes the compute controller to also sequence internal read hold changes through `command_tx`. This simplifies things as now all read hold changes flow through the same channels and we don't need to worry about having two kinds of `ChangeTx` that secretly point to different destinations. The commit removes the separate `read_holds_{tx,rx}` channel in the compute controller and moves to applying received read hold changes directly, rather than batching them. This removes a bit of complexity but also imposes a risk of a performance regression. My hope is that the performance impact will be negligible. --- Cargo.lock | 1 + src/compute-client/Cargo.toml | 1 + src/compute-client/src/controller.rs | 18 +- src/compute-client/src/controller/instance.rs | 205 ++++++++---------- 4 files changed, 97 insertions(+), 128 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index cc34653e668f3..714793cfea1cb 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4903,6 +4903,7 @@ dependencies = [ "bytesize", "chrono", "crossbeam-channel", + "derivative", "differential-dataflow", "futures", "http 1.1.0", diff --git a/src/compute-client/Cargo.toml b/src/compute-client/Cargo.toml index 2399073e4759a..f43a01736419e 100644 --- a/src/compute-client/Cargo.toml +++ b/src/compute-client/Cargo.toml @@ -17,6 +17,7 @@ bytes = "1.3.0" bytesize = "1.1.0" chrono = { version = "0.4.35", default-features = false, features = ["std"] } crossbeam-channel = "0.5.8" +derivative = "2.2.0" differential-dataflow = "0.13.2" futures = "0.3.25" http = "1.1.0" diff --git a/src/compute-client/src/controller.rs b/src/compute-client/src/controller.rs index 0408785ef1de8..9b4ba1d087681 100644 --- a/src/compute-client/src/controller.rs +++ b/src/compute-client/src/controller.rs @@ -58,9 +58,8 @@ use mz_storage_types::read_policy::ReadPolicy; use mz_storage_types::time_dependence::{TimeDependence, TimeDependenceError}; use prometheus::proto::LabelPair; use serde::{Deserialize, Serialize}; -use timely::progress::{Antichain, ChangeBatch, Timestamp}; +use timely::progress::{Antichain, Timestamp}; use timely::PartialOrder; -use tokio::sync::mpsc::error::SendError; use tokio::sync::{mpsc, oneshot}; use tokio::time::{self, MissedTickBehavior}; use tracing::debug_span; @@ -1115,7 +1114,7 @@ impl InstanceState { self.collections.get(&id).ok_or(CollectionMissing(id)) } - pub fn call(&self, f: F) + fn call(&self, f: F) where F: FnOnce(&mut Instance) + Send + 'static, { @@ -1130,7 +1129,7 @@ impl InstanceState { .expect("instance not dropped"); } - pub async fn call_sync(&self, f: F) -> R + async fn call_sync(&self, f: F) -> R where F: FnOnce(&mut Instance) -> R + Send + 'static, R: Send + 'static, @@ -1168,16 +1167,7 @@ impl InstanceState { since }); - let client = self.client.clone(); - let tx = Arc::new(move |id, change: ChangeBatch<_>| { - let cmd: instance::Command<_> = { - let change = change.clone(); - Box::new(move |i| i.report_read_hold_change(id, change)) - }; - client.send(cmd).map_err(|_| SendError((id, change))) - }); - - let hold = ReadHold::new(id, since, tx); + let hold = ReadHold::new(id, since, self.client.read_hold_tx()); Ok(hold) } diff --git a/src/compute-client/src/controller/instance.rs b/src/compute-client/src/controller/instance.rs index 866ffed48dff7..baff2738661eb 100644 --- a/src/compute-client/src/controller/instance.rs +++ b/src/compute-client/src/controller/instance.rs @@ -38,7 +38,7 @@ use mz_repr::adt::interval::Interval; use mz_repr::refresh_schedule::RefreshSchedule; use mz_repr::{Datum, Diff, GlobalId, Row}; use mz_storage_client::controller::IntrospectionType; -use mz_storage_types::read_holds::ReadHold; +use mz_storage_types::read_holds::{self, ReadHold}; use mz_storage_types::read_policy::ReadPolicy; use serde::Serialize; use thiserror::Error; @@ -130,16 +130,24 @@ impl From for ReadPolicyError { pub type Command = Box) + Send>; /// A client for an [`Instance`] task. -#[derive(Debug, Clone)] +#[derive(Clone, derivative::Derivative)] +#[derivative(Debug)] pub(super) struct Client { /// A sender for commands for the instance. command_tx: mpsc::UnboundedSender>, + /// A sender for read hold changes for collections installed on the instance. + #[derivative(Debug = "ignore")] + read_hold_tx: read_holds::ChangeTx, } impl Client { pub fn send(&self, command: Command) -> Result<(), SendError>> { self.command_tx.send(command) } + + pub fn read_hold_tx(&self) -> read_holds::ChangeTx { + Arc::clone(&self.read_hold_tx) + } } impl Client @@ -162,6 +170,17 @@ where ) -> Self { let (command_tx, command_rx) = mpsc::unbounded_channel(); + let read_hold_tx: read_holds::ChangeTx<_> = { + let command_tx = command_tx.clone(); + Arc::new(move |id, change: ChangeBatch<_>| { + let cmd: Command<_> = { + let change = change.clone(); + Box::new(move |i| i.apply_read_hold_change(id, change.clone())) + }; + command_tx.send(cmd).map_err(|_| SendError((id, change))) + }) + }; + mz_ore::task::spawn( || format!("compute-instance-{id}"), Instance::new( @@ -175,12 +194,16 @@ where dyncfg, command_rx, response_tx, + Arc::clone(&read_hold_tx), introspection_tx, ) .run(), ); - Self { command_tx } + Self { + command_tx, + read_hold_tx, + } } } @@ -273,11 +296,7 @@ pub(super) struct Instance { /// /// Copies of this sender are given to [`ReadHold`]s that are created in /// [`CollectionState::new`]. - read_holds_tx: mpsc::UnboundedSender<(GlobalId, ChangeBatch)>, - /// Receiver for updates to collection read holds. - /// - /// Received updates are applied by [`Instance::apply_read_hold_changes`]. - read_holds_rx: mpsc::UnboundedReceiver<(GlobalId, ChangeBatch)>, + read_hold_tx: read_holds::ChangeTx, /// A sender for responses from replicas. replica_tx: mz_ore::channel::InstrumentedUnboundedSender, IntCounter>, /// A receiver for responses from replicas. @@ -355,7 +374,7 @@ impl Instance { shared, storage_dependencies, compute_dependencies, - self.read_holds_tx.clone(), + Arc::clone(&self.read_hold_tx), introspection, ); // If the collection is write-only, clear its read policy to reflect that. @@ -720,17 +739,6 @@ impl Instance { .expect("Cannot error if target_replica_ids is None") } - /// Report an external read hold change. - /// - /// Changes to externally held read holds are sequenced through `command_rx`, to ensure that, - /// e.g., we don't receive changes for a collection before we have seen its creation command. - #[mz_ore::instrument(level = "debug")] - pub fn report_read_hold_change(&self, id: GlobalId, changes: ChangeBatch) { - self.read_holds_tx - .send((id, changes)) - .expect("rx is held by `self`"); - } - /// Clean up collection state that is not needed anymore. /// /// Three conditions need to be true before we can remove state for a collection: @@ -798,8 +806,7 @@ impl Instance { now: _, wallclock_lag: _, wallclock_lag_last_refresh, - read_holds_tx: _, - read_holds_rx: _, + read_hold_tx: _, replica_tx: _, replica_rx: _, } = self; @@ -867,17 +874,16 @@ where dyncfg: Arc, command_rx: mpsc::UnboundedReceiver>, response_tx: mpsc::UnboundedSender>, + read_hold_tx: read_holds::ChangeTx, introspection_tx: crossbeam_channel::Sender, ) -> Self { - let (read_holds_tx, read_holds_rx) = mpsc::unbounded_channel(); - let mut collections = BTreeMap::new(); let mut log_sources = BTreeMap::new(); for (log, id, shared) in arranged_logs { let collection = CollectionState::new_log_collection( id, shared, - read_holds_tx.clone(), + Arc::clone(&read_hold_tx), introspection_tx.clone(), ); collections.insert(id, collection); @@ -912,8 +918,7 @@ where now, wallclock_lag, wallclock_lag_last_refresh: Instant::now(), - read_holds_tx, - read_holds_rx, + read_hold_tx, replica_tx, replica_rx, } @@ -994,14 +999,11 @@ where // Apply all outstanding read hold changes. This might cause read hold downgrades to be // added to `command_tx`, so we need to apply those in a loop. - while !self.read_holds_rx.is_empty() { - self.apply_read_hold_changes(); - - // TODO(teskje): Make `Command` an enum and assert that all received commands are read - // hold downgrades. - while let Ok(cmd) = command_rx.try_recv() { - cmd(self); - } + // + // TODO(teskje): Make `Command` an enum and assert that all received commands are read + // hold downgrades. + while let Ok(cmd) = command_rx.try_recv() { + cmd(self); } // Collections might have been dropped but not cleaned up yet. @@ -1670,86 +1672,62 @@ where }); } - /// Apply collection read hold changes pending in `read_holds_rx`. - fn apply_read_hold_changes(&mut self) { - let mut allowed_compaction = BTreeMap::new(); - - // It's more efficient to apply updates for greater IDs before updates for smaller IDs, - // since ID order usually matches dependency order and downgrading read holds on a - // collection can cause downgrades on its dependencies. So instead of processing changes as - // they come in, we batch them up as much as we can and process them in reverse ID order. - let mut recv_batch = || { - let mut batch = BTreeMap::<_, ChangeBatch<_>>::new(); - while let Ok((id, mut update)) = self.read_holds_rx.try_recv() { - batch - .entry(id) - .and_modify(|e| e.extend(update.drain())) - .or_insert(update); - } - - let has_updates = !batch.is_empty(); - has_updates.then_some(batch) + /// Apply a collection read hold change. + fn apply_read_hold_change(&mut self, id: GlobalId, mut update: ChangeBatch) { + let Some(collection) = self.collections.get_mut(&id) else { + soft_panic_or_log!( + "read hold change for absent collection (id={id}, changes={update:?})" + ); + return; }; - while let Some(batch) = recv_batch() { - for (id, mut update) in batch.into_iter().rev() { - let Some(collection) = self.collections.get_mut(&id) else { - soft_panic_or_log!( - "read hold change for absent collection (id={id}, changes={update:?})" - ); - continue; - }; - - let new_since = collection.shared.lock_read_capabilities(|caps| { - // Sanity check to prevent corrupted `read_capabilities`, which can cause hard-to-debug - // issues (usually stuck read frontiers). - let read_frontier = caps.frontier(); - for (time, diff) in update.iter() { - let count = caps.count_for(time) + diff; - assert!( - count >= 0, - "invalid read capabilities update: negative capability \ - (id={id:?}, read_capabilities={caps:?}, update={update:?})", - ); - assert!( - count == 0 || read_frontier.less_equal(time), - "invalid read capabilities update: frontier regression \ - (id={id:?}, read_capabilities={caps:?}, update={update:?})", - ); - } - - // Apply read capability updates and learn about resulting changes to the read - // frontier. - let changes = caps.update_iter(update.drain()); + let new_since = collection.shared.lock_read_capabilities(|caps| { + // Sanity check to prevent corrupted `read_capabilities`, which can cause hard-to-debug + // issues (usually stuck read frontiers). + let read_frontier = caps.frontier(); + for (time, diff) in update.iter() { + let count = caps.count_for(time) + diff; + assert!( + count >= 0, + "invalid read capabilities update: negative capability \ + (id={id:?}, read_capabilities={caps:?}, update={update:?})", + ); + assert!( + count == 0 || read_frontier.less_equal(time), + "invalid read capabilities update: frontier regression \ + (id={id:?}, read_capabilities={caps:?}, update={update:?})", + ); + } - let changed = changes.count() > 0; - changed.then(|| caps.frontier().to_owned()) - }); + // Apply read capability updates and learn about resulting changes to the read + // frontier. + let changes = caps.update_iter(update.drain()); - let Some(new_since) = new_since else { - continue; // read frontier did not change - }; + let changed = changes.count() > 0; + changed.then(|| caps.frontier().to_owned()) + }); - // Propagate read frontier update to dependencies. - for read_hold in collection.compute_dependencies.values_mut() { - read_hold - .try_downgrade(new_since.clone()) - .expect("frontiers don't regress"); - } - for read_hold in collection.storage_dependencies.values_mut() { - read_hold - .try_downgrade(new_since.clone()) - .expect("frontiers don't regress"); - } + let Some(new_since) = new_since else { + return; // read frontier did not change + }; - allowed_compaction.insert(id, new_since); - } + // Propagate read frontier update to dependencies. + for read_hold in collection.compute_dependencies.values_mut() { + read_hold + .try_downgrade(new_since.clone()) + .expect("frontiers don't regress"); } - - // Produce `AllowCompaction` commands. - for (id, frontier) in allowed_compaction { - self.send(ComputeCommand::AllowCompaction { id, frontier }); + for read_hold in collection.storage_dependencies.values_mut() { + read_hold + .try_downgrade(new_since.clone()) + .expect("frontiers don't regress"); } + + // Produce `AllowCompaction` command. + self.send(ComputeCommand::AllowCompaction { + id, + frontier: new_since, + }); } /// Fulfills a registered peek and cleans up associated state. @@ -2098,7 +2076,6 @@ where pub fn maintain(&mut self) { self.rehydrate_failed_replicas(); self.downgrade_warmup_capabilities(); - self.apply_read_hold_changes(); self.schedule_collections(); self.cleanup_collections(); self.update_frontier_introspection(); @@ -2171,7 +2148,7 @@ impl CollectionState { shared: SharedCollectionState, storage_dependencies: BTreeMap>, compute_dependencies: BTreeMap>, - read_holds_tx: mpsc::UnboundedSender<(GlobalId, ChangeBatch)>, + read_hold_tx: read_holds::ChangeTx, introspection: CollectionIntrospection, ) -> Self { // A collection is not readable before the `as_of`. @@ -2187,8 +2164,8 @@ impl CollectionState { // Note that the implied read hold was already added to the `read_capabilities` when // `shared` was created, so we only need to add the warmup read hold here. let implied_read_hold = - ReadHold::with_channel(collection_id, since.clone(), read_holds_tx.clone()); - let warmup_read_hold = ReadHold::with_channel(collection_id, since.clone(), read_holds_tx); + ReadHold::new(collection_id, since.clone(), Arc::clone(&read_hold_tx)); + let warmup_read_hold = ReadHold::new(collection_id, since.clone(), read_hold_tx); let updates = warmup_read_hold.since().iter().map(|t| (t.clone(), 1)); shared.lock_read_capabilities(|c| { @@ -2213,7 +2190,7 @@ impl CollectionState { fn new_log_collection( id: GlobalId, shared: SharedCollectionState, - read_holds_tx: mpsc::UnboundedSender<(GlobalId, ChangeBatch)>, + read_hold_tx: read_holds::ChangeTx, introspection_tx: crossbeam_channel::Sender, ) -> Self { let since = Antichain::from_elem(T::minimum()); @@ -2225,7 +2202,7 @@ impl CollectionState { shared, Default::default(), Default::default(), - read_holds_tx, + read_hold_tx, introspection, ); state.log_collection = true; @@ -2277,7 +2254,7 @@ pub(super) struct SharedCollectionState { /// This accumulation contains the capabilities held by all [`ReadHold`]s given out for the /// collection, including `implied_read_hold` and `warmup_read_hold`. /// - /// NOTE: This field may only be modified by [`Instance::apply_read_hold_changes`] and + /// NOTE: This field may only be modified by [`Instance::apply_read_hold_change`] and /// `ComputeController::acquire_read_hold`. Nobody else should modify read capabilities /// directly. Instead, collection users should manage read holds through [`ReadHold`] objects /// acquired through `ComputeController::acquire_read_hold`. From ca7c42cea255e6dc0d4cc115de83cc836a10c1fa Mon Sep 17 00:00:00 2001 From: Jan Teske Date: Fri, 20 Dec 2024 18:21:00 +0100 Subject: [PATCH 08/50] envd: make tests robust against slower read hold downgrade The changes made in the previous commits make it so the compute controller is slower at applying read hold downgrades. In tests that spawn two envd instances in sequence that can lead to issues because the first instance might still try to apply read hold downgrades while the second instance is already running, causing `PersistEpoch` conflicts. This race condition existed before but apparently wasn't hit because read hold downgrades by the first instance didn't get delayed long enough. The fix chosen here is to convert the tests that want to run two envd instances in sequence from using `TestHarness::start` to using `TestHarness::start_blocking`. The latter runs the `TestServer` in a separate tokio runtime that gets dropped when the `TestServer` is dropped, ensuring that no controller tasks can still be alive when the second envd instance is started. --- src/environmentd/tests/server.rs | 47 ++++++++++++++------------------ 1 file changed, 21 insertions(+), 26 deletions(-) diff --git a/src/environmentd/tests/server.rs b/src/environmentd/tests/server.rs index 30a67ff9df7f3..992dffe3b1f15 100644 --- a/src/environmentd/tests/server.rs +++ b/src/environmentd/tests/server.rs @@ -2631,9 +2631,9 @@ fn test_internal_ws_auth() { } } -#[mz_ore::test(tokio::test(flavor = "multi_thread", worker_threads = 1))] +#[mz_ore::test] #[cfg_attr(miri, ignore)] // too slow -async fn test_leader_promotion_always_using_deploy_generation() { +fn test_leader_promotion_always_using_deploy_generation() { let tmpdir = TempDir::new().unwrap(); let harness = test_util::TestHarness::default() .unsafe_mode() @@ -2641,37 +2641,37 @@ async fn test_leader_promotion_always_using_deploy_generation() { .with_deploy_generation(2); { // propose a deploy generation for the first time - let server = harness.clone().start().await; - let client = server.connect().await.unwrap(); - client.simple_query("SELECT 1").await.unwrap(); + let server = harness.clone().start_blocking(); + let mut client = server.connect(postgres::NoTls).unwrap(); + client.simple_query("SELECT 1").unwrap(); } { // keep it the same, no need to promote the leader - let server = harness.start().await; - let client = server.connect().await.unwrap(); - client.simple_query("SELECT 1").await.unwrap(); + let server = harness.start_blocking(); + let mut client = server.connect(postgres::NoTls).unwrap(); + client.simple_query("SELECT 1").unwrap(); - let http_client = reqwest::Client::new(); + let http_client = reqwest::blocking::Client::new(); // check that we're the leader and promotion doesn't do anything let status_http_url = Url::parse(&format!( "http://{}/api/leader/status", - server.inner.internal_http_local_addr() + server.inner().internal_http_local_addr() )) .unwrap(); - let res = http_client.get(status_http_url).send().await.unwrap(); + let res = http_client.get(status_http_url).send().unwrap(); assert_eq!(res.status(), StatusCode::OK); - let response = res.text().await.unwrap(); + let response = res.text().unwrap(); assert_eq!(response, r#"{"status":"IsLeader"}"#); let promote_http_url = Url::parse(&format!( "http://{}/api/leader/promote", - server.inner.internal_http_local_addr() + server.inner().internal_http_local_addr() )) .unwrap(); - let res = http_client.post(promote_http_url).send().await.unwrap(); + let res = http_client.post(promote_http_url).send().unwrap(); assert_eq!(res.status(), StatusCode::OK); - let response = res.text().await.unwrap(); + let response = res.text().unwrap(); assert_eq!(response, r#"{"result":"Success"}"#); } } @@ -4222,44 +4222,39 @@ async fn test_startup_cluster_notice() { "###); } -#[mz_ore::test(tokio::test(flavor = "multi_thread", worker_threads = 1))] +#[mz_ore::test] #[cfg_attr(miri, ignore)] // too slow -async fn test_durable_oids() { +fn test_durable_oids() { let data_dir = tempfile::tempdir().unwrap(); let harness = test_util::TestHarness::default().data_directory(data_dir.path()); let table_oid: u32 = { - let server = harness.clone().start().await; - let client = server.connect().await.unwrap(); + let server = harness.clone().start_blocking(); + let mut client = server.connect(postgres::NoTls).unwrap(); client .execute("CREATE TABLE t (a INT);", &[]) - .await .expect("failed to create table"); client .query_one("SELECT oid FROM mz_tables WHERE name = 't'", &[]) - .await .expect("failed to select") .get(0) }; { - let server = harness.clone().start().await; - let client = server.connect().await.unwrap(); + let server = harness.clone().start_blocking(); + let mut client = server.connect(postgres::NoTls).unwrap(); let restarted_table_oid: u32 = client .query_one("SELECT oid FROM mz_tables WHERE name = 't'", &[]) - .await .expect("failed to select") .get(0); assert_eq!(table_oid, restarted_table_oid); client .execute("CREATE VIEW v AS SELECT 1;", &[]) - .await .expect("failed to create table"); let view_oid: u32 = client .query_one("SELECT oid FROM mz_views WHERE name = 'v'", &[]) - .await .expect("failed to select") .get(0); assert_ne!(table_oid, view_oid); From 3dedd375d85eff2d02775daed28348a11a979ba7 Mon Sep 17 00:00:00 2001 From: Jan Teske Date: Thu, 19 Dec 2024 19:20:46 +0100 Subject: [PATCH 09/50] kafka-source: detect topic deletion in metadata fetcher This commit moves detection of topic recreation from the reader operator into the metadata fetcher. This is done to make the metadata fetcher aware of the unrecoverable error so it can seal the collection by emitting a probe with an empty upstream frontier. Prior to this change, the collection wouldn't be sealed under the reclock-to-latest scheme, and a replica restart could cause it to receive new, incorrect updates. Apart from fixing the sealing bug, this commit makes two functional changes: * It adds detection of topic deletion. That was easy to add and not having it seemed like an oversight. * It omits the "high watermark regressed"/"partition count regressed" detail in the "topic recreated" error. It didn't feel like a useful piece of information and omitting it simplifies the code a lot. --- src/storage/src/source/kafka.rs | 239 +++++++++--------- .../kafka-recreate-topic.td | 160 +++++------- test/testdrive/kafka-recreate-topic.td | 173 +++++-------- 3 files changed, 246 insertions(+), 326 deletions(-) diff --git a/src/storage/src/source/kafka.rs b/src/storage/src/source/kafka.rs index 97b49e51e5d47..6dfb33138969b 100644 --- a/src/storage/src/source/kafka.rs +++ b/src/storage/src/source/kafka.rs @@ -15,12 +15,14 @@ use std::sync::{Arc, Mutex}; use std::thread; use std::time::Duration; -use anyhow::bail; +use anyhow::anyhow; use chrono::{DateTime, NaiveDateTime}; use differential_dataflow::{AsCollection, Hashable}; use futures::StreamExt; use maplit::btreemap; -use mz_kafka_util::client::{get_partitions, MzClientContext, PartitionId, TunnelingClientContext}; +use mz_kafka_util::client::{ + get_partitions, GetPartitionsError, MzClientContext, PartitionId, TunnelingClientContext, +}; use mz_ore::assert_none; use mz_ore::cast::CastFrom; use mz_ore::error::ErrorExt; @@ -215,7 +217,7 @@ fn render_reader>( connection: KafkaSourceConnection, config: RawSourceCreationConfig, resume_uppers: impl futures::Stream> + 'static, - metadata_stream: Stream, + metadata_stream: Stream, start_signal: impl std::future::Future + 'static, ) -> ( StackedCollection)>, @@ -517,7 +519,6 @@ fn render_reader>( let mut prev_offset_known = None; let mut prev_offset_committed = None; - let mut prev_pid_info: Option> = None; let mut metadata_update: Option = None; let mut snapshot_total = None; @@ -539,7 +540,10 @@ fn render_reader>( updates.append(&mut data); } } - metadata_update = updates.into_iter().max_by_key(|u| u.timestamp); + metadata_update = updates + .into_iter() + .max_by_key(|(ts, _)| *ts) + .map(|(_, update)| update); } // This future is not cancel safe but we are only passing a reference to it in @@ -548,8 +552,8 @@ fn render_reader>( _ = resume_uppers_process_loop.as_mut() => {}, } - match metadata_update.take().map(|m| m.info) { - Some(Ok(partitions)) => { + match metadata_update.take() { + Some(MetadataUpdate::Partitions(partitions)) => { let max_pid = partitions.keys().last().cloned(); let lower = max_pid .map(RangeBound::after) @@ -560,66 +564,6 @@ fn render_reader>( MzOffset::from(0), ); - // Topics are identified by name but it's possible that a user recreates a - // topic with the same name but different configuration. Ideally we'd want to - // catch all of these cases and immediately error out the source, since the - // data is effectively gone. Unfortunately this is not possible without - // something like KIP-516 so we're left with heuristics. - // - // The first heuristic is whether the reported number of partitions went down - if !PartialOrder::less_equal(data_cap.time(), &future_ts) { - let prev_pid_count = prev_pid_info.map(|info| info.len()).unwrap_or(0); - let pid_count = partitions.len(); - let err = DataflowError::SourceError(Box::new(SourceError { - error: SourceErrorDetails::Other( - format!( - "topic was recreated: partition count regressed from \ - {prev_pid_count} to {pid_count}" - ) - .into(), - ), - })); - let time = data_cap.time().clone(); - let err = Err(err); - for (output, err) in - outputs.iter().map(|o| o.output_index).repeat_clone(err) - { - data_output - .give_fueled(&data_cap, ((output, err), time, 1)) - .await; - } - return; - } - - // The second heuristic is whether the high watermark regressed - if let Some(prev_pid_info) = prev_pid_info { - for (pid, prev_high_watermark) in prev_pid_info { - let high_watermark = partitions[&pid]; - if !(prev_high_watermark <= high_watermark) { - let err = DataflowError::SourceError(Box::new(SourceError { - error: SourceErrorDetails::Other( - format!( - "topic was recreated: high watermark of \ - partition {pid} regressed from {} to {}", - prev_high_watermark, high_watermark - ) - .into(), - ), - })); - let time = data_cap.time().clone(); - let err = Err(err); - for (output, err) in - outputs.iter().map(|o| o.output_index).repeat_clone(err) - { - data_output - .give_fueled(&data_cap, ((output, err), time, 1)) - .await; - } - return; - } - } - } - let mut upstream_stat = 0; for (&pid, &high_watermark) in &partitions { if responsible_for_pid(&config, pid) { @@ -688,9 +632,8 @@ fn render_reader>( progress_statistics.offset_known = Some(upstream_stat); data_cap.downgrade(&future_ts); progress_cap.downgrade(&future_ts); - prev_pid_info = Some(partitions); } - Some(Err(status)) => { + Some(MetadataUpdate::TransientError(status)) => { if let Some(update) = status.kafka { for (output, update) in outputs.iter().repeat_clone(update) { health_output.give( @@ -716,6 +659,19 @@ fn render_reader>( } } } + Some(MetadataUpdate::DefiniteError(error)) => { + let error = Err(error.into()); + let time = data_cap.time().clone(); + for (output, error) in + outputs.iter().map(|o| o.output_index).repeat_clone(error) + { + data_output + .give_fueled(&data_cap, ((output, error), time, 1)) + .await; + } + + return; + } None => {} } @@ -1460,7 +1416,7 @@ fn fetch_partition_info( consumer: &BaseConsumer, topic: &str, fetch_timeout: Duration, -) -> Result, anyhow::Error> { +) -> Result, GetPartitionsError> { let pids = get_partitions(consumer.client(), topic, fetch_timeout)?; let mut offset_requests = TopicPartitionList::with_capacity(pids.len()); @@ -1474,7 +1430,7 @@ fn fetch_partition_info( for entry in offset_responses.elements() { let offset = match entry.offset() { Offset::Offset(offset) => offset, - offset => bail!("unexpected high watermark offset: {offset:?}"), + offset => Err(anyhow!("unexpected high watermark offset: {offset:?}"))?, }; let pid = entry.partition(); @@ -1486,39 +1442,45 @@ fn fetch_partition_info( } /// An update produced by the metadata fetcher. -/// -/// Either the IDs and high watermarks of the topic partitions as of `timestamp`, or a health -/// status describing a fetch error. #[derive(Clone, Debug, PartialEq, Eq, PartialOrd, Ord, Serialize, Deserialize)] -struct MetadataUpdate { - timestamp: mz_repr::Timestamp, - info: Result, HealthStatus>, +enum MetadataUpdate { + /// The current IDs and high watermarks of all topic partitions. + Partitions(BTreeMap), + /// A transient error. + /// + /// Transient errors stall the source until their cause has been resolved. + TransientError(HealthStatus), + /// A definite error. + /// + /// Definite errors cannot be recovered from. They poison the source until the end of time. + DefiniteError(SourceError), } impl MetadataUpdate { - fn to_probe(&self) -> Option> { - let Ok(partitions) = &self.info else { - return None; - }; + /// Return the upstream frontier resulting from the metadata update, if any. + fn upstream_frontier(&self) -> Option> { + match self { + Self::Partitions(partitions) => { + let max_pid = partitions.keys().last().copied(); + let lower = max_pid + .map(RangeBound::after) + .unwrap_or(RangeBound::NegInfinity); + let future_ts = + Partitioned::new_range(lower, RangeBound::PosInfinity, MzOffset::from(0)); + + let mut frontier = Antichain::from_elem(future_ts); + for (pid, high_watermark) in partitions { + frontier.insert(Partitioned::new_singleton( + RangeBound::exact(*pid), + MzOffset::from(*high_watermark), + )); + } - let max_pid = partitions.keys().last().copied(); - let lower = max_pid - .map(RangeBound::after) - .unwrap_or(RangeBound::NegInfinity); - let future_ts = Partitioned::new_range(lower, RangeBound::PosInfinity, MzOffset::from(0)); - - let mut upstream_frontier = Antichain::from_elem(future_ts); - for (pid, high_watermark) in partitions { - upstream_frontier.insert(Partitioned::new_singleton( - RangeBound::exact(*pid), - MzOffset::from(*high_watermark), - )); + Some(frontier) + } + Self::DefiniteError(_) => Some(Antichain::new()), + Self::TransientError(_) => None, } - - Some(Probe { - probe_ts: self.timestamp, - upstream_frontier, - }) } } @@ -1540,13 +1502,21 @@ fn render_metadata_fetcher>( connection: KafkaSourceConnection, config: RawSourceCreationConfig, ) -> ( - Stream, + Stream, Stream>, PressOnDropButton, ) { let active_worker_id = usize::cast_from(config.id.hashed()); let is_active_worker = active_worker_id % scope.peers() == scope.index(); + let resume_upper = Antichain::from_iter( + config + .source_resume_uppers + .values() + .map(|uppers| uppers.iter().map(KafkaTimestamp::decode_row)) + .flatten(), + ); + let name = format!("KafkaMetadataFetcher({})", config.id); let mut builder = AsyncOperatorBuilder::new(name, scope.clone()); @@ -1603,11 +1573,9 @@ fn render_metadata_fetcher>( ContextCreationError::Ssh(_) => HealthStatus::ssh(status_update), _ => HealthStatus::kafka(status_update), }; - let update = MetadataUpdate { - timestamp: 0.into(), - info: Err(status), - }; - metadata_output.give(&metadata_cap, update); + let error = MetadataUpdate::TransientError(status); + let timestamp = (config.now_fn)().into(); + metadata_output.give(&metadata_cap, (timestamp, error)); return; } }; @@ -1625,11 +1593,42 @@ fn render_metadata_fetcher>( let (tx, mut rx) = mpsc::unbounded_channel(); spawn_metadata_thread(config, consumer, topic, poll_interval, tx); - while let Some(update) = rx.recv().await { - if let Some(probe) = update.to_probe() { + let mut prev_upstream_frontier = resume_upper; + + while let Some((timestamp, mut update)) = rx.recv().await { + if prev_upstream_frontier.is_empty() { + return; + } + + if let Some(upstream_frontier) = update.upstream_frontier() { + // Topics are identified by name but it's possible that a user recreates a topic + // with the same name. Ideally we'd want to catch all of these cases and + // immediately error out the source, since the data is effectively gone. + // Unfortunately this is not possible without something like KIP-516. + // + // The best we can do is check whether the upstream frontier regressed. This tells + // us thet the topic was recreated and now contains fewer offsets and/or fewer + // partitions. Note that we are not able to detect topic recreation if neither of + // the two are true. + if !PartialOrder::less_equal(&prev_upstream_frontier, &upstream_frontier) { + let error = SourceError { + error: SourceErrorDetails::Other("topic was recreated".into()), + }; + update = MetadataUpdate::DefiniteError(error); + } + } + + if let Some(upstream_frontier) = update.upstream_frontier() { + prev_upstream_frontier = upstream_frontier.clone(); + + let probe = Probe { + probe_ts: timestamp, + upstream_frontier, + }; probe_output.give(&probe_cap, probe); } - metadata_output.give(&metadata_cap, update); + + metadata_output.give(&metadata_cap, (timestamp, update)); } }); @@ -1641,7 +1640,7 @@ fn spawn_metadata_thread( consumer: BaseConsumer>, topic: String, poll_interval: Duration, - tx: mpsc::UnboundedSender, + tx: mpsc::UnboundedSender<(mz_repr::Timestamp, MetadataUpdate)>, ) { thread::Builder::new() .name(format!("kafka-metadata-{}", config.id)) @@ -1680,10 +1679,13 @@ fn spawn_metadata_thread( "kafka metadata thread: fetched partition metadata info", ); - MetadataUpdate { - timestamp: probe_ts, - info: Ok(partitions), - } + MetadataUpdate::Partitions(partitions) + } + Err(GetPartitionsError::TopicDoesNotExist) => { + let error = SourceError { + error: SourceErrorDetails::Other("topic was deleted".into()), + }; + MetadataUpdate::DefiniteError(error) } Err(e) => { let kafka_status = Some(HealthStatusUpdate::stalled( @@ -1699,17 +1701,14 @@ fn spawn_metadata_thread( } }; - MetadataUpdate { - timestamp: probe_ts, - info: Err(HealthStatus { - kafka: kafka_status, - ssh: ssh_status, - }), - } + MetadataUpdate::TransientError(HealthStatus { + kafka: kafka_status, + ssh: ssh_status, + }) } }; - if tx.send(update).is_err() { + if tx.send((probe_ts, update)).is_err() { break; } diff --git a/test/testdrive-old-kafka-src-syntax/kafka-recreate-topic.td b/test/testdrive-old-kafka-src-syntax/kafka-recreate-topic.td index 245fc8134c218..349bd400d5c1c 100644 --- a/test/testdrive-old-kafka-src-syntax/kafka-recreate-topic.td +++ b/test/testdrive-old-kafka-src-syntax/kafka-recreate-topic.td @@ -7,74 +7,54 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. -$ kafka-create-topic topic=topic0 partitions=4 - -$ kafka-ingest key-format=bytes format=bytes key-terminator=: topic=topic0 repeat=1 -1:1 +> CREATE CONNECTION kafka_conn + TO KAFKA (BROKER '${testdrive.kafka-addr}', SECURITY PROTOCOL PLAINTEXT) -> CREATE CONNECTION IF NOT EXISTS csr_conn TO CONFLUENT SCHEMA REGISTRY ( - URL '${testdrive.schema-registry-url}' - ); +> CREATE CLUSTER to_recreate SIZE '1' -> CREATE CONNECTION kafka_conn - TO KAFKA (BROKER '${testdrive.kafka-addr}', SECURITY PROTOCOL PLAINTEXT); +# Test detection of topic deletion. -> CREATE CLUSTER to_recreate SIZE '1', REPLICATION FACTOR 1; +$ kafka-create-topic topic=topic0 partitions=4 -> CREATE SOURCE source0 - IN CLUSTER to_recreate +> CREATE SOURCE source0 IN CLUSTER to_recreate FROM KAFKA CONNECTION kafka_conn (TOPIC 'testdrive-topic0-${testdrive.seed}') - KEY FORMAT TEXT - VALUE FORMAT TEXT - ENVELOPE UPSERT + FORMAT TEXT ENVELOPE NONE > SELECT * FROM source0 -key text ----------- -1 1 - -# Now recreate the topic with fewer partitions and observe the error $ kafka-delete-topic-flaky topic=topic0 -# Even though `kafka-delete-topic` ensures that the topic no longer exists in -# the broker metadata there is still work to be done asychnronously before it's -# truly gone that must complete before we attempt to recreate it. There is no -# way to observe this work completing so the only option left is sleeping for a -# while. This is the sad state of Kafka. If this test ever becomes flaky let's -# just delete it. -# See: https://github.com/confluentinc/confluent-kafka-python/issues/541 -$ sleep-is-probably-flaky-i-have-justified-my-need-with-a-comment duration=2s - -$ kafka-create-topic topic=topic0 partitions=2 - ! SELECT * FROM source0 -contains:topic was recreated: partition count regressed from 4 to 2 +contains:topic was deleted -# We can also detect that a topic got recreated by observing the high watermark regressing +# Test detection of topic recreation. +# +# The Kafka source detects topic recreation based on regression of the upstream +# frontier. For the upstream frontier to regress, the new topic must have: +# (1) fewer partitions than the old topic, or +# (2) a lower watermark for at least one of its partitions. +# We test both cases below. -$ kafka-create-topic topic=topic1 partitions=1 +# (1) topic recreation with fewer partitions. -$ kafka-ingest format=bytes topic=topic1 repeat=1 -1 +$ kafka-create-topic topic=topic1 partitions=4 -> CREATE SOURCE source1 - IN CLUSTER to_recreate +> CREATE SOURCE source1 IN CLUSTER to_recreate FROM KAFKA CONNECTION kafka_conn (TOPIC 'testdrive-topic1-${testdrive.seed}') - FORMAT TEXT - ENVELOPE NONE + FORMAT TEXT ENVELOPE NONE > SELECT * FROM source1 -text ----- -1 -# Now recreate the topic with the same number of partitions and observe the error +# Spin down the cluster, to prevent the source from observing the topic +# deletion before the new topic was created. +> ALTER CLUSTER to_recreate SET (REPLICATION FACTOR 0) + +# Recreate the topic with fewer partitions. $ kafka-delete-topic-flaky topic=topic1 # Even though `kafka-delete-topic` ensures that the topic no longer exists in -# the broker metadata there is still work to be done asychnronously before it's +# the broker metadata there is still work to be done asynchronously before it's # truly gone that must complete before we attempt to recreate it. There is no # way to observe this work completing so the only option left is sleeping for a # while. This is the sad state of Kafka. If this test ever becomes flaky let's @@ -82,65 +62,50 @@ $ kafka-delete-topic-flaky topic=topic1 # See: https://github.com/confluentinc/confluent-kafka-python/issues/541 $ sleep-is-probably-flaky-i-have-justified-my-need-with-a-comment duration=2s -$ kafka-create-topic topic=topic1 partitions=1 +$ kafka-create-topic topic=topic1 partitions=2 + +> ALTER CLUSTER to_recreate SET (REPLICATION FACTOR 1) ! SELECT * FROM source1 -contains:topic was recreated: high watermark of partition 0 regressed from 1 to 0 +contains:topic was recreated -# Test a pathological topic recreation observed in the wild. -# See incidents-and-escalations#98. +# (2) topic recreation with a lower watermark. -# First we create a topic and successfully ingest some data. -$ kafka-create-topic topic=topic2 partitions=1 -$ kafka-ingest format=bytes topic=topic2 repeat=100 -one -> CREATE SOURCE source2 - IN CLUSTER to_recreate +$ kafka-create-topic topic=topic2 partitions=4 + +$ kafka-ingest format=bytes topic=topic2 +1 + +> CREATE SOURCE source2 IN CLUSTER to_recreate FROM KAFKA CONNECTION kafka_conn (TOPIC 'testdrive-topic2-${testdrive.seed}') - FORMAT TEXT - ENVELOPE NONE -> SELECT count(*) FROM source2 -100 + FORMAT TEXT ENVELOPE NONE + +> SELECT * FROM source2 +1 + +# Spin down the cluster, to prevent the source from observing the topic +# deletion before the new topic was created. +> ALTER CLUSTER to_recreate SET (REPLICATION FACTOR 0) -# Then we turn off the source cluster, so that we lose our record of what the -# high water mark used to be. -> ALTER CLUSTER to_recreate SET (REPLICATION FACTOR = 0) +# Recreate the topic with the same number of partitions but a lower watermark. -# Then we delete the topic and recreate it... -# See comment above about needing to sleep after deleting Kafka topics. $ kafka-delete-topic-flaky topic=topic2 + +# Even though `kafka-delete-topic` ensures that the topic no longer exists in +# the broker metadata there is still work to be done asynchronously before it's +# truly gone that must complete before we attempt to recreate it. There is no +# way to observe this work completing so the only option left is sleeping for a +# while. This is the sad state of Kafka. If this test ever becomes flaky let's +# just delete it. +# See: https://github.com/confluentinc/confluent-kafka-python/issues/541 $ sleep-is-probably-flaky-i-have-justified-my-need-with-a-comment duration=2s -$ kafka-create-topic topic=topic2 partitions=1 - -# ...crucially, with *fewer* offsets than we had previously. -$ kafka-ingest format=bytes topic=topic2 repeat=50 -one - -# Finally, we turn the source cluster back on. This would previously cause -# Materialize to panic because we'd attempt to regress the data shard's -# capability to offset 2 (the max offset in the new topic) when it was -# already at offset 3 (the max offset in the old topic). -> ALTER CLUSTER to_recreate SET (REPLICATION FACTOR = 1) - -# Give the source a few seconds to reconnect to the Kafka partitions and -# possibly read bad data. This is what actually reproduces the panic we saw in -# incidents-and-escalations#98. Unfortunately there is no signal we can wait -# for, so the best we can do is sleep. -$ sleep-is-probably-flaky-i-have-justified-my-need-with-a-comment duration=10s - -# Ensure the source reports the previous data. -> SELECT count(*) FROM source2 -100 - -# Check whether the source is still lumbering along. Correctness has gone out -# the window here. Data in the new topic will be ignored up until the first new -# offset, at which point it will start being ingested. In this case, 7 and 8 are -# the two new data rows. -$ kafka-ingest format=bytes topic=topic2 repeat=53 -one - -> SELECT count(*) FROM source2 -103 + +$ kafka-create-topic topic=topic2 partitions=4 + +> ALTER CLUSTER to_recreate SET (REPLICATION FACTOR 1) + +! SELECT * FROM source2 +contains:topic was recreated # Ensure we don't panic after we restart due to the above finished ingestions. $ kafka-create-topic topic=good-topic @@ -169,10 +134,7 @@ name status error good_source running source0 paused source1 paused -# Ideally source 2 would be permanently stalled because the topic was recreated, -# but we can't easily distingiush that situation from a temporary ingestion -# hiccup, and so at the moment we consider source2 to be fully healthy. -source2 running +source2 paused # Testdrive expects all sources to end in a healthy state, so manufacture that # by dropping sources. diff --git a/test/testdrive/kafka-recreate-topic.td b/test/testdrive/kafka-recreate-topic.td index e5c953ee50d84..0d3dbb47457df 100644 --- a/test/testdrive/kafka-recreate-topic.td +++ b/test/testdrive/kafka-recreate-topic.td @@ -7,78 +7,56 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. -$ kafka-create-topic topic=topic0 partitions=4 - -$ kafka-ingest key-format=bytes format=bytes key-terminator=: topic=topic0 repeat=1 -1:1 +> CREATE CONNECTION kafka_conn + TO KAFKA (BROKER '${testdrive.kafka-addr}', SECURITY PROTOCOL PLAINTEXT) -> CREATE CONNECTION IF NOT EXISTS csr_conn TO CONFLUENT SCHEMA REGISTRY ( - URL '${testdrive.schema-registry-url}' - ); +> CREATE CLUSTER to_recreate SIZE '1' -> CREATE CONNECTION kafka_conn - TO KAFKA (BROKER '${testdrive.kafka-addr}', SECURITY PROTOCOL PLAINTEXT); +# Test detection of topic deletion. -> CREATE CLUSTER to_recreate SIZE '1', REPLICATION FACTOR 1; +$ kafka-create-topic topic=topic0 partitions=4 -> CREATE SOURCE source0 - IN CLUSTER to_recreate +> CREATE SOURCE source0 IN CLUSTER to_recreate FROM KAFKA CONNECTION kafka_conn (TOPIC 'testdrive-topic0-${testdrive.seed}') - > CREATE TABLE source0_tbl FROM SOURCE source0 (REFERENCE "testdrive-topic0-${testdrive.seed}") - KEY FORMAT TEXT - VALUE FORMAT TEXT - ENVELOPE UPSERT + FORMAT TEXT ENVELOPE NONE > SELECT * FROM source0_tbl -key text ----------- -1 1 - -# Now recreate the topic with fewer partitions and observe the error $ kafka-delete-topic-flaky topic=topic0 -# Even though `kafka-delete-topic` ensures that the topic no longer exists in -# the broker metadata there is still work to be done asynchronously before it's -# truly gone that must complete before we attempt to recreate it. There is no -# way to observe this work completing so the only option left is sleeping for a -# while. This is the sad state of Kafka. If this test ever becomes flaky let's -# just delete it. -# See: https://github.com/confluentinc/confluent-kafka-python/issues/541 -$ sleep-is-probably-flaky-i-have-justified-my-need-with-a-comment duration=2s - -$ kafka-create-topic topic=topic0 partitions=2 - ! SELECT * FROM source0_tbl -contains:topic was recreated: partition count regressed from 4 to 2 +contains:topic was deleted -# We can also detect that a topic got recreated by observing the high watermark regressing +# Test detection of topic recreation. +# +# The Kafka source detects topic recreation based on regression of the upstream +# frontier. For the upstream frontier to regress, the new topic must have: +# (1) fewer partitions than the old topic, or +# (2) a lower watermark for at least one of its partitions. +# We test both cases below. -$ kafka-create-topic topic=topic1 partitions=1 +# (1) topic recreation with fewer partitions. -$ kafka-ingest format=bytes topic=topic1 repeat=1 -1 +$ kafka-create-topic topic=topic1 partitions=4 -> CREATE SOURCE source1 - IN CLUSTER to_recreate +> CREATE SOURCE source1 IN CLUSTER to_recreate FROM KAFKA CONNECTION kafka_conn (TOPIC 'testdrive-topic1-${testdrive.seed}') - > CREATE TABLE source1_tbl FROM SOURCE source1 (REFERENCE "testdrive-topic1-${testdrive.seed}") - FORMAT TEXT - ENVELOPE NONE + FORMAT TEXT ENVELOPE NONE > SELECT * FROM source1_tbl -text ----- -1 -# Now recreate the topic with the same number of partitions and observe the error +# Spin down the cluster, to prevent the source from observing the topic +# deletion before the new topic was created. +> ALTER CLUSTER to_recreate SET (REPLICATION FACTOR 0) + +# Recreate the topic with fewer partitions. $ kafka-delete-topic-flaky topic=topic1 # Even though `kafka-delete-topic` ensures that the topic no longer exists in -# the broker metadata there is still work to be done asychnronously before it's +# the broker metadata there is still work to be done asynchronously before it's # truly gone that must complete before we attempt to recreate it. There is no # way to observe this work completing so the only option left is sleeping for a # while. This is the sad state of Kafka. If this test ever becomes flaky let's @@ -86,70 +64,51 @@ $ kafka-delete-topic-flaky topic=topic1 # See: https://github.com/confluentinc/confluent-kafka-python/issues/541 $ sleep-is-probably-flaky-i-have-justified-my-need-with-a-comment duration=2s -$ kafka-create-topic topic=topic1 partitions=1 +$ kafka-create-topic topic=topic1 partitions=2 + +> ALTER CLUSTER to_recreate SET (REPLICATION FACTOR 1) ! SELECT * FROM source1_tbl -contains:topic was recreated: high watermark of partition 0 regressed from 1 to 0 +contains:topic was recreated -# Test a pathological topic recreation observed in the wild. -# See incidents-and-escalations#98. +# (2) topic recreation with a lower watermark. -# First we create a topic and successfully ingest some data. -$ kafka-create-topic topic=topic2 partitions=1 -$ kafka-ingest format=bytes topic=topic2 repeat=100 -one +$ kafka-create-topic topic=topic2 partitions=4 -> CREATE SOURCE source2 - IN CLUSTER to_recreate +$ kafka-ingest format=bytes topic=topic2 +1 + +> CREATE SOURCE source2 IN CLUSTER to_recreate FROM KAFKA CONNECTION kafka_conn (TOPIC 'testdrive-topic2-${testdrive.seed}') +> CREATE TABLE source2_tbl FROM SOURCE source2 (REFERENCE "testdrive-topic2-${testdrive.seed}") + FORMAT TEXT ENVELOPE NONE -> CREATE TABLE source2_tbl - FROM SOURCE source2 (REFERENCE "testdrive-topic2-${testdrive.seed}") - FORMAT TEXT - ENVELOPE NONE +> SELECT * FROM source2_tbl +1 -> SELECT count(*) FROM source2_tbl -100 +# Spin down the cluster, to prevent the source from observing the topic +# deletion before the new topic was created. +> ALTER CLUSTER to_recreate SET (REPLICATION FACTOR 0) -# Then we turn off the source cluster, so that we lose our record of what the -# high water mark used to be. -> ALTER CLUSTER to_recreate SET (REPLICATION FACTOR = 0) +# Recreate the topic with the same number of partitions but a lower watermark. -# Then we delete the topic and recreate it... -# See comment above about needing to sleep after deleting Kafka topics. $ kafka-delete-topic-flaky topic=topic2 + +# Even though `kafka-delete-topic` ensures that the topic no longer exists in +# the broker metadata there is still work to be done asynchronously before it's +# truly gone that must complete before we attempt to recreate it. There is no +# way to observe this work completing so the only option left is sleeping for a +# while. This is the sad state of Kafka. If this test ever becomes flaky let's +# just delete it. +# See: https://github.com/confluentinc/confluent-kafka-python/issues/541 $ sleep-is-probably-flaky-i-have-justified-my-need-with-a-comment duration=2s -$ kafka-create-topic topic=topic2 partitions=1 - -# ...crucially, with *fewer* offsets than we had previously. -$ kafka-ingest format=bytes topic=topic2 repeat=50 -one - -# Finally, we turn the source cluster back on. This would previously cause -# Materialize to panic because we'd attempt to regress the data shard's -# capability to offset 2 (the max offset in the new topic) when it was -# already at offset 3 (the max offset in the old topic). -> ALTER CLUSTER to_recreate SET (REPLICATION FACTOR = 1) - -# Give the source a few seconds to reconnect to the Kafka partitions and -# possibly read bad data. This is what actually reproduces the panic we saw in -# incidents-and-escalations#98. Unfortunately there is no signal we can wait -# for, so the best we can do is sleep. -$ sleep-is-probably-flaky-i-have-justified-my-need-with-a-comment duration=10s - -# Ensure the source reports the previous data. -> SELECT count(*) FROM source2_tbl -100 - -# Check whether the source is still lumbering along. Correctness has gone out -# the window here. Data in the new topic will be ignored up until the first new -# offset, at which point it will start being ingested. In this case, 7 and 8 are -# the two new data rows. -$ kafka-ingest format=bytes topic=topic2 repeat=53 -one - -> SELECT count(*) FROM source2_tbl -103 + +$ kafka-create-topic topic=topic2 partitions=4 + +> ALTER CLUSTER to_recreate SET (REPLICATION FACTOR 1) + +! SELECT * FROM source2_tbl +contains:topic was recreated # Ensure we don't panic after we restart due to the above finished ingestions. $ kafka-create-topic topic=good-topic @@ -176,14 +135,14 @@ text > SELECT name, status, error FROM mz_internal.mz_source_statuses WHERE type != 'progress' name status error ------------------------------- -good_source_tbl running -source0_tbl stalled "kafka: Source error: source must be dropped and recreated due to failure: topic was recreated: partition count regressed from 4 to 2" -source1 paused -source1_tbl stalled "kafka: Source error: source must be dropped and recreated due to failure: topic was recreated: high watermark of partition 0 regressed from 1 to 0" -source2_tbl running -good_source running -source0 paused -source2 running +good_source running +good_source_tbl running +source0 paused +source0_tbl stalled "kafka: Source error: source must be dropped and recreated due to failure: topic was deleted" +source1 paused +source1_tbl stalled "kafka: Source error: source must be dropped and recreated due to failure: topic was recreated" +source2 paused +source2_tbl stalled "kafka: Source error: source must be dropped and recreated due to failure: topic was recreated" # Testdrive expects all sources to end in a healthy state, so manufacture that # by dropping sources. From bc832468e6de1f8f18d06e550ba024f1f2c9c6df Mon Sep 17 00:00:00 2001 From: Jan Teske Date: Mon, 6 Jan 2025 09:24:16 +0100 Subject: [PATCH 10/50] kafka: don't shutdown metadata fetcher on error Shutting down the metadata fetcher operator releases its capability on the probe output. When the reclock-to-latest reclocking is enabled this makes the remap operator believe that the upstream has advanced to the empty frontier, making it seal the collection. We must thus not shut down the metadata fetcher when it encounters a connection error, as the source did not in fact advance to the empty frontier in this case. --- src/storage/src/source/kafka.rs | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/storage/src/source/kafka.rs b/src/storage/src/source/kafka.rs index 6dfb33138969b..f88f1acc23106 100644 --- a/src/storage/src/source/kafka.rs +++ b/src/storage/src/source/kafka.rs @@ -1576,7 +1576,12 @@ fn render_metadata_fetcher>( let error = MetadataUpdate::TransientError(status); let timestamp = (config.now_fn)().into(); metadata_output.give(&metadata_cap, (timestamp, error)); - return; + + // IMPORTANT: wedge forever until the `SuspendAndRestart` is processed. + // Returning would incorrectly present to the remap operator as progress to the + // empty frontier which would be incorrectly recorded to the remap shard. + std::future::pending::<()>().await; + unreachable!("pending future never returns"); } }; From 3fe91487cd75f6a984cecef5881127d48a77ff43 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Wed, 8 Jan 2025 14:20:08 +0100 Subject: [PATCH 11/50] storage: in upsert StateValue allow converting finalized tombstones Before, we were using the wrong assumption that the new feedback UPSERT operator could not yield finalized tombstone values, because it only writes finalized values that "exist". But it can happen that `ensure_decoded` puts in place a finalized tombstone when state values consolidate to a diff of zero. We say "can happen" above, because `merge_update` returns whether or not a merged value can be deleted, and this returns true when the diff_sum is zero. So in most cases the finalized tombstone would be cleaned up right away. However, not all callers immediately (or at all) act on this "can delete" information, which leads to finalized tombstones actually existing in state. --- src/storage/src/upsert/types.rs | 14 ++++---------- 1 file changed, 4 insertions(+), 10 deletions(-) diff --git a/src/storage/src/upsert/types.rs b/src/storage/src/upsert/types.rs index 55deeead8bb75..b00a20db25615 100644 --- a/src/storage/src/upsert/types.rs +++ b/src/storage/src/upsert/types.rs @@ -391,16 +391,10 @@ impl StateValue { provisional_value: (Some(provisional_value), provisional_ts, provisional_order), }) } - StateValue::Value(Value::Tombstone(_)) => { - // This cannot happen with how the new feedback UPSERT uses - // state. There are only ever provisional tombstones, and all - // updates that are merged/consolidated into upsert state come - // from the persist input, which doesn't need tombstones. - // - // Regular, finalized tombstones are only used by the classic - // UPSERT operator when doing partial processing. - panic!("cannot turn a finalized tombstone into a provisional value") - } + StateValue::Value(Value::Tombstone(_)) => StateValue::Value(Value::ProvisionalValue { + finalized_value: None, + provisional_value: (Some(provisional_value), provisional_ts, provisional_order), + }), StateValue::Value(Value::ProvisionalValue { finalized_value, provisional_value: _, From 2a904396c21ca07708d6fc74bf9c9409483f1de7 Mon Sep 17 00:00:00 2001 From: Parker Timmerman Date: Wed, 8 Jan 2025 14:05:56 -0500 Subject: [PATCH 12/50] build: Use "limited" debug info instead of "line-tables" (#30962) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Chatted about this in [Slack](https://materializeinc.slack.com/archives/CU7ELJ6E9/p1736243826039879), as-is we only see non-inlined functions which makes it hard to follow timely operators. This PR bumps our debuginfo from line-tables to limited, let's see if it helps. ### Motivation Better debug info ### Checklist - [x] This PR has adequate test coverage / QA involvement has been duly considered. ([trigger-ci for additional test/nightly runs](https://trigger-ci.dev.materialize.com/)) - [x] This PR has an associated up-to-date [design doc](https://github.com/MaterializeInc/materialize/blob/main/doc/developer/design/README.md), is a design doc ([template](https://github.com/MaterializeInc/materialize/blob/main/doc/developer/design/00000000_template.md)), or is sufficiently small to not require a design. - [x] If this PR evolves [an existing `$T ⇔ Proto$T` mapping](https://github.com/MaterializeInc/materialize/blob/main/doc/developer/command-and-response-binary-encoding.md) (possibly in a backwards-incompatible way), then it is tagged with a `T-proto` label. - [x] If this PR will require changes to cloud orchestration or tests, there is a companion cloud PR to account for those changes that is tagged with the release-blocker label ([example](https://github.com/MaterializeInc/cloud/pull/5021)). - [x] If this PR includes major [user-facing behavior changes](https://github.com/MaterializeInc/materialize/blob/main/doc/developer/guide-changes.md#what-changes-require-a-release-note), I have pinged the relevant PM to schedule a changelog post. --- .bazelrc | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.bazelrc b/.bazelrc index e95a918ddcc45..d3326f9bc5e33 100644 --- a/.bazelrc +++ b/.bazelrc @@ -116,8 +116,8 @@ build:debuginfo-full --copt=-g2 build:debuginfo-full --strip=never build:debuginfo-full --@rules_rust//:extra_rustc_flag=-Cstrip=none -build:debuginfo-limited --@rules_rust//:extra_rustc_flag=-Cdebuginfo=line-tables-only -build:debuginfo-limited --copt=-gline-tables-only +build:debuginfo-limited --@rules_rust//:extra_rustc_flag=-Cdebuginfo=1 +build:debuginfo-limited --copt=-g1 build:debuginfo-limited --strip=never build:debuginfo-limited --@rules_rust//:extra_rustc_flag=-Cstrip=none From 2ec6f5326b3ca75e23da42bc21bba82ce6e43bce Mon Sep 17 00:00:00 2001 From: Joseph Koshakow Date: Wed, 8 Jan 2025 15:24:10 -0500 Subject: [PATCH 13/50] storage: Correct output index for source exports Previously, the output index of each ingestion export was calculated assuming that index 0 was reserved for the primary source relation. When `force_source_table_syntax` is enabled, the primary source relation is not included in the exports, and therefore caused all export indexes to be off by one. This commit fixes the issue by decrementing all export indexes by 1 if the primary source relation was not included in the exports. It's unclear to the author how errors are being handled when the primary relation is not included in the exports. Works towards resolving #MaterializeInc/database-issues/issues/8620 --- src/storage-types/src/sources.rs | 22 +++++++++++++++++----- 1 file changed, 17 insertions(+), 5 deletions(-) diff --git a/src/storage-types/src/sources.rs b/src/storage-types/src/sources.rs index 56f2e1ca5dab7..96e8a762633ed 100644 --- a/src/storage-types/src/sources.rs +++ b/src/storage-types/src/sources.rs @@ -148,17 +148,21 @@ impl IngestionDescription { primary_source_id: &GlobalId, ) -> BTreeMap> { let mut source_exports = BTreeMap::new(); - // `self.source_exports` contains all source-exports (e.g. subsources & tables) as well as - // the primary source relation. It's not guaranteed that the primary source relation is - // the first element in the map, however it much be set to output 0 to align with - // assumptions in source implementations. This is the case even if the primary - // export will not have any data output for it, since output 0 is the convention + // `self.source_exports` contains all source-exports (e.g. subsources & tables) and + // sometimes the primary source relation. It's not guaranteed that the primary source + // relation is the first element in the map, however it much be set to output 0 when it + // exists to align with assumptions in source implementations. This is the case even if the + // primary export will not have any data output for it, since output 0 is the convention // used for errors that should halt the entire source dataflow. // TODO: See if we can simplify this to avoid needing to include the primary output // if no data will be exported to it. This requires refactoring all error output handling. + // TODO: We already sometimes exclude the primary source relation, so where are the errors + // going...? let mut next_output = 1; + let mut has_primary_source = false; for (id, export) in self.source_exports.iter() { let ingestion_output = if id == primary_source_id { + has_primary_source = true; 0 } else { let idx = next_output; @@ -175,6 +179,14 @@ impl IngestionDescription { ); } + // If the primary source does not exist, then we need to decrement all indexes, because we + // had reserved index 0 for the primary source. + if !has_primary_source { + for (_, output) in &mut source_exports { + output.ingestion_output -= 1; + } + } + source_exports } } From a0b12993fe3e09f574f9ddb8837b780203a769c2 Mon Sep 17 00:00:00 2001 From: Roshan Jobanputra Date: Mon, 16 Sep 2024 16:43:45 -0400 Subject: [PATCH 14/50] Opt-in catalog migration for converting subsources to source tables --- src/adapter-types/src/dyncfgs.rs | 8 ++ src/adapter/src/catalog/migrate.rs | 166 ++++++++++++++++++++++++++++- 2 files changed, 169 insertions(+), 5 deletions(-) diff --git a/src/adapter-types/src/dyncfgs.rs b/src/adapter-types/src/dyncfgs.rs index 795a342f284a4..1ddd317c835fb 100644 --- a/src/adapter-types/src/dyncfgs.rs +++ b/src/adapter-types/src/dyncfgs.rs @@ -111,6 +111,13 @@ pub const ENABLE_EXPRESSION_CACHE: Config = Config::new( "Use a cache to store optimized expressions to help speed up start times.", ); +/// Whether to enable the migration to convert all sources to use tables. +pub const ENABLE_SOURCE_TABLE_MIGRATION: Config = Config::new( + "enable_source_table_migration", + false, + "Whether to enable the migration to convert all sources to use tables.", +); + /// Adds the full set of all compute `Config`s. pub fn all_dyncfgs(configs: ConfigSet) -> ConfigSet { configs @@ -128,4 +135,5 @@ pub fn all_dyncfgs(configs: ConfigSet) -> ConfigSet { .add(&DEFAULT_SINK_PARTITION_STRATEGY) .add(&ENABLE_CONTINUAL_TASK_BUILTINS) .add(&ENABLE_EXPRESSION_CACHE) + .add(&ENABLE_SOURCE_TABLE_MIGRATION) } diff --git a/src/adapter/src/catalog/migrate.rs b/src/adapter/src/catalog/migrate.rs index c62d768a838e6..888824f0d389a 100644 --- a/src/adapter/src/catalog/migrate.rs +++ b/src/adapter/src/catalog/migrate.rs @@ -10,7 +10,7 @@ use std::collections::BTreeMap; use mz_catalog::builtin::BuiltinTable; -use mz_catalog::durable::Transaction; +use mz_catalog::durable::{Transaction, Item}; use mz_catalog::memory::objects::StateUpdate; use mz_ore::collections::CollectionExt; use mz_ore::now::NowFn; @@ -31,13 +31,23 @@ where &'a mut Transaction<'_>, CatalogItemId, &'a mut Statement, + &'a Vec<(Item, Statement)>, ) -> Result<(), anyhow::Error>, { let mut updated_items = BTreeMap::new(); + let items_with_statements = tx + .get_items() + .map(|item| { + let stmt = mz_sql::parse::parse(&item.create_sql)?.into_element().ast; + Ok((item, stmt)) + }) + .collect::, anyhow::Error>>()?; - for mut item in tx.get_items() { + // Clone this vec to be referenced within the closure if needed + let items_with_statements_ref = items_with_statements.clone(); + for (mut item, mut stmt) in items_with_statements { let mut stmt = mz_sql::parse::parse(&item.create_sql)?.into_element().ast; - f(tx, item.id, &mut stmt)?; + f(tx, item.id, &mut stmt, &items_with_statements_ref)?; item.create_sql = stmt.to_ast_string_stable(); @@ -97,7 +107,7 @@ pub(crate) async fn migrate( catalog_version ); - rewrite_ast_items(tx, |_tx, _id, _stmt| { + rewrite_ast_items(tx, |tx, _id, stmt, all_items_and_statements| { // Add per-item AST migrations below. // // Each migration should be a function that takes `stmt` (the AST @@ -107,7 +117,21 @@ pub(crate) async fn migrate( // Migration functions may also take `tx` as input to stage // arbitrary changes to the catalog. - Ok(()) + // Special block for `ast_rewrite_sources_to_tables` migration + // since it requires a feature flag. + if let Some(config_val) = tx.get_system_config("enable_source_table_migration") { + let enable_migration = config_val.parse::().map_err(|e| { + anyhow::anyhow!( + "could not parse enable_source_table_migration config value: {}", + e + ) + })?; + if enable_migration { + info!("migrate: enable_source_table_migration"); + ast_rewrite_sources_to_tables(stmt, all_items_and_statements)?; + } + } + Ok(()) })?; // Load items into catalog. We make sure to consolidate the old updates with the new updates to @@ -182,6 +206,138 @@ pub(crate) async fn migrate( // Please include the adapter team on any code reviews that add or edit // migrations. +/// Migrates all sources to use the new sources as tables model +/// +/// First we migrate existing `CREATE SUBSOURCE` statements, turning them into +/// `CREATE TABLE .. FROM SOURCE` statements. This covers existing Postgres, +/// MySQL, and multi-output (tpch, auction, marketing) load-generator subsources. +/// +/// Second we migrate existing `CREATE SOURCE` statements for these multi-output +/// sources to remove their subsource qualification option (e.g. FOR ALL TABLES..) +/// and any subsource-specific options (e.g. TEXT COLUMNS). +/// +/// Third we migrate existing `CREATE SOURCE` statements that refer to a single +/// output collection. This includes existing Kafka and single-output load-generator +/// subsources. These statements will generate an additional `CREATE TABLE .. FROM SOURCE` +/// statement that copies over all the export-specific options. This statement will be tied +/// to the existing statement's persist collection, but using a new GlobalID. The original +/// source statement will be updated to remove the export-specific options and will be +/// renamed to `_source` while keeping its same GlobalId. +/// +fn ast_rewrite_sources_to_tables( + stmt: &mut Statement, + all_items_and_statements: &Vec<(Item, Statement)>, +) -> Result<(), anyhow::Error> { + use mz_sql::ast::{ + CreateSubsourceOptionName, CreateTableFromSourceStatement, RawItemName, + TableFromSourceOption, WithOptionValue, + }; + + // Since some subsources have named-only references to their `of_source` and some have the + // global_id of their source, we first generate a reference mapping from all source names to + // items so we can correct the references in the `CREATE TABLE .. FROM SOURCE` + // statements to always use the ID of the source. + let source_name_to_item: BTreeMap<_, _> = all_items_and_statements + .iter() + .filter_map(|(item, statement)| match statement { + Statement::CreateSource(stmt) => Some((stmt.name.clone(), item)), + _ => None, + }) + .collect(); + + match stmt { + // Migrate each 'source export' `CREATE SUBSOURCE` statement to an equivalent + // `CREATE TABLE .. FROM SOURCE` statement. Note that we will not be migrating + // 'progress' `CREATE SUBSOURCE` statements as they are not currently relevant + // to the new table model. + Statement::CreateSubsource(subsource) if subsource.of_source.is_some() => { + let raw_source_name = subsource.of_source.as_ref().unwrap(); + let source = match raw_source_name { + RawItemName::Name(name) => { + // Convert the name reference to an ID reference. + let source_item = source_name_to_item.get(name).expect("source must exist"); + RawItemName::Id(source_item.id.to_string(), name.clone(), None) + } + RawItemName::Id(..) => raw_source_name.clone(), + }; + + // The external reference is a `with_option` on subsource statements but is a + // separate field on table statements. + let external_reference_option = subsource + .with_options + .iter() + .find(|o| o.name == CreateSubsourceOptionName::ExternalReference) + .expect("subsources must have external reference"); + let external_reference = match &external_reference_option.value { + Some(WithOptionValue::UnresolvedItemName(name)) => name, + _ => unreachable!("external reference must be an unresolved item name"), + }; + + // The `details` option on both subsources and tables is identical, using the same + // ProtoSourceExportStatementDetails serialized value. + let existing_details = subsource + .with_options + .iter() + .find(|o| o.name == CreateSubsourceOptionName::Details) + .expect("subsources must have details"); + + let mut with_options = vec![TableFromSourceOption { + name: mz_sql::ast::TableFromSourceOptionName::Details, + value: existing_details.value.clone(), + }]; + + // TEXT COLUMNS and EXCLUDE COLUMNS options are stored on each subsource and can be copied + // directly to the table statement. Note that we also store a 'normalized' version of the + // text columns and exclude columns in the `with_options` field of the primary source statement + // for legacy reasons, but that is redundant info and will be removed below when we migrate + // the source statements. + if let Some(text_cols_option) = subsource + .with_options + .iter() + .find(|option| option.name == CreateSubsourceOptionName::TextColumns) + { + with_options.push(TableFromSourceOption { + name: mz_sql::ast::TableFromSourceOptionName::TextColumns, + value: text_cols_option.value.clone(), + }); + } + if let Some(exclude_cols_option) = subsource + .with_options + .iter() + .find(|option| option.name == CreateSubsourceOptionName::ExcludeColumns) + { + with_options.push(TableFromSourceOption { + name: mz_sql::ast::TableFromSourceOptionName::ExcludeColumns, + value: exclude_cols_option.value.clone(), + }); + } + + let table = CreateTableFromSourceStatement { + name: subsource.name.clone(), + constraints: subsource.constraints.clone(), + columns: mz_sql::ast::TableFromSourceColumns::Defined(subsource.columns.clone()), + if_not_exists: subsource.if_not_exists, + source, + external_reference: Some(external_reference.clone()), + with_options, + // Subsources don't have `envelope`, `include_metadata`, or `format` options. + envelope: None, + include_metadata: vec![], + format: None, + }; + + info!( + "migrate: converted subsource {:?} to table {:?}", + subsource, table + ); + *stmt = Statement::CreateTableFromSource(table); + } + _ => (), + } + + Ok(()) +} + // Durable migrations /// Migrations that run only on the durable catalog before any data is loaded into memory. From 4ce1748f64bf5c279e76fcf3a83bf857368e582d Mon Sep 17 00:00:00 2001 From: Roshan Jobanputra Date: Wed, 23 Oct 2024 12:57:26 -0400 Subject: [PATCH 15/50] Add migration logic for source statements --- src/adapter/src/catalog/migrate.rs | 295 ++++++++++++++++++++++++++--- 1 file changed, 273 insertions(+), 22 deletions(-) diff --git a/src/adapter/src/catalog/migrate.rs b/src/adapter/src/catalog/migrate.rs index 888824f0d389a..acf7fa6b5d47d 100644 --- a/src/adapter/src/catalog/migrate.rs +++ b/src/adapter/src/catalog/migrate.rs @@ -29,7 +29,7 @@ fn rewrite_ast_items(tx: &mut Transaction<'_>, mut f: F) -> Result<(), anyhow where F: for<'a> FnMut( &'a mut Transaction<'_>, - CatalogItemId, + &'a mut Item, &'a mut Statement, &'a Vec<(Item, Statement)>, ) -> Result<(), anyhow::Error>, @@ -47,7 +47,7 @@ where let items_with_statements_ref = items_with_statements.clone(); for (mut item, mut stmt) in items_with_statements { let mut stmt = mz_sql::parse::parse(&item.create_sql)?.into_element().ast; - f(tx, item.id, &mut stmt, &items_with_statements_ref)?; + f(tx, &mut item, &mut stmt, &items_with_statements_ref)?; item.create_sql = stmt.to_ast_string_stable(); @@ -107,7 +107,7 @@ pub(crate) async fn migrate( catalog_version ); - rewrite_ast_items(tx, |tx, _id, stmt, all_items_and_statements| { + rewrite_ast_items(tx, |tx, item, stmt, all_items_and_statements| { // Add per-item AST migrations below. // // Each migration should be a function that takes `stmt` (the AST @@ -128,7 +128,7 @@ pub(crate) async fn migrate( })?; if enable_migration { info!("migrate: enable_source_table_migration"); - ast_rewrite_sources_to_tables(stmt, all_items_and_statements)?; + ast_rewrite_sources_to_tables(tx, item, stmt, all_items_and_statements)?; } } Ok(()) @@ -225,18 +225,30 @@ pub(crate) async fn migrate( /// renamed to `_source` while keeping its same GlobalId. /// fn ast_rewrite_sources_to_tables( + tx: &mut Transaction<'_>, + item: &mut Item, stmt: &mut Statement, all_items_and_statements: &Vec<(Item, Statement)>, ) -> Result<(), anyhow::Error> { + use maplit::btreemap; + use maplit::btreeset; + use mz_persist_types::ShardId; + use mz_proto::RustType; use mz_sql::ast::{ - CreateSubsourceOptionName, CreateTableFromSourceStatement, RawItemName, - TableFromSourceOption, WithOptionValue, + CreateSourceConnection, CreateSourceOptionName, CreateSourceStatement, + CreateSubsourceOptionName, CreateTableFromSourceStatement, Ident, LoadGenerator, + MySqlConfigOptionName, PgConfigOptionName, RawItemName, TableFromSourceColumns, + TableFromSourceOption, TableFromSourceOptionName, Value, WithOptionValue, }; + use mz_storage_client::controller::StorageTxn; + use mz_storage_types::sources::load_generator::LoadGeneratorOutput; + use mz_storage_types::sources::SourceExportStatementDetails; + use prost::Message; // Since some subsources have named-only references to their `of_source` and some have the // global_id of their source, we first generate a reference mapping from all source names to - // items so we can correct the references in the `CREATE TABLE .. FROM SOURCE` - // statements to always use the ID of the source. + // items so we can ensure we always use an ID-based reference in the stored + // `CREATE TABLE .. FROM SOURCE` statements. let source_name_to_item: BTreeMap<_, _> = all_items_and_statements .iter() .filter_map(|(item, statement)| match statement { @@ -245,13 +257,34 @@ fn ast_rewrite_sources_to_tables( }) .collect(); + // Collect any existing `CREATE TABLE .. FROM SOURCE` statements by the source they reference. + let tables_by_source: BTreeMap<_, Vec<_>> = all_items_and_statements + .iter() + .filter_map(|(_, statement)| match statement { + Statement::CreateTableFromSource(stmt) => { + let source: GlobalId = match &stmt.source { + RawItemName::Name(_) => unreachable!("tables store source as ID"), + RawItemName::Id(id, _, _) => id.parse().expect("valid id"), + }; + Some((source, stmt)) + } + _ => None, + }) + .fold(BTreeMap::new(), |mut acc, (source, stmt)| { + acc.entry(source).or_insert_with(Vec::new).push(stmt); + acc + }); + match stmt { - // Migrate each 'source export' `CREATE SUBSOURCE` statement to an equivalent - // `CREATE TABLE .. FROM SOURCE` statement. Note that we will not be migrating - // 'progress' `CREATE SUBSOURCE` statements as they are not currently relevant - // to the new table model. - Statement::CreateSubsource(subsource) if subsource.of_source.is_some() => { - let raw_source_name = subsource.of_source.as_ref().unwrap(); + // Migrate each `CREATE SUBSOURCE` statement to an equivalent + // `CREATE TABLE .. FROM SOURCE` statement. + Statement::CreateSubsource(subsource) => { + let raw_source_name = match &subsource.of_source { + // We will not be migrating 'progress' `CREATE SUBSOURCE` statements as they + // are not currently relevant to the new table model. + None => return Ok(()), + Some(name) => name, + }; let source = match raw_source_name { RawItemName::Name(name) => { // Convert the name reference to an ID reference. @@ -282,22 +315,19 @@ fn ast_rewrite_sources_to_tables( .expect("subsources must have details"); let mut with_options = vec![TableFromSourceOption { - name: mz_sql::ast::TableFromSourceOptionName::Details, + name: TableFromSourceOptionName::Details, value: existing_details.value.clone(), }]; // TEXT COLUMNS and EXCLUDE COLUMNS options are stored on each subsource and can be copied - // directly to the table statement. Note that we also store a 'normalized' version of the - // text columns and exclude columns in the `with_options` field of the primary source statement - // for legacy reasons, but that is redundant info and will be removed below when we migrate - // the source statements. + // directly to the table statement. if let Some(text_cols_option) = subsource .with_options .iter() .find(|option| option.name == CreateSubsourceOptionName::TextColumns) { with_options.push(TableFromSourceOption { - name: mz_sql::ast::TableFromSourceOptionName::TextColumns, + name: TableFromSourceOptionName::TextColumns, value: text_cols_option.value.clone(), }); } @@ -307,7 +337,7 @@ fn ast_rewrite_sources_to_tables( .find(|option| option.name == CreateSubsourceOptionName::ExcludeColumns) { with_options.push(TableFromSourceOption { - name: mz_sql::ast::TableFromSourceOptionName::ExcludeColumns, + name: TableFromSourceOptionName::ExcludeColumns, value: exclude_cols_option.value.clone(), }); } @@ -327,11 +357,232 @@ fn ast_rewrite_sources_to_tables( }; info!( - "migrate: converted subsource {:?} to table {:?}", + "migrate: converted subsource {} to table {}", subsource, table ); *stmt = Statement::CreateTableFromSource(table); } + + // Postgres sources are multi-output sources whose subsources are + // migrated above. All we need to do is remove the subsource-related + // options from this statement since they are no longer relevant. + Statement::CreateSource(CreateSourceStatement { + connection: + CreateSourceConnection::Postgres { options, .. } + | CreateSourceConnection::Yugabyte { options, .. }, + .. + }) => { + // This option storing text columns on the primary source statement is redundant + // with the option on subsource statements so can just be removed. + // This was kept for round-tripping of `CREATE SOURCE` statements that automatically + // generated subsources, which is no longer necessary. + if options + .iter() + .any(|o| matches!(o.name, PgConfigOptionName::TextColumns)) + { + options.retain(|o| !matches!(o.name, PgConfigOptionName::TextColumns)); + info!("migrate: converted postgres source {stmt} to remove subsource options"); + } + } + // MySQL sources are multi-output sources whose subsources are + // migrated above. All we need to do is remove the subsource-related + // options from this statement since they are no longer relevant. + Statement::CreateSource(CreateSourceStatement { + connection: CreateSourceConnection::MySql { options, .. }, + .. + }) => { + // These options storing text and exclude columns on the primary source statement + // are redundant with the options on subsource statements so can just be removed. + // They was kept for round-tripping of `CREATE SOURCE` statements that automatically + // generated subsources, which is no longer necessary. + if options.iter().any(|o| { + matches!( + o.name, + MySqlConfigOptionName::TextColumns | MySqlConfigOptionName::ExcludeColumns + ) + }) { + options.retain(|o| { + !matches!( + o.name, + MySqlConfigOptionName::TextColumns | MySqlConfigOptionName::ExcludeColumns + ) + }); + info!("migrate: converted mysql source {stmt} to remove subsource options"); + } + } + // Multi-output load generator sources whose subsources are already + // migrated above. There is no need to remove any options from this + // statement since they are not export-specific. + Statement::CreateSource(CreateSourceStatement { + connection: + CreateSourceConnection::LoadGenerator { + generator: + LoadGenerator::Auction | LoadGenerator::Marketing | LoadGenerator::Tpch, + .. + }, + .. + }) => {} + // Single-output sources that need to be migrated to tables. These sources currently output + // data to the primary collection of the source statement. We will create a new table + // statement for them and move all export-specific options over from the source statement, + // while moving the `CREATE SOURCE` statement to a new name and moving its shard to the + // new table statement. + Statement::CreateSource(CreateSourceStatement { + connection: + conn @ (CreateSourceConnection::Kafka { .. } + | CreateSourceConnection::LoadGenerator { + generator: + LoadGenerator::Clock + | LoadGenerator::Datums + | LoadGenerator::Counter + | LoadGenerator::KeyValue, + .. + }), + name, + col_names, + include_metadata, + format, + envelope, + with_options, + .. + }) => { + // To check if this is a source that has already been migrated we use a basic + // heuristic: if there is at least one existing table for the source, and if + // the envelope/format/include_metadata options are empty, we assume it's + // already been migrated. + if let Some(existing_tables) = tables_by_source.get(&item.id) { + if !existing_tables.is_empty() + && envelope.is_none() + && format.is_none() + && include_metadata.is_empty() + { + return Ok(()); + } + } + + // Use the current source name as the new table name, and rename the source to + // `_source`. This is intended to allow users to continue using + // queries that reference the source name, since they will now need to query the + // table instead. + let new_source_name_ident = Ident::new_unchecked( + name.0.last().expect("at least one ident").to_string() + "_source", + ); + let mut new_source_name = name.clone(); + *new_source_name.0.last_mut().expect("at least one ident") = new_source_name_ident; + let table_name = std::mem::replace(name, new_source_name.clone()); + + // Also update the name of the source 'item' + let mut table_item_name = item.name.clone() + "_source"; + std::mem::swap(&mut item.name, &mut table_item_name); + + // A reference to the source that will be included in the table statement + let source_ref = RawItemName::Id(item.id.to_string(), new_source_name, None); + + let columns = if col_names.is_empty() { + TableFromSourceColumns::NotSpecified + } else { + TableFromSourceColumns::Named(col_names.drain(..).collect()) + }; + + // All source tables must have a `details` option, which is a serialized proto + // describing any source-specific details for this table statement. + let details = match conn { + // For kafka sources this proto is currently empty. + CreateSourceConnection::Kafka { .. } => SourceExportStatementDetails::Kafka {}, + CreateSourceConnection::LoadGenerator { .. } => { + // Since these load generators are single-output we use the default output. + SourceExportStatementDetails::LoadGenerator { + output: LoadGeneratorOutput::Default, + } + } + _ => unreachable!("match determined above"), + }; + let mut table_with_options = vec![TableFromSourceOption { + name: TableFromSourceOptionName::Details, + value: Some(WithOptionValue::Value(Value::String(hex::encode( + details.into_proto().encode_to_vec(), + )))), + }]; + + // Move over the IgnoreKeys option if it exists. + let mut i = 0; + while i < with_options.len() { + if with_options[i].name == CreateSourceOptionName::IgnoreKeys { + let option = with_options.remove(i); + table_with_options.push(TableFromSourceOption { + name: TableFromSourceOptionName::IgnoreKeys, + value: option.value, + }); + } else { + i += 1; + } + } + + // Move over the Timeline option if it exists. + i = 0; + while i < with_options.len() { + if with_options[i].name == CreateSourceOptionName::Timeline { + let option = with_options.remove(i); + table_with_options.push(TableFromSourceOption { + name: TableFromSourceOptionName::Timeline, + value: option.value, + }); + } else { + i += 1; + } + } + + // The new table statement, stealing the export-specific options from the + // create source statement. + let table = CreateTableFromSourceStatement { + name: table_name, + constraints: vec![], + columns, + if_not_exists: false, + source: source_ref, + // external_reference is not required for single-output sources + external_reference: None, + with_options: table_with_options, + envelope: envelope.take(), + include_metadata: include_metadata.drain(..).collect(), + format: format.take(), + }; + + // Insert the new table statement into the catalog with a new id. + let ids = tx.allocate_user_item_ids(1)?; + let new_table_id = ids[0]; + tx.insert_user_item( + new_table_id, + item.schema_id.clone(), + &table_item_name, + table.to_ast_string_stable(), + item.owner_id.clone(), + item.privileges.clone(), + &Default::default(), + )?; + // We need to move the shard currently attached to the source statement to the + // table statement such that the existing data in the shard is preserved and can + // be queried on the new table statement. However, we need to keep the GlobalId of + // the source the same, to preserve existing references to that statement in + // external tools such as DBT and Terraform. We will insert a new shard for the source + // statement which will be automatically created after the migration is complete. + let new_source_shard = ShardId::new(); + let (source_id, existing_source_shard) = tx + .delete_collection_metadata(btreeset! {item.id}) + .pop() + .expect("shard should exist"); + tx.insert_collection_metadata(btreemap! { + new_table_id => existing_source_shard, + source_id => new_source_shard + })?; + + info!("migrate: updated source {stmt} and added table {table}"); + } + + // When we upgrade to > rust 1.81 we should use #[expect(unreachable_patterns)] + // to enforce that we have covered all CreateSourceStatement variants. + #[allow(unreachable_patterns)] + Statement::CreateSource(_) => {} _ => (), } From f9b3c0125454ed5d8a91948f021e00c027928f6a Mon Sep 17 00:00:00 2001 From: Roshan Jobanputra Date: Wed, 23 Oct 2024 14:08:31 -0400 Subject: [PATCH 16/50] Rename the feature flag --- src/adapter-types/src/dyncfgs.rs | 6 ++++-- src/adapter/src/catalog/migrate.rs | 6 +++--- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/src/adapter-types/src/dyncfgs.rs b/src/adapter-types/src/dyncfgs.rs index 1ddd317c835fb..14ca83651565f 100644 --- a/src/adapter-types/src/dyncfgs.rs +++ b/src/adapter-types/src/dyncfgs.rs @@ -112,8 +112,9 @@ pub const ENABLE_EXPRESSION_CACHE: Config = Config::new( ); /// Whether to enable the migration to convert all sources to use tables. -pub const ENABLE_SOURCE_TABLE_MIGRATION: Config = Config::new( - "enable_source_table_migration", +/// TODO(#8678): This should also disable usage of the old source syntax. +pub const FORCE_SOURCE_TABLE_SYNTAX: Config = Config::new( + "force_source_table_syntax", false, "Whether to enable the migration to convert all sources to use tables.", ); @@ -136,4 +137,5 @@ pub fn all_dyncfgs(configs: ConfigSet) -> ConfigSet { .add(&ENABLE_CONTINUAL_TASK_BUILTINS) .add(&ENABLE_EXPRESSION_CACHE) .add(&ENABLE_SOURCE_TABLE_MIGRATION) + .add(&FORCE_SOURCE_TABLE_SYNTAX) } diff --git a/src/adapter/src/catalog/migrate.rs b/src/adapter/src/catalog/migrate.rs index acf7fa6b5d47d..abbf5653a42a5 100644 --- a/src/adapter/src/catalog/migrate.rs +++ b/src/adapter/src/catalog/migrate.rs @@ -119,15 +119,15 @@ pub(crate) async fn migrate( // Special block for `ast_rewrite_sources_to_tables` migration // since it requires a feature flag. - if let Some(config_val) = tx.get_system_config("enable_source_table_migration") { + if let Some(config_val) = tx.get_system_config("force_source_table_syntax") { let enable_migration = config_val.parse::().map_err(|e| { anyhow::anyhow!( - "could not parse enable_source_table_migration config value: {}", + "could not parse force_source_table_syntax config value: {}", e ) })?; if enable_migration { - info!("migrate: enable_source_table_migration"); + info!("migrate: force_source_table_syntax"); ast_rewrite_sources_to_tables(tx, item, stmt, all_items_and_statements)?; } } From 1094734de8f444a2bc7d2a8cac75264b73395f7f Mon Sep 17 00:00:00 2001 From: Roshan Jobanputra Date: Wed, 23 Oct 2024 16:24:18 -0400 Subject: [PATCH 17/50] Add new table to audit log --- src/adapter/src/catalog/migrate.rs | 40 ++++++++++++++++++++++++-- src/catalog/src/durable/transaction.rs | 7 +++++ 2 files changed, 45 insertions(+), 2 deletions(-) diff --git a/src/adapter/src/catalog/migrate.rs b/src/adapter/src/catalog/migrate.rs index abbf5653a42a5..155c42d8b478f 100644 --- a/src/adapter/src/catalog/migrate.rs +++ b/src/adapter/src/catalog/migrate.rs @@ -93,7 +93,7 @@ pub(crate) async fn migrate( tx: &mut Transaction<'_>, local_expr_cache: &mut LocalExpressionCache, item_updates: Vec, - _now: NowFn, + now: NowFn, _boot_ts: Timestamp, ) -> Result>, anyhow::Error> { let catalog_version = tx.get_catalog_content_version(); @@ -108,6 +108,7 @@ pub(crate) async fn migrate( ); rewrite_ast_items(tx, |tx, item, stmt, all_items_and_statements| { + let now = now.clone(); // Add per-item AST migrations below. // // Each migration should be a function that takes `stmt` (the AST @@ -128,7 +129,7 @@ pub(crate) async fn migrate( })?; if enable_migration { info!("migrate: force_source_table_syntax"); - ast_rewrite_sources_to_tables(tx, item, stmt, all_items_and_statements)?; + ast_rewrite_sources_to_tables(tx, now, item, stmt, all_items_and_statements)?; } } Ok(()) @@ -226,6 +227,7 @@ pub(crate) async fn migrate( /// fn ast_rewrite_sources_to_tables( tx: &mut Transaction<'_>, + now: NowFn, item: &mut Item, stmt: &mut Statement, all_items_and_statements: &Vec<(Item, Statement)>, @@ -576,6 +578,26 @@ fn ast_rewrite_sources_to_tables( source_id => new_source_shard })?; + let schema = tx.get_schema(&item.schema_id).expect("schema must exist"); + + add_to_audit_log( + tx, + mz_audit_log::EventType::Create, + mz_audit_log::ObjectType::Table, + mz_audit_log::EventDetails::IdFullNameV1(mz_audit_log::IdFullNameV1 { + id: new_table_id.to_string(), + name: mz_audit_log::FullNameV1 { + database: schema + .database_id + .map(|d| d.to_string()) + .unwrap_or_default(), + schema: schema.name, + item: table_item_name, + }, + }), + now(), + )?; + info!("migrate: updated source {stmt} and added table {table}"); } @@ -609,3 +631,17 @@ pub(crate) fn durable_migrate( // // Please include the adapter team on any code reviews that add or edit // migrations. + +fn add_to_audit_log( + tx: &mut Transaction, + event_type: mz_audit_log::EventType, + object_type: mz_audit_log::ObjectType, + details: mz_audit_log::EventDetails, + occurred_at: mz_ore::now::EpochMillis, +) -> Result<(), anyhow::Error> { + let id = tx.get_and_increment_id(mz_catalog::durable::AUDIT_LOG_ID_ALLOC_KEY.to_string())?; + let event = + mz_audit_log::VersionedEvent::new(id, event_type, object_type, details, None, occurred_at); + tx.insert_audit_log_event(event); + Ok(()) +} diff --git a/src/catalog/src/durable/transaction.rs b/src/catalog/src/durable/transaction.rs index 81ec6e9cb65aa..4bdc8df768a21 100644 --- a/src/catalog/src/durable/transaction.rs +++ b/src/catalog/src/durable/transaction.rs @@ -2050,6 +2050,13 @@ impl<'a> Transaction<'a> { .map(|(k, v)| DurableType::from_key_value(k, v)) } + pub fn get_schema(&self, id: &SchemaId) -> Option { + let key = SchemaKey { id: *id }; + self.schemas + .get(&key) + .map(|v| DurableType::from_key_value(key, v.clone())) + } + pub fn get_introspection_source_indexes( &self, cluster_id: ClusterId, From aa9d33c41ac567553e7974689c7b3563ccdf94ce Mon Sep 17 00:00:00 2001 From: Roshan Jobanputra Date: Wed, 23 Oct 2024 16:37:47 -0400 Subject: [PATCH 18/50] Add platform check scenario to test migration --- .../checks/all_checks/source_tables.py | 1 - .../materialize/checks/all_checks/upsert.py | 56 +++++++++++++++++++ .../materialize/checks/scenarios_upgrade.py | 46 +++++++++++++++ 3 files changed, 102 insertions(+), 1 deletion(-) diff --git a/misc/python/materialize/checks/all_checks/source_tables.py b/misc/python/materialize/checks/all_checks/source_tables.py index 279ede6a4022e..c36bb1eb8af12 100644 --- a/misc/python/materialize/checks/all_checks/source_tables.py +++ b/misc/python/materialize/checks/all_checks/source_tables.py @@ -28,7 +28,6 @@ class TableFromPgSource(TableFromSourceBase): suffix = "tbl_from_pg_source" def initialize(self) -> Testdrive: - return Testdrive( self.generic_setup() + dedent( diff --git a/misc/python/materialize/checks/all_checks/upsert.py b/misc/python/materialize/checks/all_checks/upsert.py index dd7f92caf831e..739bc9d5effa9 100644 --- a/misc/python/materialize/checks/all_checks/upsert.py +++ b/misc/python/materialize/checks/all_checks/upsert.py @@ -164,3 +164,59 @@ def validate(self) -> Testdrive: """ ) ) + + +class UpsertLegacy(Check): + """ + An upsert source test that uses the legacy syntax to create the source + on all versions to ensure the source is properly migrated with the + ActivateSourceVersioningMigration scenario + """ + + def initialize(self) -> Testdrive: + return Testdrive( + schemas() + + dedent( + """ + $ kafka-create-topic topic=upsert-legacy-syntax + + $ kafka-ingest format=avro key-format=avro topic=upsert-legacy-syntax key-schema=${keyschema} schema=${schema} repeat=10000 + {"key1": "A${kafka-ingest.iteration}"} {"f1": "A${kafka-ingest.iteration}"} + + > CREATE SOURCE upsert_insert + FROM KAFKA CONNECTION kafka_conn (TOPIC 'testdrive-upsert-legacy-syntax-${testdrive.seed}') + FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn + ENVELOPE UPSERT + + > CREATE MATERIALIZED VIEW upsert_insert_view AS SELECT COUNT(DISTINCT key1 || ' ' || f1) FROM upsert_insert; + """ + ) + ) + + def manipulate(self) -> list[Testdrive]: + return [ + Testdrive(schemas() + dedent(s)) + for s in [ + """ + $ kafka-ingest format=avro key-format=avro topic=upsert-legacy-syntax key-schema=${keyschema} schema=${schema} repeat=10000 + {"key1": "A${kafka-ingest.iteration}"} {"f1": "A${kafka-ingest.iteration}"} + """, + """ + $ kafka-ingest format=avro key-format=avro topic=upsert-legacy-syntax key-schema=${keyschema} schema=${schema} repeat=10000 + {"key1": "A${kafka-ingest.iteration}"} {"f1": "A${kafka-ingest.iteration}"} + """, + ] + ] + + def validate(self) -> Testdrive: + return Testdrive( + dedent( + """ + > SELECT COUNT(*), COUNT(DISTINCT key1), COUNT(DISTINCT f1) FROM upsert_insert + 10000 10000 10000 + + > SELECT * FROM upsert_insert_view; + 10000 + """ + ) + ) diff --git a/misc/python/materialize/checks/scenarios_upgrade.py b/misc/python/materialize/checks/scenarios_upgrade.py index 7411b7b7484f8..26aee25584f89 100644 --- a/misc/python/materialize/checks/scenarios_upgrade.py +++ b/misc/python/materialize/checks/scenarios_upgrade.py @@ -383,3 +383,49 @@ def actions(self) -> list[Action]: ), Validate(self), ] + + +class ActivateSourceVersioningMigration(Scenario): + """ + Starts MZ, initializes and manipulates, then forces the migration + of sources to the new table model (introducing Source Versioning). + """ + + def base_version(self) -> MzVersion: + return get_last_version() + + def actions(self) -> list[Action]: + print(f"Upgrading from tag {self.base_version()}") + return [ + StartMz( + self, + tag=self.base_version(), + ), + Initialize(self), + Manipulate(self, phase=1), + KillMz( + capture_logs=True + ), # We always use True here otherwise docker-compose will lose the pre-upgrade logs + StartMz( + self, + tag=None, + # Activate the `force_source_table_syntax` flag + # which should trigger the migration of sources + # using the old syntax to the new table model. + additional_system_parameter_defaults={ + "force_source_table_syntax": "true", + }, + ), + Manipulate(self, phase=2), + Validate(self), + # A second restart while already on the new version + KillMz(capture_logs=True), + StartMz( + self, + tag=None, + additional_system_parameter_defaults={ + "force_source_table_syntax": "true", + }, + ), + Validate(self), + ] From b8105986ed9e5ef04af25f9adc4c813e6df6db1d Mon Sep 17 00:00:00 2001 From: Roshan Jobanputra Date: Thu, 24 Oct 2024 10:00:47 -0400 Subject: [PATCH 19/50] Switch to using system vars instead of flags to allow console access to values m --- src/adapter-types/src/dyncfgs.rs | 9 --------- src/adapter/src/catalog/migrate.rs | 16 +++++----------- 2 files changed, 5 insertions(+), 20 deletions(-) diff --git a/src/adapter-types/src/dyncfgs.rs b/src/adapter-types/src/dyncfgs.rs index 14ca83651565f..0ab56c9b41144 100644 --- a/src/adapter-types/src/dyncfgs.rs +++ b/src/adapter-types/src/dyncfgs.rs @@ -111,14 +111,6 @@ pub const ENABLE_EXPRESSION_CACHE: Config = Config::new( "Use a cache to store optimized expressions to help speed up start times.", ); -/// Whether to enable the migration to convert all sources to use tables. -/// TODO(#8678): This should also disable usage of the old source syntax. -pub const FORCE_SOURCE_TABLE_SYNTAX: Config = Config::new( - "force_source_table_syntax", - false, - "Whether to enable the migration to convert all sources to use tables.", -); - /// Adds the full set of all compute `Config`s. pub fn all_dyncfgs(configs: ConfigSet) -> ConfigSet { configs @@ -137,5 +129,4 @@ pub fn all_dyncfgs(configs: ConfigSet) -> ConfigSet { .add(&ENABLE_CONTINUAL_TASK_BUILTINS) .add(&ENABLE_EXPRESSION_CACHE) .add(&ENABLE_SOURCE_TABLE_MIGRATION) - .add(&FORCE_SOURCE_TABLE_SYNTAX) } diff --git a/src/adapter/src/catalog/migrate.rs b/src/adapter/src/catalog/migrate.rs index 155c42d8b478f..923c733b9c395 100644 --- a/src/adapter/src/catalog/migrate.rs +++ b/src/adapter/src/catalog/migrate.rs @@ -107,6 +107,8 @@ pub(crate) async fn migrate( catalog_version ); + let enable_source_table_migration = state.system_config().force_source_table_syntax(); + rewrite_ast_items(tx, |tx, item, stmt, all_items_and_statements| { let now = now.clone(); // Add per-item AST migrations below. @@ -118,20 +120,12 @@ pub(crate) async fn migrate( // Migration functions may also take `tx` as input to stage // arbitrary changes to the catalog. - // Special block for `ast_rewrite_sources_to_tables` migration - // since it requires a feature flag. - if let Some(config_val) = tx.get_system_config("force_source_table_syntax") { - let enable_migration = config_val.parse::().map_err(|e| { - anyhow::anyhow!( - "could not parse force_source_table_syntax config value: {}", - e - ) - })?; - if enable_migration { + // Special block for `ast_rewrite_sources_to_tables` migration + // since it requires a feature flag. + if enable_source_table_migration { info!("migrate: force_source_table_syntax"); ast_rewrite_sources_to_tables(tx, now, item, stmt, all_items_and_statements)?; } - } Ok(()) })?; From a9ac9dc02ce120b8a3514e000000f8f18b91c59d Mon Sep 17 00:00:00 2001 From: Roshan Jobanputra Date: Thu, 24 Oct 2024 14:37:21 -0400 Subject: [PATCH 20/50] Fixes to migration based on testing --- src/adapter/src/catalog/migrate.rs | 916 +++++++++++------- src/adapter/src/catalog/open.rs | 30 +- src/sql-parser/src/ast/defs/ddl.rs | 18 + src/sql/src/names.rs | 39 + .../src/sources/load_generator.rs | 2 + 5 files changed, 624 insertions(+), 381 deletions(-) diff --git a/src/adapter/src/catalog/migrate.rs b/src/adapter/src/catalog/migrate.rs index 923c733b9c395..ff21c13ce89fb 100644 --- a/src/adapter/src/catalog/migrate.rs +++ b/src/adapter/src/catalog/migrate.rs @@ -11,11 +11,12 @@ use std::collections::BTreeMap; use mz_catalog::builtin::BuiltinTable; use mz_catalog::durable::{Transaction, Item}; -use mz_catalog::memory::objects::StateUpdate; +use mz_catalog::memory::objects::{StateUpdate,BootstrapStateUpdateKind}; use mz_ore::collections::CollectionExt; use mz_ore::now::NowFn; use mz_repr::{CatalogItemId, Timestamp}; use mz_sql::ast::display::AstDisplay; +use mz_sql::names::FullItemName; use mz_sql_parser::ast::{Raw, Statement}; use semver::Version; use tracing::info; @@ -29,26 +30,15 @@ fn rewrite_ast_items(tx: &mut Transaction<'_>, mut f: F) -> Result<(), anyhow where F: for<'a> FnMut( &'a mut Transaction<'_>, - &'a mut Item, + GlobalId, &'a mut Statement, - &'a Vec<(Item, Statement)>, ) -> Result<(), anyhow::Error>, { let mut updated_items = BTreeMap::new(); - let items_with_statements = tx - .get_items() - .map(|item| { - let stmt = mz_sql::parse::parse(&item.create_sql)?.into_element().ast; - Ok((item, stmt)) - }) - .collect::, anyhow::Error>>()?; - // Clone this vec to be referenced within the closure if needed - let items_with_statements_ref = items_with_statements.clone(); - for (mut item, mut stmt) in items_with_statements { + for mut item in tx.get_items() { let mut stmt = mz_sql::parse::parse(&item.create_sql)?.into_element().ast; - f(tx, &mut item, &mut stmt, &items_with_statements_ref)?; - + f(tx, item.id, &mut stmt).await?; item.create_sql = stmt.to_ast_string_stable(); updated_items.insert(item.id, item); @@ -85,6 +75,11 @@ where Ok(()) } +pub(crate) struct MigrateResult { + pub(crate) builtin_table_updates: Vec>, + pub(crate) post_item_updates: Vec<(BootstrapStateUpdateKind, Timestamp, i64)>, +} + /// Migrates all user items and loads them into `state`. /// /// Returns the builtin updates corresponding to all user items. @@ -95,7 +90,7 @@ pub(crate) async fn migrate( item_updates: Vec, now: NowFn, _boot_ts: Timestamp, -) -> Result>, anyhow::Error> { +) -> Result { let catalog_version = tx.get_catalog_content_version(); let catalog_version = match catalog_version { Some(v) => Version::parse(&v)?, @@ -107,7 +102,11 @@ pub(crate) async fn migrate( catalog_version ); - let enable_source_table_migration = state.system_config().force_source_table_syntax(); + // Special block for `ast_rewrite_sources_to_tables` migration + // since it requires a feature flag needs to update multiple AST items at once. + if state.system_config().force_source_table_syntax() { + ast_rewrite_sources_to_tables(tx, now)?; + } rewrite_ast_items(tx, |tx, item, stmt, all_items_and_statements| { let now = now.clone(); @@ -120,12 +119,6 @@ pub(crate) async fn migrate( // Migration functions may also take `tx` as input to stage // arbitrary changes to the catalog. - // Special block for `ast_rewrite_sources_to_tables` migration - // since it requires a feature flag. - if enable_source_table_migration { - info!("migrate: force_source_table_syntax"); - ast_rewrite_sources_to_tables(tx, now, item, stmt, all_items_and_statements)?; - } Ok(()) })?; @@ -137,6 +130,18 @@ pub(crate) async fn migrate( let op_item_updates = into_consolidatable_updates_startup(op_item_updates, commit_ts); item_updates.extend(op_item_updates); differential_dataflow::consolidation::consolidate_updates(&mut item_updates); + + // Since some migrations might introduce non-item 'post-item' updates, we sequester those + // so they can be applied with other post-item updates after migrations to avoid + // accumulating negative diffs. + let (post_item_updates, item_updates): (Vec<_>, Vec<_>) = item_updates + .into_iter() + // The only post-item update kind we currently generate is to + // update storage collection metadata. + .partition(|(kind, _, _)| { + matches!(kind, BootstrapStateUpdateKind::StorageCollectionMetadata(_)) + }); + let item_updates = item_updates .into_iter() .map(|(kind, ts, diff)| StateUpdate { @@ -188,7 +193,10 @@ pub(crate) async fn migrate( "migration from catalog version {:?} complete", catalog_version ); - Ok(ast_builtin_table_updates) + Ok(MigrateResult { + builtin_table_updates: ast_builtin_table_updates, + post_item_updates, + }) } // Add new migrations below their appropriate heading, and precede them with a @@ -208,23 +216,20 @@ pub(crate) async fn migrate( /// MySQL, and multi-output (tpch, auction, marketing) load-generator subsources. /// /// Second we migrate existing `CREATE SOURCE` statements for these multi-output -/// sources to remove their subsource qualification option (e.g. FOR ALL TABLES..) -/// and any subsource-specific options (e.g. TEXT COLUMNS). +/// sources to remove any subsource-specific options (e.g. TEXT COLUMNS). /// -/// Third we migrate existing `CREATE SOURCE` statements that refer to a single -/// output collection. This includes existing Kafka and single-output load-generator -/// subsources. These statements will generate an additional `CREATE TABLE .. FROM SOURCE` -/// statement that copies over all the export-specific options. This statement will be tied -/// to the existing statement's persist collection, but using a new GlobalID. The original -/// source statement will be updated to remove the export-specific options and will be -/// renamed to `_source` while keeping its same GlobalId. +/// Third we migrate existing single-output `CREATE SOURCE` statements. +/// This includes existing Kafka and single-output load-generator +/// subsources. This will generate an additional `CREATE TABLE .. FROM SOURCE` +/// statement that copies over all the export-specific options. This table will use +/// to the existing source statement's persist shard but use a new GlobalID. +/// The original source statement will be updated to remove the export-specific options, +/// renamed to `_source`, and use a new empty shard while keeping its +/// same GlobalId. /// fn ast_rewrite_sources_to_tables( tx: &mut Transaction<'_>, now: NowFn, - item: &mut Item, - stmt: &mut Statement, - all_items_and_statements: &Vec<(Item, Statement)>, ) -> Result<(), anyhow::Error> { use maplit::btreemap; use maplit::btreeset; @@ -232,374 +237,543 @@ fn ast_rewrite_sources_to_tables( use mz_proto::RustType; use mz_sql::ast::{ CreateSourceConnection, CreateSourceOptionName, CreateSourceStatement, - CreateSubsourceOptionName, CreateTableFromSourceStatement, Ident, LoadGenerator, - MySqlConfigOptionName, PgConfigOptionName, RawItemName, TableFromSourceColumns, - TableFromSourceOption, TableFromSourceOptionName, Value, WithOptionValue, + CreateSubsourceOptionName, CreateSubsourceStatement, CreateTableFromSourceStatement, Ident, + KafkaSourceConfigOptionName, LoadGenerator, MySqlConfigOptionName, PgConfigOptionName, + RawItemName, TableFromSourceColumns, TableFromSourceOption, TableFromSourceOptionName, + UnresolvedItemName, Value, WithOptionValue, }; use mz_storage_client::controller::StorageTxn; use mz_storage_types::sources::load_generator::LoadGeneratorOutput; use mz_storage_types::sources::SourceExportStatementDetails; use prost::Message; - // Since some subsources have named-only references to their `of_source` and some have the - // global_id of their source, we first generate a reference mapping from all source names to - // items so we can ensure we always use an ID-based reference in the stored - // `CREATE TABLE .. FROM SOURCE` statements. - let source_name_to_item: BTreeMap<_, _> = all_items_and_statements - .iter() - .filter_map(|(item, statement)| match statement { - Statement::CreateSource(stmt) => Some((stmt.name.clone(), item)), - _ => None, + let items_with_statements = tx + .get_items() + .map(|item| { + let stmt = mz_sql::parse::parse(&item.create_sql)?.into_element().ast; + Ok((item, stmt)) }) - .collect(); + .collect::, anyhow::Error>>()?; + let items_with_statements_copied = items_with_statements.clone(); + + // Any GlobalId that should be changed to a new GlobalId in any statements that + // reference it. This is necessary for ensuring downstream statements (e.g. + // mat views, indexes) that reference a single-output source (e.g. kafka) + // will now reference the corresponding new table, with the same data, instead. + let mut changed_ids = BTreeMap::new(); + + for (mut item, stmt) in items_with_statements { + match stmt { + // Migrate each `CREATE SUBSOURCE` statement to an equivalent + // `CREATE TABLE .. FROM SOURCE` statement. + Statement::CreateSubsource(CreateSubsourceStatement { + name, + columns, + constraints, + of_source, + if_not_exists, + mut with_options, + }) => { + let raw_source_name = match of_source { + // If `of_source` is None then this is a `progress` subsource which we + // are not migrating as they are not currently relevant to the new table model. + None => return Ok(()), + Some(name) => name, + }; + let source = match raw_source_name { + // Some legacy subsources have named-only references to their `of_source` + // so we ensure we always use an ID-based reference in the stored + // `CREATE TABLE .. FROM SOURCE` statements. + RawItemName::Name(name) => { + // Convert the name reference to an ID reference. + let (source_item, _) = items_with_statements_copied + .iter() + .find(|(_, statement)| match statement { + Statement::CreateSource(stmt) => stmt.name == name, + _ => false, + }) + .expect("source must exist"); + RawItemName::Id(source_item.id.to_string(), name, None) + } + RawItemName::Id(..) => raw_source_name, + }; - // Collect any existing `CREATE TABLE .. FROM SOURCE` statements by the source they reference. - let tables_by_source: BTreeMap<_, Vec<_>> = all_items_and_statements - .iter() - .filter_map(|(_, statement)| match statement { - Statement::CreateTableFromSource(stmt) => { - let source: GlobalId = match &stmt.source { - RawItemName::Name(_) => unreachable!("tables store source as ID"), - RawItemName::Id(id, _, _) => id.parse().expect("valid id"), + // The external reference is a `with_option` on subsource statements but is a + // separate field on table statements. + let mut i = 0; + let external_reference = loop { + if i >= with_options.len() { + panic!("subsource must have an external reference"); + } + if with_options[i].name == CreateSubsourceOptionName::ExternalReference { + let option = with_options.remove(i); + match option.value { + Some(WithOptionValue::UnresolvedItemName(name)) => break name, + _ => unreachable!("external reference must be an unresolved item name"), + }; + } else { + i += 1; + } + }; + let with_options = with_options + .into_iter() + .map(|option| { + match option.name { + CreateSubsourceOptionName::Details => TableFromSourceOption { + name: TableFromSourceOptionName::Details, + // The `details` option on both subsources and tables is identical, using the same + // ProtoSourceExportStatementDetails serialized value. + value: option.value, + }, + CreateSubsourceOptionName::TextColumns => TableFromSourceOption { + name: TableFromSourceOptionName::TextColumns, + value: option.value, + }, + CreateSubsourceOptionName::ExcludeColumns => TableFromSourceOption { + name: TableFromSourceOptionName::ExcludeColumns, + value: option.value, + }, + CreateSubsourceOptionName::Progress => { + panic!("progress option should not exist on this subsource") + } + CreateSubsourceOptionName::ExternalReference => { + // This option is handled separately above. + unreachable!() + } + } + }) + .collect::>(); + + let table = CreateTableFromSourceStatement { + name, + constraints, + columns: mz_sql::ast::TableFromSourceColumns::Defined(columns), + if_not_exists, + source, + external_reference: Some(external_reference.clone()), + with_options, + // Subsources don't have `envelope`, `include_metadata`, or `format` options. + envelope: None, + include_metadata: vec![], + format: None, }; - Some((source, stmt)) - } - _ => None, - }) - .fold(BTreeMap::new(), |mut acc, (source, stmt)| { - acc.entry(source).or_insert_with(Vec::new).push(stmt); - acc - }); - match stmt { - // Migrate each `CREATE SUBSOURCE` statement to an equivalent - // `CREATE TABLE .. FROM SOURCE` statement. - Statement::CreateSubsource(subsource) => { - let raw_source_name = match &subsource.of_source { - // We will not be migrating 'progress' `CREATE SUBSOURCE` statements as they - // are not currently relevant to the new table model. - None => return Ok(()), - Some(name) => name, - }; - let source = match raw_source_name { - RawItemName::Name(name) => { - // Convert the name reference to an ID reference. - let source_item = source_name_to_item.get(name).expect("source must exist"); - RawItemName::Id(source_item.id.to_string(), name.clone(), None) - } - RawItemName::Id(..) => raw_source_name.clone(), - }; - - // The external reference is a `with_option` on subsource statements but is a - // separate field on table statements. - let external_reference_option = subsource - .with_options - .iter() - .find(|o| o.name == CreateSubsourceOptionName::ExternalReference) - .expect("subsources must have external reference"); - let external_reference = match &external_reference_option.value { - Some(WithOptionValue::UnresolvedItemName(name)) => name, - _ => unreachable!("external reference must be an unresolved item name"), - }; - - // The `details` option on both subsources and tables is identical, using the same - // ProtoSourceExportStatementDetails serialized value. - let existing_details = subsource - .with_options - .iter() - .find(|o| o.name == CreateSubsourceOptionName::Details) - .expect("subsources must have details"); - - let mut with_options = vec![TableFromSourceOption { - name: TableFromSourceOptionName::Details, - value: existing_details.value.clone(), - }]; - - // TEXT COLUMNS and EXCLUDE COLUMNS options are stored on each subsource and can be copied - // directly to the table statement. - if let Some(text_cols_option) = subsource - .with_options - .iter() - .find(|option| option.name == CreateSubsourceOptionName::TextColumns) - { - with_options.push(TableFromSourceOption { - name: TableFromSourceOptionName::TextColumns, - value: text_cols_option.value.clone(), - }); - } - if let Some(exclude_cols_option) = subsource - .with_options - .iter() - .find(|option| option.name == CreateSubsourceOptionName::ExcludeColumns) - { - with_options.push(TableFromSourceOption { - name: TableFromSourceOptionName::ExcludeColumns, - value: exclude_cols_option.value.clone(), - }); + info!( + "migrate: converted subsource {} to table {}", + item.create_sql, table + ); + item.create_sql = Statement::CreateTableFromSource(table).to_ast_string_stable(); + tx.update_item(item.id, item)?; } - let table = CreateTableFromSourceStatement { - name: subsource.name.clone(), - constraints: subsource.constraints.clone(), - columns: mz_sql::ast::TableFromSourceColumns::Defined(subsource.columns.clone()), - if_not_exists: subsource.if_not_exists, - source, - external_reference: Some(external_reference.clone()), + // Postgres sources are multi-output sources whose subsources are + // migrated above. All we need to do is remove the subsource-related + // options from this statement since they are no longer relevant. + Statement::CreateSource(CreateSourceStatement { + connection: + mut conn @ (CreateSourceConnection::Postgres { .. } + | CreateSourceConnection::Yugabyte { .. }), + name, + if_not_exists, + in_cluster, + include_metadata, + format, + envelope, + col_names, with_options, - // Subsources don't have `envelope`, `include_metadata`, or `format` options. - envelope: None, - include_metadata: vec![], - format: None, - }; - - info!( - "migrate: converted subsource {} to table {}", - subsource, table - ); - *stmt = Statement::CreateTableFromSource(table); - } - - // Postgres sources are multi-output sources whose subsources are - // migrated above. All we need to do is remove the subsource-related - // options from this statement since they are no longer relevant. - Statement::CreateSource(CreateSourceStatement { - connection: - CreateSourceConnection::Postgres { options, .. } - | CreateSourceConnection::Yugabyte { options, .. }, - .. - }) => { - // This option storing text columns on the primary source statement is redundant - // with the option on subsource statements so can just be removed. - // This was kept for round-tripping of `CREATE SOURCE` statements that automatically - // generated subsources, which is no longer necessary. - if options - .iter() - .any(|o| matches!(o.name, PgConfigOptionName::TextColumns)) - { - options.retain(|o| !matches!(o.name, PgConfigOptionName::TextColumns)); - info!("migrate: converted postgres source {stmt} to remove subsource options"); + key_constraint, + external_references, + progress_subsource, + }) => { + let options = match &mut conn { + CreateSourceConnection::Postgres { options, .. } => options, + CreateSourceConnection::Yugabyte { options, .. } => options, + _ => unreachable!("match determined above"), + }; + // This option storing text columns on the primary source statement is redundant + // with the option on subsource statements so can just be removed. + // This was kept for round-tripping of `CREATE SOURCE` statements that automatically + // generated subsources, which is no longer necessary. + if options + .iter() + .any(|o| matches!(o.name, PgConfigOptionName::TextColumns)) + { + options.retain(|o| !matches!(o.name, PgConfigOptionName::TextColumns)); + let stmt = Statement::CreateSource(CreateSourceStatement { + connection: conn, + name, + if_not_exists, + in_cluster, + include_metadata, + format, + envelope, + col_names, + with_options, + key_constraint, + external_references, + progress_subsource, + }); + item.create_sql = stmt.to_ast_string_stable(); + tx.update_item(item.id, item)?; + info!("migrate: converted postgres source {stmt} to remove subsource options"); + } } - } - // MySQL sources are multi-output sources whose subsources are - // migrated above. All we need to do is remove the subsource-related - // options from this statement since they are no longer relevant. - Statement::CreateSource(CreateSourceStatement { - connection: CreateSourceConnection::MySql { options, .. }, - .. - }) => { - // These options storing text and exclude columns on the primary source statement - // are redundant with the options on subsource statements so can just be removed. - // They was kept for round-tripping of `CREATE SOURCE` statements that automatically - // generated subsources, which is no longer necessary. - if options.iter().any(|o| { - matches!( - o.name, - MySqlConfigOptionName::TextColumns | MySqlConfigOptionName::ExcludeColumns - ) - }) { - options.retain(|o| { - !matches!( + // MySQL sources are multi-output sources whose subsources are + // migrated above. All we need to do is remove the subsource-related + // options from this statement since they are no longer relevant. + Statement::CreateSource(CreateSourceStatement { + connection: mut conn @ CreateSourceConnection::MySql { .. }, + name, + if_not_exists, + in_cluster, + include_metadata, + format, + envelope, + col_names, + with_options, + key_constraint, + external_references, + progress_subsource, + .. + }) => { + let options = match &mut conn { + CreateSourceConnection::MySql { options, .. } => options, + _ => unreachable!("match determined above"), + }; + // These options storing text and exclude columns on the primary source statement + // are redundant with the options on subsource statements so can just be removed. + // They was kept for round-tripping of `CREATE SOURCE` statements that automatically + // generated subsources, which is no longer necessary. + if options.iter().any(|o| { + matches!( o.name, MySqlConfigOptionName::TextColumns | MySqlConfigOptionName::ExcludeColumns ) - }); - info!("migrate: converted mysql source {stmt} to remove subsource options"); + }) { + options.retain(|o| { + !matches!( + o.name, + MySqlConfigOptionName::TextColumns + | MySqlConfigOptionName::ExcludeColumns + ) + }); + let stmt = Statement::CreateSource(CreateSourceStatement { + connection: conn, + name, + if_not_exists, + in_cluster, + include_metadata, + format, + envelope, + col_names, + with_options, + key_constraint, + external_references, + progress_subsource, + }); + item.create_sql = stmt.to_ast_string_stable(); + tx.update_item(item.id, item)?; + info!("migrate: converted mysql source {stmt} to remove subsource options"); + } } - } - // Multi-output load generator sources whose subsources are already - // migrated above. There is no need to remove any options from this - // statement since they are not export-specific. - Statement::CreateSource(CreateSourceStatement { - connection: - CreateSourceConnection::LoadGenerator { - generator: - LoadGenerator::Auction | LoadGenerator::Marketing | LoadGenerator::Tpch, - .. - }, - .. - }) => {} - // Single-output sources that need to be migrated to tables. These sources currently output - // data to the primary collection of the source statement. We will create a new table - // statement for them and move all export-specific options over from the source statement, - // while moving the `CREATE SOURCE` statement to a new name and moving its shard to the - // new table statement. - Statement::CreateSource(CreateSourceStatement { - connection: - conn @ (CreateSourceConnection::Kafka { .. } - | CreateSourceConnection::LoadGenerator { - generator: - LoadGenerator::Clock - | LoadGenerator::Datums - | LoadGenerator::Counter - | LoadGenerator::KeyValue, - .. - }), - name, - col_names, - include_metadata, - format, - envelope, - with_options, - .. - }) => { - // To check if this is a source that has already been migrated we use a basic - // heuristic: if there is at least one existing table for the source, and if - // the envelope/format/include_metadata options are empty, we assume it's - // already been migrated. - if let Some(existing_tables) = tables_by_source.get(&item.id) { - if !existing_tables.is_empty() + // Multi-output load generator sources whose subsources are already + // migrated above. There is no need to remove any options from this + // statement since they are not export-specific. + Statement::CreateSource(CreateSourceStatement { + connection: + CreateSourceConnection::LoadGenerator { + generator: + LoadGenerator::Auction | LoadGenerator::Marketing | LoadGenerator::Tpch, + .. + }, + .. + }) => {} + // Single-output sources that need to be migrated to tables. These sources currently output + // data to the primary collection of the source statement. We will create a new table + // statement for them and move all export-specific options over from the source statement, + // while moving the `CREATE SOURCE` statement to a new name and moving its shard to the + // new table statement. + Statement::CreateSource(CreateSourceStatement { + connection: + conn @ (CreateSourceConnection::Kafka { .. } + | CreateSourceConnection::LoadGenerator { + generator: + LoadGenerator::Clock + | LoadGenerator::Datums + | LoadGenerator::Counter + | LoadGenerator::KeyValue, + .. + }), + name, + col_names, + include_metadata, + format, + envelope, + mut with_options, + if_not_exists, + in_cluster, + progress_subsource, + external_references, + key_constraint, + }) => { + // To check if this is a source that has already been migrated we use a basic + // heuristic: if there is at least one existing table for the source, and if + // the envelope/format/include_metadata options are empty, we assume it's + // already been migrated. + let tables_for_source = + items_with_statements_copied + .iter() + .any(|(item, statement)| match statement { + Statement::CreateTableFromSource(stmt) => { + let source: GlobalId = match &stmt.source { + RawItemName::Name(_) => { + unreachable!("tables store source as ID") + } + RawItemName::Id(source_id, _, _) => { + source_id.parse().expect("valid id") + } + }; + source == item.id + } + _ => false, + }); + if tables_for_source && envelope.is_none() && format.is_none() && include_metadata.is_empty() { + info!("migrate: skipping already migrated source: {}", name); return Ok(()); } - } - // Use the current source name as the new table name, and rename the source to - // `_source`. This is intended to allow users to continue using - // queries that reference the source name, since they will now need to query the - // table instead. - let new_source_name_ident = Ident::new_unchecked( - name.0.last().expect("at least one ident").to_string() + "_source", - ); - let mut new_source_name = name.clone(); - *new_source_name.0.last_mut().expect("at least one ident") = new_source_name_ident; - let table_name = std::mem::replace(name, new_source_name.clone()); - - // Also update the name of the source 'item' - let mut table_item_name = item.name.clone() + "_source"; - std::mem::swap(&mut item.name, &mut table_item_name); - - // A reference to the source that will be included in the table statement - let source_ref = RawItemName::Id(item.id.to_string(), new_source_name, None); - - let columns = if col_names.is_empty() { - TableFromSourceColumns::NotSpecified - } else { - TableFromSourceColumns::Named(col_names.drain(..).collect()) - }; - - // All source tables must have a `details` option, which is a serialized proto - // describing any source-specific details for this table statement. - let details = match conn { - // For kafka sources this proto is currently empty. - CreateSourceConnection::Kafka { .. } => SourceExportStatementDetails::Kafka {}, - CreateSourceConnection::LoadGenerator { .. } => { - // Since these load generators are single-output we use the default output. - SourceExportStatementDetails::LoadGenerator { - output: LoadGeneratorOutput::Default, + // Use the current source name as the new table name, and rename the source to + // `_source`. This is intended to allow users to continue using + // queries that reference the source name, since they will now need to query the + // table instead. + let new_source_name_ident = Ident::new_unchecked( + name.0.last().expect("at least one ident").to_string() + "_source", + ); + let mut new_source_name = name.clone(); + *new_source_name.0.last_mut().expect("at least one ident") = new_source_name_ident; + + // Also update the name of the source 'item' + let mut table_item_name = item.name.clone() + "_source"; + std::mem::swap(&mut item.name, &mut table_item_name); + + // A reference to the source that will be included in the table statement + let source_ref = + RawItemName::Id(item.id.to_string(), new_source_name.clone(), None); + + let columns = if col_names.is_empty() { + TableFromSourceColumns::NotSpecified + } else { + TableFromSourceColumns::Named(col_names) + }; + + // All source tables must have a `details` option, which is a serialized proto + // describing any source-specific details for this table statement. + let details = match &conn { + // For kafka sources this proto is currently empty. + CreateSourceConnection::Kafka { .. } => SourceExportStatementDetails::Kafka {}, + CreateSourceConnection::LoadGenerator { .. } => { + // Since these load generators are single-output we use the default output. + SourceExportStatementDetails::LoadGenerator { + output: LoadGeneratorOutput::Default, + } + } + _ => unreachable!("match determined above"), + }; + let mut table_with_options = vec![TableFromSourceOption { + name: TableFromSourceOptionName::Details, + value: Some(WithOptionValue::Value(Value::String(hex::encode( + details.into_proto().encode_to_vec(), + )))), + }]; + + // Move over the IgnoreKeys option if it exists. + let mut i = 0; + while i < with_options.len() { + if with_options[i].name == CreateSourceOptionName::IgnoreKeys { + let option = with_options.remove(i); + table_with_options.push(TableFromSourceOption { + name: TableFromSourceOptionName::IgnoreKeys, + value: option.value, + }); + } else { + i += 1; } } - _ => unreachable!("match determined above"), - }; - let mut table_with_options = vec![TableFromSourceOption { - name: TableFromSourceOptionName::Details, - value: Some(WithOptionValue::Value(Value::String(hex::encode( - details.into_proto().encode_to_vec(), - )))), - }]; - - // Move over the IgnoreKeys option if it exists. - let mut i = 0; - while i < with_options.len() { - if with_options[i].name == CreateSourceOptionName::IgnoreKeys { - let option = with_options.remove(i); - table_with_options.push(TableFromSourceOption { - name: TableFromSourceOptionName::IgnoreKeys, - value: option.value, - }); - } else { - i += 1; + // Move over the Timeline option if it exists. + i = 0; + while i < with_options.len() { + if with_options[i].name == CreateSourceOptionName::Timeline { + let option = with_options.remove(i); + table_with_options.push(TableFromSourceOption { + name: TableFromSourceOptionName::Timeline, + value: option.value, + }); + } else { + i += 1; + } } - } - // Move over the Timeline option if it exists. - i = 0; - while i < with_options.len() { - if with_options[i].name == CreateSourceOptionName::Timeline { - let option = with_options.remove(i); - table_with_options.push(TableFromSourceOption { - name: TableFromSourceOptionName::Timeline, - value: option.value, - }); - } else { - i += 1; - } - } + // Generate the same external-reference that would have been generated + // during purification for single-output sources. + let external_reference = match &conn { + // For kafka sources this proto is currently empty. + CreateSourceConnection::Kafka { options, .. } => { + let topic_option = options + .iter() + .find(|o| matches!(o.name, KafkaSourceConfigOptionName::Topic)) + .expect("kafka sources must have a topic"); + let topic = match &topic_option.value { + Some(WithOptionValue::Value(Value::String(topic))) => topic, + _ => unreachable!("topic must be a string"), + }; + + Some(UnresolvedItemName::qualified(&[Ident::new(topic)?])) + } + CreateSourceConnection::LoadGenerator { generator, .. } => { + // Since these load generators are single-output the external reference + // uses the schema-name for both namespace and name. + let name = FullItemName { + database: mz_sql::names::RawDatabaseSpecifier::Name( + mz_storage_types::sources::load_generator::LOAD_GENERATOR_DATABASE_NAME + .to_owned(), + ), + schema: generator.schema_name().to_string(), + item: generator.schema_name().to_string(), + }; + Some(UnresolvedItemName::from(name)) + } + _ => unreachable!("match determined above"), + }; - // The new table statement, stealing the export-specific options from the - // create source statement. - let table = CreateTableFromSourceStatement { - name: table_name, - constraints: vec![], - columns, - if_not_exists: false, - source: source_ref, - // external_reference is not required for single-output sources - external_reference: None, - with_options: table_with_options, - envelope: envelope.take(), - include_metadata: include_metadata.drain(..).collect(), - format: format.take(), - }; - - // Insert the new table statement into the catalog with a new id. - let ids = tx.allocate_user_item_ids(1)?; - let new_table_id = ids[0]; - tx.insert_user_item( - new_table_id, - item.schema_id.clone(), - &table_item_name, - table.to_ast_string_stable(), - item.owner_id.clone(), - item.privileges.clone(), - &Default::default(), - )?; - // We need to move the shard currently attached to the source statement to the - // table statement such that the existing data in the shard is preserved and can - // be queried on the new table statement. However, we need to keep the GlobalId of - // the source the same, to preserve existing references to that statement in - // external tools such as DBT and Terraform. We will insert a new shard for the source - // statement which will be automatically created after the migration is complete. - let new_source_shard = ShardId::new(); - let (source_id, existing_source_shard) = tx - .delete_collection_metadata(btreeset! {item.id}) - .pop() - .expect("shard should exist"); - tx.insert_collection_metadata(btreemap! { - new_table_id => existing_source_shard, - source_id => new_source_shard - })?; - - let schema = tx.get_schema(&item.schema_id).expect("schema must exist"); - - add_to_audit_log( - tx, - mz_audit_log::EventType::Create, - mz_audit_log::ObjectType::Table, - mz_audit_log::EventDetails::IdFullNameV1(mz_audit_log::IdFullNameV1 { - id: new_table_id.to_string(), - name: mz_audit_log::FullNameV1 { - database: schema - .database_id - .map(|d| d.to_string()) - .unwrap_or_default(), - schema: schema.name, - item: table_item_name, - }, - }), - now(), - )?; + // The new table statement, stealing the name and the export-specific fields from + // the create source statement. + let table = CreateTableFromSourceStatement { + name, + constraints: vec![], + columns, + if_not_exists: false, + source: source_ref, + external_reference, + with_options: table_with_options, + envelope, + include_metadata, + format, + }; - info!("migrate: updated source {stmt} and added table {table}"); + // The source statement with a new name and many of its fields emptied + let source = CreateSourceStatement { + connection: conn, + name: new_source_name, + if_not_exists, + in_cluster, + include_metadata: vec![], + format: None, + envelope: None, + col_names: vec![], + with_options, + key_constraint, + external_references, + progress_subsource, + }; + + let source_id = item.id; + let schema_id = item.schema_id.clone(); + let schema = tx.get_schema(&item.schema_id).expect("schema must exist"); + + let owner_id = item.owner_id.clone(); + let privileges = item.privileges.clone(); + + // Update the source statement in the catalog first, since the name will + // otherwise conflict with the new table statement. + info!("migrate: updated source {} to {source}", item.create_sql); + item.create_sql = Statement::CreateSource(source).to_ast_string_stable(); + tx.update_item(item.id, item)?; + + // Insert the new table statement into the catalog with a new id. + let ids = tx.allocate_user_item_ids(1)?; + let new_table_id = ids[0]; + info!("migrate: added table {new_table_id}: {table}"); + tx.insert_user_item( + new_table_id, + schema_id, + &table_item_name, + table.to_ast_string_stable(), + owner_id, + privileges, + &Default::default(), + )?; + // We need to move the shard currently attached to the source statement to the + // table statement such that the existing data in the shard is preserved and can + // be queried on the new table statement. However, we need to keep the GlobalId of + // the source the same, to preserve existing references to that statement in + // external tools such as DBT and Terraform. We will insert a new shard for the source + // statement which will be automatically created after the migration is complete. + let new_source_shard = ShardId::new(); + let (source_id, existing_source_shard) = tx + .delete_collection_metadata(btreeset! {source_id}) + .pop() + .expect("shard should exist"); + tx.insert_collection_metadata(btreemap! { + new_table_id => existing_source_shard, + source_id => new_source_shard + })?; + + add_to_audit_log( + tx, + mz_audit_log::EventType::Create, + mz_audit_log::ObjectType::Table, + mz_audit_log::EventDetails::IdFullNameV1(mz_audit_log::IdFullNameV1 { + id: new_table_id.to_string(), + name: mz_audit_log::FullNameV1 { + database: schema + .database_id + .map(|d| d.to_string()) + .unwrap_or_default(), + schema: schema.name, + item: table_item_name, + }, + }), + now(), + )?; + + // We also need to update any other statements that reference the source to use the new + // table id/name instead. + changed_ids.insert(source_id, new_table_id); + } + + // When we upgrade to > rust 1.81 we should use #[expect(unreachable_patterns)] + // to enforce that we have covered all CreateSourceStatement variants. + #[allow(unreachable_patterns)] + Statement::CreateSource(_) => {} + _ => (), } + } - // When we upgrade to > rust 1.81 we should use #[expect(unreachable_patterns)] - // to enforce that we have covered all CreateSourceStatement variants. - #[allow(unreachable_patterns)] - Statement::CreateSource(_) => {} - _ => (), + let mut updated_items = BTreeMap::new(); + for (mut item, mut statement) in items_with_statements_copied { + match &statement { + // Don’t rewrite any of the statements we just migrated. + Statement::CreateSource(_) => {} + Statement::CreateSubsource(_) => {} + Statement::CreateTableFromSource(_) => {} + // We need to rewrite any statements that reference a source id to use the new + // table id instead, since any contained data in the source will now be in the table. + // This assumes the table has stolen the source's name, which is the case + // for all sources that were migrated. + _ => { + if mz_sql::names::modify_dependency_item_ids(&mut statement, &changed_ids) { + item.create_sql = statement.to_ast_string_stable(); + updated_items.insert(item.id, item); + } + } + } + } + if !updated_items.is_empty() { + tx.update_items(updated_items)?; } Ok(()) diff --git a/src/adapter/src/catalog/open.rs b/src/adapter/src/catalog/open.rs index 6d18f935ecae3..83176ebf4a3b7 100644 --- a/src/adapter/src/catalog/open.rs +++ b/src/adapter/src/catalog/open.rs @@ -412,11 +412,7 @@ impl Catalog { | BootstrapStateUpdateKind::StorageCollectionMetadata(_) | BootstrapStateUpdateKind::SourceReferences(_) | BootstrapStateUpdateKind::UnfinalizedShard(_) => { - post_item_updates.push(StateUpdate { - kind: kind.into(), - ts, - diff, - }) + post_item_updates.push((kind, ts, diff)); } BootstrapStateUpdateKind::AuditLog(_) => { audit_log_updates.push(StateUpdate { @@ -500,7 +496,7 @@ impl Catalog { // Migrate item ASTs. let builtin_table_update = if !config.skip_migrations { - migrate::migrate( + let migrate_result = migrate::migrate( &mut state, &mut txn, &mut local_expr_cache, @@ -515,7 +511,15 @@ impl Catalog { this_version: config.build_info.version, cause: e.to_string(), }) - })? + })?; + if !migrate_result.post_item_updates.is_empty() { + // Include any post-item-updates generated by migrations, and then consolidate + // them to ensure diffs are all positive. + post_item_updates.extend(migrate_result.post_item_updates); + differential_dataflow::consolidation::consolidate_updates(&mut post_item_updates); + } + + migrate_result.builtin_table_updates } else { state .apply_updates_for_bootstrap(item_updates, &mut local_expr_cache) @@ -523,9 +527,15 @@ impl Catalog { }; builtin_table_updates.extend(builtin_table_update); - let builtin_table_update = state - .apply_updates_for_bootstrap(post_item_updates, &mut local_expr_cache) - .await; + let post_item_updates = post_item_updates + .into_iter() + .map(|(kind, ts, diff)| StateUpdate { + kind: kind.into(), + ts, + diff: diff.try_into().expect("valid diff"), + }) + .collect(); + let builtin_table_update = state.apply_updates_for_bootstrap(post_item_updates, &mut local_expr_cache).await; builtin_table_updates.extend(builtin_table_update); // We don't need to apply the audit logs in memory, yet apply can be expensive when the diff --git a/src/sql-parser/src/ast/defs/ddl.rs b/src/sql-parser/src/ast/defs/ddl.rs index 7bffdcc990681..573aa56f4dfce 100644 --- a/src/sql-parser/src/ast/defs/ddl.rs +++ b/src/sql-parser/src/ast/defs/ddl.rs @@ -1294,6 +1294,24 @@ impl AstDisplay for LoadGenerator { } impl_display!(LoadGenerator); +impl LoadGenerator { + /// Corresponds with the same mapping on the `LoadGenerator` enum defined in + /// src/storage-types/src/sources/load_generator.rs, but re-defined here for + /// cases where we only have the AST representation. This can be removed once + /// the `ast_rewrite_sources_to_tables` migration is removed. + pub fn schema_name(&self) -> &'static str { + match self { + LoadGenerator::Counter => "counter", + LoadGenerator::Clock => "clock", + LoadGenerator::Marketing => "marketing", + LoadGenerator::Auction => "auction", + LoadGenerator::Datums => "datums", + LoadGenerator::Tpch => "tpch", + LoadGenerator::KeyValue => "key_value", + } + } +} + #[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Ord, Hash)] pub enum LoadGeneratorOptionName { ScaleFactor, diff --git a/src/sql/src/names.rs b/src/sql/src/names.rs index 33e26a9552959..d32f247f0e202 100644 --- a/src/sql/src/names.rs +++ b/src/sql/src/names.rs @@ -24,6 +24,7 @@ use mz_repr::network_policy_id::NetworkPolicyId; use mz_repr::role_id::RoleId; use mz_repr::{CatalogItemId, GlobalId}; use mz_repr::{ColumnName, RelationVersionSelector}; +use mz_sql_parser::ast::visit_mut::VisitMutNode; use mz_sql_parser::ast::{CreateContinualTaskStatement, Expr, RawNetworkPolicyName, Version}; use mz_sql_parser::ident; use proptest_derive::Arbitrary; @@ -2351,6 +2352,44 @@ where ResolvedIds::new(visitor.ids) } +#[derive(Debug)] +pub struct ItemDependencyModifier<'a> { + pub modified: bool, + pub id_map: &'a BTreeMap, +} + +impl<'ast, 'a> VisitMut<'ast, Raw> for ItemDependencyModifier<'a> { + fn visit_item_name_mut(&mut self, item_name: &mut RawItemName) { + if let RawItemName::Id(id, _, _) = item_name { + let parsed_id = id.parse::().unwrap(); + if let Some(new_id) = self.id_map.get(&parsed_id) { + *id = new_id.to_string(); + self.modified = true; + } + } + } +} + +/// Updates any references in the provided AST node that are keys in `id_map`. +/// If an id is found it will be updated to the value of the key in `id_map`. +/// This assumes the names of the reference(s) are unmodified (e.g. each pair of +/// ids refer to an item of the same name, whose id has changed). +pub fn modify_dependency_item_ids<'ast, N>( + node: &'ast mut N, + id_map: &BTreeMap, +) -> bool +where + N: VisitMutNode<'ast, Raw>, +{ + let mut modifier = ItemDependencyModifier { + id_map, + modified: false, + }; + node.visit_mut(&mut modifier); + + modifier.modified +} + // Used when displaying a view's source for human creation. If the name // specified is the same as the name in the catalog, we don't use the ID format. #[derive(Debug)] diff --git a/src/storage-types/src/sources/load_generator.rs b/src/storage-types/src/sources/load_generator.rs index 3d9dd471cd71a..0d6ea646992f1 100644 --- a/src/storage-types/src/sources/load_generator.rs +++ b/src/storage-types/src/sources/load_generator.rs @@ -196,6 +196,8 @@ pub enum LoadGenerator { pub const LOAD_GENERATOR_DATABASE_NAME: &str = "mz_load_generators"; impl LoadGenerator { + /// Must be kept in-sync with the same mapping on the `LoadGenerator` enum defined in + /// src/sql-parser/src/ast/defs/ddl.rs. pub fn schema_name(&self) -> &'static str { match self { LoadGenerator::Counter { .. } => "counter", From 869dde331b407c1dd62eddd8062761567ef07b18 Mon Sep 17 00:00:00 2001 From: Roshan Jobanputra Date: Thu, 24 Oct 2024 14:37:35 -0400 Subject: [PATCH 21/50] Also test the migration in the legacy upgrade tests --- misc/python/materialize/mzcompose/__init__.py | 5 +- test/legacy-upgrade/mzcompose.py | 54 +++++++++++++++++-- 2 files changed, 53 insertions(+), 6 deletions(-) diff --git a/misc/python/materialize/mzcompose/__init__.py b/misc/python/materialize/mzcompose/__init__.py index 9fbc90c183c92..dac8cfb3b40bd 100644 --- a/misc/python/materialize/mzcompose/__init__.py +++ b/misc/python/materialize/mzcompose/__init__.py @@ -53,7 +53,9 @@ def get_default_system_parameters( - version: MzVersion | None = None, zero_downtime: bool = False + version: MzVersion | None = None, + zero_downtime: bool = False, + force_source_table_syntax: bool = False, ) -> dict[str, str]: """For upgrade tests we only want parameters set when all environmentd / clusterd processes have reached a specific version (or higher) @@ -123,6 +125,7 @@ def get_default_system_parameters( "persist_record_schema_id": ( "true" if version > MzVersion.parse_mz("v0.127.0-dev") else "false" ), + "force_source_table_syntax": "true" if force_source_table_syntax else "false", "persist_batch_columnar_format": "both_v2", "persist_batch_columnar_format_percent": "100", "persist_batch_delete_enabled": "true", diff --git a/test/legacy-upgrade/mzcompose.py b/test/legacy-upgrade/mzcompose.py index 97dbc8e976288..30dc6df865c4b 100644 --- a/test/legacy-upgrade/mzcompose.py +++ b/test/legacy-upgrade/mzcompose.py @@ -108,20 +108,56 @@ def workflow_default(c: Composition, parser: WorkflowArgumentParser) -> None: else: if parallelism_count == 1 or parallelism_index == 0: test_upgrade_from_version( - c, f"{version}", priors, filter=args.filter, zero_downtime=True + c, + f"{version}", + priors, + filter=args.filter, + zero_downtime=True, + force_source_table_syntax=False, ) if parallelism_count == 1 or parallelism_index == 1: test_upgrade_from_version( - c, f"{version}", priors, filter=args.filter, zero_downtime=False + c, + f"{version}", + priors, + filter=args.filter, + zero_downtime=False, + force_source_table_syntax=False, + ) + test_upgrade_from_version( + c, + f"{version}", + priors, + filter=args.filter, + zero_downtime=False, + force_source_table_syntax=True, ) if parallelism_count == 1 or parallelism_index == 0: test_upgrade_from_version( - c, "current_source", priors=[], filter=args.filter, zero_downtime=True + c, + "current_source", + priors=[], + filter=args.filter, + zero_downtime=True, + force_source_table_syntax=False, ) if parallelism_count == 1 or parallelism_index == 1: test_upgrade_from_version( - c, "current_source", priors=[], filter=args.filter, zero_downtime=False + c, + "current_source", + priors=[], + filter=args.filter, + zero_downtime=False, + force_source_table_syntax=False, + ) + test_upgrade_from_version( + c, + "current_source", + priors=[], + filter=args.filter, + zero_downtime=False, + force_source_table_syntax=True, ) @@ -144,13 +180,14 @@ def test_upgrade_from_version( priors: list[MzVersion], filter: str, zero_downtime: bool, + force_source_table_syntax: bool, ) -> None: print( f"+++ Testing {'0dt upgrade' if zero_downtime else 'regular upgrade'} from Materialize {from_version} to current_source." ) system_parameter_defaults = get_default_system_parameters( - zero_downtime=zero_downtime + zero_downtime=zero_downtime, ) deploy_generation = 0 @@ -280,6 +317,13 @@ def test_upgrade_from_version( c.rm(mz_service) print(f"{'0dt-' if zero_downtime else ''}Upgrading to final version") + system_parameter_defaults = get_default_system_parameters( + zero_downtime=zero_downtime, + # We can only force the syntax on the final version so that the migration to convert + # sources to the new model can be applied without preventing sources from being + # created in the old syntax on the older version. + force_source_table_syntax=force_source_table_syntax, + ) mz_to = Materialized( name=mz_service, options=list(mz_options.values()), From 7daeefca04b345d7f22affd86328ad3f017d0218 Mon Sep 17 00:00:00 2001 From: Rainer Niedermayr Date: Fri, 25 Oct 2024 11:28:38 +0200 Subject: [PATCH 22/50] platform checks: unique source name --- misc/python/materialize/checks/all_checks/upsert.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/misc/python/materialize/checks/all_checks/upsert.py b/misc/python/materialize/checks/all_checks/upsert.py index 739bc9d5effa9..543478f0a0fe8 100644 --- a/misc/python/materialize/checks/all_checks/upsert.py +++ b/misc/python/materialize/checks/all_checks/upsert.py @@ -183,12 +183,12 @@ def initialize(self) -> Testdrive: $ kafka-ingest format=avro key-format=avro topic=upsert-legacy-syntax key-schema=${keyschema} schema=${schema} repeat=10000 {"key1": "A${kafka-ingest.iteration}"} {"f1": "A${kafka-ingest.iteration}"} - > CREATE SOURCE upsert_insert + > CREATE SOURCE upsert_insert_legacy FROM KAFKA CONNECTION kafka_conn (TOPIC 'testdrive-upsert-legacy-syntax-${testdrive.seed}') FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn ENVELOPE UPSERT - > CREATE MATERIALIZED VIEW upsert_insert_view AS SELECT COUNT(DISTINCT key1 || ' ' || f1) FROM upsert_insert; + > CREATE MATERIALIZED VIEW upsert_insert_legacy_view AS SELECT COUNT(DISTINCT key1 || ' ' || f1) FROM upsert_insert_legacy; """ ) ) @@ -212,10 +212,10 @@ def validate(self) -> Testdrive: return Testdrive( dedent( """ - > SELECT COUNT(*), COUNT(DISTINCT key1), COUNT(DISTINCT f1) FROM upsert_insert + > SELECT COUNT(*), COUNT(DISTINCT key1), COUNT(DISTINCT f1) FROM upsert_insert_legacy 10000 10000 10000 - > SELECT * FROM upsert_insert_view; + > SELECT * FROM upsert_insert_legacy_view; 10000 """ ) From 3144c3e31ae2b20ec0bc5777eb5704504f2d6e20 Mon Sep 17 00:00:00 2001 From: Roshan Jobanputra Date: Fri, 25 Oct 2024 10:23:56 -0400 Subject: [PATCH 23/50] Migration structure cleanup from feedback --- src/adapter/src/catalog/migrate.rs | 71 +++++++++++++----------------- 1 file changed, 31 insertions(+), 40 deletions(-) diff --git a/src/adapter/src/catalog/migrate.rs b/src/adapter/src/catalog/migrate.rs index ff21c13ce89fb..da264a06fa1c6 100644 --- a/src/adapter/src/catalog/migrate.rs +++ b/src/adapter/src/catalog/migrate.rs @@ -277,7 +277,7 @@ fn ast_rewrite_sources_to_tables( let raw_source_name = match of_source { // If `of_source` is None then this is a `progress` subsource which we // are not migrating as they are not currently relevant to the new table model. - None => return Ok(()), + None => continue, Some(name) => name, }; let source = match raw_source_name { @@ -300,21 +300,17 @@ fn ast_rewrite_sources_to_tables( // The external reference is a `with_option` on subsource statements but is a // separate field on table statements. - let mut i = 0; - let external_reference = loop { - if i >= with_options.len() { - panic!("subsource must have an external reference"); - } - if with_options[i].name == CreateSubsourceOptionName::ExternalReference { - let option = with_options.remove(i); - match option.value { - Some(WithOptionValue::UnresolvedItemName(name)) => break name, - _ => unreachable!("external reference must be an unresolved item name"), - }; - } else { - i += 1; - } + let external_reference = match with_options + .iter() + .position(|opt| opt.name == CreateSubsourceOptionName::ExternalReference) + { + Some(i) => match with_options.remove(i).value { + Some(WithOptionValue::UnresolvedItemName(name)) => name, + _ => unreachable!("external reference must be an unresolved item name"), + }, + None => panic!("subsource must have an external reference"), }; + let with_options = with_options .into_iter() .map(|option| { @@ -337,8 +333,7 @@ fn ast_rewrite_sources_to_tables( panic!("progress option should not exist on this subsource") } CreateSubsourceOptionName::ExternalReference => { - // This option is handled separately above. - unreachable!() + unreachable!("This option is handled separately above.") } } }) @@ -591,30 +586,26 @@ fn ast_rewrite_sources_to_tables( }]; // Move over the IgnoreKeys option if it exists. - let mut i = 0; - while i < with_options.len() { - if with_options[i].name == CreateSourceOptionName::IgnoreKeys { - let option = with_options.remove(i); - table_with_options.push(TableFromSourceOption { - name: TableFromSourceOptionName::IgnoreKeys, - value: option.value, - }); - } else { - i += 1; - } - } + if let Some(i) = with_options + .iter() + .position(|opt| opt.name == CreateSourceOptionName::IgnoreKeys) + { + let option = with_options.remove(i); + table_with_options.push(TableFromSourceOption { + name: TableFromSourceOptionName::IgnoreKeys, + value: option.value, + }); + }; // Move over the Timeline option if it exists. - i = 0; - while i < with_options.len() { - if with_options[i].name == CreateSourceOptionName::Timeline { - let option = with_options.remove(i); - table_with_options.push(TableFromSourceOption { - name: TableFromSourceOptionName::Timeline, - value: option.value, - }); - } else { - i += 1; - } + if let Some(i) = with_options + .iter() + .position(|opt| opt.name == CreateSourceOptionName::Timeline) + { + let option = with_options.remove(i); + table_with_options.push(TableFromSourceOption { + name: TableFromSourceOptionName::Timeline, + value: option.value, + }); } // Generate the same external-reference that would have been generated From b08465cfc7cb69e6f3946445a0180868d692f2ae Mon Sep 17 00:00:00 2001 From: Roshan Jobanputra Date: Fri, 25 Oct 2024 12:13:42 -0400 Subject: [PATCH 24/50] Address more feedback; ensure new source name is unique --- src/adapter/src/catalog/migrate.rs | 47 +++++++++++++++++++++++------- 1 file changed, 37 insertions(+), 10 deletions(-) diff --git a/src/adapter/src/catalog/migrate.rs b/src/adapter/src/catalog/migrate.rs index da264a06fa1c6..b23eb29500d76 100644 --- a/src/adapter/src/catalog/migrate.rs +++ b/src/adapter/src/catalog/migrate.rs @@ -256,6 +256,16 @@ fn ast_rewrite_sources_to_tables( .collect::, anyhow::Error>>()?; let items_with_statements_copied = items_with_statements.clone(); + let item_names_per_schema = items_with_statements_copied + .iter() + .map(|(item, _)| (item.schema_id.clone(), &item.name)) + .fold(BTreeMap::new(), |mut acc, (schema_id, name)| { + acc.entry(schema_id) + .or_insert_with(|| btreeset! {}) + .insert(name); + acc + }); + // Any GlobalId that should be changed to a new GlobalId in any statements that // reference it. This is necessary for ensuring downstream statements (e.g. // mat views, indexes) that reference a single-output source (e.g. kafka) @@ -538,22 +548,40 @@ fn ast_rewrite_sources_to_tables( && include_metadata.is_empty() { info!("migrate: skipping already migrated source: {}", name); - return Ok(()); + continue; } // Use the current source name as the new table name, and rename the source to // `_source`. This is intended to allow users to continue using // queries that reference the source name, since they will now need to query the // table instead. - let new_source_name_ident = Ident::new_unchecked( - name.0.last().expect("at least one ident").to_string() + "_source", - ); - let mut new_source_name = name.clone(); - *new_source_name.0.last_mut().expect("at least one ident") = new_source_name_ident; - // Also update the name of the source 'item' - let mut table_item_name = item.name.clone() + "_source"; - std::mem::swap(&mut item.name, &mut table_item_name); + // First find an unused name within the same schema to avoid conflicts. + let mut new_source_item_name = format!("{}_source", item.name); + let mut new_source_name_inner = + format!("{}_source", name.0.last().expect("at least one ident")); + let mut i = 0; + while item_names_per_schema + .get(&item.schema_id) + .expect("schema must exist") + .contains(&new_source_item_name) + { + new_source_item_name = format!("{}_source_{}", item.name, i); + new_source_name_inner = format!( + "{}_source_{}", + name.0.last().expect("at least one ident"), + i + ); + i += 1; + } + // We will use the original item name for the new table item. + let table_item_name = item.name.clone(); + + // Update the source item/statement to use the new name. + let mut new_source_name = name.clone(); + *new_source_name.0.last_mut().expect("at least one ident") = + Ident::new_unchecked(new_source_name_inner); + item.name = new_source_item_name; // A reference to the source that will be included in the table statement let source_ref = @@ -611,7 +639,6 @@ fn ast_rewrite_sources_to_tables( // Generate the same external-reference that would have been generated // during purification for single-output sources. let external_reference = match &conn { - // For kafka sources this proto is currently empty. CreateSourceConnection::Kafka { options, .. } => { let topic_option = options .iter() From 8cc26c9b92dade3d4bbdbeea1d07f803a21f8060 Mon Sep 17 00:00:00 2001 From: Roshan Jobanputra Date: Fri, 25 Oct 2024 14:32:08 -0400 Subject: [PATCH 25/50] Fix legacy-upgrade checks --- test/legacy-upgrade/check-from-v0.27.0-compile-proto-sources.td | 2 -- test/legacy-upgrade/check-from-v0.45.0-kafka-progress.td | 2 -- 2 files changed, 4 deletions(-) diff --git a/test/legacy-upgrade/check-from-v0.27.0-compile-proto-sources.td b/test/legacy-upgrade/check-from-v0.27.0-compile-proto-sources.td index 4240d3a2229de..fcc08c2529d2d 100644 --- a/test/legacy-upgrade/check-from-v0.27.0-compile-proto-sources.td +++ b/test/legacy-upgrade/check-from-v0.27.0-compile-proto-sources.td @@ -24,5 +24,3 @@ c h a e - -> DROP SOURCE kafka_proto_source CASCADE; diff --git a/test/legacy-upgrade/check-from-v0.45.0-kafka-progress.td b/test/legacy-upgrade/check-from-v0.45.0-kafka-progress.td index 69af351dbbe33..9592f9055b7d5 100644 --- a/test/legacy-upgrade/check-from-v0.45.0-kafka-progress.td +++ b/test/legacy-upgrade/check-from-v0.45.0-kafka-progress.td @@ -18,5 +18,3 @@ $ set-regex match=\d+ replacement= $ set-regex match=testdrive-upgrade-kafka-source-.*?' replacement=' >[version<8100] SHOW CREATE SOURCE kafka_source materialize.public.kafka_source "CREATE SOURCE \"materialize\".\"public\".\"kafka_source\" FROM KAFKA CONNECTION \"materialize\".\"public\".\"kafka_conn\" (TOPIC = '') FORMAT AVRO USING SCHEMA '{ \"type\": \"record\", \"name\": \"cpx\", \"fields\": [ {\"name\": \"a\", \"type\": \"long\"}, {\"name\": \"b\", \"type\": \"long\"} ] }' ENVELOPE NONE EXPOSE PROGRESS AS \"materialize\".\"public\".\"kafka_source_progress\"" - -> DROP SOURCE kafka_source; From a061b34aaebb2d30f242697d25c19a8fd54ffb4a Mon Sep 17 00:00:00 2001 From: Roshan Jobanputra Date: Fri, 25 Oct 2024 14:49:52 -0400 Subject: [PATCH 26/50] Fixes caused by rebase on main --- src/adapter/src/catalog/migrate.rs | 21 +++++++++++---------- 1 file changed, 11 insertions(+), 10 deletions(-) diff --git a/src/adapter/src/catalog/migrate.rs b/src/adapter/src/catalog/migrate.rs index b23eb29500d76..2c645ec9e56d3 100644 --- a/src/adapter/src/catalog/migrate.rs +++ b/src/adapter/src/catalog/migrate.rs @@ -38,7 +38,8 @@ where for mut item in tx.get_items() { let mut stmt = mz_sql::parse::parse(&item.create_sql)?.into_element().ast; - f(tx, item.id, &mut stmt).await?; + // TODO(alter_table): Switch this to CatalogItemId. + f(tx, item.global_id, &mut stmt).await?; item.create_sql = stmt.to_ast_string_stable(); updated_items.insert(item.id, item); @@ -303,7 +304,7 @@ fn ast_rewrite_sources_to_tables( _ => false, }) .expect("source must exist"); - RawItemName::Id(source_item.id.to_string(), name, None) + RawItemName::Id(source_item.global_id.to_string(), name, None) } RawItemName::Id(..) => raw_source_name, }; @@ -368,7 +369,7 @@ fn ast_rewrite_sources_to_tables( item.create_sql, table ); item.create_sql = Statement::CreateTableFromSource(table).to_ast_string_stable(); - tx.update_item(item.id, item)?; + tx.update_item(item.global_id, item)?; } // Postgres sources are multi-output sources whose subsources are @@ -419,7 +420,7 @@ fn ast_rewrite_sources_to_tables( progress_subsource, }); item.create_sql = stmt.to_ast_string_stable(); - tx.update_item(item.id, item)?; + tx.update_item(item.global_id, item)?; info!("migrate: converted postgres source {stmt} to remove subsource options"); } } @@ -477,7 +478,7 @@ fn ast_rewrite_sources_to_tables( progress_subsource, }); item.create_sql = stmt.to_ast_string_stable(); - tx.update_item(item.id, item)?; + tx.update_item(item.global_id, item)?; info!("migrate: converted mysql source {stmt} to remove subsource options"); } } @@ -538,7 +539,7 @@ fn ast_rewrite_sources_to_tables( source_id.parse().expect("valid id") } }; - source == item.id + source == item.global_id } _ => false, }); @@ -585,7 +586,7 @@ fn ast_rewrite_sources_to_tables( // A reference to the source that will be included in the table statement let source_ref = - RawItemName::Id(item.id.to_string(), new_source_name.clone(), None); + RawItemName::Id(item.global_id.to_string(), new_source_name.clone(), None); let columns = if col_names.is_empty() { TableFromSourceColumns::NotSpecified @@ -698,7 +699,7 @@ fn ast_rewrite_sources_to_tables( progress_subsource, }; - let source_id = item.id; + let source_id = item.global_id; let schema_id = item.schema_id.clone(); let schema = tx.get_schema(&item.schema_id).expect("schema must exist"); @@ -709,7 +710,7 @@ fn ast_rewrite_sources_to_tables( // otherwise conflict with the new table statement. info!("migrate: updated source {} to {source}", item.create_sql); item.create_sql = Statement::CreateSource(source).to_ast_string_stable(); - tx.update_item(item.id, item)?; + tx.update_item(item.global_id, item)?; // Insert the new table statement into the catalog with a new id. let ids = tx.allocate_user_item_ids(1)?; @@ -785,7 +786,7 @@ fn ast_rewrite_sources_to_tables( _ => { if mz_sql::names::modify_dependency_item_ids(&mut statement, &changed_ids) { item.create_sql = statement.to_ast_string_stable(); - updated_items.insert(item.id, item); + updated_items.insert(item.global_id, item); } } } From 12428f756ac11ccab8082b27b2c9cf7bf925a05a Mon Sep 17 00:00:00 2001 From: Rainer Niedermayr Date: Mon, 28 Oct 2024 11:24:18 +0100 Subject: [PATCH 27/50] ci: print source table migration issues --- misc/python/materialize/cli/ci_annotate_errors.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/misc/python/materialize/cli/ci_annotate_errors.py b/misc/python/materialize/cli/ci_annotate_errors.py index 53b7fc82d272b..84865cbbcaff4 100644 --- a/misc/python/materialize/cli/ci_annotate_errors.py +++ b/misc/python/materialize/cli/ci_annotate_errors.py @@ -101,6 +101,8 @@ | (FAIL|TIMEOUT)\s+\[\s*\d+\.\d+s\] # parallel-workload | worker_.*\ still\ running: [\s\S]* Threads\ have\ not\ stopped\ within\ 5\ minutes,\ exiting\ hard + # source-table migration + | source-table-migration\ issue ) .* $ """, From ebcd57e47628dd5128086a849ced963c9923bdc0 Mon Sep 17 00:00:00 2001 From: Rainer Niedermayr Date: Fri, 25 Oct 2024 14:10:37 +0200 Subject: [PATCH 28/50] migration tests: pg-cdc-old-syntax --- ci/nightly/pipeline.template.yml | 11 ++ test/pg-cdc-old-syntax/mzcompose.py | 132 +++++++++++++++++- .../subsource-resolution-duplicates.td | 3 - 3 files changed, 141 insertions(+), 5 deletions(-) diff --git a/ci/nightly/pipeline.template.yml b/ci/nightly/pipeline.template.yml index 4c0c4cbfc7c69..52adde622096d 100644 --- a/ci/nightly/pipeline.template.yml +++ b/ci/nightly/pipeline.template.yml @@ -602,6 +602,17 @@ steps: queue: hetzner-aarch64-4cpu-8gb # the mzbuild postgres version will be used, which depends on the Dockerfile specification + - id: pg-cdc-migration + label: Postgres CDC source-versioning migration tests + depends_on: build-aarch64 + timeout_in_minutes: 360 + plugins: + - ./ci/plugins/mzcompose: + composition: pg-cdc-old-syntax + run: migration + agents: + queue: hetzner-aarch64-4cpu-8gb + - id: pg-cdc-resumption-old-syntax label: Postgres CDC resumption tests (before source versioning) depends_on: build-aarch64 diff --git a/test/pg-cdc-old-syntax/mzcompose.py b/test/pg-cdc-old-syntax/mzcompose.py index f44e64c31615b..5560e259197ad 100644 --- a/test/pg-cdc-old-syntax/mzcompose.py +++ b/test/pg-cdc-old-syntax/mzcompose.py @@ -18,10 +18,16 @@ from pg8000 import Connection from materialize import buildkite +from materialize.mz_version import MzVersion from materialize.mzcompose.composition import Composition, WorkflowArgumentParser from materialize.mzcompose.service import Service, ServiceConfig from materialize.mzcompose.services.materialized import Materialized -from materialize.mzcompose.services.postgres import Postgres +from materialize.mzcompose.services.minio import Minio +from materialize.mzcompose.services.postgres import ( + METADATA_STORE, + CockroachOrPostgresMetadata, + Postgres, +) from materialize.mzcompose.services.test_certs import TestCerts from materialize.mzcompose.services.testdrive import Testdrive from materialize.mzcompose.services.toxiproxy import Toxiproxy @@ -88,8 +94,11 @@ def create_postgres( additional_system_parameter_defaults={ "log_filter": "mz_storage::source::postgres=trace,debug,info,warn,error" }, + external_minio=True, ), Testdrive(), + CockroachOrPostgresMetadata(), + Minio(setup_materialize=True), TestCerts(), Toxiproxy(), create_postgres(pg_version=None), @@ -323,7 +332,11 @@ def workflow_cdc(c: Composition, parser: WorkflowArgumentParser) -> None: def workflow_default(c: Composition, parser: WorkflowArgumentParser) -> None: workflows_with_internal_sharding = ["cdc"] sharded_workflows = workflows_with_internal_sharding + buildkite.shard_list( - [w for w in c.workflows if w not in workflows_with_internal_sharding], + [ + w + for w in c.workflows + if w not in workflows_with_internal_sharding and w != "migration" + ], lambda w: w, ) print( @@ -348,3 +361,118 @@ def workflow_default(c: Composition, parser: WorkflowArgumentParser) -> None: with c.test_case(name): c.workflow(name, *parser.args) + + +def workflow_migration(c: Composition, parser: WorkflowArgumentParser) -> None: + parser.add_argument( + "filter", + nargs="*", + default=["*.td"], + help="limit to only the files matching filter", + ) + args = parser.parse_args() + + matching_files = [] + for filter in args.filter: + matching_files.extend(glob.glob(filter, root_dir="test/pg-cdc-old-syntax")) + sharded_files: list[str] = sorted( + buildkite.shard_list(matching_files, lambda file: file) + ) + print(f"Files: {sharded_files}") + + ssl_ca = c.run("test-certs", "cat", "/secrets/ca.crt", capture=True).stdout + ssl_cert = c.run("test-certs", "cat", "/secrets/certuser.crt", capture=True).stdout + ssl_key = c.run("test-certs", "cat", "/secrets/certuser.key", capture=True).stdout + ssl_wrong_cert = c.run( + "test-certs", "cat", "/secrets/postgres.crt", capture=True + ).stdout + ssl_wrong_key = c.run( + "test-certs", "cat", "/secrets/postgres.key", capture=True + ).stdout + + pg_version = get_targeted_pg_version(parser) + + mz_old_image = "materialize/materialized:v0.122.0" + mz_new_image = None + + assert MzVersion.parse_cargo() < MzVersion.parse_mz( + "v0.130.0" + ), "migration test probably no longer needed" + + for file in sharded_files: + mz_old = Materialized( + name="materialized", + image=mz_old_image, + volumes_extra=["secrets:/share/secrets"], + external_metadata_store=True, + external_minio=True, + additional_system_parameter_defaults={ + "log_filter": "mz_storage::source::postgres=trace,debug,info,warn,error" + }, + ) + + mz_new = Materialized( + name="materialized", + image=mz_new_image, + volumes_extra=["secrets:/share/secrets"], + external_metadata_store=True, + external_minio=True, + additional_system_parameter_defaults={ + "log_filter": "mz_storage::source::postgres=trace,debug,info,warn,error", + "force_source_table_syntax": "true", + }, + ) + with c.override(mz_old, create_postgres(pg_version=pg_version)): + c.up("materialized", "test-certs", "postgres") + + print(f"Running {file} with mz_old") + + c.run_testdrive_files( + f"--var=ssl-ca={ssl_ca}", + f"--var=ssl-cert={ssl_cert}", + f"--var=ssl-key={ssl_key}", + f"--var=ssl-wrong-cert={ssl_wrong_cert}", + f"--var=ssl-wrong-key={ssl_wrong_key}", + f"--var=default-replica-size={Materialized.Size.DEFAULT_SIZE}-{Materialized.Size.DEFAULT_SIZE}", + f"--var=default-storage-size={Materialized.Size.DEFAULT_SIZE}-1", + "--no-reset", + file, + ) + c.kill("materialized", wait=True) + + with c.override(mz_new): + c.up("materialized") + + print("Running mz_new") + verify_sources(c, file) + + c.kill("materialized", wait=True) + c.kill("postgres", wait=True) + c.kill(METADATA_STORE, wait=True) + c.rm("materialized") + c.rm(METADATA_STORE) + c.rm("postgres") + c.rm_volumes("mzdata") + + +def verify_sources(c: Composition, file: str) -> None: + source_names = c.sql_query("SELECT name FROM mz_sources WHERE id LIKE 'u%';") + + print(f"Sources created in {file} are: {source_names}") + + for row in source_names: + verify_source(c, file, row[0]) + + +def verify_source(c: Composition, file: str, source_name: str) -> None: + try: + print(f"Checking source: {source_name}") + # must not crash + c.sql_query(f"SELECT count(*) FROM {source_name};") + + result = c.sql_query(f"SHOW CREATE SOURCE {source_name};") + sql = result[0][1] + assert "FOR TABLE" not in sql, f"FOR TABLE found in: {sql}" + assert "FOR ALL TABLES" not in sql, f"FOR ALL TABLES found in: {sql}" + except Exception as e: + print(f"source-table-migration issue in {file}: {str(e)}") diff --git a/test/pg-cdc-old-syntax/subsource-resolution-duplicates.td b/test/pg-cdc-old-syntax/subsource-resolution-duplicates.td index 3f510be4088b8..8f053fd552679 100644 --- a/test/pg-cdc-old-syntax/subsource-resolution-duplicates.td +++ b/test/pg-cdc-old-syntax/subsource-resolution-duplicates.td @@ -57,6 +57,3 @@ detail: subsources referencing table: x, y mz_source postgres quickstart "" mz_source_progress progress "" t subsource quickstart "" - -$ postgres-execute connection=postgres://postgres:postgres@postgres -DROP SCHEMA other CASCADE; From db36e657050ac6e42ab310c66b2ca0d78a85cd03 Mon Sep 17 00:00:00 2001 From: Rainer Niedermayr Date: Mon, 28 Oct 2024 11:35:03 +0100 Subject: [PATCH 29/50] migration tests: extract logic --- .../materialize/source_table_migration.py | 50 +++++++++++++++++++ test/pg-cdc-old-syntax/mzcompose.py | 42 +++------------- 2 files changed, 58 insertions(+), 34 deletions(-) create mode 100644 misc/python/materialize/source_table_migration.py diff --git a/misc/python/materialize/source_table_migration.py b/misc/python/materialize/source_table_migration.py new file mode 100644 index 0000000000000..ed420cbf172f8 --- /dev/null +++ b/misc/python/materialize/source_table_migration.py @@ -0,0 +1,50 @@ +# Copyright Materialize, Inc. and contributors. All rights reserved. +# +# Use of this software is governed by the Business Source License +# included in the LICENSE file at the root of this repository. +# +# As of the Change Date specified in that file, in accordance with +# the Business Source License, use of this software will be governed +# by the Apache License, Version 2.0. + +"""Utilities for testing the source table migration""" +from materialize.mz_version import MzVersion +from materialize.mzcompose.composition import Composition + + +def verify_sources_after_source_table_migration(c: Composition, file: str) -> None: + source_names = c.sql_query("SELECT name FROM mz_sources WHERE id LIKE 'u%';") + + print(f"Sources created in {file} are: {source_names}") + + for row in source_names: + _verify_source(c, file, row[0]) + + +def _verify_source(c: Composition, file: str, source_name: str) -> None: + try: + print(f"Checking source: {source_name}") + # must not crash + c.sql("SET statement_timeout = '20s'") + c.sql_query(f"SELECT count(*) FROM {source_name};") + + result = c.sql_query(f"SHOW CREATE SOURCE {source_name};") + sql = result[0][1] + assert "FOR TABLE" not in sql, f"FOR TABLE found in: {sql}" + assert "FOR ALL TABLES" not in sql, f"FOR ALL TABLES found in: {sql}" + except Exception as e: + print(f"source-table-migration issue in {file}: {str(e)}") + + +def check_source_table_migration_test_sensible() -> None: + assert MzVersion.parse_cargo() < MzVersion.parse_mz( + "v0.130.0" + ), "migration test probably no longer needed" + + +def get_old_image_for_source_table_migration_test() -> str: + return "materialize/materialized:v0.122.0" + + +def get_new_image_for_source_table_migration_test() -> str | None: + return None diff --git a/test/pg-cdc-old-syntax/mzcompose.py b/test/pg-cdc-old-syntax/mzcompose.py index 5560e259197ad..51ab5ed98d0e9 100644 --- a/test/pg-cdc-old-syntax/mzcompose.py +++ b/test/pg-cdc-old-syntax/mzcompose.py @@ -18,7 +18,6 @@ from pg8000 import Connection from materialize import buildkite -from materialize.mz_version import MzVersion from materialize.mzcompose.composition import Composition, WorkflowArgumentParser from materialize.mzcompose.service import Service, ServiceConfig from materialize.mzcompose.services.materialized import Materialized @@ -31,6 +30,11 @@ from materialize.mzcompose.services.test_certs import TestCerts from materialize.mzcompose.services.testdrive import Testdrive from materialize.mzcompose.services.toxiproxy import Toxiproxy +from materialize.source_table_migration import ( + get_new_image_for_source_table_migration_test, + get_old_image_for_source_table_migration_test, + verify_sources_after_source_table_migration, +) # Set the max slot WAL keep size to 10MB DEFAULT_PG_EXTRA_COMMAND = ["-c", "max_slot_wal_keep_size=10"] @@ -392,17 +396,10 @@ def workflow_migration(c: Composition, parser: WorkflowArgumentParser) -> None: pg_version = get_targeted_pg_version(parser) - mz_old_image = "materialize/materialized:v0.122.0" - mz_new_image = None - - assert MzVersion.parse_cargo() < MzVersion.parse_mz( - "v0.130.0" - ), "migration test probably no longer needed" - for file in sharded_files: mz_old = Materialized( name="materialized", - image=mz_old_image, + image=get_old_image_for_source_table_migration_test(), volumes_extra=["secrets:/share/secrets"], external_metadata_store=True, external_minio=True, @@ -413,7 +410,7 @@ def workflow_migration(c: Composition, parser: WorkflowArgumentParser) -> None: mz_new = Materialized( name="materialized", - image=mz_new_image, + image=get_new_image_for_source_table_migration_test(), volumes_extra=["secrets:/share/secrets"], external_metadata_store=True, external_minio=True, @@ -444,7 +441,7 @@ def workflow_migration(c: Composition, parser: WorkflowArgumentParser) -> None: c.up("materialized") print("Running mz_new") - verify_sources(c, file) + verify_sources_after_source_table_migration(c, file) c.kill("materialized", wait=True) c.kill("postgres", wait=True) @@ -453,26 +450,3 @@ def workflow_migration(c: Composition, parser: WorkflowArgumentParser) -> None: c.rm(METADATA_STORE) c.rm("postgres") c.rm_volumes("mzdata") - - -def verify_sources(c: Composition, file: str) -> None: - source_names = c.sql_query("SELECT name FROM mz_sources WHERE id LIKE 'u%';") - - print(f"Sources created in {file} are: {source_names}") - - for row in source_names: - verify_source(c, file, row[0]) - - -def verify_source(c: Composition, file: str, source_name: str) -> None: - try: - print(f"Checking source: {source_name}") - # must not crash - c.sql_query(f"SELECT count(*) FROM {source_name};") - - result = c.sql_query(f"SHOW CREATE SOURCE {source_name};") - sql = result[0][1] - assert "FOR TABLE" not in sql, f"FOR TABLE found in: {sql}" - assert "FOR ALL TABLES" not in sql, f"FOR ALL TABLES found in: {sql}" - except Exception as e: - print(f"source-table-migration issue in {file}: {str(e)}") From 7a36b2a91c9edd86d6ed2b946f1885afcca6fc80 Mon Sep 17 00:00:00 2001 From: Rainer Niedermayr Date: Mon, 28 Oct 2024 16:29:49 +0100 Subject: [PATCH 30/50] migration tests: improve verification --- .../materialize/source_table_migration.py | 22 ++++++++++++++----- 1 file changed, 17 insertions(+), 5 deletions(-) diff --git a/misc/python/materialize/source_table_migration.py b/misc/python/materialize/source_table_migration.py index ed420cbf172f8..21ecc1ee4d7c1 100644 --- a/misc/python/materialize/source_table_migration.py +++ b/misc/python/materialize/source_table_migration.py @@ -12,16 +12,23 @@ from materialize.mzcompose.composition import Composition -def verify_sources_after_source_table_migration(c: Composition, file: str) -> None: - source_names = c.sql_query("SELECT name FROM mz_sources WHERE id LIKE 'u%';") +def verify_sources_after_source_table_migration( + c: Composition, file: str, fail: bool = False +) -> None: + source_names_rows = c.sql_query( + "SELECT sm.name || '.' || src.name FROM mz_sources src INNER JOIN mz_schemas sm ON src.schema_id = sm.id WHERE src.id LIKE 'u%';" + ) + source_names = [row[0] for row in source_names_rows] print(f"Sources created in {file} are: {source_names}") - for row in source_names: - _verify_source(c, file, row[0]) + for source_name in source_names: + _verify_source(c, file, source_name, fail=fail) -def _verify_source(c: Composition, file: str, source_name: str) -> None: +def _verify_source( + c: Composition, file: str, source_name: str, fail: bool = False +) -> None: try: print(f"Checking source: {source_name}") # must not crash @@ -32,9 +39,14 @@ def _verify_source(c: Composition, file: str, source_name: str) -> None: sql = result[0][1] assert "FOR TABLE" not in sql, f"FOR TABLE found in: {sql}" assert "FOR ALL TABLES" not in sql, f"FOR ALL TABLES found in: {sql}" + assert "CREATE SUBSOURCE" not in sql, f"FOR ALL TABLES found in: {sql}" + print("OK.") except Exception as e: print(f"source-table-migration issue in {file}: {str(e)}") + if fail: + raise e + def check_source_table_migration_test_sensible() -> None: assert MzVersion.parse_cargo() < MzVersion.parse_mz( From 093bc6c3e3672e5731dc27fc0e111bac754ce62e Mon Sep 17 00:00:00 2001 From: Rainer Niedermayr Date: Mon, 28 Oct 2024 13:47:06 +0100 Subject: [PATCH 31/50] migration tests: mysql-cdc-old-syntax --- ci/nightly/pipeline.template.yml | 11 ++ test/mysql-cdc-old-syntax/mzcompose.py | 106 +++++++++++++++++- .../subsource-resolution-duplicates.td | 3 - 3 files changed, 116 insertions(+), 4 deletions(-) diff --git a/ci/nightly/pipeline.template.yml b/ci/nightly/pipeline.template.yml index 52adde622096d..705b84efa471b 100644 --- a/ci/nightly/pipeline.template.yml +++ b/ci/nightly/pipeline.template.yml @@ -571,6 +571,17 @@ steps: agents: queue: hetzner-aarch64-4cpu-8gb + - id: mysql-cdc-migration + label: MySQL CDC source-versioning migration tests + depends_on: build-aarch64 + timeout_in_minutes: 360 + plugins: + - ./ci/plugins/mzcompose: + composition: mysql-cdc-old-syntax + run: migration + agents: + queue: hetzner-aarch64-4cpu-8gb + - id: mysql-cdc-resumption-old-syntax label: MySQL CDC resumption tests (before source versioning) depends_on: build-aarch64 diff --git a/test/mysql-cdc-old-syntax/mzcompose.py b/test/mysql-cdc-old-syntax/mzcompose.py index 32d78456c45c2..46986c98c3190 100644 --- a/test/mysql-cdc-old-syntax/mzcompose.py +++ b/test/mysql-cdc-old-syntax/mzcompose.py @@ -22,9 +22,19 @@ ) from materialize.mzcompose.composition import Composition, WorkflowArgumentParser from materialize.mzcompose.services.materialized import Materialized +from materialize.mzcompose.services.minio import Minio from materialize.mzcompose.services.mysql import MySql +from materialize.mzcompose.services.postgres import ( + METADATA_STORE, + CockroachOrPostgresMetadata, +) from materialize.mzcompose.services.test_certs import TestCerts from materialize.mzcompose.services.testdrive import Testdrive +from materialize.source_table_migration import ( + get_new_image_for_source_table_migration_test, + get_old_image_for_source_table_migration_test, + verify_sources_after_source_table_migration, +) def create_mysql(mysql_version: str) -> MySql: @@ -46,6 +56,7 @@ def create_mysql_replica(mysql_version: str) -> MySql: SERVICES = [ Materialized( + external_minio=True, additional_system_parameter_defaults={ "log_filter": "mz_storage::source::mysql=trace,info" }, @@ -53,6 +64,8 @@ def create_mysql_replica(mysql_version: str) -> MySql: create_mysql(MySql.DEFAULT_VERSION), create_mysql_replica(MySql.DEFAULT_VERSION), TestCerts(), + CockroachOrPostgresMetadata(), + Minio(setup_materialize=True), Testdrive(default_timeout="60s"), ] @@ -72,7 +85,11 @@ def get_targeted_mysql_version(parser: WorkflowArgumentParser) -> str: def workflow_default(c: Composition, parser: WorkflowArgumentParser) -> None: workflows_with_internal_sharding = ["cdc"] sharded_workflows = workflows_with_internal_sharding + buildkite.shard_list( - [w for w in c.workflows if w not in workflows_with_internal_sharding], + [ + w + for w in c.workflows + if w not in workflows_with_internal_sharding and w != "migration" + ], lambda w: w, ) print( @@ -273,3 +290,90 @@ def do_inserts(c: Composition): """ ), ) + + +def workflow_migration(c: Composition, parser: WorkflowArgumentParser) -> None: + parser.add_argument( + "filter", + nargs="*", + default=["*.td"], + help="limit to only the files matching filter", + ) + args = parser.parse_args() + + matching_files = [] + for filter in args.filter: + matching_files.extend(glob.glob(filter, root_dir="test/mysql-cdc-old-syntax")) + + # TODO: database-issues#8706 + matching_files = [file for file in matching_files if file != "alter-source.td"] + + sharded_files: list[str] = sorted( + buildkite.shard_list(matching_files, lambda file: file) + ) + print(f"Files: {sharded_files}") + + mysql_version = get_targeted_mysql_version(parser) + + for file in sharded_files: + + mz_old = Materialized( + name="materialized", + image=get_old_image_for_source_table_migration_test(), + external_metadata_store=True, + external_minio=True, + additional_system_parameter_defaults={ + "log_filter": "mz_storage::source::mysql=trace,info" + }, + ) + + mz_new = Materialized( + name="materialized", + image=get_new_image_for_source_table_migration_test(), + external_metadata_store=True, + external_minio=True, + additional_system_parameter_defaults={ + "log_filter": "mz_storage::source::mysql=trace,info" + }, + ) + + with c.override(mz_old, create_mysql(mysql_version)): + c.up("materialized", "mysql") + + print(f"Running {file} with mz_old") + + valid_ssl_context = retrieve_ssl_context_for_mysql(c) + wrong_ssl_context = retrieve_invalid_ssl_context_for_mysql(c) + + c.sources_and_sinks_ignored_from_validation.add("drop_table") + + c.run_testdrive_files( + f"--var=ssl-ca={valid_ssl_context.ca}", + f"--var=ssl-client-cert={valid_ssl_context.client_cert}", + f"--var=ssl-client-key={valid_ssl_context.client_key}", + f"--var=ssl-wrong-ca={wrong_ssl_context.ca}", + f"--var=ssl-wrong-client-cert={wrong_ssl_context.client_cert}", + f"--var=ssl-wrong-client-key={wrong_ssl_context.client_key}", + f"--var=mysql-root-password={MySql.DEFAULT_ROOT_PASSWORD}", + "--var=mysql-user-password=us3rp4ssw0rd", + f"--var=default-replica-size={Materialized.Size.DEFAULT_SIZE}-{Materialized.Size.DEFAULT_SIZE}", + f"--var=default-storage-size={Materialized.Size.DEFAULT_SIZE}-1", + "--no-reset", + file, + ) + + c.kill("materialized", wait=True) + + with c.override(mz_new): + c.up("materialized") + + print("Running mz_new") + verify_sources_after_source_table_migration(c, file) + + c.kill("materialized", wait=True) + c.kill("mysql", wait=True) + c.kill(METADATA_STORE, wait=True) + c.rm("materialized") + c.rm(METADATA_STORE) + c.rm("mysql") + c.rm_volumes("mzdata") diff --git a/test/mysql-cdc-old-syntax/subsource-resolution-duplicates.td b/test/mysql-cdc-old-syntax/subsource-resolution-duplicates.td index 1b5662cc78956..c762d583849e5 100644 --- a/test/mysql-cdc-old-syntax/subsource-resolution-duplicates.td +++ b/test/mysql-cdc-old-syntax/subsource-resolution-duplicates.td @@ -53,6 +53,3 @@ detail: subsources referencing table: x, y mz_source mysql quickstart "" mz_source_progress progress "" t subsource quickstart "" - -$ mysql-execute name=mysql -DROP DATABASE other; From e2ec8a68f5e6a81aba8c975dbdb1ab2edb33d118 Mon Sep 17 00:00:00 2001 From: Rainer Niedermayr Date: Mon, 28 Oct 2024 14:10:55 +0100 Subject: [PATCH 32/50] migration tests: testdrive-old-kafka-syntax --- ci/nightly/pipeline.template.yml | 11 + .../mzcompose.py | 227 +++++++++++++++++- 2 files changed, 235 insertions(+), 3 deletions(-) diff --git a/ci/nightly/pipeline.template.yml b/ci/nightly/pipeline.template.yml index 705b84efa471b..3d892a9ddf2e3 100644 --- a/ci/nightly/pipeline.template.yml +++ b/ci/nightly/pipeline.template.yml @@ -654,6 +654,17 @@ steps: agents: queue: hetzner-aarch64-8cpu-16gb + - id: testdrive-kafka-migration + label: "Testdrive %N migration tests" + depends_on: build-aarch64 + timeout_in_minutes: 180 + parallelism: 8 + plugins: + - ./ci/plugins/mzcompose: + composition: testdrive-old-kafka-src-syntax + run: migration + agents: + queue: hetzner-aarch64-8cpu-16gb - group: AWS key: aws diff --git a/test/testdrive-old-kafka-src-syntax/mzcompose.py b/test/testdrive-old-kafka-src-syntax/mzcompose.py index c946b1a2de564..8135b14a74295 100644 --- a/test/testdrive-old-kafka-src-syntax/mzcompose.py +++ b/test/testdrive-old-kafka-src-syntax/mzcompose.py @@ -12,10 +12,10 @@ the expected-result/actual-result (aka golden testing) paradigm. A query is retried until it produces the desired result. """ - +import glob from pathlib import Path -from materialize import ci_util +from materialize import ci_util, spawn from materialize.mzcompose import get_default_system_parameters from materialize.mzcompose.composition import Composition, WorkflowArgumentParser from materialize.mzcompose.services.fivetran_destination import FivetranDestination @@ -23,11 +23,20 @@ from materialize.mzcompose.services.materialized import Materialized from materialize.mzcompose.services.minio import Minio from materialize.mzcompose.services.mysql import MySql -from materialize.mzcompose.services.postgres import Postgres +from materialize.mzcompose.services.postgres import ( + METADATA_STORE, + CockroachOrPostgresMetadata, + Postgres, +) from materialize.mzcompose.services.redpanda import Redpanda from materialize.mzcompose.services.schema_registry import SchemaRegistry from materialize.mzcompose.services.testdrive import Testdrive from materialize.mzcompose.services.zookeeper import Zookeeper +from materialize.source_table_migration import ( + get_new_image_for_source_table_migration_test, + get_old_image_for_source_table_migration_test, + verify_sources_after_source_table_migration, +) SERVICES = [ Zookeeper(), @@ -38,12 +47,25 @@ MySql(), Minio(setup_materialize=True, additional_directories=["copytos3"]), Materialized(external_minio=True), + CockroachOrPostgresMetadata(), FivetranDestination(volumes_extra=["tmp:/share/tmp"]), Testdrive(external_minio=True), ] def workflow_default(c: Composition, parser: WorkflowArgumentParser) -> None: + for name in c.workflows: + if name == "default": + continue + + if name == "migration": + continue + + with c.test_case(name): + c.workflow(name) + + +def workflow_kafka(c: Composition, parser: WorkflowArgumentParser) -> None: """Run testdrive.""" parser.add_argument( "--redpanda", @@ -230,3 +252,202 @@ def workflow_default(c: Composition, parser: WorkflowArgumentParser) -> None: ci_util.upload_junit_report( "testdrive", Path(__file__).parent / junit_report ) + + +def workflow_migration(c: Composition, parser: WorkflowArgumentParser) -> None: + """Run testdrive.""" + parser.add_argument( + "--redpanda", + action="store_true", + help="run against Redpanda instead of the Confluent Platform", + ) + parser.add_argument( + "--aws-region", + help="run against the specified AWS region instead of localstack", + ) + parser.add_argument( + "--kafka-default-partitions", + type=int, + metavar="N", + help="set the default number of kafka partitions per topic", + ) + parser.add_argument( + "--system-param", + type=str, + action="append", + nargs="*", + help="System parameters to set in Materialize, i.e. what you would set with `ALTER SYSTEM SET`", + ) + + parser.add_argument("--replicas", type=int, default=1, help="use multiple replicas") + + parser.add_argument( + "--default-timeout", + type=str, + help="set the default timeout for Testdrive", + ) + parser.add_argument( + "files", + nargs="*", + default=["*.td"], + help="run against the specified files", + ) + + (args, _) = parser.parse_known_args() + + matching_files = [] + for filter in args.files: + matching_files.extend( + glob.glob(filter, root_dir="test/testdrive-old-kafka-src-syntax") + ) + matching_files = [file for file in matching_files if file != "session.td"] + + dependencies = [ + "fivetran-destination", + "minio", + "materialized", + "postgres", + "mysql", + ] + + if args.redpanda: + kafka_deps = ["redpanda"] + else: + kafka_deps = ["zookeeper", "kafka", "schema-registry"] + + dependencies += kafka_deps + + testdrive = Testdrive( + forward_buildkite_shard=True, + kafka_default_partitions=args.kafka_default_partitions, + aws_region=args.aws_region, + # validate_catalog_store=True, + default_timeout=args.default_timeout, + volumes_extra=["mzdata:/mzdata"], + external_minio=True, + fivetran_destination=True, + fivetran_destination_files_path="/share/tmp", + entrypoint_extra=[f"--var=uses-redpanda={args.redpanda}"], + ) + + sysparams = args.system_param + if not args.system_param: + sysparams = [] + + additional_system_parameter_defaults = {} + for val in sysparams: + x = val[0].split("=", maxsplit=1) + assert len(x) == 2, f"--system-param '{val}' should be the format =" + key = x[0] + val = x[1] + + additional_system_parameter_defaults[key] = val + + mz_old = Materialized( + default_size=Materialized.Size.DEFAULT_SIZE, + image=get_old_image_for_source_table_migration_test(), + external_metadata_store=True, + external_minio=True, + additional_system_parameter_defaults=additional_system_parameter_defaults, + ) + mz_new = Materialized( + default_size=Materialized.Size.DEFAULT_SIZE, + image=get_new_image_for_source_table_migration_test(), + external_metadata_store=True, + external_minio=True, + additional_system_parameter_defaults=additional_system_parameter_defaults, + ) + + for file in matching_files: + with c.override(testdrive, mz_old): + c.up(*dependencies) + + c.sql( + "ALTER SYSTEM SET max_clusters = 50;", + port=6877, + user="mz_system", + ) + + non_default_testdrive_vars = [] + + if args.replicas > 1: + c.sql("DROP CLUSTER quickstart CASCADE", user="mz_system", port=6877) + # Make sure a replica named 'r1' always exists + replica_names = [ + "r1" if replica_id == 0 else f"replica{replica_id}" + for replica_id in range(0, args.replicas) + ] + replica_string = ",".join( + f"{replica_name} (SIZE '{mz_old.default_replica_size}')" + for replica_name in replica_names + ) + c.sql( + f"CREATE CLUSTER quickstart REPLICAS ({replica_string})", + user="mz_system", + port=6877, + ) + + # Note that any command that outputs SHOW CLUSTERS will have output + # that depends on the number of replicas testdrive has. This means + # it might be easier to skip certain tests if the number of replicas + # is > 1. + c.sql( + f""" + CREATE CLUSTER testdrive_single_replica_cluster SIZE = '{mz_old.default_replica_size}'; + GRANT ALL PRIVILEGES ON CLUSTER testdrive_single_replica_cluster TO materialize; + """, + user="mz_system", + port=6877, + ) + + non_default_testdrive_vars.append(f"--var=replicas={args.replicas}") + non_default_testdrive_vars.append( + "--var=single-replica-cluster=testdrive_single_replica_cluster" + ) + + non_default_testdrive_vars.append( + f"--var=default-replica-size={mz_old.default_replica_size}" + ) + non_default_testdrive_vars.append( + f"--var=default-storage-size={mz_old.default_storage_size}" + ) + + print(f"Running {file} with mz_old") + + c.run_testdrive_files( + *non_default_testdrive_vars, + "--no-reset", + file, + ) + + c.kill("materialized", wait=True) + + with c.override(mz_new): + c.up("materialized") + + print("Running mz_new") + verify_sources_after_source_table_migration(c, file) + + c.kill("materialized", wait=True) + c.kill("postgres", wait=True) + c.kill("mysql", wait=True) + c.kill(METADATA_STORE, wait=True) + + for dep in kafka_deps: + c.kill(dep, wait=True) + + for dep in kafka_deps: + c.rm(dep) + + c.rm("materialized") + c.rm(METADATA_STORE) + c.rm("postgres") + c.rm("mysql") + + # remove the testdrive container which uses the mzdata volume + testdrive_container_id = spawn.capture( + ["docker", "ps", "-a", "--filter", f"volume={c.name}_mzdata", "-q"] + ).strip() + spawn.runv(["docker", "rm", testdrive_container_id]) + + c.rm_volumes("mzdata", force=True) From 08b674883918ead051175b3361217f2207d29d92 Mon Sep 17 00:00:00 2001 From: Rainer Niedermayr Date: Tue, 29 Oct 2024 09:32:42 +0100 Subject: [PATCH 33/50] migration tests: improve output --- misc/python/materialize/source_table_migration.py | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/misc/python/materialize/source_table_migration.py b/misc/python/materialize/source_table_migration.py index 21ecc1ee4d7c1..ffafc1d7c3dc4 100644 --- a/misc/python/materialize/source_table_migration.py +++ b/misc/python/materialize/source_table_migration.py @@ -22,6 +22,8 @@ def verify_sources_after_source_table_migration( print(f"Sources created in {file} are: {source_names}") + c.sql("SET statement_timeout = '20s'") + for source_name in source_names: _verify_source(c, file, source_name, fail=fail) @@ -31,11 +33,15 @@ def _verify_source( ) -> None: try: print(f"Checking source: {source_name}") + # must not crash - c.sql("SET statement_timeout = '20s'") - c.sql_query(f"SELECT count(*) FROM {source_name};") + statement = f"SELECT count(*) FROM {source_name};" + print(statement) + c.sql_query(statement) - result = c.sql_query(f"SHOW CREATE SOURCE {source_name};") + statement = f"SHOW CREATE SOURCE {source_name};" + print(statement) + result = c.sql_query(statement) sql = result[0][1] assert "FOR TABLE" not in sql, f"FOR TABLE found in: {sql}" assert "FOR ALL TABLES" not in sql, f"FOR ALL TABLES found in: {sql}" From 9879b1b3775d81a0dc4d97398e1b153c86e04e7d Mon Sep 17 00:00:00 2001 From: Rainer Niedermayr Date: Tue, 29 Oct 2024 14:15:53 +0100 Subject: [PATCH 34/50] migration tests: fixes --- test/mysql-cdc-old-syntax/mzcompose.py | 6 ++---- test/testdrive-old-kafka-src-syntax/mzcompose.py | 2 ++ 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/test/mysql-cdc-old-syntax/mzcompose.py b/test/mysql-cdc-old-syntax/mzcompose.py index 46986c98c3190..308cd4a9c2856 100644 --- a/test/mysql-cdc-old-syntax/mzcompose.py +++ b/test/mysql-cdc-old-syntax/mzcompose.py @@ -305,9 +305,6 @@ def workflow_migration(c: Composition, parser: WorkflowArgumentParser) -> None: for filter in args.filter: matching_files.extend(glob.glob(filter, root_dir="test/mysql-cdc-old-syntax")) - # TODO: database-issues#8706 - matching_files = [file for file in matching_files if file != "alter-source.td"] - sharded_files: list[str] = sorted( buildkite.shard_list(matching_files, lambda file: file) ) @@ -333,7 +330,8 @@ def workflow_migration(c: Composition, parser: WorkflowArgumentParser) -> None: external_metadata_store=True, external_minio=True, additional_system_parameter_defaults={ - "log_filter": "mz_storage::source::mysql=trace,info" + "log_filter": "mz_storage::source::mysql=trace,info", + "force_source_table_syntax": "true", }, ) diff --git a/test/testdrive-old-kafka-src-syntax/mzcompose.py b/test/testdrive-old-kafka-src-syntax/mzcompose.py index 8135b14a74295..8fb3f627de6e3 100644 --- a/test/testdrive-old-kafka-src-syntax/mzcompose.py +++ b/test/testdrive-old-kafka-src-syntax/mzcompose.py @@ -343,6 +343,8 @@ def workflow_migration(c: Composition, parser: WorkflowArgumentParser) -> None: additional_system_parameter_defaults[key] = val + additional_system_parameter_defaults["force_source_table_syntax"] = "true" + mz_old = Materialized( default_size=Materialized.Size.DEFAULT_SIZE, image=get_old_image_for_source_table_migration_test(), From 6ac9b65659b1d453c0ad7bf1f2c45204133b99a3 Mon Sep 17 00:00:00 2001 From: Rainer Niedermayr Date: Tue, 29 Oct 2024 16:12:33 +0100 Subject: [PATCH 35/50] migration tests: fixes --- misc/python/materialize/source_table_migration.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/misc/python/materialize/source_table_migration.py b/misc/python/materialize/source_table_migration.py index ffafc1d7c3dc4..423443da73fb2 100644 --- a/misc/python/materialize/source_table_migration.py +++ b/misc/python/materialize/source_table_migration.py @@ -45,7 +45,10 @@ def _verify_source( sql = result[0][1] assert "FOR TABLE" not in sql, f"FOR TABLE found in: {sql}" assert "FOR ALL TABLES" not in sql, f"FOR ALL TABLES found in: {sql}" - assert "CREATE SUBSOURCE" not in sql, f"FOR ALL TABLES found in: {sql}" + + if not source_name.endswith("_progress"): + assert "CREATE SUBSOURCE" not in sql, f"CREATE SUBSOURCE found in: {sql}" + print("OK.") except Exception as e: print(f"source-table-migration issue in {file}: {str(e)}") From 03f7f3a9273658b50147268d383c74de7c9c6b0b Mon Sep 17 00:00:00 2001 From: Roshan Jobanputra Date: Tue, 29 Oct 2024 15:17:24 -0400 Subject: [PATCH 36/50] Fix for mysql source being restarted after new table added --- src/storage/src/source/mysql/replication/events.rs | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/storage/src/source/mysql/replication/events.rs b/src/storage/src/source/mysql/replication/events.rs index 14a628e8db25a..56f0016a3faa4 100644 --- a/src/storage/src/source/mysql/replication/events.rs +++ b/src/storage/src/source/mysql/replication/events.rs @@ -197,6 +197,11 @@ pub(super) async fn handle_query_event( (Some("commit"), None) => { is_complete_event = true; } + // Detect `CREATE TABLE ` statements which don't affect existing tables but do + // signify a complete event (e.g. for the purposes of advancing the GTID) + (Some("create"), Some("table")) => { + is_complete_event = true; + } _ => {} } From cb5201871aa995314b7be4ff2ae17efa28843610 Mon Sep 17 00:00:00 2001 From: Roshan Jobanputra Date: Tue, 29 Oct 2024 17:54:37 -0400 Subject: [PATCH 37/50] Avoid needing to rewrite ids of dependent statements by changing the sorting of item updates --- src/adapter/src/catalog/apply.rs | 79 ++++++++++++++++++++---------- src/adapter/src/catalog/migrate.rs | 1 + src/adapter/src/coord.rs | 23 --------- src/sql/src/names.rs | 26 ++++++++++ 4 files changed, 81 insertions(+), 48 deletions(-) diff --git a/src/adapter/src/catalog/apply.rs b/src/adapter/src/catalog/apply.rs index 4b03c9124f8f7..11f2675f85753 100644 --- a/src/adapter/src/catalog/apply.rs +++ b/src/adapter/src/catalog/apply.rs @@ -25,7 +25,7 @@ use mz_catalog::builtin::{ use mz_catalog::durable::objects::{ ClusterKey, DatabaseKey, DurableType, ItemKey, NetworkPolicyKey, RoleKey, SchemaKey, }; -use mz_catalog::durable::{CatalogError, DurableCatalogError, SystemObjectMapping}; +use mz_catalog::durable::{CatalogError, SystemObjectMapping}; use mz_catalog::memory::error::{Error, ErrorKind}; use mz_catalog::memory::objects::{ CatalogEntry, CatalogItem, Cluster, ClusterReplica, DataSourceDesc, Database, Func, Index, Log, @@ -37,6 +37,7 @@ use mz_compute_types::config::ComputeReplicaConfig; use mz_controller::clusters::{ReplicaConfig, ReplicaLogging}; use mz_controller_types::ClusterId; use mz_expr::MirScalarExpr; +use mz_ore::collections::CollectionExt; use mz_ore::tracing::OpenTelemetryContext; use mz_ore::{instrument, soft_assert_no_log}; use mz_pgrepr::oid::INVALID_OID; @@ -54,7 +55,7 @@ use mz_sql::session::vars::{VarError, VarInput}; use mz_sql::{plan, rbac}; use mz_sql_parser::ast::Expr; use mz_storage_types::sources::Timeline; -use tracing::{error, info_span, warn, Instrument}; +use tracing::{info_span, warn, Instrument}; use crate::catalog::state::LocalExpressionCache; use crate::catalog::{BuiltinTableUpdate, CatalogState}; @@ -1023,17 +1024,7 @@ impl CatalogState { } } }; - // We allow sinks to break this invariant due to a know issue with `ALTER SINK`. - // https://github.com/MaterializeInc/materialize/pull/28708. - if !entry.is_sink() && entry.uses().iter().any(|id| *id > entry.id) { - let msg = format!( - "item cannot depend on items with larger GlobalIds, item: {:?}, dependencies: {:?}", - entry, - entry.uses() - ); - error!("internal catalog errr: {msg}"); - return Err(CatalogError::Durable(DurableCatalogError::Internal(msg))); - } + self.insert_entry(entry); } StateDiff::Retraction => { @@ -1887,25 +1878,63 @@ fn sort_updates_inner(updates: Vec) -> Vec { } } - /// Sort item updates by [`CatalogItemId`]. + /// Sort item updates by parsing statements to identify any id-based dependencies within + /// this set of updates and then performing a topological sort. fn sort_item_updates( item_updates: Vec<(mz_catalog::durable::Item, Timestamp, StateDiff)>, ) -> VecDeque<(mz_catalog::durable::Item, Timestamp, StateDiff)> { - item_updates + let mut items_with_dependencies = item_updates .into_iter() - // HACK: due to `ALTER SINK`, sinks can appear before the objects they - // depend upon. Fortunately, because sinks can never have dependencies - // and can never depend upon one another, to fix the topological sort, - // we can just always move sinks to the end. - .sorted_by_key(|(item, _ts, _diff)| { - if item.create_sql.starts_with("CREATE SINK") { - CatalogItemId::User(u64::MAX) - } else { - item.id - } + .map(|(item, ts, diff)| { + let parsed = mz_sql::parse::parse(&item.create_sql) + .expect("failed to parse persisted SQL") + .into_element() + .ast; + let deps = mz_sql::names::raw_item_dependency_ids(&parsed); + + (item.global_id, (deps, (item, ts, diff))) }) + .collect::>(); + let mut visited = BTreeSet::new(); + let mut sorted = Vec::new(); + fn dfs( + id: GlobalId, + visited: &mut BTreeSet, + sorted: &mut Vec, + items_with_dependencies: &BTreeMap< + GlobalId, + ( + BTreeSet, + (mz_catalog::durable::Item, Timestamp, StateDiff), + ), + >, + ) { + visited.insert(id); + let deps = items_with_dependencies + .get(&id) + .map(|(deps, _)| deps) + .expect("item should be in the map"); + for dep in deps { + // We only want to visit dependencies that are in the current set of updates. + if !visited.contains(dep) && items_with_dependencies.contains_key(dep) { + dfs(*dep, visited, sorted, items_with_dependencies); + } + } + sorted.push(id); + } + for id in items_with_dependencies.keys() { + if !visited.contains(id) { + dfs(*id, &mut visited, &mut sorted, &items_with_dependencies); + } + } + // return the values from items_with_dependencies in the order of sorted + sorted + .into_iter() + .filter_map(|id| items_with_dependencies.remove(&id)) + .map(|item| item.1) .collect() } + let item_retractions = sort_item_updates(item_retractions); let item_additions = sort_item_updates(item_additions); diff --git a/src/adapter/src/catalog/migrate.rs b/src/adapter/src/catalog/migrate.rs index 2c645ec9e56d3..196e18250a0f5 100644 --- a/src/adapter/src/catalog/migrate.rs +++ b/src/adapter/src/catalog/migrate.rs @@ -785,6 +785,7 @@ fn ast_rewrite_sources_to_tables( // for all sources that were migrated. _ => { if mz_sql::names::modify_dependency_item_ids(&mut statement, &changed_ids) { + info!("migrate: updated dependency reference in statement {statement}"); item.create_sql = statement.to_ast_string_stable(); updated_items.insert(item.global_id, item); } diff --git a/src/adapter/src/coord.rs b/src/adapter/src/coord.rs index 002bf4dd6416f..cde49cb059e8e 100644 --- a/src/adapter/src/coord.rs +++ b/src/adapter/src/coord.rs @@ -1932,29 +1932,6 @@ impl Coordinator { let mut privatelink_connections = BTreeMap::new(); for entry in &entries { - // TODO(database-issues#7922): we should move this invariant into `CatalogEntry`. - mz_ore::soft_assert_or_log!( - // We only expect user objects to objects obey this invariant. - // System objects, for instance, can depend on other system - // objects that belong to a schema that is simply loaded first. - // To meaningfully resolve this, we could need more careful - // loading order or more complex IDs, neither of which seem very - // beneficial. - // - // HACK: sinks are permitted to depend on items with larger IDs, - // due to `ALTER SINK`. - !entry.id().is_user() - || entry.is_sink() - || entry - .uses() - .iter() - .all(|dependency_id| *dependency_id <= entry.id), - "entries should only use to items with lesser `GlobalId`s, but \ - {:?} uses {:?}", - entry.id, - entry.uses() - ); - debug!( "coordinator init: installing {} {}", entry.item().typ(), diff --git a/src/sql/src/names.rs b/src/sql/src/names.rs index d32f247f0e202..4cb7d98412997 100644 --- a/src/sql/src/names.rs +++ b/src/sql/src/names.rs @@ -2352,6 +2352,32 @@ where ResolvedIds::new(visitor.ids) } +#[derive(Debug)] +pub struct RawItemDependencyIds { + pub ids: BTreeSet, +} + +impl<'ast> Visit<'ast, Raw> for RawItemDependencyIds { + fn visit_item_name(&mut self, item_name: &RawItemName) { + if let RawItemName::Id(id, _, _) = item_name { + let parsed_id = id.parse::().unwrap(); + self.ids.insert(parsed_id); + } + } +} + +/// Collect any ID-based dependencies of the provided raw AST node. +pub fn raw_item_dependency_ids<'ast, N>(node: &'ast N) -> BTreeSet +where + N: VisitNode<'ast, Raw>, +{ + let mut deps = RawItemDependencyIds { + ids: BTreeSet::new(), + }; + node.visit(&mut deps); + deps.ids +} + #[derive(Debug)] pub struct ItemDependencyModifier<'a> { pub modified: bool, From 6a42fca2db749c015891b12375377eadbdabb240 Mon Sep 17 00:00:00 2001 From: Joseph Koshakow Date: Thu, 14 Nov 2024 11:35:44 -0500 Subject: [PATCH 38/50] Fix merge skew --- src/adapter/src/catalog/migrate.rs | 22 +++++++++++++--------- 1 file changed, 13 insertions(+), 9 deletions(-) diff --git a/src/adapter/src/catalog/migrate.rs b/src/adapter/src/catalog/migrate.rs index 196e18250a0f5..85b25f474f01c 100644 --- a/src/adapter/src/catalog/migrate.rs +++ b/src/adapter/src/catalog/migrate.rs @@ -14,13 +14,14 @@ use mz_catalog::durable::{Transaction, Item}; use mz_catalog::memory::objects::{StateUpdate,BootstrapStateUpdateKind}; use mz_ore::collections::CollectionExt; use mz_ore::now::NowFn; -use mz_repr::{CatalogItemId, Timestamp}; +use mz_repr::{CatalogItemId, GlobalId, Timestamp}; use mz_sql::ast::display::AstDisplay; use mz_sql::names::FullItemName; use mz_sql_parser::ast::{Raw, Statement}; use semver::Version; use tracing::info; use uuid::Uuid; + // DO NOT add any more imports from `crate` outside of `crate::catalog`. use crate::catalog::open::into_consolidatable_updates_startup; use crate::catalog::state::LocalExpressionCache; @@ -369,7 +370,7 @@ fn ast_rewrite_sources_to_tables( item.create_sql, table ); item.create_sql = Statement::CreateTableFromSource(table).to_ast_string_stable(); - tx.update_item(item.global_id, item)?; + tx.update_item(item.id, item)?; } // Postgres sources are multi-output sources whose subsources are @@ -420,7 +421,7 @@ fn ast_rewrite_sources_to_tables( progress_subsource, }); item.create_sql = stmt.to_ast_string_stable(); - tx.update_item(item.global_id, item)?; + tx.update_item(item.id, item)?; info!("migrate: converted postgres source {stmt} to remove subsource options"); } } @@ -478,7 +479,7 @@ fn ast_rewrite_sources_to_tables( progress_subsource, }); item.create_sql = stmt.to_ast_string_stable(); - tx.update_item(item.global_id, item)?; + tx.update_item(item.id, item)?; info!("migrate: converted mysql source {stmt} to remove subsource options"); } } @@ -705,25 +706,28 @@ fn ast_rewrite_sources_to_tables( let owner_id = item.owner_id.clone(); let privileges = item.privileges.clone(); + let extra_versions = item.extra_versions.clone(); // Update the source statement in the catalog first, since the name will // otherwise conflict with the new table statement. info!("migrate: updated source {} to {source}", item.create_sql); item.create_sql = Statement::CreateSource(source).to_ast_string_stable(); - tx.update_item(item.global_id, item)?; + tx.update_item(item.id, item)?; // Insert the new table statement into the catalog with a new id. let ids = tx.allocate_user_item_ids(1)?; - let new_table_id = ids[0]; + let (new_table_id, new_table_global_id) = ids[0]; info!("migrate: added table {new_table_id}: {table}"); tx.insert_user_item( new_table_id, + new_table_global_id, schema_id, &table_item_name, table.to_ast_string_stable(), owner_id, privileges, &Default::default(), + extra_versions, )?; // We need to move the shard currently attached to the source statement to the // table statement such that the existing data in the shard is preserved and can @@ -737,7 +741,7 @@ fn ast_rewrite_sources_to_tables( .pop() .expect("shard should exist"); tx.insert_collection_metadata(btreemap! { - new_table_id => existing_source_shard, + new_table_global_id => existing_source_shard, source_id => new_source_shard })?; @@ -761,7 +765,7 @@ fn ast_rewrite_sources_to_tables( // We also need to update any other statements that reference the source to use the new // table id/name instead. - changed_ids.insert(source_id, new_table_id); + changed_ids.insert(source_id, new_table_global_id); } // When we upgrade to > rust 1.81 we should use #[expect(unreachable_patterns)] @@ -787,7 +791,7 @@ fn ast_rewrite_sources_to_tables( if mz_sql::names::modify_dependency_item_ids(&mut statement, &changed_ids) { info!("migrate: updated dependency reference in statement {statement}"); item.create_sql = statement.to_ast_string_stable(); - updated_items.insert(item.global_id, item); + updated_items.insert(item.id, item); } } } From 5cb65cfed1a69c8afa2cc5208038dcf523adceac Mon Sep 17 00:00:00 2001 From: Joseph Koshakow Date: Thu, 14 Nov 2024 14:17:41 -0500 Subject: [PATCH 39/50] Fix dependency tracking --- src/adapter/src/catalog/apply.rs | 60 ++++++++++++++++++++++++++------ src/sql/src/names.rs | 29 +++++++++------ 2 files changed, 69 insertions(+), 20 deletions(-) diff --git a/src/adapter/src/catalog/apply.rs b/src/adapter/src/catalog/apply.rs index 11f2675f85753..254159e7a1054 100644 --- a/src/adapter/src/catalog/apply.rs +++ b/src/adapter/src/catalog/apply.rs @@ -37,7 +37,7 @@ use mz_compute_types::config::ComputeReplicaConfig; use mz_controller::clusters::{ReplicaConfig, ReplicaLogging}; use mz_controller_types::ClusterId; use mz_expr::MirScalarExpr; -use mz_ore::collections::CollectionExt; +use mz_ore::collections::{CollectionExt, HashMap}; use mz_ore::tracing::OpenTelemetryContext; use mz_ore::{instrument, soft_assert_no_log}; use mz_pgrepr::oid::INVALID_OID; @@ -101,7 +101,7 @@ impl CatalogState { local_expression_cache: &mut LocalExpressionCache, ) -> Vec> { let mut builtin_table_updates = Vec::with_capacity(updates.len()); - let updates = sort_updates(updates); + let updates = sort_updates(updates, &self); let mut groups: Vec> = Vec::new(); for (_, updates) in &updates.into_iter().group_by(|update| update.ts) { @@ -143,7 +143,7 @@ impl CatalogState { updates: Vec, ) -> Result>, CatalogError> { let mut builtin_table_updates = Vec::with_capacity(updates.len()); - let updates = sort_updates(updates); + let updates = sort_updates(updates, self); for (_, updates) in &updates.into_iter().group_by(|update| update.ts) { let mut retractions = InProgressRetractions::default(); @@ -1733,12 +1733,12 @@ impl CatalogState { } /// Sort [`StateUpdate`]s in timestamp then dependency order -fn sort_updates(mut updates: Vec) -> Vec { +fn sort_updates(mut updates: Vec, state: &CatalogState) -> Vec { let mut sorted_updates = Vec::with_capacity(updates.len()); updates.sort_by_key(|update| update.ts); for (_, updates) in &updates.into_iter().group_by(|update| update.ts) { - let sorted_ts_updates = sort_updates_inner(updates.collect()); + let sorted_ts_updates = sort_updates_inner(updates.collect(), state); sorted_updates.extend(sorted_ts_updates); } @@ -1746,7 +1746,7 @@ fn sort_updates(mut updates: Vec) -> Vec { } /// Sort [`StateUpdate`]s in dependency order for a single timestamp. -fn sort_updates_inner(updates: Vec) -> Vec { +fn sort_updates_inner(updates: Vec, state: &CatalogState) -> Vec { fn push_update( update: T, diff: StateDiff, @@ -1882,7 +1882,16 @@ fn sort_updates_inner(updates: Vec) -> Vec { /// this set of updates and then performing a topological sort. fn sort_item_updates( item_updates: Vec<(mz_catalog::durable::Item, Timestamp, StateDiff)>, + state: &CatalogState, ) -> VecDeque<(mz_catalog::durable::Item, Timestamp, StateDiff)> { + let mut item_id_lookup: HashMap<_, HashMap<_, _>> = HashMap::new(); + for (item, _, _) in &item_updates { + item_id_lookup + .entry(item.schema_id) + .or_insert_with(|| HashMap::new()) + .insert(item.name.clone(), item.global_id); + } + let mut items_with_dependencies = item_updates .into_iter() .map(|(item, ts, diff)| { @@ -1890,9 +1899,40 @@ fn sort_updates_inner(updates: Vec) -> Vec { .expect("failed to parse persisted SQL") .into_element() .ast; - let deps = mz_sql::names::raw_item_dependency_ids(&parsed); + let (mut id_deps, name_deps) = mz_sql::names::raw_item_dependency_ids(&parsed); + + // Convert named deps into ID deps. Ideally this is empty and all dependencies are + // specified by ID. However, this is not the case and require some changes and a + // migration to fix. + for name in name_deps { + let (db, schema, item) = match name.0.len() { + 3 => ( + Some(name.0[0].as_str()), + name.0[1].as_str(), + name.0[2].as_str(), + ), + 2 => (None, name.0[1].as_str(), name.0[2].as_str()), + _ => panic!("Invalid item name: {name:?}"), + }; + let schema = state + .resolve_schema(None, db, schema, &SYSTEM_CONN_ID) + .expect("schema must be loaded before an item"); + // If `name` is not also being applied in this batch then the relative order of + // `item` and `name` doesn't matter, so we can ignore it. + let schema_id = match schema.id { + SchemaSpecifier::Id(id) => id, + SchemaSpecifier::Temporary => { + panic!("temporary item {name:?} persisted as dependency of {item:?}") + } + }; + if let Some(ids) = item_id_lookup.get(&schema_id) { + if let Some(id) = ids.get(item) { + id_deps.insert(*id); + } + } + } - (item.global_id, (deps, (item, ts, diff))) + (item.global_id, (id_deps, (item, ts, diff))) }) .collect::>(); let mut visited = BTreeSet::new(); @@ -1935,8 +1975,8 @@ fn sort_updates_inner(updates: Vec) -> Vec { .collect() } - let item_retractions = sort_item_updates(item_retractions); - let item_additions = sort_item_updates(item_additions); + let item_retractions = sort_item_updates(item_retractions, state); + let item_additions = sort_item_updates(item_additions, state); /// Sort temporary item updates by GlobalId. fn sort_temp_item_updates( diff --git a/src/sql/src/names.rs b/src/sql/src/names.rs index 4cb7d98412997..ebd25677fc38d 100644 --- a/src/sql/src/names.rs +++ b/src/sql/src/names.rs @@ -2353,29 +2353,38 @@ where } #[derive(Debug)] -pub struct RawItemDependencyIds { - pub ids: BTreeSet, +struct RawItemDependencyIds<'a> { + ids: BTreeSet, + names: BTreeSet<&'a UnresolvedItemName>, } -impl<'ast> Visit<'ast, Raw> for RawItemDependencyIds { - fn visit_item_name(&mut self, item_name: &RawItemName) { - if let RawItemName::Id(id, _, _) = item_name { - let parsed_id = id.parse::().unwrap(); - self.ids.insert(parsed_id); +impl<'ast> Visit<'ast, Raw> for RawItemDependencyIds<'ast> { + fn visit_item_name(&mut self, item_name: &'ast RawItemName) { + match item_name { + RawItemName::Name(name) => { + self.names.insert(name); + } + RawItemName::Id(id, _, _) => { + let parsed_id = id.parse::().unwrap(); + self.ids.insert(parsed_id); + } } } } -/// Collect any ID-based dependencies of the provided raw AST node. -pub fn raw_item_dependency_ids<'ast, N>(node: &'ast N) -> BTreeSet +/// Collect any dependencies of the provided raw AST node. +pub fn raw_item_dependency_ids<'ast, N>( + node: &'ast N, +) -> (BTreeSet, BTreeSet<&'ast UnresolvedItemName>) where N: VisitNode<'ast, Raw>, { let mut deps = RawItemDependencyIds { ids: BTreeSet::new(), + names: BTreeSet::new(), }; node.visit(&mut deps); - deps.ids + (deps.ids, deps.names) } #[derive(Debug)] From dc1a48a31c3ef7a86cf13b0426b855929ddd492e Mon Sep 17 00:00:00 2001 From: Joseph Koshakow Date: Thu, 14 Nov 2024 14:18:24 -0500 Subject: [PATCH 40/50] Fix lint --- src/adapter/src/catalog/open.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/adapter/src/catalog/open.rs b/src/adapter/src/catalog/open.rs index 83176ebf4a3b7..2194c8572fac8 100644 --- a/src/adapter/src/catalog/open.rs +++ b/src/adapter/src/catalog/open.rs @@ -535,7 +535,9 @@ impl Catalog { diff: diff.try_into().expect("valid diff"), }) .collect(); - let builtin_table_update = state.apply_updates_for_bootstrap(post_item_updates, &mut local_expr_cache).await; + let builtin_table_update = state + .apply_updates_for_bootstrap(post_item_updates, &mut local_expr_cache) + .await; builtin_table_updates.extend(builtin_table_update); // We don't need to apply the audit logs in memory, yet apply can be expensive when the From b14cc63b15af42a223e323b1029dd5f81df42eb6 Mon Sep 17 00:00:00 2001 From: Joseph Koshakow Date: Thu, 14 Nov 2024 14:54:23 -0500 Subject: [PATCH 41/50] Fix some issues --- src/adapter/src/catalog/apply.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/adapter/src/catalog/apply.rs b/src/adapter/src/catalog/apply.rs index 254159e7a1054..7a1633725671d 100644 --- a/src/adapter/src/catalog/apply.rs +++ b/src/adapter/src/catalog/apply.rs @@ -101,7 +101,7 @@ impl CatalogState { local_expression_cache: &mut LocalExpressionCache, ) -> Vec> { let mut builtin_table_updates = Vec::with_capacity(updates.len()); - let updates = sort_updates(updates, &self); + let updates = sort_updates(updates, self); let mut groups: Vec> = Vec::new(); for (_, updates) in &updates.into_iter().group_by(|update| update.ts) { @@ -1888,7 +1888,7 @@ fn sort_updates_inner(updates: Vec, state: &CatalogState) -> Vec, state: &CatalogState) -> Vec (None, name.0[1].as_str(), name.0[2].as_str()), + 2 => (None, name.0[0].as_str(), name.0[1].as_str()), _ => panic!("Invalid item name: {name:?}"), }; let schema = state From 1763a8676f1194b1f32f15df43ed1fe8d9b42a46 Mon Sep 17 00:00:00 2001 From: Joseph Koshakow Date: Thu, 14 Nov 2024 16:50:25 -0500 Subject: [PATCH 42/50] Fix dependency tracking --- src/adapter/src/catalog/apply.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/adapter/src/catalog/apply.rs b/src/adapter/src/catalog/apply.rs index 7a1633725671d..65a64c162f4a1 100644 --- a/src/adapter/src/catalog/apply.rs +++ b/src/adapter/src/catalog/apply.rs @@ -1912,7 +1912,8 @@ fn sort_updates_inner(updates: Vec, state: &CatalogState) -> Vec (None, name.0[0].as_str(), name.0[1].as_str()), - _ => panic!("Invalid item name: {name:?}"), + // This must be a CTE. + _ => continue, }; let schema = state .resolve_schema(None, db, schema, &SYSTEM_CONN_ID) From ace0355d1e1445955b670daf8bfe37b4510422e3 Mon Sep 17 00:00:00 2001 From: Joseph Koshakow Date: Fri, 15 Nov 2024 10:01:41 -0500 Subject: [PATCH 43/50] Fix merge skew --- src/adapter/src/catalog/open.rs | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/src/adapter/src/catalog/open.rs b/src/adapter/src/catalog/open.rs index 2194c8572fac8..5046004e46966 100644 --- a/src/adapter/src/catalog/open.rs +++ b/src/adapter/src/catalog/open.rs @@ -378,7 +378,6 @@ impl Catalog { let mut post_item_updates = Vec::new(); let mut audit_log_updates = Vec::new(); for (kind, ts, diff) in updates { - let diff = diff.try_into().expect("valid diff"); match kind { BootstrapStateUpdateKind::Role(_) | BootstrapStateUpdateKind::Database(_) @@ -392,7 +391,7 @@ impl Catalog { pre_item_updates.push(StateUpdate { kind: kind.into(), ts, - diff, + diff: diff.try_into().expect("valid diff"), }) } BootstrapStateUpdateKind::IntrospectionSourceIndex(_) @@ -400,13 +399,13 @@ impl Catalog { system_item_updates.push(StateUpdate { kind: kind.into(), ts, - diff, + diff: diff.try_into().expect("valid diff"), }) } BootstrapStateUpdateKind::Item(_) => item_updates.push(StateUpdate { kind: kind.into(), ts, - diff, + diff: diff.try_into().expect("valid diff"), }), BootstrapStateUpdateKind::Comment(_) | BootstrapStateUpdateKind::StorageCollectionMetadata(_) @@ -418,7 +417,7 @@ impl Catalog { audit_log_updates.push(StateUpdate { kind: kind.into(), ts, - diff, + diff: diff.try_into().expect("valid diff"), }); } } From bdb57832b88ad89c6c6223759ed6cb5fa95c5f1f Mon Sep 17 00:00:00 2001 From: Joseph Koshakow Date: Fri, 15 Nov 2024 12:11:40 -0500 Subject: [PATCH 44/50] Update test versions --- misc/python/materialize/source_table_migration.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/misc/python/materialize/source_table_migration.py b/misc/python/materialize/source_table_migration.py index 423443da73fb2..0f56806d2d9ba 100644 --- a/misc/python/materialize/source_table_migration.py +++ b/misc/python/materialize/source_table_migration.py @@ -59,12 +59,12 @@ def _verify_source( def check_source_table_migration_test_sensible() -> None: assert MzVersion.parse_cargo() < MzVersion.parse_mz( - "v0.130.0" + "v0.133.0" ), "migration test probably no longer needed" def get_old_image_for_source_table_migration_test() -> str: - return "materialize/materialized:v0.122.0" + return "materialize/materialized:v0.125.0" def get_new_image_for_source_table_migration_test() -> str | None: From 032cc8434e5ec00ba52ad1d125c802306cf38fdf Mon Sep 17 00:00:00 2001 From: Joseph Koshakow Date: Thu, 12 Dec 2024 16:05:38 -0500 Subject: [PATCH 45/50] More merge skew fixes --- .../materialize/checks/all_checks/source_tables.py | 1 + src/adapter-types/src/dyncfgs.rs | 1 - src/adapter/src/catalog/apply.rs | 2 +- src/adapter/src/catalog/migrate.rs | 11 +++++------ 4 files changed, 7 insertions(+), 8 deletions(-) diff --git a/misc/python/materialize/checks/all_checks/source_tables.py b/misc/python/materialize/checks/all_checks/source_tables.py index c36bb1eb8af12..279ede6a4022e 100644 --- a/misc/python/materialize/checks/all_checks/source_tables.py +++ b/misc/python/materialize/checks/all_checks/source_tables.py @@ -28,6 +28,7 @@ class TableFromPgSource(TableFromSourceBase): suffix = "tbl_from_pg_source" def initialize(self) -> Testdrive: + return Testdrive( self.generic_setup() + dedent( diff --git a/src/adapter-types/src/dyncfgs.rs b/src/adapter-types/src/dyncfgs.rs index 0ab56c9b41144..795a342f284a4 100644 --- a/src/adapter-types/src/dyncfgs.rs +++ b/src/adapter-types/src/dyncfgs.rs @@ -128,5 +128,4 @@ pub fn all_dyncfgs(configs: ConfigSet) -> ConfigSet { .add(&DEFAULT_SINK_PARTITION_STRATEGY) .add(&ENABLE_CONTINUAL_TASK_BUILTINS) .add(&ENABLE_EXPRESSION_CACHE) - .add(&ENABLE_SOURCE_TABLE_MIGRATION) } diff --git a/src/adapter/src/catalog/apply.rs b/src/adapter/src/catalog/apply.rs index 65a64c162f4a1..4eb98bc0f07ad 100644 --- a/src/adapter/src/catalog/apply.rs +++ b/src/adapter/src/catalog/apply.rs @@ -25,7 +25,7 @@ use mz_catalog::builtin::{ use mz_catalog::durable::objects::{ ClusterKey, DatabaseKey, DurableType, ItemKey, NetworkPolicyKey, RoleKey, SchemaKey, }; -use mz_catalog::durable::{CatalogError, SystemObjectMapping}; +use mz_catalog::durable::{CatalogError, SystemObjectMapping}; use mz_catalog::memory::error::{Error, ErrorKind}; use mz_catalog::memory::objects::{ CatalogEntry, CatalogItem, Cluster, ClusterReplica, DataSourceDesc, Database, Func, Index, Log, diff --git a/src/adapter/src/catalog/migrate.rs b/src/adapter/src/catalog/migrate.rs index 85b25f474f01c..bfadeaa3cd2bb 100644 --- a/src/adapter/src/catalog/migrate.rs +++ b/src/adapter/src/catalog/migrate.rs @@ -31,7 +31,7 @@ fn rewrite_ast_items(tx: &mut Transaction<'_>, mut f: F) -> Result<(), anyhow where F: for<'a> FnMut( &'a mut Transaction<'_>, - GlobalId, + CatalogItemId, &'a mut Statement, ) -> Result<(), anyhow::Error>, { @@ -39,8 +39,8 @@ where for mut item in tx.get_items() { let mut stmt = mz_sql::parse::parse(&item.create_sql)?.into_element().ast; - // TODO(alter_table): Switch this to CatalogItemId. - f(tx, item.global_id, &mut stmt).await?; + f(tx, item.id, &mut stmt)?; + item.create_sql = stmt.to_ast_string_stable(); updated_items.insert(item.id, item); @@ -110,8 +110,7 @@ pub(crate) async fn migrate( ast_rewrite_sources_to_tables(tx, now)?; } - rewrite_ast_items(tx, |tx, item, stmt, all_items_and_statements| { - let now = now.clone(); + rewrite_ast_items(tx, |_tx, _id, _stmt| { // Add per-item AST migrations below. // // Each migration should be a function that takes `stmt` (the AST @@ -121,7 +120,7 @@ pub(crate) async fn migrate( // Migration functions may also take `tx` as input to stage // arbitrary changes to the catalog. - Ok(()) + Ok(()) })?; // Load items into catalog. We make sure to consolidate the old updates with the new updates to From 31fb7604a671b6fb461ff0669c3fc5c14f52d18d Mon Sep 17 00:00:00 2001 From: Joseph Koshakow Date: Mon, 16 Dec 2024 14:06:13 -0500 Subject: [PATCH 46/50] Update item sorting to only sort within item groups --- src/adapter/src/catalog/apply.rs | 209 ++++++++++++++--------------- src/catalog/src/durable/objects.rs | 60 +++++---- src/sql/src/names.rs | 35 ----- 3 files changed, 139 insertions(+), 165 deletions(-) diff --git a/src/adapter/src/catalog/apply.rs b/src/adapter/src/catalog/apply.rs index 4eb98bc0f07ad..97c9030c33193 100644 --- a/src/adapter/src/catalog/apply.rs +++ b/src/adapter/src/catalog/apply.rs @@ -37,7 +37,6 @@ use mz_compute_types::config::ComputeReplicaConfig; use mz_controller::clusters::{ReplicaConfig, ReplicaLogging}; use mz_controller_types::ClusterId; use mz_expr::MirScalarExpr; -use mz_ore::collections::{CollectionExt, HashMap}; use mz_ore::tracing::OpenTelemetryContext; use mz_ore::{instrument, soft_assert_no_log}; use mz_pgrepr::oid::INVALID_OID; @@ -101,7 +100,7 @@ impl CatalogState { local_expression_cache: &mut LocalExpressionCache, ) -> Vec> { let mut builtin_table_updates = Vec::with_capacity(updates.len()); - let updates = sort_updates(updates, self); + let updates = sort_updates(updates); let mut groups: Vec> = Vec::new(); for (_, updates) in &updates.into_iter().group_by(|update| update.ts) { @@ -143,7 +142,7 @@ impl CatalogState { updates: Vec, ) -> Result>, CatalogError> { let mut builtin_table_updates = Vec::with_capacity(updates.len()); - let updates = sort_updates(updates, self); + let updates = sort_updates(updates); for (_, updates) in &updates.into_iter().group_by(|update| update.ts) { let mut retractions = InProgressRetractions::default(); @@ -1733,12 +1732,12 @@ impl CatalogState { } /// Sort [`StateUpdate`]s in timestamp then dependency order -fn sort_updates(mut updates: Vec, state: &CatalogState) -> Vec { +fn sort_updates(mut updates: Vec) -> Vec { let mut sorted_updates = Vec::with_capacity(updates.len()); updates.sort_by_key(|update| update.ts); for (_, updates) in &updates.into_iter().group_by(|update| update.ts) { - let sorted_ts_updates = sort_updates_inner(updates.collect(), state); + let sorted_ts_updates = sort_updates_inner(updates.collect()); sorted_updates.extend(sorted_ts_updates); } @@ -1746,7 +1745,7 @@ fn sort_updates(mut updates: Vec, state: &CatalogState) -> Vec, state: &CatalogState) -> Vec { +fn sort_updates_inner(updates: Vec) -> Vec { fn push_update( update: T, diff: StateDiff, @@ -1882,117 +1881,117 @@ fn sort_updates_inner(updates: Vec, state: &CatalogState) -> Vec, - state: &CatalogState, ) -> VecDeque<(mz_catalog::durable::Item, Timestamp, StateDiff)> { - let mut item_id_lookup: HashMap<_, HashMap<_, _>> = HashMap::new(); - for (item, _, _) in &item_updates { - item_id_lookup - .entry(item.schema_id) - .or_insert_with(HashMap::new) - .insert(item.name.clone(), item.global_id); - } - - let mut items_with_dependencies = item_updates - .into_iter() - .map(|(item, ts, diff)| { - let parsed = mz_sql::parse::parse(&item.create_sql) - .expect("failed to parse persisted SQL") - .into_element() - .ast; - let (mut id_deps, name_deps) = mz_sql::names::raw_item_dependency_ids(&parsed); - - // Convert named deps into ID deps. Ideally this is empty and all dependencies are - // specified by ID. However, this is not the case and require some changes and a - // migration to fix. - for name in name_deps { - let (db, schema, item) = match name.0.len() { - 3 => ( - Some(name.0[0].as_str()), - name.0[1].as_str(), - name.0[2].as_str(), - ), - 2 => (None, name.0[0].as_str(), name.0[1].as_str()), - // This must be a CTE. - _ => continue, - }; - let schema = state - .resolve_schema(None, db, schema, &SYSTEM_CONN_ID) - .expect("schema must be loaded before an item"); - // If `name` is not also being applied in this batch then the relative order of - // `item` and `name` doesn't matter, so we can ignore it. - let schema_id = match schema.id { - SchemaSpecifier::Id(id) => id, - SchemaSpecifier::Temporary => { - panic!("temporary item {name:?} persisted as dependency of {item:?}") - } - }; - if let Some(ids) = item_id_lookup.get(&schema_id) { - if let Some(id) = ids.get(item) { - id_deps.insert(*id); - } - } - } - - (item.global_id, (id_deps, (item, ts, diff))) - }) - .collect::>(); - let mut visited = BTreeSet::new(); - let mut sorted = Vec::new(); - fn dfs( - id: GlobalId, - visited: &mut BTreeSet, - sorted: &mut Vec, - items_with_dependencies: &BTreeMap< - GlobalId, - ( - BTreeSet, - (mz_catalog::durable::Item, Timestamp, StateDiff), - ), - >, - ) { - visited.insert(id); - let deps = items_with_dependencies - .get(&id) - .map(|(deps, _)| deps) - .expect("item should be in the map"); - for dep in deps { - // We only want to visit dependencies that are in the current set of updates. - if !visited.contains(dep) && items_with_dependencies.contains_key(dep) { - dfs(*dep, visited, sorted, items_with_dependencies); - } + // Partition items into groups s.t. each item in one group has a predefined order with all + // items in other groups. For example, all sinks are ordered greater than all tables. + let mut types = Vec::new(); + let mut secrets = Vec::new(); + let mut connections = Vec::new(); + let mut sources = Vec::new(); + let mut tables = Vec::new(); + let mut derived_items = Vec::new(); + let mut sinks = Vec::new(); + let mut continual_tasks = Vec::new(); + + for update in item_updates { + match update.0.item_type() { + CatalogItemType::Type => types.push(update), + CatalogItemType::Secret => secrets.push(update), + CatalogItemType::Connection => connections.push(update), + CatalogItemType::Source => sources.push(update), + CatalogItemType::Table => tables.push(update), + CatalogItemType::View + | CatalogItemType::MaterializedView + | CatalogItemType::Index + | CatalogItemType::Func => derived_items.push(update), + CatalogItemType::Sink => sinks.push(update), + CatalogItemType::ContinualTask => continual_tasks.push(update), } - sorted.push(id); } - for id in items_with_dependencies.keys() { - if !visited.contains(id) { - dfs(*id, &mut visited, &mut sorted, &items_with_dependencies); - } + + // Within each group, sort by ID. + for group in [ + &mut types, + &mut secrets, + &mut connections, + &mut sources, + &mut tables, + &mut derived_items, + &mut sinks, + &mut continual_tasks, + ] { + group.sort_by_key(|(item, _, _)| item.id); } - // return the values from items_with_dependencies in the order of sorted - sorted - .into_iter() - .filter_map(|id| items_with_dependencies.remove(&id)) - .map(|item| item.1) + + iter::empty() + .chain(types) + .chain(secrets) + .chain(connections) + .chain(sources) + .chain(tables) + .chain(derived_items) + .chain(sinks) + .chain(continual_tasks) .collect() } - let item_retractions = sort_item_updates(item_retractions, state); - let item_additions = sort_item_updates(item_additions, state); + let item_retractions = sort_item_updates(item_retractions); + let item_additions = sort_item_updates(item_additions); /// Sort temporary item updates by GlobalId. fn sort_temp_item_updates( temp_item_updates: Vec<(TemporaryItem, Timestamp, StateDiff)>, ) -> VecDeque<(TemporaryItem, Timestamp, StateDiff)> { - temp_item_updates - .into_iter() - // HACK: due to `ALTER SINK`, sinks can appear before the objects they - // depend upon. Fortunately, because sinks can never have dependencies - // and can never depend upon one another, to fix the topological sort, - // we can just always move sinks to the end. - .sorted_by_key(|(item, _ts, _diff)| match item.item.typ() { - CatalogItemType::Sink => CatalogItemId::User(u64::MAX), - _ => item.id, - }) + // Partition items into groups s.t. each item in one group has a predefined order with all + // items in other groups. For example, all sinks are ordered greater than all tables. + let mut types = Vec::new(); + let mut secrets = Vec::new(); + let mut connections = Vec::new(); + let mut sources = Vec::new(); + let mut tables = Vec::new(); + let mut derived_items = Vec::new(); + let mut sinks = Vec::new(); + let mut continual_tasks = Vec::new(); + + for update in temp_item_updates { + match update.0.item.typ() { + CatalogItemType::Type => types.push(update), + CatalogItemType::Secret => secrets.push(update), + CatalogItemType::Connection => connections.push(update), + CatalogItemType::Source => sources.push(update), + CatalogItemType::Table => tables.push(update), + CatalogItemType::View + | CatalogItemType::MaterializedView + | CatalogItemType::Index + | CatalogItemType::Func => derived_items.push(update), + CatalogItemType::Sink => sinks.push(update), + CatalogItemType::ContinualTask => continual_tasks.push(update), + } + } + + // Within each group, sort by ID. + for group in [ + &mut types, + &mut secrets, + &mut connections, + &mut sources, + &mut tables, + &mut derived_items, + &mut sinks, + &mut continual_tasks, + ] { + group.sort_by_key(|(item, _, _)| item.id); + } + + iter::empty() + .chain(types) + .chain(secrets) + .chain(connections) + .chain(sources) + .chain(tables) + .chain(derived_items) + .chain(sinks) + .chain(continual_tasks) .collect() } let temp_item_retractions = sort_temp_item_updates(temp_item_retractions); diff --git a/src/catalog/src/durable/objects.rs b/src/catalog/src/durable/objects.rs index e6bb869ce84fb..1d0002461d61f 100644 --- a/src/catalog/src/durable/objects.rs +++ b/src/catalog/src/durable/objects.rs @@ -509,6 +509,12 @@ pub struct Item { pub extra_versions: BTreeMap, } +impl Item { + pub fn item_type(&self) -> CatalogItemType { + item_type(&self.create_sql) + } +} + impl DurableType for Item { type Key = ItemKey; type Value = ItemValue; @@ -1289,32 +1295,36 @@ pub struct ItemValue { } impl ItemValue { - pub(crate) fn item_type(&self) -> CatalogItemType { - // NOTE(benesch): the implementation of this method is hideous, but is - // there a better alternative? Storing the object type alongside the - // `create_sql` would introduce the possibility of skew. - let mut tokens = self.create_sql.split_whitespace(); - assert_eq!(tokens.next(), Some("CREATE")); - match tokens.next() { - Some("TABLE") => CatalogItemType::Table, - Some("SOURCE") | Some("SUBSOURCE") => CatalogItemType::Source, - Some("SINK") => CatalogItemType::Sink, - Some("VIEW") => CatalogItemType::View, - Some("MATERIALIZED") => { - assert_eq!(tokens.next(), Some("VIEW")); - CatalogItemType::MaterializedView - } - Some("CONTINUAL") => { - assert_eq!(tokens.next(), Some("TASK")); - CatalogItemType::ContinualTask - } - Some("INDEX") => CatalogItemType::Index, - Some("TYPE") => CatalogItemType::Type, - Some("FUNCTION") => CatalogItemType::Func, - Some("SECRET") => CatalogItemType::Secret, - Some("CONNECTION") => CatalogItemType::Connection, - _ => panic!("unexpected create sql: {}", self.create_sql), + pub fn item_type(&self) -> CatalogItemType { + item_type(&self.create_sql) + } +} + +fn item_type(create_sql: &str) -> CatalogItemType { + // NOTE(benesch): the implementation of this method is hideous, but is + // there a better alternative? Storing the object type alongside the + // `create_sql` would introduce the possibility of skew. + let mut tokens = create_sql.split_whitespace(); + assert_eq!(tokens.next(), Some("CREATE")); + match tokens.next() { + Some("TABLE") => CatalogItemType::Table, + Some("SOURCE") | Some("SUBSOURCE") => CatalogItemType::Source, + Some("SINK") => CatalogItemType::Sink, + Some("VIEW") => CatalogItemType::View, + Some("MATERIALIZED") => { + assert_eq!(tokens.next(), Some("VIEW")); + CatalogItemType::MaterializedView + } + Some("CONTINUAL") => { + assert_eq!(tokens.next(), Some("TASK")); + CatalogItemType::ContinualTask } + Some("INDEX") => CatalogItemType::Index, + Some("TYPE") => CatalogItemType::Type, + Some("FUNCTION") => CatalogItemType::Func, + Some("SECRET") => CatalogItemType::Secret, + Some("CONNECTION") => CatalogItemType::Connection, + _ => panic!("unexpected create sql: {}", create_sql), } } diff --git a/src/sql/src/names.rs b/src/sql/src/names.rs index ebd25677fc38d..d32f247f0e202 100644 --- a/src/sql/src/names.rs +++ b/src/sql/src/names.rs @@ -2352,41 +2352,6 @@ where ResolvedIds::new(visitor.ids) } -#[derive(Debug)] -struct RawItemDependencyIds<'a> { - ids: BTreeSet, - names: BTreeSet<&'a UnresolvedItemName>, -} - -impl<'ast> Visit<'ast, Raw> for RawItemDependencyIds<'ast> { - fn visit_item_name(&mut self, item_name: &'ast RawItemName) { - match item_name { - RawItemName::Name(name) => { - self.names.insert(name); - } - RawItemName::Id(id, _, _) => { - let parsed_id = id.parse::().unwrap(); - self.ids.insert(parsed_id); - } - } - } -} - -/// Collect any dependencies of the provided raw AST node. -pub fn raw_item_dependency_ids<'ast, N>( - node: &'ast N, -) -> (BTreeSet, BTreeSet<&'ast UnresolvedItemName>) -where - N: VisitNode<'ast, Raw>, -{ - let mut deps = RawItemDependencyIds { - ids: BTreeSet::new(), - names: BTreeSet::new(), - }; - node.visit(&mut deps); - (deps.ids, deps.names) -} - #[derive(Debug)] pub struct ItemDependencyModifier<'a> { pub modified: bool, From ac821faaa5611fdab4a4eb0f4b5925be71c0559a Mon Sep 17 00:00:00 2001 From: Joseph Koshakow Date: Tue, 17 Dec 2024 12:12:01 -0500 Subject: [PATCH 47/50] Experiment for migrate --- src/adapter/src/catalog/open.rs | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/adapter/src/catalog/open.rs b/src/adapter/src/catalog/open.rs index 5046004e46966..cfd169c46055c 100644 --- a/src/adapter/src/catalog/open.rs +++ b/src/adapter/src/catalog/open.rs @@ -515,6 +515,11 @@ impl Catalog { // Include any post-item-updates generated by migrations, and then consolidate // them to ensure diffs are all positive. post_item_updates.extend(migrate_result.post_item_updates); + if let Some(max_ts) = post_item_updates.iter().map(|(_, ts, _)| ts).max().cloned() { + for (_, ts, _) in &mut post_item_updates { + *ts = max_ts; + } + } differential_dataflow::consolidation::consolidate_updates(&mut post_item_updates); } From f56fac05d0fee031d441a853d17a3c11db882bd5 Mon Sep 17 00:00:00 2001 From: Joseph Koshakow Date: Wed, 18 Dec 2024 11:14:41 -0500 Subject: [PATCH 48/50] Fix migration idempotency --- src/adapter/src/catalog/migrate.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/adapter/src/catalog/migrate.rs b/src/adapter/src/catalog/migrate.rs index bfadeaa3cd2bb..f03611ddc90ef 100644 --- a/src/adapter/src/catalog/migrate.rs +++ b/src/adapter/src/catalog/migrate.rs @@ -529,7 +529,7 @@ fn ast_rewrite_sources_to_tables( let tables_for_source = items_with_statements_copied .iter() - .any(|(item, statement)| match statement { + .any(|(_, statement)| match statement { Statement::CreateTableFromSource(stmt) => { let source: GlobalId = match &stmt.source { RawItemName::Name(_) => { From 51647b66ed3f38ba1b1ffd3b8ab82c58692d6223 Mon Sep 17 00:00:00 2001 From: Joseph Koshakow Date: Wed, 18 Dec 2024 11:24:38 -0500 Subject: [PATCH 49/50] Fixup --- src/adapter/src/catalog/apply.rs | 16 ++++++++++++---- src/adapter/src/catalog/open.rs | 1 + 2 files changed, 13 insertions(+), 4 deletions(-) diff --git a/src/adapter/src/catalog/apply.rs b/src/adapter/src/catalog/apply.rs index 97c9030c33193..02b59e787bf1a 100644 --- a/src/adapter/src/catalog/apply.rs +++ b/src/adapter/src/catalog/apply.rs @@ -1885,6 +1885,8 @@ fn sort_updates_inner(updates: Vec) -> Vec { // Partition items into groups s.t. each item in one group has a predefined order with all // items in other groups. For example, all sinks are ordered greater than all tables. let mut types = Vec::new(); + // N.B. Functions can depend on system tables, but not user tables. + let mut funcs = Vec::new(); let mut secrets = Vec::new(); let mut connections = Vec::new(); let mut sources = Vec::new(); @@ -1896,14 +1898,14 @@ fn sort_updates_inner(updates: Vec) -> Vec { for update in item_updates { match update.0.item_type() { CatalogItemType::Type => types.push(update), + CatalogItemType::Func => funcs.push(update), CatalogItemType::Secret => secrets.push(update), CatalogItemType::Connection => connections.push(update), CatalogItemType::Source => sources.push(update), CatalogItemType::Table => tables.push(update), CatalogItemType::View | CatalogItemType::MaterializedView - | CatalogItemType::Index - | CatalogItemType::Func => derived_items.push(update), + | CatalogItemType::Index => derived_items.push(update), CatalogItemType::Sink => sinks.push(update), CatalogItemType::ContinualTask => continual_tasks.push(update), } @@ -1912,6 +1914,7 @@ fn sort_updates_inner(updates: Vec) -> Vec { // Within each group, sort by ID. for group in [ &mut types, + &mut funcs, &mut secrets, &mut connections, &mut sources, @@ -1925,6 +1928,7 @@ fn sort_updates_inner(updates: Vec) -> Vec { iter::empty() .chain(types) + .chain(funcs) .chain(secrets) .chain(connections) .chain(sources) @@ -1945,6 +1949,8 @@ fn sort_updates_inner(updates: Vec) -> Vec { // Partition items into groups s.t. each item in one group has a predefined order with all // items in other groups. For example, all sinks are ordered greater than all tables. let mut types = Vec::new(); + // N.B. Functions can depend on system tables, but not user tables. + let mut funcs = Vec::new(); let mut secrets = Vec::new(); let mut connections = Vec::new(); let mut sources = Vec::new(); @@ -1956,14 +1962,14 @@ fn sort_updates_inner(updates: Vec) -> Vec { for update in temp_item_updates { match update.0.item.typ() { CatalogItemType::Type => types.push(update), + CatalogItemType::Func => funcs.push(update), CatalogItemType::Secret => secrets.push(update), CatalogItemType::Connection => connections.push(update), CatalogItemType::Source => sources.push(update), CatalogItemType::Table => tables.push(update), CatalogItemType::View | CatalogItemType::MaterializedView - | CatalogItemType::Index - | CatalogItemType::Func => derived_items.push(update), + | CatalogItemType::Index => derived_items.push(update), CatalogItemType::Sink => sinks.push(update), CatalogItemType::ContinualTask => continual_tasks.push(update), } @@ -1972,6 +1978,7 @@ fn sort_updates_inner(updates: Vec) -> Vec { // Within each group, sort by ID. for group in [ &mut types, + &mut funcs, &mut secrets, &mut connections, &mut sources, @@ -1985,6 +1992,7 @@ fn sort_updates_inner(updates: Vec) -> Vec { iter::empty() .chain(types) + .chain(funcs) .chain(secrets) .chain(connections) .chain(sources) diff --git a/src/adapter/src/catalog/open.rs b/src/adapter/src/catalog/open.rs index cfd169c46055c..8cf82938d5dbc 100644 --- a/src/adapter/src/catalog/open.rs +++ b/src/adapter/src/catalog/open.rs @@ -515,6 +515,7 @@ impl Catalog { // Include any post-item-updates generated by migrations, and then consolidate // them to ensure diffs are all positive. post_item_updates.extend(migrate_result.post_item_updates); + // Push everything to the same timestamp so it consolidates cleanly. if let Some(max_ts) = post_item_updates.iter().map(|(_, ts, _)| ts).max().cloned() { for (_, ts, _) in &mut post_item_updates { *ts = max_ts; From 6462bfcded1ef2cd192368697656f9976e6a7bc8 Mon Sep 17 00:00:00 2001 From: Joseph Koshakow Date: Thu, 2 Jan 2025 13:09:23 -0500 Subject: [PATCH 50/50] resolve merge conflicts --- src/adapter/src/catalog/migrate.rs | 35 +++++------------------------- 1 file changed, 6 insertions(+), 29 deletions(-) diff --git a/src/adapter/src/catalog/migrate.rs b/src/adapter/src/catalog/migrate.rs index f03611ddc90ef..4c3e2668b9b4f 100644 --- a/src/adapter/src/catalog/migrate.rs +++ b/src/adapter/src/catalog/migrate.rs @@ -10,8 +10,8 @@ use std::collections::BTreeMap; use mz_catalog::builtin::BuiltinTable; -use mz_catalog::durable::{Transaction, Item}; -use mz_catalog::memory::objects::{StateUpdate,BootstrapStateUpdateKind}; +use mz_catalog::durable::Transaction; +use mz_catalog::memory::objects::{BootstrapStateUpdateKind, StateUpdate}; use mz_ore::collections::CollectionExt; use mz_ore::now::NowFn; use mz_repr::{CatalogItemId, GlobalId, Timestamp}; @@ -237,8 +237,8 @@ fn ast_rewrite_sources_to_tables( use mz_persist_types::ShardId; use mz_proto::RustType; use mz_sql::ast::{ - CreateSourceConnection, CreateSourceOptionName, CreateSourceStatement, - CreateSubsourceOptionName, CreateSubsourceStatement, CreateTableFromSourceStatement, Ident, + CreateSourceConnection, CreateSourceStatement, CreateSubsourceOptionName, + CreateSubsourceStatement, CreateTableFromSourceStatement, Ident, KafkaSourceConfigOptionName, LoadGenerator, MySqlConfigOptionName, PgConfigOptionName, RawItemName, TableFromSourceColumns, TableFromSourceOption, TableFromSourceOptionName, UnresolvedItemName, Value, WithOptionValue, @@ -515,7 +515,7 @@ fn ast_rewrite_sources_to_tables( include_metadata, format, envelope, - mut with_options, + with_options, if_not_exists, in_cluster, progress_subsource, @@ -607,36 +607,13 @@ fn ast_rewrite_sources_to_tables( } _ => unreachable!("match determined above"), }; - let mut table_with_options = vec![TableFromSourceOption { + let table_with_options = vec![TableFromSourceOption { name: TableFromSourceOptionName::Details, value: Some(WithOptionValue::Value(Value::String(hex::encode( details.into_proto().encode_to_vec(), )))), }]; - // Move over the IgnoreKeys option if it exists. - if let Some(i) = with_options - .iter() - .position(|opt| opt.name == CreateSourceOptionName::IgnoreKeys) - { - let option = with_options.remove(i); - table_with_options.push(TableFromSourceOption { - name: TableFromSourceOptionName::IgnoreKeys, - value: option.value, - }); - }; - // Move over the Timeline option if it exists. - if let Some(i) = with_options - .iter() - .position(|opt| opt.name == CreateSourceOptionName::Timeline) - { - let option = with_options.remove(i); - table_with_options.push(TableFromSourceOption { - name: TableFromSourceOptionName::Timeline, - value: option.value, - }); - } - // Generate the same external-reference that would have been generated // during purification for single-output sources. let external_reference = match &conn {