mz_sql_server_util/cdc.rs
1// Copyright Materialize, Inc. and contributors. All rights reserved.
2//
3// Use of this software is governed by the Business Source License
4// included in the LICENSE file.
5//
6// As of the Change Date specified in that file, in accordance with
7// the Business Source License, use of this software will be governed
8// by the Apache License, Version 2.0.
9
10//! Replicate a table from SQL Server using their Change-Data-Capture (CDC) primitives.
11//!
12//! This module provides a [`CdcStream`] type that provides the following API for
13//! replicating a table:
14//!
15//! 1. [`CdcStream::snapshot`] returns an initial snapshot of a table and the [`Lsn`] at
16//! which the snapshot was taken.
17//! 2. [`CdcStream::into_stream`] returns a [`futures::Stream`] of [`CdcEvent`]s
18//! optionally from the [`Lsn`] returned in step 1.
19//!
20//! The snapshot process is responsible for identifying an [`Lsn`] that corresponds to
21//! a point-in-time view of the data for the table(s) being copied. Similarly to
22//! MySQL, Microsoft SQL server, as far as we know, does not provide an API to
23//! achieve this.
24//!
25//! SQL Server `SNAPSHOT` isolation provides guarantees that a reader will only
26//! see writes committed before the transaction began. More specficially, this
27//! snapshot is implemented using versions that are visibile based on the
28//! transaction sequence number (`XSN`). The `XSN` is set at the first
29//! read or write, not at `BEGIN TRANSACTION`, see [here](https://learn.microsoft.com/en-us/sql/relational-databases/sql-server-transaction-locking-and-row-versioning-guide?view=sql-server-ver17).
30//! This provides us a suitable starting point for capturing the table data.
31//! To force an `XSN` to be assigned, experiments have shown that a table must
32//! be read. We choose a well-known table that we should already have access to,
33//! [cdc.change_tables](https://learn.microsoft.com/en-us/sql/relational-databases/system-tables/cdc-change-tables-transact-sql?view=sql-server-ver17),
34//! and read a single value from it.
35//!
36//! Due to the asynchronous nature of CDC, we can assume that the [`Lsn`]
37//! returned from any CDC tables or CDC functions will always be stale,
38//! in relation to the source table that CDC is tracking. The system table
39//! [sys.dm_tran_database_transactions](https://learn.microsoft.com/en-us/sql/relational-databases/system-dynamic-management-views/sys-dm-tran-database-transactions-transact-sql?view=sql-server-ver17)
40//! will contain an [`Lsn`] for any transaction that performs a write operation.
41//! Creating a savepoint using [SAVE TRANSACTION](https://learn.microsoft.com/en-us/sql/t-sql/language-elements/save-transaction-transact-sql?view=sql-server-ver17)
42//! is sufficient to generate an [`Lsn`] in this case.
43//!
44//! To ensure that the the point-in-time view is established atomically with
45//! collection of the [`Lsn`], we lock the tables to prevent writes from being
46//! interleaved between the 2 commands (read to establish `XSN` and creation of
47//! the savepoint).
48//!
49//! SQL server supports table locks, but those will only be released
50//! once the outermost transaction completes. For this reason, this module
51//! uses two connections for the snapshot process. The first connection is used
52//! to initiate a transaction and lock the upstream tables under
53//! [`TransactionIsolationLevel::ReadCommitted`] isolation. While the first
54//! connection maintains the locks, the second connection starts a
55//! transaction with [`TransactionIsolationLevel::Snapshot`] isolation and
56//! creates a savepoint. Once the savepoint is created, SQL server has assigned
57//! an [`Lsn`] and the the first connection rolls back the transaction.
58//! The [`Lsn`] and snapshot are captured by the second connection within the
59//! existing transaction.
60//!
61//! After completing the snapshot we use [`crate::inspect::get_changes_asc`] which will return
62//! all changes between a `[lower, upper)` bound of [`Lsn`]s.
63
64use std::collections::BTreeMap;
65use std::fmt;
66use std::sync::Arc;
67use std::time::Duration;
68
69use derivative::Derivative;
70use futures::{Stream, StreamExt};
71use mz_repr::GlobalId;
72use proptest_derive::Arbitrary;
73use serde::{Deserialize, Serialize};
74use tiberius::numeric::Numeric;
75
76use crate::desc::{SqlServerQualifiedTableName, SqlServerTableRaw};
77use crate::inspect::DDLEvent;
78use crate::{Client, SqlServerError, TransactionIsolationLevel};
79
80/// A stream of changes from a table in SQL Server that has CDC enabled.
81///
82/// SQL Server does not have an API to push or notify consumers of changes, so we periodically
83/// poll the upstream source.
84///
85/// See: <https://learn.microsoft.com/en-us/sql/relational-databases/system-tables/change-data-capture-tables-transact-sql?view=sql-server-ver16>
86pub struct CdcStream<'a> {
87 /// Client we use for querying SQL Server.
88 client: &'a mut Client,
89 /// Upstream capture instances we'll list changes from.
90 capture_instances: BTreeMap<Arc<str>, Option<Lsn>>,
91 /// How often we poll the upstream for changes.
92 poll_interval: Duration,
93 /// How long we'll wait for SQL Server to return a max LSN before taking a snapshot.
94 ///
95 /// Note: When CDC is first enabled in an instance of SQL Server it can take a moment
96 /// for it to "completely" startup. Before starting a `TRANSACTION` for our snapshot
97 /// we'll wait this duration for SQL Server to report an [`Lsn`] and thus indicate CDC is
98 /// ready to go.
99 max_lsn_wait: Duration,
100}
101
102impl<'a> CdcStream<'a> {
103 pub(crate) fn new(
104 client: &'a mut Client,
105 capture_instances: BTreeMap<Arc<str>, Option<Lsn>>,
106 ) -> Self {
107 CdcStream {
108 client,
109 capture_instances,
110 poll_interval: Duration::from_secs(1),
111 max_lsn_wait: Duration::from_secs(10),
112 }
113 }
114
115 /// Set the [`Lsn`] that we should start streaming changes from.
116 ///
117 /// If the provided [`Lsn`] is not available, the stream will return an error
118 /// when first polled.
119 pub fn start_lsn(mut self, capture_instance: &str, lsn: Lsn) -> Self {
120 let start_lsn = self
121 .capture_instances
122 .get_mut(capture_instance)
123 .expect("capture instance does not exist");
124 *start_lsn = Some(lsn);
125 self
126 }
127
128 /// The cadence at which we'll poll the upstream SQL Server database for changes.
129 ///
130 /// Default is 1 second.
131 pub fn poll_interval(mut self, interval: Duration) -> Self {
132 self.poll_interval = interval;
133 self
134 }
135
136 /// The max duration we'll wait for SQL Server to return an [`Lsn`] before taking a
137 /// snapshot.
138 ///
139 /// When CDC is first enabled in SQL Server it can take a moment before it is fully
140 /// setup and starts reporting LSNs.
141 ///
142 /// Default is 10 seconds.
143 pub fn max_lsn_wait(mut self, wait: Duration) -> Self {
144 self.max_lsn_wait = wait;
145 self
146 }
147
148 /// Takes a snapshot of the upstream table that the specified `table` represents.
149 pub async fn snapshot<'b>(
150 &'b mut self,
151 table: &SqlServerTableRaw,
152 worker_id: usize,
153 source_id: GlobalId,
154 ) -> Result<
155 (
156 Lsn,
157 impl Stream<Item = Result<tiberius::Row, SqlServerError>>,
158 ),
159 SqlServerError,
160 > {
161 static SAVEPOINT_NAME: &str = "_mz_snap_";
162
163 // The client that will be used for fencing does not need any special isolation level
164 // as it will be just be locking the table(s).
165 let mut fencing_client = self.client.new_connection().await?;
166 let mut fence_txn = fencing_client.transaction().await?;
167 fence_txn
168 .lock_table_shared(&table.schema_name, &table.name)
169 .await?;
170 tracing::info!(%source_id, %table.schema_name, %table.name, "timely-{worker_id} locked table");
171
172 self.client
173 .set_transaction_isolation(TransactionIsolationLevel::Snapshot)
174 .await?;
175 let mut txn = self.client.transaction().await?;
176 // Creating a savepoint forces a write to the transaction log, which will
177 // assign an LSN, but it does not force a transaction sequence number to be
178 // assigned as far as I can tell. I have not observed any entries added to
179 // `sys.dm_tran_active_snapshot_database_transactions` when creating a savepoint
180 // or when reading system views to retrieve the LSN.
181 //
182 // We choose cdc.change_tables because it is a system table that will exist
183 // when CDC is enabled, it has a well known schema, and as a CDC client,
184 // we should be able to read from it already.
185 let res = txn
186 .simple_query("SELECT TOP 1 object_id FROM cdc.change_tables")
187 .await?;
188 if res.len() != 1 {
189 Err(SqlServerError::InvariantViolated(
190 "No objects found in cdc.change_tables".into(),
191 ))?
192 }
193
194 // Because the table is locked, any write operation has either
195 // completed, or is blocked. The LSN and XSN acquired now will represent a
196 // consistent point-in-time view, such that any committed write will be
197 // visible to this snapshot and the LSN of such a write will be less than
198 // or equal to the LSN captured here. Creating the savepoint sets the LSN,
199 // we can read it after rolling back the locks.
200 txn.create_savepoint(SAVEPOINT_NAME).await?;
201 tracing::info!(%source_id, %table.schema_name, %table.name, %SAVEPOINT_NAME, "timely-{worker_id} created savepoint");
202
203 // Once the savepoint is created (which establishes the XSN and captures the LSN),
204 // the table no longer needs to be locked. Any writes that happen to the upstream table
205 // will have an LSN higher than our captured LSN, and will be read from CDC.
206 fence_txn.rollback().await?;
207
208 let lsn = txn.get_lsn().await?;
209
210 tracing::info!(%source_id, ?lsn, "timely-{worker_id} starting snapshot");
211 let rows = async_stream::try_stream! {
212 {
213 let snapshot_stream = crate::inspect::snapshot(txn.client, table);
214 tokio::pin!(snapshot_stream);
215
216 while let Some(row) = snapshot_stream.next().await {
217 yield row?;
218 }
219 }
220
221 txn.rollback().await?
222 };
223
224 Ok((lsn, rows))
225 }
226
227 /// Consume `self` returning a [`Stream`] of [`CdcEvent`]s.
228 pub fn into_stream(mut self) -> impl Stream<Item = Result<CdcEvent, SqlServerError>> + use<'a> {
229 async_stream::try_stream! {
230 // Initialize all of our start LSNs.
231 self.initialize_start_lsns().await?;
232
233 // When starting the stream we'll emit one progress event if we've already observed
234 // everything the DB currently has.
235 if let Some(starting_lsn) = self.capture_instances.values().filter_map(|x| *x).min() {
236 let db_curr_lsn = crate::inspect::get_max_lsn(self.client).await?;
237 let next_lsn = db_curr_lsn.increment();
238 if starting_lsn >= db_curr_lsn {
239 tracing::debug!(
240 %starting_lsn,
241 %db_curr_lsn,
242 %next_lsn,
243 "yielding initial progress",
244 );
245 yield CdcEvent::Progress { next_lsn };
246 }
247 }
248
249 loop {
250 // Measure the tick before we do any operation so the time it takes
251 // to query SQL Server is included in the time that we wait.
252 let next_tick = tokio::time::Instant::now()
253 .checked_add(self.poll_interval)
254 .expect("tick interval overflowed!");
255
256 // We always check for changes based on the "global" minimum LSN of any
257 // one capture instance.
258 let maybe_curr_lsn = self.capture_instances.values().filter_map(|x| *x).min();
259 let Some(curr_lsn) = maybe_curr_lsn else {
260 tracing::warn!("shutting down CDC stream because nothing to replicate");
261 break;
262 };
263
264 // Get the max LSN for the DB.
265 let db_max_lsn = crate::inspect::get_max_lsn(self.client).await?;
266 tracing::debug!(?db_max_lsn, ?curr_lsn, "got max LSN");
267
268 // If the LSN of the DB has increased then get all of our changes.
269 if db_max_lsn > curr_lsn {
270 for (instance, instance_lsn) in &self.capture_instances {
271 let Some(instance_lsn) = instance_lsn.as_ref() else {
272 tracing::error!(?instance, "found uninitialized LSN!");
273 continue;
274 };
275
276 // We've already replicated everything up-to db_max_lsn, so
277 // nothing to do.
278 if db_max_lsn < *instance_lsn {
279 continue;
280 }
281
282 {
283 // Get a stream of all the changes for the current instance.
284 let changes = crate::inspect::get_changes_asc(
285 self.client,
286 &*instance,
287 *instance_lsn,
288 db_max_lsn,
289 RowFilterOption::AllUpdateOld,
290 )
291 // TODO(sql_server3): Make this chunk size configurable.
292 .ready_chunks(64);
293 let mut changes = std::pin::pin!(changes);
294
295 // Map and stream all the rows to our listener.
296 while let Some(chunk) = changes.next().await {
297 // Group events by LSN.
298 //
299 // TODO(sql_server3): Can we maybe re-use this BTreeMap or these Vec
300 // allocations? Something to be careful of is shrinking the allocations
301 // if/when they grow to large, e.g. from a large spike of changes.
302 // Alternatively we could also use a single Vec here since we know the
303 // changes are ordered by LSN.
304 let mut events: BTreeMap<Lsn, Vec<Operation>> = BTreeMap::default();
305 for change in chunk {
306 let (lsn, operation) = change.and_then(Operation::try_parse)?;
307 events.entry(lsn).or_default().push(operation);
308 }
309
310 // Emit the groups of events.
311 for (lsn, changes) in events {
312 yield CdcEvent::Data {
313 capture_instance: Arc::clone(instance),
314 lsn,
315 changes,
316 };
317 }
318 }
319 }
320
321 let ddl_history = crate::inspect::get_ddl_history(self.client, instance, instance_lsn, &db_max_lsn)
322 .await?;
323 for (table, ddl_events) in ddl_history {
324 for ddl_event in ddl_events {
325 yield CdcEvent::SchemaUpdate {
326 capture_instance: Arc::clone(instance),
327 table: table.clone(),
328 ddl_event
329 }
330 }
331 }
332 }
333
334 // Increment our LSN (`get_changes` is inclusive).
335 //
336 // TODO(sql_server2): We should occassionally check to see how close the LSN we
337 // generate is to the LSN returned from incrementing via SQL Server itself.
338 let next_lsn = db_max_lsn.increment();
339 tracing::debug!(?curr_lsn, ?next_lsn, "incrementing LSN");
340
341 // Notify our listener that we've emitted all changes __less than__ this LSN.
342 //
343 // Note: This aligns well with timely's semantics of progress tracking.
344 yield CdcEvent::Progress { next_lsn };
345
346 // We just listed everything upto next_lsn.
347 for instance_lsn in self.capture_instances.values_mut() {
348 let instance_lsn = instance_lsn.as_mut().expect("should be initialized");
349 // Ensure LSNs don't go backwards.
350 *instance_lsn = std::cmp::max(*instance_lsn, next_lsn);
351 }
352 }
353
354 tokio::time::sleep_until(next_tick).await;
355 }
356 }
357 }
358
359 /// Determine the [`Lsn`] to start streaming changes from.
360 async fn initialize_start_lsns(&mut self) -> Result<(), SqlServerError> {
361 // First, initialize all start LSNs. If a capture instance didn't have
362 // one specified then we'll start from the current max.
363 let max_lsn = crate::inspect::get_max_lsn(self.client).await?;
364 for (_instance, requsted_lsn) in self.capture_instances.iter_mut() {
365 if requsted_lsn.is_none() {
366 requsted_lsn.replace(max_lsn);
367 }
368 }
369
370 // For each instance, ensure their requested LSN is available.
371 for (instance, requested_lsn) in self.capture_instances.iter() {
372 let requested_lsn = requested_lsn
373 .as_ref()
374 .expect("initialized all values above");
375
376 // Get the minimum Lsn available for this instance.
377 let available_lsn = crate::inspect::get_min_lsn(self.client, &*instance).await?;
378
379 // If we cannot start at our desired LSN, we must return an error!.
380 if *requested_lsn < available_lsn {
381 return Err(CdcError::LsnNotAvailable {
382 capture_instance: Arc::clone(instance),
383 requested: *requested_lsn,
384 minimum: available_lsn,
385 }
386 .into());
387 }
388 }
389
390 Ok(())
391 }
392
393 /// If CDC was recently enabled on an instance of SQL Server then it will report
394 /// `NULL` for the minimum LSN of a capture instance and/or the maximum LSN of the
395 /// entire database.
396 ///
397 /// This method runs a retry loop that waits for the upstream DB to report good
398 /// values. It should be called before taking the initial [`CdcStream::snapshot`]
399 /// to ensure the system is ready to proceed with CDC.
400 pub async fn wait_for_ready(&mut self) -> Result<(), SqlServerError> {
401 // Ensure all of the capture instances are reporting an LSN.
402 for instance in self.capture_instances.keys() {
403 crate::inspect::get_min_lsn_retry(self.client, instance, self.max_lsn_wait).await?;
404 }
405
406 // Ensure the database is reporting a max LSN.
407 crate::inspect::get_max_lsn_retry(self.client, self.max_lsn_wait).await?;
408
409 Ok(())
410 }
411}
412
413/// A change event from a [`CdcStream`].
414#[derive(Derivative)]
415#[derivative(Debug)]
416pub enum CdcEvent {
417 /// Changes have occurred upstream.
418 Data {
419 /// The capture instance these changes are for.
420 capture_instance: Arc<str>,
421 /// The LSN that this change occurred at.
422 lsn: Lsn,
423 /// The change itself.
424 changes: Vec<Operation>,
425 },
426 /// We've made progress and observed all the changes less than `next_lsn`.
427 Progress {
428 /// We've received all of the data for [`Lsn`]s __less than__ this one.
429 next_lsn: Lsn,
430 },
431 /// DDL change has occured for the upstream table.
432 SchemaUpdate {
433 /// The capture instance.
434 capture_instance: Arc<str>,
435 /// The upstream table that was updated.
436 table: SqlServerQualifiedTableName,
437 /// DDL event
438 ddl_event: DDLEvent,
439 },
440}
441
442#[derive(Debug, thiserror::Error)]
443pub enum CdcError {
444 #[error(
445 "the requested LSN '{requested:?}' is less than the minimum '{minimum:?}' for `{capture_instance}'"
446 )]
447 LsnNotAvailable {
448 capture_instance: Arc<str>,
449 requested: Lsn,
450 minimum: Lsn,
451 },
452 #[error("failed to get the required column '{column_name}': {error}")]
453 RequiredColumn {
454 column_name: &'static str,
455 error: String,
456 },
457 #[error("failed to cleanup values for '{capture_instance}' at {low_water_mark}")]
458 CleanupFailed {
459 capture_instance: String,
460 low_water_mark: Lsn,
461 },
462}
463
464/// This type is used to represent the progress of each SQL Server instance in
465/// the ingestion dataflow.
466///
467/// A SQL Server LSN is a three part "number" that provides a __total order__
468/// to all transations within a database. Interally we don't really care what
469/// these parts mean, but they are:
470///
471/// 1. A Virtual Log File (VLF) sequence number, bytes [0, 4)
472/// 2. Log block number, bytes [4, 8)
473/// 3. Log record number, bytes [8, 10)
474///
475/// For more info on log sequence numbers in SQL Server see:
476/// <https://learn.microsoft.com/en-us/sql/relational-databases/sql-server-transaction-log-architecture-and-management-guide?view=sql-server-ver16#Logical_Arch>
477///
478/// Note: The derived impl of [`PartialOrd`] and [`Ord`] relies on the field
479/// ordering so do not change it.
480#[derive(
481 Default,
482 Copy,
483 Clone,
484 Debug,
485 Eq,
486 PartialEq,
487 PartialOrd,
488 Ord,
489 Hash,
490 Serialize,
491 Deserialize,
492 Arbitrary,
493)]
494pub struct Lsn {
495 /// Virtual Log File sequence number.
496 pub vlf_id: u32,
497 /// Log block number.
498 pub block_id: u32,
499 /// Log record number.
500 pub record_id: u16,
501}
502
503impl Lsn {
504 const SIZE: usize = 10;
505
506 /// Interpret the provided bytes as an [`Lsn`].
507 pub fn try_from_bytes(bytes: &[u8]) -> Result<Self, String> {
508 if bytes.len() != Self::SIZE {
509 return Err(format!("incorrect length, expected 10 got {}", bytes.len()));
510 }
511
512 let vlf_id: [u8; 4] = bytes[0..4].try_into().expect("known good length");
513 let block_id: [u8; 4] = bytes[4..8].try_into().expect("known good length");
514 let record_id: [u8; 2] = bytes[8..].try_into().expect("known good length");
515
516 Ok(Lsn {
517 vlf_id: u32::from_be_bytes(vlf_id),
518 block_id: u32::from_be_bytes(block_id),
519 record_id: u16::from_be_bytes(record_id),
520 })
521 }
522
523 /// Return the underlying byte slice for this [`Lsn`].
524 pub fn as_bytes(&self) -> [u8; 10] {
525 let mut raw: [u8; Self::SIZE] = [0; 10];
526
527 raw[0..4].copy_from_slice(&self.vlf_id.to_be_bytes());
528 raw[4..8].copy_from_slice(&self.block_id.to_be_bytes());
529 raw[8..].copy_from_slice(&self.record_id.to_be_bytes());
530
531 raw
532 }
533
534 /// Increment this [`Lsn`].
535 ///
536 /// The returned [`Lsn`] may not exist upstream yet, but it's guaranteed to
537 /// sort greater than `self`.
538 pub fn increment(self) -> Lsn {
539 let (record_id, carry) = self.record_id.overflowing_add(1);
540 let (block_id, carry) = self.block_id.overflowing_add(carry.into());
541 let (vlf_id, overflow) = self.vlf_id.overflowing_add(carry.into());
542 assert!(!overflow, "overflowed Lsn, {self:?}");
543
544 Lsn {
545 vlf_id,
546 block_id,
547 record_id,
548 }
549 }
550
551 /// Drops the `record_id` portion of the [`Lsn`] so we can fit an "abbreviation"
552 /// of this [`Lsn`] into a [`u64`], without losing the total order.
553 pub fn abbreviate(&self) -> u64 {
554 let mut abbreviated: u64 = 0;
555
556 #[allow(clippy::as_conversions)]
557 {
558 abbreviated += (self.vlf_id as u64) << 32;
559 abbreviated += self.block_id as u64;
560 }
561
562 abbreviated
563 }
564}
565
566impl fmt::Display for Lsn {
567 fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
568 write!(f, "{}:{}:{}", self.vlf_id, self.block_id, self.record_id)
569 }
570}
571
572impl TryFrom<&[u8]> for Lsn {
573 type Error = String;
574
575 fn try_from(value: &[u8]) -> Result<Self, Self::Error> {
576 Lsn::try_from_bytes(value)
577 }
578}
579
580impl TryFrom<Numeric> for Lsn {
581 type Error = String;
582
583 fn try_from(value: Numeric) -> Result<Self, Self::Error> {
584 if value.dec_part() != 0 {
585 return Err(format!(
586 "LSN expect Numeric(25,0), but found decimal portion {}",
587 value.dec_part()
588 ));
589 }
590 let mut decimal_lsn = value.int_part();
591 // LSN is composed of 4 bytes : 4 bytes : 2 bytes
592 // and MS provided the method to decode that here
593 // https://github.com/microsoft/sql-server-samples/blob/master/samples/features/ssms-templates/Sql/Change%20Data%20Capture/Enumeration/Create%20Function%20fn_convertnumericlsntobinary.sql
594
595 let vlf_id = u32::try_from(decimal_lsn / 10_i128.pow(15))
596 .map_err(|e| format!("Failed to decode vlf_id for lsn {decimal_lsn}: {e:?}"))?;
597 decimal_lsn -= i128::from(vlf_id) * 10_i128.pow(15);
598
599 let block_id = u32::try_from(decimal_lsn / 10_i128.pow(5))
600 .map_err(|e| format!("Failed to decode block_id for lsn {decimal_lsn}: {e:?}"))?;
601 decimal_lsn -= i128::from(block_id) * 10_i128.pow(5);
602
603 let record_id = u16::try_from(decimal_lsn)
604 .map_err(|e| format!("Failed to decode record_id for lsn {decimal_lsn}: {e:?}"))?;
605
606 Ok(Lsn {
607 vlf_id,
608 block_id,
609 record_id,
610 })
611 }
612}
613
614impl columnation::Columnation for Lsn {
615 type InnerRegion = columnation::CopyRegion<Lsn>;
616}
617
618impl timely::progress::Timestamp for Lsn {
619 // No need to describe complex summaries.
620 type Summary = ();
621
622 fn minimum() -> Self {
623 Lsn::default()
624 }
625}
626
627impl timely::progress::PathSummary<Lsn> for () {
628 fn results_in(&self, src: &Lsn) -> Option<Lsn> {
629 Some(*src)
630 }
631
632 fn followed_by(&self, _other: &Self) -> Option<Self> {
633 Some(())
634 }
635}
636
637impl timely::progress::timestamp::Refines<()> for Lsn {
638 fn to_inner(_other: ()) -> Self {
639 use timely::progress::Timestamp;
640 Self::minimum()
641 }
642 fn to_outer(self) -> () {}
643
644 fn summarize(_path: <Self as timely::progress::Timestamp>::Summary) -> () {}
645}
646
647impl timely::order::PartialOrder for Lsn {
648 fn less_equal(&self, other: &Self) -> bool {
649 self <= other
650 }
651
652 fn less_than(&self, other: &Self) -> bool {
653 self < other
654 }
655}
656impl timely::order::TotalOrder for Lsn {}
657
658/// Structured format of an [`Lsn`].
659///
660/// Note: The derived impl of [`PartialOrd`] and [`Ord`] relies on the field
661/// ordering so do not change it.
662#[derive(Copy, Clone, Debug, PartialEq, Eq, PartialOrd, Ord)]
663pub struct StructuredLsn {
664 vlf_id: u32,
665 block_id: u32,
666 record_id: u16,
667}
668
669/// When querying CDC functions like `cdc.fn_cdc_get_all_changes_<capture_instance>` this governs
670/// what content is returned.
671///
672/// Note: There exists another option `All` that exclude the _before_ value from an `UPDATE`. We
673/// don't support this for SQL Server sources yet, so it's not included in this enum.
674///
675/// See: <https://learn.microsoft.com/en-us/sql/relational-databases/system-functions/cdc-fn-cdc-get-all-changes-capture-instance-transact-sql?view=sql-server-ver16#row_filter_option>
676#[derive(Debug, Copy, Clone)]
677pub enum RowFilterOption {
678 /// Includes both the before and after values of a row when changed because of an `UPDATE`.
679 AllUpdateOld,
680}
681
682impl RowFilterOption {
683 /// Returns this option formatted in a way that can be used in a query.
684 pub fn to_sql_string(&self) -> &'static str {
685 match self {
686 RowFilterOption::AllUpdateOld => "all update old",
687 }
688 }
689}
690
691impl fmt::Display for RowFilterOption {
692 fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
693 write!(f, "{}", self.to_sql_string())
694 }
695}
696
697/// Identifies what change was made to the SQL Server table tracked by CDC.
698#[derive(Debug)]
699pub enum Operation {
700 /// Row was `INSERT`-ed.
701 Insert(tiberius::Row),
702 /// Row was `DELETE`-ed.
703 Delete(tiberius::Row),
704 /// Original value of the row when `UPDATE`-ed.
705 UpdateOld(tiberius::Row),
706 /// New value of the row when `UPDATE`-ed.
707 UpdateNew(tiberius::Row),
708}
709
710impl Operation {
711 /// Parse the provided [`tiberius::Row`] to determine what [`Operation`] occurred.
712 ///
713 /// See <https://learn.microsoft.com/en-us/sql/relational-databases/system-functions/cdc-fn-cdc-get-all-changes-capture-instance-transact-sql?view=sql-server-ver16#table-returned>.
714 fn try_parse(data: tiberius::Row) -> Result<(Lsn, Self), SqlServerError> {
715 static START_LSN_COLUMN: &str = "__$start_lsn";
716 static OPERATION_COLUMN: &str = "__$operation";
717
718 let lsn: &[u8] = data
719 .try_get(START_LSN_COLUMN)
720 .map_err(|e| CdcError::RequiredColumn {
721 column_name: START_LSN_COLUMN,
722 error: e.to_string(),
723 })?
724 .ok_or_else(|| CdcError::RequiredColumn {
725 column_name: START_LSN_COLUMN,
726 error: "got null value".to_string(),
727 })?;
728 let operation: i32 = data
729 .try_get(OPERATION_COLUMN)
730 .map_err(|e| CdcError::RequiredColumn {
731 column_name: OPERATION_COLUMN,
732 error: e.to_string(),
733 })?
734 .ok_or_else(|| CdcError::RequiredColumn {
735 column_name: OPERATION_COLUMN,
736 error: "got null value".to_string(),
737 })?;
738
739 let lsn = Lsn::try_from(lsn).map_err(|msg| SqlServerError::InvalidData {
740 column_name: START_LSN_COLUMN.to_string(),
741 error: msg,
742 })?;
743 let operation = match operation {
744 1 => Operation::Delete(data),
745 2 => Operation::Insert(data),
746 3 => Operation::UpdateOld(data),
747 4 => Operation::UpdateNew(data),
748 other => {
749 return Err(SqlServerError::InvalidData {
750 column_name: OPERATION_COLUMN.to_string(),
751 error: format!("unrecognized operation {other}"),
752 });
753 }
754 };
755
756 Ok((lsn, operation))
757 }
758}
759
760#[cfg(test)]
761mod tests {
762 use super::Lsn;
763 use proptest::prelude::*;
764 use tiberius::numeric::Numeric;
765
766 #[mz_ore::test]
767 fn smoketest_lsn_ordering() {
768 let a = hex::decode("0000003D000019B80004").unwrap();
769 let a = Lsn::try_from(&a[..]).unwrap();
770
771 let b = hex::decode("0000003D000019F00011").unwrap();
772 let b = Lsn::try_from(&b[..]).unwrap();
773
774 let c = hex::decode("0000003D00001A500003").unwrap();
775 let c = Lsn::try_from(&c[..]).unwrap();
776
777 assert!(a < b);
778 assert!(b < c);
779 assert!(a < c);
780
781 assert_eq!(a, a);
782 assert_eq!(b, b);
783 assert_eq!(c, c);
784 }
785
786 #[mz_ore::test]
787 fn smoketest_lsn_roundtrips() {
788 #[track_caller]
789 fn test_case(hex: &str) {
790 let og = hex::decode(hex).unwrap();
791 let lsn = Lsn::try_from(&og[..]).unwrap();
792 let rnd = lsn.as_bytes();
793 assert_eq!(og[..], rnd[..]);
794 }
795
796 test_case("0000003D000019B80004");
797 test_case("0000003D000019F00011");
798 test_case("0000003D00001A500003");
799 }
800
801 #[mz_ore::test]
802 fn proptest_lsn_roundtrips() {
803 #[track_caller]
804 fn test_case(bytes: [u8; 10]) {
805 let lsn = Lsn::try_from_bytes(&bytes[..]).unwrap();
806 let rnd = lsn.as_bytes();
807 assert_eq!(&bytes[..], &rnd[..]);
808 }
809 proptest!(|(random_bytes in any::<[u8; 10]>())| {
810 test_case(random_bytes)
811 })
812 }
813
814 #[mz_ore::test]
815 fn proptest_lsn_increment() {
816 #[track_caller]
817 fn test_case(bytes: [u8; 10]) {
818 let lsn = Lsn::try_from_bytes(&bytes[..]).unwrap();
819 let new = lsn.increment();
820 assert!(lsn < new);
821 }
822 proptest!(|(random_bytes in any::<[u8; 10]>())| {
823 test_case(random_bytes)
824 })
825 }
826
827 #[mz_ore::test]
828 fn proptest_lsn_abbreviate_total_order() {
829 #[track_caller]
830 fn test_case(bytes: [u8; 10], num_increment: u8) {
831 let lsn = Lsn::try_from_bytes(&bytes[..]).unwrap();
832 let mut new = lsn;
833 for _ in 0..num_increment {
834 new = new.increment();
835 }
836
837 let a = lsn.abbreviate();
838 let b = new.abbreviate();
839
840 assert!(a <= b);
841 }
842 proptest!(|(random_bytes in any::<[u8; 10]>(), num_increment in any::<u8>())| {
843 test_case(random_bytes, num_increment)
844 })
845 }
846
847 #[mz_ore::test]
848 fn test_numeric_lsn_ordering() {
849 let a = Lsn::try_from(Numeric::new_with_scale(45_0000008784_00001_i128, 0)).unwrap();
850 let b = Lsn::try_from(Numeric::new_with_scale(45_0000008784_00002_i128, 0)).unwrap();
851 let c = Lsn::try_from(Numeric::new_with_scale(45_0000008785_00002_i128, 0)).unwrap();
852 let d = Lsn::try_from(Numeric::new_with_scale(49_0000008784_00002_i128, 0)).unwrap();
853 assert!(a < b);
854 assert!(b < c);
855 assert!(c < d);
856 assert!(a < d);
857
858 assert_eq!(a, a);
859 assert_eq!(b, b);
860 assert_eq!(c, c);
861 assert_eq!(d, d);
862 }
863
864 #[mz_ore::test]
865 fn test_numeric_lsn_invalid() {
866 let with_decimal = Numeric::new_with_scale(1, 20);
867 assert!(Lsn::try_from(with_decimal).is_err());
868
869 for v in [
870 4294967296_0000000000_00000_i128, // vlf_id is too large
871 1_4294967296_00000_i128, // block_id is too large
872 1_0000000001_65536_i128, // record_id is too large
873 -49_0000008784_00002_i128, // negative is invalid
874 ] {
875 let invalid_lsn = Numeric::new_with_scale(v, 0);
876 assert!(Lsn::try_from(invalid_lsn).is_err());
877 }
878 }
879}