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 schema_name = &*table.schema_name;
212        let table_name = &*table.name;
213        let rows = async_stream::try_stream! {
214            {
215                let snapshot_stream = crate::inspect::snapshot(txn.client, &*schema_name, &*table_name);
216                tokio::pin!(snapshot_stream);
217
218                while let Some(row) = snapshot_stream.next().await {
219                    yield row?;
220                }
221            }
222
223            txn.rollback().await?
224        };
225
226        Ok((lsn, rows))
227    }
228
229    /// Consume `self` returning a [`Stream`] of [`CdcEvent`]s.
230    pub fn into_stream(mut self) -> impl Stream<Item = Result<CdcEvent, SqlServerError>> + use<'a> {
231        async_stream::try_stream! {
232            // Initialize all of our start LSNs.
233            self.initialize_start_lsns().await?;
234
235            // When starting the stream we'll emit one progress event if we've already observed
236            // everything the DB currently has.
237            if let Some(starting_lsn) = self.capture_instances.values().filter_map(|x| *x).min() {
238                let db_curr_lsn = crate::inspect::get_max_lsn(self.client).await?;
239                let next_lsn = db_curr_lsn.increment();
240                if starting_lsn >= db_curr_lsn {
241                    tracing::debug!(
242                        %starting_lsn,
243                        %db_curr_lsn,
244                        %next_lsn,
245                        "yielding initial progress",
246                    );
247                    yield CdcEvent::Progress { next_lsn };
248                }
249            }
250
251            loop {
252                // Measure the tick before we do any operation so the time it takes
253                // to query SQL Server is included in the time that we wait.
254                let next_tick = tokio::time::Instant::now()
255                    .checked_add(self.poll_interval)
256                    .expect("tick interval overflowed!");
257
258                // We always check for changes based on the "global" minimum LSN of any
259                // one capture instance.
260                let maybe_curr_lsn = self.capture_instances.values().filter_map(|x| *x).min();
261                let Some(curr_lsn) = maybe_curr_lsn else {
262                    tracing::warn!("shutting down CDC stream because nothing to replicate");
263                    break;
264                };
265
266                // Get the max LSN for the DB.
267                let db_max_lsn = crate::inspect::get_max_lsn(self.client).await?;
268                tracing::debug!(?db_max_lsn, ?curr_lsn, "got max LSN");
269
270                // If the LSN of the DB has increased then get all of our changes.
271                if db_max_lsn > curr_lsn {
272                    for (instance, instance_lsn) in &self.capture_instances {
273                        let Some(instance_lsn) = instance_lsn.as_ref() else {
274                            tracing::error!(?instance, "found uninitialized LSN!");
275                            continue;
276                        };
277
278                        // We've already replicated everything up-to db_max_lsn, so
279                        // nothing to do.
280                        if db_max_lsn < *instance_lsn {
281                            continue;
282                        }
283
284                        {
285                            // Get a stream of all the changes for the current instance.
286                            let changes = crate::inspect::get_changes_asc(
287                                self.client,
288                                &*instance,
289                                *instance_lsn,
290                                db_max_lsn,
291                                RowFilterOption::AllUpdateOld,
292                            )
293                            // TODO(sql_server3): Make this chunk size configurable.
294                            .ready_chunks(64);
295                            let mut changes = std::pin::pin!(changes);
296
297                            // Map and stream all the rows to our listener.
298                            while let Some(chunk) = changes.next().await {
299                                // Group events by LSN.
300                                //
301                                // TODO(sql_server3): Can we maybe re-use this BTreeMap or these Vec
302                                // allocations? Something to be careful of is shrinking the allocations
303                                // if/when they grow to large, e.g. from a large spike of changes.
304                                // Alternatively we could also use a single Vec here since we know the
305                                // changes are ordered by LSN.
306                                let mut events: BTreeMap<Lsn, Vec<Operation>> = BTreeMap::default();
307                                for change in chunk {
308                                    let (lsn, operation) = change.and_then(Operation::try_parse)?;
309                                    events.entry(lsn).or_default().push(operation);
310                                }
311
312                                // Emit the groups of events.
313                                for (lsn, changes) in events {
314                                    yield CdcEvent::Data {
315                                        capture_instance: Arc::clone(instance),
316                                        lsn,
317                                        changes,
318                                    };
319                                }
320                            }
321                        }
322
323                        let ddl_history = crate::inspect::get_ddl_history(self.client, instance, instance_lsn, &db_max_lsn)
324                            .await?;
325                        for (table, ddl_events) in ddl_history {
326                            for ddl_event in ddl_events {
327                                yield CdcEvent::SchemaUpdate {
328                                    capture_instance: Arc::clone(instance),
329                                    table: table.clone(),
330                                    ddl_event
331                                }
332                            }
333                        }
334                    }
335
336                    // Increment our LSN (`get_changes` is inclusive).
337                    //
338                    // TODO(sql_server2): We should occassionally check to see how close the LSN we
339                    // generate is to the LSN returned from incrementing via SQL Server itself.
340                    let next_lsn = db_max_lsn.increment();
341                    tracing::debug!(?curr_lsn, ?next_lsn, "incrementing LSN");
342
343                    // Notify our listener that we've emitted all changes __less than__ this LSN.
344                    //
345                    // Note: This aligns well with timely's semantics of progress tracking.
346                    yield CdcEvent::Progress { next_lsn };
347
348                    // We just listed everything upto next_lsn.
349                    for instance_lsn in self.capture_instances.values_mut() {
350                        let instance_lsn = instance_lsn.as_mut().expect("should be initialized");
351                        // Ensure LSNs don't go backwards.
352                        *instance_lsn = std::cmp::max(*instance_lsn, next_lsn);
353                    }
354                }
355
356                tokio::time::sleep_until(next_tick).await;
357            }
358        }
359    }
360
361    /// Determine the [`Lsn`] to start streaming changes from.
362    async fn initialize_start_lsns(&mut self) -> Result<(), SqlServerError> {
363        // First, initialize all start LSNs. If a capture instance didn't have
364        // one specified then we'll start from the current max.
365        let max_lsn = crate::inspect::get_max_lsn(self.client).await?;
366        for (_instance, requsted_lsn) in self.capture_instances.iter_mut() {
367            if requsted_lsn.is_none() {
368                requsted_lsn.replace(max_lsn);
369            }
370        }
371
372        // For each instance, ensure their requested LSN is available.
373        for (instance, requested_lsn) in self.capture_instances.iter() {
374            let requested_lsn = requested_lsn
375                .as_ref()
376                .expect("initialized all values above");
377
378            // Get the minimum Lsn available for this instance.
379            let available_lsn = crate::inspect::get_min_lsn(self.client, &*instance).await?;
380
381            // If we cannot start at our desired LSN, we must return an error!.
382            if *requested_lsn < available_lsn {
383                return Err(CdcError::LsnNotAvailable {
384                    capture_instance: Arc::clone(instance),
385                    requested: *requested_lsn,
386                    minimum: available_lsn,
387                }
388                .into());
389            }
390        }
391
392        Ok(())
393    }
394
395    /// If CDC was recently enabled on an instance of SQL Server then it will report
396    /// `NULL` for the minimum LSN of a capture instance and/or the maximum LSN of the
397    /// entire database.
398    ///
399    /// This method runs a retry loop that waits for the upstream DB to report good
400    /// values. It should be called before taking the initial [`CdcStream::snapshot`]
401    /// to ensure the system is ready to proceed with CDC.
402    pub async fn wait_for_ready(&mut self) -> Result<(), SqlServerError> {
403        // Ensure all of the capture instances are reporting an LSN.
404        for instance in self.capture_instances.keys() {
405            crate::inspect::get_min_lsn_retry(self.client, instance, self.max_lsn_wait).await?;
406        }
407
408        // Ensure the database is reporting a max LSN.
409        crate::inspect::get_max_lsn_retry(self.client, self.max_lsn_wait).await?;
410
411        Ok(())
412    }
413}
414
415/// A change event from a [`CdcStream`].
416#[derive(Derivative)]
417#[derivative(Debug)]
418pub enum CdcEvent {
419    /// Changes have occurred upstream.
420    Data {
421        /// The capture instance these changes are for.
422        capture_instance: Arc<str>,
423        /// The LSN that this change occurred at.
424        lsn: Lsn,
425        /// The change itself.
426        changes: Vec<Operation>,
427    },
428    /// We've made progress and observed all the changes less than `next_lsn`.
429    Progress {
430        /// We've received all of the data for [`Lsn`]s __less than__ this one.
431        next_lsn: Lsn,
432    },
433    /// DDL change has occured for the upstream table.
434    SchemaUpdate {
435        /// The capture instance.
436        capture_instance: Arc<str>,
437        /// The upstream table that was updated.
438        table: SqlServerQualifiedTableName,
439        /// DDL event
440        ddl_event: DDLEvent,
441    },
442}
443
444#[derive(Debug, thiserror::Error)]
445pub enum CdcError {
446    #[error(
447        "the requested LSN '{requested:?}' is less than the minimum '{minimum:?}' for `{capture_instance}'"
448    )]
449    LsnNotAvailable {
450        capture_instance: Arc<str>,
451        requested: Lsn,
452        minimum: Lsn,
453    },
454    #[error("failed to get the required column '{column_name}': {error}")]
455    RequiredColumn {
456        column_name: &'static str,
457        error: String,
458    },
459    #[error("failed to cleanup values for '{capture_instance}' at {low_water_mark}")]
460    CleanupFailed {
461        capture_instance: String,
462        low_water_mark: Lsn,
463    },
464}
465
466/// This type is used to represent the progress of each SQL Server instance in
467/// the ingestion dataflow.
468///
469/// A SQL Server LSN is a three part "number" that provides a __total order__
470/// to all transations within a database. Interally we don't really care what
471/// these parts mean, but they are:
472///
473/// 1. A Virtual Log File (VLF) sequence number, bytes [0, 4)
474/// 2. Log block number, bytes [4, 8)
475/// 3. Log record number, bytes [8, 10)
476///
477/// For more info on log sequence numbers in SQL Server see:
478/// <https://learn.microsoft.com/en-us/sql/relational-databases/sql-server-transaction-log-architecture-and-management-guide?view=sql-server-ver16#Logical_Arch>
479///
480/// Note: The derived impl of [`PartialOrd`] and [`Ord`] relies on the field
481/// ordering so do not change it.
482#[derive(
483    Default,
484    Copy,
485    Clone,
486    Debug,
487    Eq,
488    PartialEq,
489    PartialOrd,
490    Ord,
491    Hash,
492    Serialize,
493    Deserialize,
494    Arbitrary,
495)]
496pub struct Lsn {
497    /// Virtual Log File sequence number.
498    pub vlf_id: u32,
499    /// Log block number.
500    pub block_id: u32,
501    /// Log record number.
502    pub record_id: u16,
503}
504
505impl Lsn {
506    const SIZE: usize = 10;
507
508    /// Interpret the provided bytes as an [`Lsn`].
509    pub fn try_from_bytes(bytes: &[u8]) -> Result<Self, String> {
510        if bytes.len() != Self::SIZE {
511            return Err(format!("incorrect length, expected 10 got {}", bytes.len()));
512        }
513
514        let vlf_id: [u8; 4] = bytes[0..4].try_into().expect("known good length");
515        let block_id: [u8; 4] = bytes[4..8].try_into().expect("known good length");
516        let record_id: [u8; 2] = bytes[8..].try_into().expect("known good length");
517
518        Ok(Lsn {
519            vlf_id: u32::from_be_bytes(vlf_id),
520            block_id: u32::from_be_bytes(block_id),
521            record_id: u16::from_be_bytes(record_id),
522        })
523    }
524
525    /// Return the underlying byte slice for this [`Lsn`].
526    pub fn as_bytes(&self) -> [u8; 10] {
527        let mut raw: [u8; Self::SIZE] = [0; 10];
528
529        raw[0..4].copy_from_slice(&self.vlf_id.to_be_bytes());
530        raw[4..8].copy_from_slice(&self.block_id.to_be_bytes());
531        raw[8..].copy_from_slice(&self.record_id.to_be_bytes());
532
533        raw
534    }
535
536    /// Increment this [`Lsn`].
537    ///
538    /// The returned [`Lsn`] may not exist upstream yet, but it's guaranteed to
539    /// sort greater than `self`.
540    pub fn increment(self) -> Lsn {
541        let (record_id, carry) = self.record_id.overflowing_add(1);
542        let (block_id, carry) = self.block_id.overflowing_add(carry.into());
543        let (vlf_id, overflow) = self.vlf_id.overflowing_add(carry.into());
544        assert!(!overflow, "overflowed Lsn, {self:?}");
545
546        Lsn {
547            vlf_id,
548            block_id,
549            record_id,
550        }
551    }
552
553    /// Drops the `record_id` portion of the [`Lsn`] so we can fit an "abbreviation"
554    /// of this [`Lsn`] into a [`u64`], without losing the total order.
555    pub fn abbreviate(&self) -> u64 {
556        let mut abbreviated: u64 = 0;
557
558        #[allow(clippy::as_conversions)]
559        {
560            abbreviated += (self.vlf_id as u64) << 32;
561            abbreviated += self.block_id as u64;
562        }
563
564        abbreviated
565    }
566}
567
568impl fmt::Display for Lsn {
569    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
570        write!(f, "{}:{}:{}", self.vlf_id, self.block_id, self.record_id)
571    }
572}
573
574impl TryFrom<&[u8]> for Lsn {
575    type Error = String;
576
577    fn try_from(value: &[u8]) -> Result<Self, Self::Error> {
578        Lsn::try_from_bytes(value)
579    }
580}
581
582impl TryFrom<Numeric> for Lsn {
583    type Error = String;
584
585    fn try_from(value: Numeric) -> Result<Self, Self::Error> {
586        if value.dec_part() != 0 {
587            return Err(format!(
588                "LSN expect Numeric(25,0), but found decimal portion {}",
589                value.dec_part()
590            ));
591        }
592        let mut decimal_lsn = value.int_part();
593        // LSN is composed of 4 bytes : 4 bytes : 2 bytes
594        // and MS provided the method to decode that here
595        // https://github.com/microsoft/sql-server-samples/blob/master/samples/features/ssms-templates/Sql/Change%20Data%20Capture/Enumeration/Create%20Function%20fn_convertnumericlsntobinary.sql
596
597        let vlf_id = u32::try_from(decimal_lsn / 10_i128.pow(15))
598            .map_err(|e| format!("Failed to decode vlf_id for lsn {decimal_lsn}: {e:?}"))?;
599        decimal_lsn -= i128::from(vlf_id) * 10_i128.pow(15);
600
601        let block_id = u32::try_from(decimal_lsn / 10_i128.pow(5))
602            .map_err(|e| format!("Failed to decode block_id for lsn {decimal_lsn}: {e:?}"))?;
603        decimal_lsn -= i128::from(block_id) * 10_i128.pow(5);
604
605        let record_id = u16::try_from(decimal_lsn)
606            .map_err(|e| format!("Failed to decode record_id for lsn {decimal_lsn}: {e:?}"))?;
607
608        Ok(Lsn {
609            vlf_id,
610            block_id,
611            record_id,
612        })
613    }
614}
615
616impl columnation::Columnation for Lsn {
617    type InnerRegion = columnation::CopyRegion<Lsn>;
618}
619
620impl timely::progress::Timestamp for Lsn {
621    // No need to describe complex summaries.
622    type Summary = ();
623
624    fn minimum() -> Self {
625        Lsn::default()
626    }
627}
628
629impl timely::progress::PathSummary<Lsn> for () {
630    fn results_in(&self, src: &Lsn) -> Option<Lsn> {
631        Some(*src)
632    }
633
634    fn followed_by(&self, _other: &Self) -> Option<Self> {
635        Some(())
636    }
637}
638
639impl timely::progress::timestamp::Refines<()> for Lsn {
640    fn to_inner(_other: ()) -> Self {
641        use timely::progress::Timestamp;
642        Self::minimum()
643    }
644    fn to_outer(self) -> () {}
645
646    fn summarize(_path: <Self as timely::progress::Timestamp>::Summary) -> () {}
647}
648
649impl timely::order::PartialOrder for Lsn {
650    fn less_equal(&self, other: &Self) -> bool {
651        self <= other
652    }
653
654    fn less_than(&self, other: &Self) -> bool {
655        self < other
656    }
657}
658impl timely::order::TotalOrder for Lsn {}
659
660/// Structured format of an [`Lsn`].
661///
662/// Note: The derived impl of [`PartialOrd`] and [`Ord`] relies on the field
663/// ordering so do not change it.
664#[derive(Copy, Clone, Debug, PartialEq, Eq, PartialOrd, Ord)]
665pub struct StructuredLsn {
666    vlf_id: u32,
667    block_id: u32,
668    record_id: u16,
669}
670
671/// When querying CDC functions like `cdc.fn_cdc_get_all_changes_<capture_instance>` this governs
672/// what content is returned.
673///
674/// Note: There exists another option `All` that exclude the _before_ value from an `UPDATE`. We
675/// don't support this for SQL Server sources yet, so it's not included in this enum.
676///
677/// 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>
678#[derive(Debug, Copy, Clone)]
679pub enum RowFilterOption {
680    /// Includes both the before and after values of a row when changed because of an `UPDATE`.
681    AllUpdateOld,
682}
683
684impl RowFilterOption {
685    /// Returns this option formatted in a way that can be used in a query.
686    pub fn to_sql_string(&self) -> &'static str {
687        match self {
688            RowFilterOption::AllUpdateOld => "all update old",
689        }
690    }
691}
692
693impl fmt::Display for RowFilterOption {
694    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
695        write!(f, "{}", self.to_sql_string())
696    }
697}
698
699/// Identifies what change was made to the SQL Server table tracked by CDC.
700#[derive(Debug)]
701pub enum Operation {
702    /// Row was `INSERT`-ed.
703    Insert(tiberius::Row),
704    /// Row was `DELETE`-ed.
705    Delete(tiberius::Row),
706    /// Original value of the row when `UPDATE`-ed.
707    UpdateOld(tiberius::Row),
708    /// New value of the row when `UPDATE`-ed.
709    UpdateNew(tiberius::Row),
710}
711
712impl Operation {
713    /// Parse the provided [`tiberius::Row`] to determine what [`Operation`] occurred.
714    ///
715    /// 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>.
716    fn try_parse(data: tiberius::Row) -> Result<(Lsn, Self), SqlServerError> {
717        static START_LSN_COLUMN: &str = "__$start_lsn";
718        static OPERATION_COLUMN: &str = "__$operation";
719
720        let lsn: &[u8] = data
721            .try_get(START_LSN_COLUMN)
722            .map_err(|e| CdcError::RequiredColumn {
723                column_name: START_LSN_COLUMN,
724                error: e.to_string(),
725            })?
726            .ok_or_else(|| CdcError::RequiredColumn {
727                column_name: START_LSN_COLUMN,
728                error: "got null value".to_string(),
729            })?;
730        let operation: i32 = data
731            .try_get(OPERATION_COLUMN)
732            .map_err(|e| CdcError::RequiredColumn {
733                column_name: OPERATION_COLUMN,
734                error: e.to_string(),
735            })?
736            .ok_or_else(|| CdcError::RequiredColumn {
737                column_name: OPERATION_COLUMN,
738                error: "got null value".to_string(),
739            })?;
740
741        let lsn = Lsn::try_from(lsn).map_err(|msg| SqlServerError::InvalidData {
742            column_name: START_LSN_COLUMN.to_string(),
743            error: msg,
744        })?;
745        let operation = match operation {
746            1 => Operation::Delete(data),
747            2 => Operation::Insert(data),
748            3 => Operation::UpdateOld(data),
749            4 => Operation::UpdateNew(data),
750            other => {
751                return Err(SqlServerError::InvalidData {
752                    column_name: OPERATION_COLUMN.to_string(),
753                    error: format!("unrecognized operation {other}"),
754                });
755            }
756        };
757
758        Ok((lsn, operation))
759    }
760}
761
762#[cfg(test)]
763mod tests {
764    use super::Lsn;
765    use proptest::prelude::*;
766    use tiberius::numeric::Numeric;
767
768    #[mz_ore::test]
769    fn smoketest_lsn_ordering() {
770        let a = hex::decode("0000003D000019B80004").unwrap();
771        let a = Lsn::try_from(&a[..]).unwrap();
772
773        let b = hex::decode("0000003D000019F00011").unwrap();
774        let b = Lsn::try_from(&b[..]).unwrap();
775
776        let c = hex::decode("0000003D00001A500003").unwrap();
777        let c = Lsn::try_from(&c[..]).unwrap();
778
779        assert!(a < b);
780        assert!(b < c);
781        assert!(a < c);
782
783        assert_eq!(a, a);
784        assert_eq!(b, b);
785        assert_eq!(c, c);
786    }
787
788    #[mz_ore::test]
789    fn smoketest_lsn_roundtrips() {
790        #[track_caller]
791        fn test_case(hex: &str) {
792            let og = hex::decode(hex).unwrap();
793            let lsn = Lsn::try_from(&og[..]).unwrap();
794            let rnd = lsn.as_bytes();
795            assert_eq!(og[..], rnd[..]);
796        }
797
798        test_case("0000003D000019B80004");
799        test_case("0000003D000019F00011");
800        test_case("0000003D00001A500003");
801    }
802
803    #[mz_ore::test]
804    fn proptest_lsn_roundtrips() {
805        #[track_caller]
806        fn test_case(bytes: [u8; 10]) {
807            let lsn = Lsn::try_from_bytes(&bytes[..]).unwrap();
808            let rnd = lsn.as_bytes();
809            assert_eq!(&bytes[..], &rnd[..]);
810        }
811        proptest!(|(random_bytes in any::<[u8; 10]>())| {
812            test_case(random_bytes)
813        })
814    }
815
816    #[mz_ore::test]
817    fn proptest_lsn_increment() {
818        #[track_caller]
819        fn test_case(bytes: [u8; 10]) {
820            let lsn = Lsn::try_from_bytes(&bytes[..]).unwrap();
821            let new = lsn.increment();
822            assert!(lsn < new);
823        }
824        proptest!(|(random_bytes in any::<[u8; 10]>())| {
825            test_case(random_bytes)
826        })
827    }
828
829    #[mz_ore::test]
830    fn proptest_lsn_abbreviate_total_order() {
831        #[track_caller]
832        fn test_case(bytes: [u8; 10], num_increment: u8) {
833            let lsn = Lsn::try_from_bytes(&bytes[..]).unwrap();
834            let mut new = lsn;
835            for _ in 0..num_increment {
836                new = new.increment();
837            }
838
839            let a = lsn.abbreviate();
840            let b = new.abbreviate();
841
842            assert!(a <= b);
843        }
844        proptest!(|(random_bytes in any::<[u8; 10]>(), num_increment in any::<u8>())| {
845            test_case(random_bytes, num_increment)
846        })
847    }
848
849    #[mz_ore::test]
850    fn test_numeric_lsn_ordering() {
851        let a = Lsn::try_from(Numeric::new_with_scale(45_0000008784_00001_i128, 0)).unwrap();
852        let b = Lsn::try_from(Numeric::new_with_scale(45_0000008784_00002_i128, 0)).unwrap();
853        let c = Lsn::try_from(Numeric::new_with_scale(45_0000008785_00002_i128, 0)).unwrap();
854        let d = Lsn::try_from(Numeric::new_with_scale(49_0000008784_00002_i128, 0)).unwrap();
855        assert!(a < b);
856        assert!(b < c);
857        assert!(c < d);
858        assert!(a < d);
859
860        assert_eq!(a, a);
861        assert_eq!(b, b);
862        assert_eq!(c, c);
863        assert_eq!(d, d);
864    }
865
866    #[mz_ore::test]
867    fn test_numeric_lsn_invalid() {
868        let with_decimal = Numeric::new_with_scale(1, 20);
869        assert!(Lsn::try_from(with_decimal).is_err());
870
871        for v in [
872            4294967296_0000000000_00000_i128, // vlf_id is too large
873            1_4294967296_00000_i128,          // block_id is too large
874            1_0000000001_65536_i128,          // record_id is too large
875            -49_0000008784_00002_i128,        // negative is invalid
876        ] {
877            let invalid_lsn = Numeric::new_with_scale(v, 0);
878            assert!(Lsn::try_from(invalid_lsn).is_err());
879        }
880    }
881}