mz_sql_server_util/
lib.rs

1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
// Copyright Materialize, Inc. and contributors. All rights reserved.
//
// Use of this software is governed by the Business Source License
// included in the LICENSE file.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0.

use std::borrow::Cow;
use std::future::IntoFuture;
use std::pin::Pin;
use std::sync::Arc;

use anyhow::Context;
use derivative::Derivative;
use futures::future::BoxFuture;
use futures::{FutureExt, Stream, StreamExt, TryStreamExt};
use mz_ore::result::ResultExt;
use smallvec::{smallvec, SmallVec};
use tiberius::ToSql;
use tokio::net::TcpStream;
use tokio::sync::mpsc::{UnboundedReceiver, UnboundedSender};
use tokio::sync::oneshot;
use tokio_util::compat::{Compat, TokioAsyncWriteCompatExt};

pub mod cdc;
pub mod desc;
pub mod inspect;

// Re-export tiberius' Config type since it's needed by our Client wrapper.
pub use tiberius::Config;

/// Higher level wrapper around a [`tiberius::Client`] that models transaction
/// management like other database clients.
///
/// When creating a [`Client`] we return a [`Connection`] which implements [`std::future::Future`]
/// and must be polled for queries to make progress. Internally a [`Client`] holds the sending side
/// of a channel and the [`Connection`] receives query requests to run. This enables us to
/// introduce a [`Transaction`] type that when dropped will cause the `TRANSACTION` in the
/// connected SQL Server instance to get rolled back.
#[derive(Debug)]
pub struct Client {
    tx: UnboundedSender<Request>,
}
// While a Client could implement Clone, it's not obvious how multiple Clients
// using the same SQL Server connection would interact, so ban it for now.
static_assertions::assert_not_impl_all!(Client: Clone);

impl Client {
    pub async fn connect(config: tiberius::Config) -> Result<(Self, Connection), SqlServerError> {
        let tcp = TcpStream::connect(config.get_addr()).await?;
        tcp.set_nodelay(true)?;
        Self::connect_raw(config, tcp).await
    }

    pub async fn connect_raw(
        config: tiberius::Config,
        tcp: tokio::net::TcpStream,
    ) -> Result<(Self, Connection), SqlServerError> {
        let client = tiberius::Client::connect(config, tcp.compat_write())
            .await
            .context("connecting to SQL Server")?;
        let (tx, rx) = tokio::sync::mpsc::unbounded_channel();

        Ok((Client { tx }, Connection { rx, client }))
    }

    /// Executes SQL statements in SQL Server, returning the number of rows effected.
    ///
    /// Passthrough method for [`tiberius::Client::execute`].
    ///
    /// Note: The returned [`Future`] does not need to be awaited for the query
    /// to be sent.
    ///
    /// [`Future`]: std::future::Future
    pub async fn execute<'a>(
        &mut self,
        query: impl Into<Cow<'a, str>>,
        params: &[&dyn ToSql],
    ) -> Result<SmallVec<[u64; 1]>, SqlServerError> {
        let (tx, rx) = tokio::sync::oneshot::channel();

        let params = params
            .iter()
            .map(|p| OwnedColumnData::from(p.to_sql()))
            .collect();
        let kind = RequestKind::Execute {
            query: query.into().to_string(),
            params,
        };
        self.tx
            .send(Request { tx, kind })
            .context("sending request")?;

        let response = rx.await.context("channel")?.context("execute")?;
        match response {
            Response::Execute { rows_affected } => Ok(rows_affected),
            other @ Response::Rows(_) | other @ Response::RowStream { .. } => {
                Err(SqlServerError::ProgrammingError(format!(
                    "expected Response::Execute, got {other:?}"
                )))
            }
        }
    }

    /// Executes SQL statements in SQL Server, returning the resulting rows.
    ///
    /// Passthrough method for [`tiberius::Client::query`].
    ///
    /// Note: The returned [`Future`] does not need to be awaited for the query
    /// to be sent.
    ///
    /// [`Future`]: std::future::Future
    pub async fn query<'a>(
        &mut self,
        query: impl Into<Cow<'a, str>>,
        params: &[&dyn tiberius::ToSql],
    ) -> Result<SmallVec<[tiberius::Row; 1]>, SqlServerError> {
        let (tx, rx) = tokio::sync::oneshot::channel();

        let params = params
            .iter()
            .map(|p| OwnedColumnData::from(p.to_sql()))
            .collect();
        let kind = RequestKind::Query {
            query: query.into().to_string(),
            params,
        };
        self.tx
            .send(Request { tx, kind })
            .context("sending request")?;

        let response = rx.await.context("channel")?.context("query")?;
        match response {
            Response::Rows(rows) => Ok(rows),
            other @ Response::Execute { .. } | other @ Response::RowStream { .. } => Err(
                SqlServerError::ProgrammingError(format!("expected Response::Rows, got {other:?}")),
            ),
        }
    }

    /// Executes SQL statements in SQL Server, returning a [`Stream`] of
    /// resulting rows.
    ///
    /// Passthrough method for [`tiberius::Client::query`].
    pub fn query_streaming<'a>(
        &mut self,
        query: impl Into<Cow<'a, str>>,
        params: &[&dyn tiberius::ToSql],
    ) -> impl Stream<Item = Result<tiberius::Row, SqlServerError>> + Send + '_ {
        let (tx, rx) = tokio::sync::oneshot::channel();
        let params = params
            .iter()
            .map(|p| OwnedColumnData::from(p.to_sql()))
            .collect();
        let kind = RequestKind::QueryStreamed {
            query: query.into().to_string(),
            params,
        };

        // Make our initial request which will return a Stream of Rows.
        let request_future = async move {
            self.tx
                .send(Request { tx, kind })
                .context("sending request")?;

            let response = rx.await.context("channel")??;
            match response {
                Response::RowStream { stream } => {
                    Ok(tokio_stream::wrappers::ReceiverStream::new(stream))
                }
                other @ Response::Execute { .. } | other @ Response::Rows(_) => {
                    Err(SqlServerError::ProgrammingError(format!(
                        "expected Response::Rows, got {other:?}"
                    )))
                }
            }
        };

        // "flatten" our initial request into the returned stream.
        futures::stream::once(request_future).try_flatten()
    }

    /// Executes multiple queries, delimited with `;` and return multiple
    /// result sets; one for each query.
    ///
    /// Passthrough method for [`tiberius::Client::simple_query`].
    ///
    /// Note: The returned [`Future`] does not need to be awaited for the query
    /// to be sent.
    ///
    /// [`Future`]: std::future::Future
    pub async fn simple_query<'a>(
        &mut self,
        query: impl Into<Cow<'a, str>>,
    ) -> Result<SmallVec<[tiberius::Row; 1]>, SqlServerError> {
        let (tx, rx) = tokio::sync::oneshot::channel();
        let kind = RequestKind::SimpleQuery {
            query: query.into().to_string(),
        };
        self.tx
            .send(Request { tx, kind })
            .context("sending request")?;

        let response = rx.await.context("channel")?.context("simple_query")?;
        match response {
            Response::Rows(rows) => Ok(rows),
            other @ Response::Execute { .. } | other @ Response::RowStream { .. } => Err(
                SqlServerError::ProgrammingError(format!("expected Response::Rows, got {other:?}")),
            ),
        }
    }

    /// Starts a transaction which is automatically rolled back on drop.
    ///
    /// To commit or rollback the transaction, see [`Transaction::commit`] and
    /// [`Transaction::rollback`] respectively.
    pub async fn transaction(&mut self) -> Result<Transaction<'_>, SqlServerError> {
        Transaction::new(self).await
    }

    /// Sets the transaction isolation level for the current session.
    pub async fn set_transaction_isolation(
        &mut self,
        level: TransactionIsolationLevel,
    ) -> Result<(), SqlServerError> {
        let query = format!("SET TRANSACTION ISOLATION LEVEL {}", level.as_str());
        self.simple_query(query).await?;
        Ok(())
    }

    /// Returns the current transaction isolation level for the current session.
    pub async fn get_transaction_isolation(
        &mut self,
    ) -> Result<TransactionIsolationLevel, SqlServerError> {
        const QUERY: &str = "SELECT transaction_isolation_level FROM sys.dm_exec_sessions where session_id = @@SPID;";
        let rows = self.simple_query(QUERY).await?;
        match &rows[..] {
            [row] => {
                let val: i16 = row
                    .try_get(0)
                    .context("getting 0th column")?
                    .ok_or_else(|| anyhow::anyhow!("no 0th column?"))?;
                let level = TransactionIsolationLevel::try_from_sql_server(val)?;
                Ok(level)
            }
            other => Err(SqlServerError::InvariantViolated(format!(
                "expected one row, got {other:?}"
            ))),
        }
    }

    /// Return a [`CdcStream`] that can be used to track changes for the specified
    /// `capture_instances`.
    ///
    /// [`CdcStream`]: crate::cdc::CdcStream
    pub fn cdc<I>(&mut self, capture_instances: I) -> crate::cdc::CdcStream<'_>
    where
        I: IntoIterator,
        I::Item: Into<Arc<str>>,
    {
        let instances = capture_instances
            .into_iter()
            .map(|i| (i.into(), None))
            .collect();
        crate::cdc::CdcStream::new(self, instances)
    }
}

/// A stream of [`tiberius::Row`]s.
pub type RowStream<'a> =
    Pin<Box<dyn Stream<Item = Result<tiberius::Row, SqlServerError>> + Send + 'a>>;

#[derive(Debug)]
pub struct Transaction<'a> {
    client: &'a mut Client,
    closed: bool,
}

impl<'a> Transaction<'a> {
    async fn new(client: &'a mut Client) -> Result<Self, SqlServerError> {
        let results = client
            .simple_query("BEGIN TRANSACTION")
            .await
            .context("begin")?;
        if !results.is_empty() {
            Err(SqlServerError::InvariantViolated(format!(
                "expected empty result from BEGIN TRANSACTION. Got: {results:?}"
            )))
        } else {
            Ok(Transaction {
                client,
                closed: false,
            })
        }
    }

    /// See [`Client::execute`].
    pub async fn execute<'q>(
        &mut self,
        query: impl Into<Cow<'q, str>>,
        params: &[&dyn ToSql],
    ) -> Result<SmallVec<[u64; 1]>, SqlServerError> {
        self.client.execute(query, params).await
    }

    /// See [`Client::query`].
    pub async fn query<'q>(
        &mut self,
        query: impl Into<Cow<'q, str>>,
        params: &[&dyn tiberius::ToSql],
    ) -> Result<SmallVec<[tiberius::Row; 1]>, SqlServerError> {
        self.client.query(query, params).await
    }

    /// See [`Client::query_streaming`]
    pub fn query_streaming<'q>(
        &mut self,
        query: impl Into<Cow<'q, str>>,
        params: &[&dyn tiberius::ToSql],
    ) -> impl Stream<Item = Result<tiberius::Row, SqlServerError>> + Send + '_ {
        self.client.query_streaming(query, params)
    }

    /// See [`Client::simple_query`].
    pub async fn simple_query<'q>(
        &mut self,
        query: impl Into<Cow<'q, str>>,
    ) -> Result<SmallVec<[tiberius::Row; 1]>, SqlServerError> {
        self.client.simple_query(query).await
    }

    /// Rollback the [`Transaction`].
    pub async fn rollback(mut self) -> Result<(), SqlServerError> {
        static ROLLBACK_QUERY: &str = "ROLLBACK TRANSACTION";
        // N.B. Mark closed _before_ running the query. This prevents us from
        // double closing the transaction if this query itself fails.
        self.closed = true;
        self.client.simple_query(ROLLBACK_QUERY).await?;
        Ok(())
    }

    /// Commit the [`Transaction`].
    pub async fn commit(mut self) -> Result<(), SqlServerError> {
        static COMMIT_QUERY: &str = "COMMIT TRANSACTION";
        // N.B. Mark closed _before_ running the query. This prevents us from
        // double closing the transaction if this query itself fails.
        self.closed = true;
        self.client.simple_query(COMMIT_QUERY).await?;
        Ok(())
    }
}

impl Drop for Transaction<'_> {
    fn drop(&mut self) {
        // Internally the query is synchronously sent down a channel, and the response is what
        // we await. In other words, we don't need to `.await` here for the query to be run.
        if !self.closed {
            let _fut = self.client.simple_query("ROLLBACK TRANSACTION");
        }
    }
}

/// Transaction isolation levels defined by Microsoft's SQL Server.
///
/// See: <https://learn.microsoft.com/en-us/sql/t-sql/statements/set-transaction-isolation-level-transact-sql>
#[derive(Debug, PartialEq, Eq)]
pub enum TransactionIsolationLevel {
    ReadUncommitted,
    ReadCommitted,
    RepeatableRead,
    Snapshot,
    Serializable,
}

impl TransactionIsolationLevel {
    /// Return the string representation of a transaction isolation level.
    fn as_str(&self) -> &'static str {
        match self {
            TransactionIsolationLevel::ReadUncommitted => "READ UNCOMMITTED",
            TransactionIsolationLevel::ReadCommitted => "READ COMMITTED",
            TransactionIsolationLevel::RepeatableRead => "REPEATABLE READ",
            TransactionIsolationLevel::Snapshot => "SNAPSHOT",
            TransactionIsolationLevel::Serializable => "SERIALIZABLE",
        }
    }

    /// Try to parse a [`TransactionIsolationLevel`] from the value returned from SQL Server.
    fn try_from_sql_server(val: i16) -> Result<TransactionIsolationLevel, anyhow::Error> {
        let level = match val {
            1 => TransactionIsolationLevel::ReadUncommitted,
            2 => TransactionIsolationLevel::ReadCommitted,
            3 => TransactionIsolationLevel::RepeatableRead,
            4 => TransactionIsolationLevel::Serializable,
            5 => TransactionIsolationLevel::Snapshot,
            x => anyhow::bail!("unknown level {x}"),
        };
        Ok(level)
    }
}

#[derive(Derivative)]
#[derivative(Debug)]
enum Response {
    Execute {
        rows_affected: SmallVec<[u64; 1]>,
    },
    Rows(SmallVec<[tiberius::Row; 1]>),
    RowStream {
        #[derivative(Debug = "ignore")]
        stream: tokio::sync::mpsc::Receiver<Result<tiberius::Row, SqlServerError>>,
    },
}

#[derive(Debug)]
struct Request {
    tx: oneshot::Sender<Result<Response, SqlServerError>>,
    kind: RequestKind,
}

#[derive(Derivative)]
#[derivative(Debug)]
enum RequestKind {
    Execute {
        query: String,
        #[derivative(Debug = "ignore")]
        params: SmallVec<[OwnedColumnData; 4]>,
    },
    Query {
        query: String,
        #[derivative(Debug = "ignore")]
        params: SmallVec<[OwnedColumnData; 4]>,
    },
    QueryStreamed {
        query: String,
        #[derivative(Debug = "ignore")]
        params: SmallVec<[OwnedColumnData; 4]>,
    },
    SimpleQuery {
        query: String,
    },
}

pub struct Connection {
    rx: UnboundedReceiver<Request>,
    client: tiberius::Client<Compat<TcpStream>>,
}

impl Connection {
    async fn run(mut self) {
        while let Some(Request { tx, kind }) = self.rx.recv().await {
            tracing::debug!(?kind, "processing SQL Server query");
            let result = Connection::handle_request(&mut self.client, kind).await;
            let (response, maybe_extra_work) = match result {
                Ok((response, work)) => (Ok(response), work),
                Err(err) => (Err(err), None),
            };

            // We don't care if our listener for this query has gone away.
            let _ = tx.send(response);

            // After we handle a request there might still be something in-flight
            // that we need to continue driving, e.g. when the response is a
            // Stream of Rows.
            if let Some(extra_work) = maybe_extra_work {
                extra_work.await;
            }
        }
        tracing::debug!("channel closed, SQL Server InnerClient shutting down");
    }

    async fn handle_request<'c>(
        client: &'c mut tiberius::Client<Compat<TcpStream>>,
        kind: RequestKind,
    ) -> Result<(Response, Option<BoxFuture<'c, ()>>), SqlServerError> {
        match kind {
            RequestKind::Execute { query, params } => {
                #[allow(clippy::as_conversions)]
                let params: SmallVec<[&dyn ToSql; 4]> =
                    params.iter().map(|x| x as &dyn ToSql).collect();
                let result = client.execute(query, &params[..]).await?;

                match result.rows_affected() {
                    [] => Err(SqlServerError::InvariantViolated(
                        "got empty response".into(),
                    )),
                    rows_affected => {
                        let response = Response::Execute {
                            rows_affected: rows_affected.into(),
                        };
                        Ok((response, None))
                    }
                }
            }
            RequestKind::Query { query, params } => {
                #[allow(clippy::as_conversions)]
                let params: SmallVec<[&dyn ToSql; 4]> =
                    params.iter().map(|x| x as &dyn ToSql).collect();
                let result = client.query(query, params.as_slice()).await?;

                let mut results = result.into_results().await.context("into results")?;
                if results.is_empty() {
                    Err(SqlServerError::InvariantViolated(
                        "got empty response".into(),
                    ))
                } else if results.len() == 1 {
                    // TODO(sql_server3): Don't use `into_results()` above, instead directly
                    // push onto a SmallVec to avoid the heap allocations.
                    let rows = results.pop().expect("checked len").into();
                    Ok((Response::Rows(rows), None))
                } else {
                    Err(SqlServerError::ProgrammingError(format!(
                        "Query only supports 1 statement, got {}",
                        results.len()
                    )))
                }
            }
            RequestKind::QueryStreamed { query, params } => {
                #[allow(clippy::as_conversions)]
                let params: SmallVec<[&dyn ToSql; 4]> =
                    params.iter().map(|x| x as &dyn ToSql).collect();
                let result = client.query(query, params.as_slice()).await?;

                // ~~ Rust Lifetimes ~~
                //
                // What's going on here, why do we have some extra channel and
                // this 'work' future?
                //
                // Remember, we run the actual `tiberius::Client` in a separate
                // `tokio::task` and the `mz::Client` sends query requests via
                // a channel, this allows us to "automatically" manage
                // transactions.
                //
                // But the returned `QueryStream` from a `tiberius::Client` has
                // a lifetime associated with said client running in this
                // separate task. Thus we cannot send the `QueryStream` back to
                // the `mz::Client` because the lifetime of these two clients
                // is not linked at all. The fix is to create a separate owned
                // channel and return the receiving end, while this work future
                // pulls events off the `QueryStream` and sends them over the
                // channel we just returned.
                let (tx, rx) = tokio::sync::mpsc::channel(256);
                let work = Box::pin(async move {
                    let mut stream = result.into_row_stream();
                    while let Some(result) = stream.next().await {
                        if let Err(err) = tx.send(result.err_into()).await {
                            tracing::warn!(?err, "SQL Server row stream receiver went away");
                        }
                    }
                    tracing::info!("SQL Server row stream complete");
                });

                Ok((Response::RowStream { stream: rx }, Some(work)))
            }
            RequestKind::SimpleQuery { query } => {
                let result = client.simple_query(query).await?;

                let mut results = result.into_results().await.context("into results")?;
                if results.is_empty() {
                    Ok((Response::Rows(smallvec![]), None))
                } else if results.len() == 1 {
                    // TODO(sql_server3): Don't use `into_results()` above, instead directly
                    // push onto a SmallVec to avoid the heap allocations.
                    let rows = results.pop().expect("checked len").into();
                    Ok((Response::Rows(rows), None))
                } else {
                    Err(SqlServerError::ProgrammingError(format!(
                        "Simple query only supports 1 statement, got {}",
                        results.len()
                    )))
                }
            }
        }
    }
}

impl IntoFuture for Connection {
    type Output = ();
    type IntoFuture = BoxFuture<'static, Self::Output>;

    fn into_future(self) -> Self::IntoFuture {
        self.run().boxed()
    }
}

/// Owned version of [`tiberius::ColumnData`] that can be more easily sent
/// across threads or through a channel.
#[derive(Debug)]
enum OwnedColumnData {
    U8(Option<u8>),
    I16(Option<i16>),
    I32(Option<i32>),
    I64(Option<i64>),
    F32(Option<f32>),
    F64(Option<f64>),
    Bit(Option<bool>),
    String(Option<String>),
    Guid(Option<uuid::Uuid>),
    Binary(Option<Vec<u8>>),
    Numeric(Option<tiberius::numeric::Numeric>),
    Xml(Option<tiberius::xml::XmlData>),
    DateTime(Option<tiberius::time::DateTime>),
    SmallDateTime(Option<tiberius::time::SmallDateTime>),
    Time(Option<tiberius::time::Time>),
    Date(Option<tiberius::time::Date>),
    DateTime2(Option<tiberius::time::DateTime2>),
    DateTimeOffset(Option<tiberius::time::DateTimeOffset>),
}

impl<'a> From<tiberius::ColumnData<'a>> for OwnedColumnData {
    fn from(value: tiberius::ColumnData<'a>) -> Self {
        match value {
            tiberius::ColumnData::U8(inner) => OwnedColumnData::U8(inner),
            tiberius::ColumnData::I16(inner) => OwnedColumnData::I16(inner),
            tiberius::ColumnData::I32(inner) => OwnedColumnData::I32(inner),
            tiberius::ColumnData::I64(inner) => OwnedColumnData::I64(inner),
            tiberius::ColumnData::F32(inner) => OwnedColumnData::F32(inner),
            tiberius::ColumnData::F64(inner) => OwnedColumnData::F64(inner),
            tiberius::ColumnData::Bit(inner) => OwnedColumnData::Bit(inner),
            tiberius::ColumnData::String(inner) => {
                OwnedColumnData::String(inner.map(|s| s.to_string()))
            }
            tiberius::ColumnData::Guid(inner) => OwnedColumnData::Guid(inner),
            tiberius::ColumnData::Binary(inner) => {
                OwnedColumnData::Binary(inner.map(|b| b.to_vec()))
            }
            tiberius::ColumnData::Numeric(inner) => OwnedColumnData::Numeric(inner),
            tiberius::ColumnData::Xml(inner) => OwnedColumnData::Xml(inner.map(|x| x.into_owned())),
            tiberius::ColumnData::DateTime(inner) => OwnedColumnData::DateTime(inner),
            tiberius::ColumnData::SmallDateTime(inner) => OwnedColumnData::SmallDateTime(inner),
            tiberius::ColumnData::Time(inner) => OwnedColumnData::Time(inner),
            tiberius::ColumnData::Date(inner) => OwnedColumnData::Date(inner),
            tiberius::ColumnData::DateTime2(inner) => OwnedColumnData::DateTime2(inner),
            tiberius::ColumnData::DateTimeOffset(inner) => OwnedColumnData::DateTimeOffset(inner),
        }
    }
}

impl tiberius::ToSql for OwnedColumnData {
    fn to_sql(&self) -> tiberius::ColumnData<'_> {
        match self {
            OwnedColumnData::U8(inner) => tiberius::ColumnData::U8(*inner),
            OwnedColumnData::I16(inner) => tiberius::ColumnData::I16(*inner),
            OwnedColumnData::I32(inner) => tiberius::ColumnData::I32(*inner),
            OwnedColumnData::I64(inner) => tiberius::ColumnData::I64(*inner),
            OwnedColumnData::F32(inner) => tiberius::ColumnData::F32(*inner),
            OwnedColumnData::F64(inner) => tiberius::ColumnData::F64(*inner),
            OwnedColumnData::Bit(inner) => tiberius::ColumnData::Bit(*inner),
            OwnedColumnData::String(inner) => {
                tiberius::ColumnData::String(inner.as_deref().map(Cow::Borrowed))
            }
            OwnedColumnData::Guid(inner) => tiberius::ColumnData::Guid(*inner),
            OwnedColumnData::Binary(inner) => {
                tiberius::ColumnData::Binary(inner.as_deref().map(Cow::Borrowed))
            }
            OwnedColumnData::Numeric(inner) => tiberius::ColumnData::Numeric(*inner),
            OwnedColumnData::Xml(inner) => {
                tiberius::ColumnData::Xml(inner.as_ref().map(Cow::Borrowed))
            }
            OwnedColumnData::DateTime(inner) => tiberius::ColumnData::DateTime(*inner),
            OwnedColumnData::SmallDateTime(inner) => tiberius::ColumnData::SmallDateTime(*inner),
            OwnedColumnData::Time(inner) => tiberius::ColumnData::Time(*inner),
            OwnedColumnData::Date(inner) => tiberius::ColumnData::Date(*inner),
            OwnedColumnData::DateTime2(inner) => tiberius::ColumnData::DateTime2(*inner),
            OwnedColumnData::DateTimeOffset(inner) => tiberius::ColumnData::DateTimeOffset(*inner),
        }
    }
}

impl<'a, T: tiberius::ToSql> From<&'a T> for OwnedColumnData {
    fn from(value: &'a T) -> Self {
        OwnedColumnData::from(value.to_sql())
    }
}

#[derive(Debug, thiserror::Error)]
pub enum SqlServerError {
    #[error(transparent)]
    SqlServer(#[from] tiberius::error::Error),
    #[error(transparent)]
    CdcError(#[from] crate::cdc::CdcError),
    #[error("'{column_type}' from column '{column_name}' is not supported: {reason}")]
    UnsupportedDataType {
        column_name: String,
        column_type: String,
        reason: String,
    },
    #[error("sql server client encountered I/O error: {0}")]
    IO(#[from] tokio::io::Error),
    #[error("found invalid data in the column '{column_name}': {error}")]
    InvalidData { column_name: String, error: String },
    #[error("invariant was violated: {0}")]
    InvariantViolated(String),
    #[error(transparent)]
    Generic(#[from] anyhow::Error),
    #[error("programming error! {0}")]
    ProgrammingError(String),
}