1use std::collections::BTreeMap;
54use std::convert::Infallible;
55use std::fmt;
56use std::io;
57use std::rc::Rc;
58
59use differential_dataflow::AsCollection;
60use differential_dataflow::containers::TimelyStack;
61use itertools::Itertools;
62use mz_mysql_util::quote_identifier;
63use mz_ore::cast::CastFrom;
64use mz_repr::Diff;
65use mz_repr::GlobalId;
66use mz_storage_types::errors::{DataflowError, SourceError};
67use mz_storage_types::sources::SourceExport;
68use mz_timely_util::containers::stack::AccountedStackBuilder;
69use serde::{Deserialize, Serialize};
70use timely::container::CapacityContainerBuilder;
71use timely::dataflow::channels::pushers::Tee;
72use timely::dataflow::operators::core::Partition;
73use timely::dataflow::operators::{CapabilitySet, Concat, Map, ToStream};
74use timely::dataflow::{Scope, Stream};
75use timely::progress::Antichain;
76use uuid::Uuid;
77
78use mz_mysql_util::{
79 MySqlError, MySqlTableDesc, ensure_full_row_binlog_format, ensure_gtid_consistency,
80 ensure_replication_commit_order,
81};
82use mz_ore::error::ErrorExt;
83use mz_storage_types::errors::SourceErrorDetails;
84use mz_storage_types::sources::mysql::{GtidPartition, GtidState, gtid_set_frontier};
85use mz_storage_types::sources::{MySqlSourceConnection, SourceExportDetails, SourceTimestamp};
86use mz_timely_util::builder_async::{AsyncOutputHandle, PressOnDropButton};
87use mz_timely_util::order::Extrema;
88
89use crate::healthcheck::{HealthStatusMessage, HealthStatusUpdate, StatusNamespace};
90use crate::source::types::Probe;
91use crate::source::types::{SourceRender, StackedCollection};
92use crate::source::{RawSourceCreationConfig, SourceMessage};
93
94mod replication;
95mod schemas;
96mod snapshot;
97mod statistics;
98
99impl SourceRender for MySqlSourceConnection {
100 type Time = GtidPartition;
101
102 const STATUS_NAMESPACE: StatusNamespace = StatusNamespace::MySql;
103
104 fn render<G: Scope<Timestamp = GtidPartition>>(
107 self,
108 scope: &mut G,
109 config: &RawSourceCreationConfig,
110 resume_uppers: impl futures::Stream<Item = Antichain<GtidPartition>> + 'static,
111 _start_signal: impl std::future::Future<Output = ()> + 'static,
112 ) -> (
113 BTreeMap<GlobalId, StackedCollection<G, Result<SourceMessage, DataflowError>>>,
114 Stream<G, Infallible>,
115 Stream<G, HealthStatusMessage>,
116 Option<Stream<G, Probe<GtidPartition>>>,
117 Vec<PressOnDropButton>,
118 ) {
119 let mut source_outputs = Vec::new();
121 for (idx, (id, export)) in config.source_exports.iter().enumerate() {
122 let SourceExport {
123 details,
124 storage_metadata: _,
125 data_config: _,
126 } = export;
127 let details = match details {
128 SourceExportDetails::MySql(details) => details,
129 SourceExportDetails::None => continue,
131 _ => panic!("unexpected source export details: {:?}", details),
132 };
133
134 let desc = details.table.clone();
135 let initial_gtid_set = details.initial_gtid_set.to_string();
136 let resume_upper = Antichain::from_iter(
137 config
138 .source_resume_uppers
139 .get(id)
140 .expect("missing resume upper")
141 .iter()
142 .map(GtidPartition::decode_row),
143 );
144 let name = MySqlTableName::new(&desc.schema_name, &desc.name);
145 source_outputs.push(SourceOutputInfo {
146 output_index: idx,
147 table_name: name.clone(),
148 desc,
149 text_columns: details.text_columns.clone(),
150 exclude_columns: details.exclude_columns.clone(),
151 initial_gtid_set: gtid_set_frontier(&initial_gtid_set).expect("invalid gtid set"),
152 resume_upper,
153 export_id: id.clone(),
154 });
155 }
156
157 let metrics = config.metrics.get_mysql_source_metrics(config.id);
158
159 let (snapshot_updates, rewinds, snapshot_err, snapshot_token) = snapshot::render(
160 scope.clone(),
161 config.clone(),
162 self.clone(),
163 source_outputs.clone(),
164 metrics.snapshot_metrics.clone(),
165 );
166
167 let (repl_updates, uppers, repl_err, repl_token) = replication::render(
168 scope.clone(),
169 config.clone(),
170 self.clone(),
171 source_outputs,
172 &rewinds,
173 metrics,
174 );
175
176 let (stats_err, probe_stream, stats_token) =
177 statistics::render(scope.clone(), config.clone(), self, resume_uppers);
178
179 let updates = snapshot_updates.concat(&repl_updates);
180 let partition_count = u64::cast_from(config.source_exports.len());
181 let data_streams: Vec<_> = updates
182 .inner
183 .partition::<CapacityContainerBuilder<_>, _, _>(
184 partition_count,
185 |((output, data), time, diff): &(
186 (usize, Result<SourceMessage, DataflowError>),
187 _,
188 Diff,
189 )| {
190 let output = u64::cast_from(*output);
191 (output, (data.clone(), time.clone(), diff.clone()))
192 },
193 );
194 let mut data_collections = BTreeMap::new();
195 for (id, data_stream) in config.source_exports.keys().zip_eq(data_streams) {
196 data_collections.insert(*id, data_stream.as_collection());
197 }
198
199 let health_init = std::iter::once(HealthStatusMessage {
200 id: None,
201 namespace: Self::STATUS_NAMESPACE,
202 update: HealthStatusUpdate::Running,
203 })
204 .to_stream(scope);
205
206 let health_errs = snapshot_err
207 .concat(&repl_err)
208 .concat(&stats_err)
209 .map(move |err| {
210 let err_string = err.display_with_causes().to_string();
212 let update = HealthStatusUpdate::halting(err_string.clone(), None);
213
214 let namespace = match err {
215 ReplicationError::Transient(err)
216 if matches!(&*err, TransientError::MySqlError(MySqlError::Ssh(_))) =>
217 {
218 StatusNamespace::Ssh
219 }
220 _ => Self::STATUS_NAMESPACE,
221 };
222
223 HealthStatusMessage {
224 id: None,
225 namespace: namespace.clone(),
226 update,
227 }
228 });
229 let health = health_init.concat(&health_errs);
230
231 (
232 data_collections,
233 uppers,
234 health,
235 Some(probe_stream),
236 vec![snapshot_token, repl_token, stats_token],
237 )
238 }
239}
240
241#[derive(Clone, Debug)]
242struct SourceOutputInfo {
243 output_index: usize,
244 table_name: MySqlTableName,
245 desc: MySqlTableDesc,
246 text_columns: Vec<String>,
247 exclude_columns: Vec<String>,
248 initial_gtid_set: Antichain<GtidPartition>,
249 resume_upper: Antichain<GtidPartition>,
250 export_id: GlobalId,
251}
252
253#[derive(Clone, Debug, thiserror::Error)]
254pub enum ReplicationError {
255 #[error(transparent)]
256 Transient(#[from] Rc<TransientError>),
257 #[error(transparent)]
258 Definite(#[from] Rc<DefiniteError>),
259}
260
261#[derive(Debug, thiserror::Error)]
263pub enum TransientError {
264 #[error("couldn't decode binlog row")]
265 BinlogRowDecodeError(#[from] mysql_async::binlog::row::BinlogRowToRowError),
266 #[error("stream ended prematurely")]
267 ReplicationEOF,
268 #[error(transparent)]
269 IoError(#[from] io::Error),
270 #[error("sql client error")]
271 SQLClient(#[from] mysql_async::Error),
272 #[error("ident decode error")]
273 IdentError(#[from] mz_sql_parser::ast::IdentError),
274 #[error(transparent)]
275 MySqlError(#[from] MySqlError),
276 #[error(transparent)]
277 Generic(#[from] anyhow::Error),
278}
279
280#[derive(Debug, Clone, Serialize, Deserialize, thiserror::Error)]
282pub enum DefiniteError {
283 #[error("unable to decode: {0}")]
284 ValueDecodeError(String),
285 #[error("table was truncated: {0}")]
286 TableTruncated(String),
287 #[error("table was dropped: {0}")]
288 TableDropped(String),
289 #[error("incompatible schema change: {0}")]
290 IncompatibleSchema(String),
291 #[error("received a gtid set from the server that violates our requirements: {0}")]
292 UnsupportedGtidState(String),
293 #[error("received out of order gtids for source {0} at transaction-id {1}")]
294 BinlogGtidMonotonicityViolation(String, GtidState),
295 #[error("mysql server does not have the binlog available at the requested gtid set")]
296 BinlogNotAvailable,
297 #[error("mysql server binlog frontier at {0} is beyond required frontier {1}")]
298 BinlogMissingResumePoint(String, String),
299 #[error("mysql server configuration: {0}")]
300 ServerConfigurationError(String),
301}
302
303impl From<DefiniteError> for DataflowError {
304 fn from(err: DefiniteError) -> Self {
305 let m = err.to_string().into();
306 DataflowError::SourceError(Box::new(SourceError {
307 error: match &err {
308 DefiniteError::ValueDecodeError(_) => SourceErrorDetails::Other(m),
309 DefiniteError::TableTruncated(_) => SourceErrorDetails::Other(m),
310 DefiniteError::TableDropped(_) => SourceErrorDetails::Other(m),
311 DefiniteError::IncompatibleSchema(_) => SourceErrorDetails::Other(m),
312 DefiniteError::UnsupportedGtidState(_) => SourceErrorDetails::Other(m),
313 DefiniteError::BinlogGtidMonotonicityViolation(_, _) => {
314 SourceErrorDetails::Other(m)
315 }
316 DefiniteError::BinlogNotAvailable => SourceErrorDetails::Initialization(m),
317 DefiniteError::BinlogMissingResumePoint(_, _) => {
318 SourceErrorDetails::Initialization(m)
319 }
320 DefiniteError::ServerConfigurationError(_) => SourceErrorDetails::Initialization(m),
321 },
322 }))
323 }
324}
325
326#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Ord, Serialize, Deserialize, Hash)]
330pub(crate) struct MySqlTableName(pub(crate) String, pub(crate) String);
331
332impl MySqlTableName {
333 pub(crate) fn new(schema_name: &str, table_name: &str) -> Self {
334 Self(schema_name.to_string(), table_name.to_string())
335 }
336}
337
338impl fmt::Display for MySqlTableName {
339 fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
340 write!(
341 f,
342 "{}.{}",
343 quote_identifier(&self.0),
344 quote_identifier(&self.1)
345 )
346 }
347}
348
349impl From<&MySqlTableDesc> for MySqlTableName {
350 fn from(desc: &MySqlTableDesc) -> Self {
351 Self::new(&desc.schema_name, &desc.name)
352 }
353}
354
355#[derive(Debug, Clone, Deserialize, Serialize)]
356pub(crate) struct RewindRequest {
357 pub(crate) output_index: usize,
359 pub(crate) snapshot_upper: Antichain<GtidPartition>,
362}
363
364type StackedAsyncOutputHandle<T, D> = AsyncOutputHandle<
365 T,
366 AccountedStackBuilder<CapacityContainerBuilder<TimelyStack<(D, T, Diff)>>>,
367 Tee<T, TimelyStack<(D, T, Diff)>>,
368>;
369
370async fn return_definite_error(
371 err: DefiniteError,
372 outputs: &[usize],
373 data_handle: &StackedAsyncOutputHandle<
374 GtidPartition,
375 (usize, Result<SourceMessage, DataflowError>),
376 >,
377 data_cap_set: &CapabilitySet<GtidPartition>,
378 definite_error_handle: &AsyncOutputHandle<
379 GtidPartition,
380 CapacityContainerBuilder<Vec<ReplicationError>>,
381 Tee<GtidPartition, Vec<ReplicationError>>,
382 >,
383 definite_error_cap_set: &CapabilitySet<GtidPartition>,
384) {
385 for output_index in outputs {
386 let update = (
387 (*output_index, Err(err.clone().into())),
388 GtidPartition::new_range(Uuid::minimum(), Uuid::maximum(), GtidState::MAX),
389 Diff::ONE,
390 );
391 data_handle.give_fueled(&data_cap_set[0], update).await;
392 }
393 definite_error_handle.give(
394 &definite_error_cap_set[0],
395 ReplicationError::Definite(Rc::new(err)),
396 );
397 ()
398}
399
400async fn validate_mysql_repl_settings(conn: &mut mysql_async::Conn) -> Result<(), MySqlError> {
401 ensure_gtid_consistency(conn).await?;
402 ensure_full_row_binlog_format(conn).await?;
403 ensure_replication_commit_order(conn).await?;
404
405 Ok(())
406}