Skip to main content

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