risingwave_connector/parser/sql_server.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
// Copyright 2024 RisingWave Labs
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
use std::sync::LazyLock;
use chrono::{DateTime, NaiveDate, NaiveDateTime, NaiveTime, Utc};
use risingwave_common::catalog::Schema;
use risingwave_common::log::LogSuppresser;
use risingwave_common::row::OwnedRow;
use risingwave_common::types::{Date, Decimal, ScalarImpl, Time, Timestamp, Timestamptz};
use rust_decimal::Decimal as RustDecimal;
use thiserror_ext::AsReport;
use tiberius::xml::XmlData;
use tiberius::Row;
use uuid::Uuid;
use crate::parser::util::log_error;
static LOG_SUPPERSSER: LazyLock<LogSuppresser> = LazyLock::new(LogSuppresser::default);
pub fn sql_server_row_to_owned_row(row: &mut Row, schema: &Schema) -> OwnedRow {
let mut datums: Vec<Option<ScalarImpl>> = vec![];
for i in 0..schema.fields.len() {
let rw_field = &schema.fields[i];
let name = rw_field.name.as_str();
let datum = match row.try_get::<ScalarImplTiberiusWrapper, usize>(i) {
Ok(datum) => datum.map(|d| d.0),
Err(err) => {
log_error!(name, err, "parse column failed");
None
}
};
datums.push(datum);
}
OwnedRow::new(datums)
}
macro_rules! impl_tiberius_wrapper {
($wrapper_name:ident, $variant_name:ident) => {
pub struct $wrapper_name($variant_name);
impl From<$variant_name> for $wrapper_name {
fn from(value: $variant_name) -> Self {
Self(value)
}
}
};
}
impl_tiberius_wrapper!(ScalarImplTiberiusWrapper, ScalarImpl);
impl_tiberius_wrapper!(TimeTiberiusWrapper, Time);
impl_tiberius_wrapper!(DateTiberiusWrapper, Date);
impl_tiberius_wrapper!(TimestampTiberiusWrapper, Timestamp);
impl_tiberius_wrapper!(TimestamptzTiberiusWrapper, Timestamptz);
impl_tiberius_wrapper!(DecimalTiberiusWrapper, Decimal);
macro_rules! impl_chrono_tiberius_wrapper {
($wrapper_name:ident, $variant_name:ident, $chrono:ty) => {
impl<'a> tiberius::IntoSql<'a> for $wrapper_name {
fn into_sql(self) -> tiberius::ColumnData<'a> {
self.0 .0.into_sql()
}
}
impl<'a> tiberius::FromSql<'a> for $wrapper_name {
fn from_sql(
value: &'a tiberius::ColumnData<'static>,
) -> tiberius::Result<Option<Self>> {
let instant = <$chrono>::from_sql(value)?;
let time = instant.map($variant_name::from).map($wrapper_name::from);
tiberius::Result::Ok(time)
}
}
};
}
impl_chrono_tiberius_wrapper!(TimeTiberiusWrapper, Time, NaiveTime);
impl_chrono_tiberius_wrapper!(DateTiberiusWrapper, Date, NaiveDate);
impl_chrono_tiberius_wrapper!(TimestampTiberiusWrapper, Timestamp, NaiveDateTime);
impl<'a> tiberius::IntoSql<'a> for DecimalTiberiusWrapper {
fn into_sql(self) -> tiberius::ColumnData<'a> {
match self.0 {
Decimal::Normalized(d) => d.into_sql(),
Decimal::NaN => tiberius::ColumnData::Numeric(None),
Decimal::PositiveInf => tiberius::ColumnData::Numeric(None),
Decimal::NegativeInf => tiberius::ColumnData::Numeric(None),
}
}
}
impl<'a> tiberius::FromSql<'a> for DecimalTiberiusWrapper {
// TODO(kexiang): will sql server have inf/-inf/nan for decimal?
fn from_sql(value: &'a tiberius::ColumnData<'static>) -> tiberius::Result<Option<Self>> {
tiberius::Result::Ok(
RustDecimal::from_sql(value)?
.map(Decimal::Normalized)
.map(DecimalTiberiusWrapper::from),
)
}
}
impl<'a> tiberius::IntoSql<'a> for TimestamptzTiberiusWrapper {
fn into_sql(self) -> tiberius::ColumnData<'a> {
self.0.to_datetime_utc().into_sql()
}
}
impl<'a> tiberius::FromSql<'a> for TimestamptzTiberiusWrapper {
fn from_sql(value: &'a tiberius::ColumnData<'static>) -> tiberius::Result<Option<Self>> {
let instant = DateTime::<Utc>::from_sql(value)?;
let time = instant
.map(Timestamptz::from)
.map(TimestamptzTiberiusWrapper::from);
tiberius::Result::Ok(time)
}
}
/// The following table shows the mapping between Rust types and Sql Server types in tiberius.
/// |Rust Type|Sql Server Type|
/// |`u8`|`tinyint`|
/// |`i16`|`smallint`|
/// |`i32`|`int`|
/// |`i64`|`bigint`|
/// |`f32`|`float(24)`|
/// |`f64`|`float(53)`|
/// |`bool`|`bit`|
/// |`String`/`&str`|`nvarchar`/`varchar`/`nchar`/`char`/`ntext`/`text`|
/// |`Vec<u8>`/`&[u8]`|`binary`/`varbinary`/`image`|
/// |[`Uuid`]|`uniqueidentifier`|
/// |[`Numeric`]|`numeric`/`decimal`|
/// |[`Decimal`] (with feature flag `rust_decimal`)|`numeric`/`decimal`|
/// |[`XmlData`]|`xml`|
/// |[`NaiveDateTime`] (with feature flag `chrono`)|`datetime`/`datetime2`/`smalldatetime`|
/// |[`NaiveDate`] (with feature flag `chrono`)|`date`|
/// |[`NaiveTime`] (with feature flag `chrono`)|`time`|
/// |[`DateTime`] (with feature flag `chrono`)|`datetimeoffset`|
///
/// See the [`time`] module for more information about the date and time structs.
///
/// [`Row#get`]: struct.Row.html#method.get
/// [`Row#try_get`]: struct.Row.html#method.try_get
/// [`time`]: time/index.html
/// [`Uuid`]: struct.Uuid.html
/// [`Numeric`]: numeric/struct.Numeric.html
/// [`Decimal`]: numeric/struct.Decimal.html
/// [`XmlData`]: xml/struct.XmlData.html
/// [`NaiveDateTime`]: time/chrono/struct.NaiveDateTime.html
/// [`NaiveDate`]: time/chrono/struct.NaiveDate.html
/// [`NaiveTime`]: time/chrono/struct.NaiveTime.html
/// [`DateTime`]: time/chrono/struct.DateTime.html
impl<'a> tiberius::FromSql<'a> for ScalarImplTiberiusWrapper {
fn from_sql(value: &'a tiberius::ColumnData<'static>) -> tiberius::Result<Option<Self>> {
Ok(match &value {
tiberius::ColumnData::U8(_) => u8::from_sql(value)?
.map(|v| ScalarImplTiberiusWrapper::from(ScalarImpl::from(v as i16))),
tiberius::ColumnData::I16(_) => i16::from_sql(value)?
.map(ScalarImpl::from)
.map(ScalarImplTiberiusWrapper::from),
tiberius::ColumnData::I32(_) => i32::from_sql(value)?
.map(ScalarImpl::from)
.map(ScalarImplTiberiusWrapper::from),
tiberius::ColumnData::I64(_) => i64::from_sql(value)?
.map(ScalarImpl::from)
.map(ScalarImplTiberiusWrapper::from),
tiberius::ColumnData::F32(_) => f32::from_sql(value)?
.map(ScalarImpl::from)
.map(ScalarImplTiberiusWrapper::from),
tiberius::ColumnData::F64(_) => f64::from_sql(value)?
.map(ScalarImpl::from)
.map(ScalarImplTiberiusWrapper::from),
tiberius::ColumnData::Bit(_) => bool::from_sql(value)?
.map(ScalarImpl::from)
.map(ScalarImplTiberiusWrapper::from),
tiberius::ColumnData::String(_) => <&str>::from_sql(value)?
.map(ScalarImpl::from)
.map(ScalarImplTiberiusWrapper::from),
tiberius::ColumnData::Numeric(_) => DecimalTiberiusWrapper::from_sql(value)?
.map(|w| ScalarImpl::from(w.0))
.map(ScalarImplTiberiusWrapper::from),
tiberius::ColumnData::DateTime(_)
| tiberius::ColumnData::DateTime2(_)
| tiberius::ColumnData::SmallDateTime(_) => TimestampTiberiusWrapper::from_sql(value)?
.map(|w| ScalarImpl::from(w.0))
.map(ScalarImplTiberiusWrapper::from),
tiberius::ColumnData::Time(_) => TimeTiberiusWrapper::from_sql(value)?
.map(|w| ScalarImpl::from(w.0))
.map(ScalarImplTiberiusWrapper::from),
tiberius::ColumnData::Date(_) => DateTiberiusWrapper::from_sql(value)?
.map(|w| ScalarImpl::from(w.0))
.map(ScalarImplTiberiusWrapper::from),
tiberius::ColumnData::DateTimeOffset(_) => TimestamptzTiberiusWrapper::from_sql(value)?
.map(|w| ScalarImpl::from(w.0))
.map(ScalarImplTiberiusWrapper::from),
tiberius::ColumnData::Binary(_) => <&[u8]>::from_sql(value)?
.map(ScalarImpl::from)
.map(ScalarImplTiberiusWrapper::from),
tiberius::ColumnData::Guid(_) => <Uuid>::from_sql(value)?
.map(|uuid| uuid.to_string().to_uppercase())
.map(ScalarImpl::from)
.map(ScalarImplTiberiusWrapper::from),
tiberius::ColumnData::Xml(_) => <&XmlData>::from_sql(value)?
.map(|xml| xml.clone().into_string())
.map(ScalarImpl::from)
.map(ScalarImplTiberiusWrapper::from),
})
}
}
/// The following table shows the mapping between Rust types and Sql Server types in tiberius.
/// |Rust type|Sql Server type|
/// |--------|--------|
/// |`u8`|`tinyint`|
/// |`i16`|`smallint`|
/// |`i32`|`int`|
/// |`i64`|`bigint`|
/// |`f32`|`float(24)`|
/// |`f64`|`float(53)`|
/// |`bool`|`bit`|
/// |`String`/`&str` (< 4000 characters)|`nvarchar(4000)`|
/// |`String`/`&str`|`nvarchar(max)`|
/// |`Vec<u8>`/`&[u8]` (< 8000 bytes)|`varbinary(8000)`|
/// |`Vec<u8>`/`&[u8]`|`varbinary(max)`|
/// |[`Uuid`]|`uniqueidentifier`|
/// |[`Numeric`]|`numeric`/`decimal`|
/// |[`Decimal`] (with feature flag `rust_decimal`)|`numeric`/`decimal`|
/// |[`BigDecimal`] (with feature flag `bigdecimal`)|`numeric`/`decimal`|
/// |[`XmlData`]|`xml`|
/// |[`NaiveDate`] (with `chrono` feature, TDS 7.3 >)|`date`|
/// |[`NaiveTime`] (with `chrono` feature, TDS 7.3 >)|`time`|
/// |[`DateTime`] (with `chrono` feature, TDS 7.3 >)|`datetimeoffset`|
/// |[`NaiveDateTime`] (with `chrono` feature, TDS 7.3 >)|`datetime2`|
/// |[`NaiveDateTime`] (with `chrono` feature, TDS 7.2)|`datetime`|
///
/// It is possible to use some of the types to write into columns that are not
/// of the same type. For example on systems following the TDS 7.3 standard (SQL
/// Server 2008 and later), the chrono type `NaiveDateTime` can also be used to
/// write to `datetime`, `datetime2` and `smalldatetime` columns. All string
/// types can also be used with `ntext`, `text`, `varchar`, `nchar` and `char`
/// columns. All binary types can also be used with `binary` and `image`
/// columns.
///
/// See the [`time`] module for more information about the date and time structs.
///
/// [`Client#query`]: struct.Client.html#method.query
/// [`Client#execute`]: struct.Client.html#method.execute
/// [`time`]: time/index.html
/// [`Uuid`]: struct.Uuid.html
/// [`Numeric`]: numeric/struct.Numeric.html
/// [`Decimal`]: numeric/struct.Decimal.html
/// [`BigDecimal`]: numeric/struct.BigDecimal.html
/// [`XmlData`]: xml/struct.XmlData.html
/// [`NaiveDateTime`]: time/chrono/struct.NaiveDateTime.html
/// [`NaiveDate`]: time/chrono/struct.NaiveDate.html
/// [`NaiveTime`]: time/chrono/struct.NaiveTime.html
/// [`DateTime`]: time/chrono/struct.DateTime.html
impl<'a> tiberius::IntoSql<'a> for ScalarImplTiberiusWrapper {
fn into_sql(self) -> tiberius::ColumnData<'a> {
match self.0 {
ScalarImpl::Int16(v) => v.into_sql(),
ScalarImpl::Int32(v) => v.into_sql(),
ScalarImpl::Int64(v) => v.into_sql(),
ScalarImpl::Float32(v) => v.0.into_sql(),
ScalarImpl::Float64(v) => v.0.into_sql(),
ScalarImpl::Bool(v) => v.into_sql(),
ScalarImpl::Decimal(v) => DecimalTiberiusWrapper::from(v).into_sql(),
ScalarImpl::Date(v) => DateTiberiusWrapper::from(v).into_sql(),
ScalarImpl::Timestamp(v) => TimestampTiberiusWrapper::from(v).into_sql(),
ScalarImpl::Timestamptz(v) => TimestamptzTiberiusWrapper::from(v).into_sql(),
ScalarImpl::Time(v) => TimeTiberiusWrapper::from(v).into_sql(),
// ScalarImpl::Bytea(v) => (*v.clone()).into_sql(),
value => {
// Utf8, Serial, Interval, Jsonb, Int256, Struct, List are not supported yet
unimplemented!("the sql server decoding for {:?} is unsupported", value);
}
}
}
}