diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index fe6ab78..26819c8 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -34,7 +34,7 @@ jobs: - run: rustup show active-toolchain -v - run: cargo build - run: cargo build --no-default-features - - run: cargo build --features uuid,time + - run: cargo build --features uuid,time,chrono - run: cargo build --all-features rustfmt: diff --git a/Cargo.toml b/Cargo.toml index 58867a5..1bf6b88 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -51,7 +51,7 @@ required-features = ["rustls-tls"] [[example]] name = "data_types_derive_simple" -required-features = ["time", "uuid"] +required-features = ["time", "uuid", "chrono"] [[example]] name = "data_types_variant" @@ -69,17 +69,29 @@ watch = ["dep:sha-1", "dep:serde_json", "serde/derive"] uuid = ["dep:uuid"] time = ["dep:time"] lz4 = ["dep:lz4_flex", "dep:cityhash-rs"] - +chrono = ["dep:chrono"] ## TLS native-tls = ["dep:hyper-tls"] # ext: native-tls-alpn # ext: native-tls-vendored rustls-tls = ["rustls-tls-aws-lc", "rustls-tls-webpki-roots"] -rustls-tls-aws-lc = ["dep:rustls", "dep:hyper-rustls", "hyper-rustls?/aws-lc-rs"] +rustls-tls-aws-lc = [ + "dep:rustls", + "dep:hyper-rustls", + "hyper-rustls?/aws-lc-rs", +] rustls-tls-ring = ["dep:rustls", "dep:hyper-rustls", "hyper-rustls?/ring"] -rustls-tls-webpki-roots = ["dep:rustls", "dep:hyper-rustls", "hyper-rustls?/webpki-tokio"] -rustls-tls-native-roots = ["dep:rustls", "dep:hyper-rustls", "hyper-rustls?/native-tokio"] +rustls-tls-webpki-roots = [ + "dep:rustls", + "dep:hyper-rustls", + "hyper-rustls?/webpki-tokio", +] +rustls-tls-native-roots = [ + "dep:rustls", + "dep:hyper-rustls", + "hyper-rustls?/native-tokio", +] [dependencies] clickhouse-derive = { version = "0.2.0", path = "derive" } @@ -110,6 +122,7 @@ lz4_flex = { version = "0.11.3", default-features = false, features = [ cityhash-rs = { version = "=1.0.1", optional = true } # exact version for safety uuid = { version = "1", optional = true } time = { version = "0.3", optional = true } +chrono = { version = "0.4", optional = true, features = ["serde"] } bstr = { version = "1.11.0", default-features = false } quanta = { version = "0.12", optional = true } replace_with = { version = "0.1.7" } diff --git a/README.md b/README.md index 157a47a..f13666c 100644 --- a/README.md +++ b/README.md @@ -224,6 +224,7 @@ See [examples](https://github.com/ClickHouse/clickhouse-rs/tree/main/examples). * `watch` — enables `client.watch` functionality. See the corresponding section for details. * `uuid` — adds `serde::uuid` to work with [uuid](https://docs.rs/uuid) crate. * `time` — adds `serde::time` to work with [time](https://docs.rs/time) crate. +* `chrono` — adds `serde::chrono` to work with [chrono](https://docs.rs/chrono) crate. ### TLS By default, TLS is disabled and one or more following features must be enabled to use HTTPS urls: @@ -328,7 +329,9 @@ How to choose between all these features? Here are some considerations: } ``` -* `Date` maps to/from `u16` or a newtype around it and represents a number of days elapsed since `1970-01-01`. Also, [`time::Date`](https://docs.rs/time/latest/time/struct.Date.html) is supported by using `serde::time::date`, that requires the `time` feature. +* `Date` maps to/from `u16` or a newtype around it and represents a number of days elapsed since `1970-01-01`. The following external types are supported: + * [`time::Date`](https://docs.rs/time/latest/time/struct.Date.html) is supported by using `serde::time::date`, requiring the `time` feature. + * [`chrono::NaiveDate`](https://docs.rs/chrono/latest/chrono/struct.NaiveDate.html) is supported by using `serde::chrono::date`, requiring the `chrono` feature.
Example @@ -338,10 +341,16 @@ How to choose between all these features? Here are some considerations: days: u16, #[serde(with = "clickhouse::serde::time::date")] date: Date, + // if you prefer using chrono: + #[serde(with = "clickhouse::serde::chrono::date")] + date_chrono: NaiveDate, } + ```
-* `Date32` maps to/from `i32` or a newtype around it and represents a number of days elapsed since `1970-01-01`. Also, [`time::Date`](https://docs.rs/time/latest/time/struct.Date.html) is supported by using `serde::time::date32`, that requires the `time` feature. +* `Date32` maps to/from `i32` or a newtype around it and represents a number of days elapsed since `1970-01-01`. The following external types are supported: + * [`time::Date`](https://docs.rs/time/latest/time/struct.Date.html) is supported by using `serde::time::date32`, requiring the `time` feature. + * [`chrono::NaiveDate`](https://docs.rs/chrono/latest/chrono/struct.NaiveDate.html) is supported by using `serde::chrono::date32`, requiring the `chrono` feature.
Example @@ -351,10 +360,17 @@ How to choose between all these features? Here are some considerations: days: i32, #[serde(with = "clickhouse::serde::time::date32")] date: Date, + // if you prefer using chrono: + #[serde(with = "clickhouse::serde::chrono::date32")] + date_chrono: NaiveDate, + } + ```
-* `DateTime` maps to/from `u32` or a newtype around it and represents a number of seconds elapsed since UNIX epoch. Also, [`time::OffsetDateTime`](https://docs.rs/time/latest/time/struct.OffsetDateTime.html) is supported by using `serde::time::datetime`, that requires the `time` feature. +* `DateTime` maps to/from `u32` or a newtype around it and represents a number of seconds elapsed since UNIX epoch. The following external types are supported: + * [`time::OffsetDateTime`](https://docs.rs/time/latest/time/struct.OffsetDateTime.html) is supported by using `serde::time::datetime`, requiring the `time` feature. + * [`chrono::DateTime`](https://docs.rs/chrono/latest/chrono/struct.DateTime.html) is supported by using `serde::chrono::datetime`, requiring the `chrono` feature.
Example @@ -364,10 +380,15 @@ How to choose between all these features? Here are some considerations: ts: u32, #[serde(with = "clickhouse::serde::time::datetime")] dt: OffsetDateTime, + // if you prefer using chrono: + #[serde(with = "clickhouse::serde::chrono::datetime")] + dt_chrono: DateTime, } ```
-* `DateTime64(_)` maps to/from `i64` or a newtype around it and represents a time elapsed since UNIX epoch. Also, [`time::OffsetDateTime`](https://docs.rs/time/latest/time/struct.OffsetDateTime.html) is supported by using `serde::time::datetime64::*`, that requires the `time` feature. +* `DateTime64(_)` maps to/from `i64` or a newtype around it and represents a time elapsed since UNIX epoch. The following external types are supported: + * [`time::OffsetDateTime`](https://docs.rs/time/latest/time/struct.OffsetDateTime.html) is supported by using `serde::time::datetime64::*`, requiring the `time` feature. + * [`chrono::DateTime`](https://docs.rs/chrono/latest/chrono/struct.DateTime.html) is supported by using `serde::chrono::datetime64::*`, requiring the `chrono` feature.
Example @@ -383,7 +404,18 @@ How to choose between all these features? Here are some considerations: dt64us: OffsetDateTime, // `DateTime64(6)` #[serde(with = "clickhouse::serde::time::datetime64::nanos")] dt64ns: OffsetDateTime, // `DateTime64(9)` + // if you prefer using chrono: + #[serde(with = "clickhouse::serde::chrono::datetime64::secs")] + dt64s_chrono: DateTime, // `DateTime64(0)` + #[serde(with = "clickhouse::serde::chrono::datetime64::millis")] + dt64ms_chrono: DateTime, // `DateTime64(3)` + #[serde(with = "clickhouse::serde::chrono::datetime64::micros")] + dt64us_chrono: DateTime, // `DateTime64(6)` + #[serde(with = "clickhouse::serde::chrono::datetime64::nanos")] + dt64ns_chrono: DateTime, // `DateTime64(9)` } + + ```
* `Tuple(A, B, ...)` maps to/from `(A, B, ...)` or a newtype around it. diff --git a/examples/data_types_derive_simple.rs b/examples/data_types_derive_simple.rs index 1163746..d633a63 100644 --- a/examples/data_types_derive_simple.rs +++ b/examples/data_types_derive_simple.rs @@ -1,14 +1,14 @@ use std::str::FromStr; -use fixnum::typenum::{U12, U4, U8}; -use fixnum::FixedPoint; -use rand::distributions::Alphanumeric; -use rand::seq::SliceRandom; -use rand::Rng; +use chrono::{DateTime, NaiveDate, Utc}; +use fixnum::{ + typenum::{U12, U4, U8}, + FixedPoint, +}; +use rand::{distributions::Alphanumeric, seq::SliceRandom, Rng}; use time::{Date, Month, OffsetDateTime, Time}; -use clickhouse::sql::Identifier; -use clickhouse::{error::Result, Client}; +use clickhouse::{error::Result, sql::Identifier, Client}; // This example covers derivation of _simpler_ ClickHouse data types. // See also: https://clickhouse.com/docs/en/sql-reference/data-types @@ -121,23 +121,42 @@ pub struct Row { pub decimal64_18_8: Decimal64, pub decimal128_38_12: Decimal128, #[serde(with = "clickhouse::serde::time::date")] - pub date: Date, + pub time_date: Date, #[serde(with = "clickhouse::serde::time::date32")] - pub date32: Date, + pub time_date32: Date, #[serde(with = "clickhouse::serde::time::datetime")] - pub datetime: OffsetDateTime, + pub time_datetime: OffsetDateTime, #[serde(with = "clickhouse::serde::time::datetime")] - pub datetime_tz: OffsetDateTime, + pub time_datetime_tz: OffsetDateTime, #[serde(with = "clickhouse::serde::time::datetime64::secs")] - pub datetime64_0: OffsetDateTime, + pub time_datetime64_0: OffsetDateTime, #[serde(with = "clickhouse::serde::time::datetime64::millis")] - pub datetime64_3: OffsetDateTime, + pub time_datetime64_3: OffsetDateTime, #[serde(with = "clickhouse::serde::time::datetime64::micros")] - pub datetime64_6: OffsetDateTime, + pub time_datetime64_6: OffsetDateTime, #[serde(with = "clickhouse::serde::time::datetime64::nanos")] - pub datetime64_9: OffsetDateTime, + pub time_datetime64_9: OffsetDateTime, #[serde(with = "clickhouse::serde::time::datetime64::nanos")] - pub datetime64_9_tz: OffsetDateTime, + pub time_datetime64_9_tz: OffsetDateTime, + + #[serde(with = "clickhouse::serde::chrono::date")] + pub chrono_date: NaiveDate, + #[serde(with = "clickhouse::serde::chrono::date32")] + pub chrono_date32: NaiveDate, + #[serde(with = "clickhouse::serde::chrono::datetime")] + pub chrono_datetime: DateTime, + #[serde(with = "clickhouse::serde::chrono::datetime")] + pub chrono_datetime_tz: DateTime, + #[serde(with = "clickhouse::serde::chrono::datetime64::secs")] + pub chrono_datetime64_0: DateTime, + #[serde(with = "clickhouse::serde::chrono::datetime64::millis")] + pub chrono_datetime64_3: DateTime, + #[serde(with = "clickhouse::serde::chrono::datetime64::micros")] + pub chrono_datetime64_6: DateTime, + #[serde(with = "clickhouse::serde::chrono::datetime64::nanos")] + pub chrono_datetime64_9: DateTime, + #[serde(with = "clickhouse::serde::chrono::datetime64::nanos")] + pub chrono_datetime64_9_tz: DateTime, } // See ClickHouse decimal sizes: https://clickhouse.com/docs/en/sql-reference/data-types/decimal @@ -206,15 +225,25 @@ impl Row { // See // - https://clickhouse.com/docs/en/sql-reference/data-types/date // - https://clickhouse.com/docs/en/sql-reference/data-types/date32 - date: Date::from_calendar_date(2149, Month::June, 6).unwrap(), - date32: Date::from_calendar_date(2299, Month::December, 31).unwrap(), - datetime: max_datetime(), - datetime_tz: max_datetime(), - datetime64_0: max_datetime64(), - datetime64_3: max_datetime64(), - datetime64_6: max_datetime64(), - datetime64_9: max_datetime64_nanos(), - datetime64_9_tz: max_datetime64_nanos(), + time_date: Date::from_calendar_date(2149, Month::June, 6).unwrap(), + time_date32: Date::from_calendar_date(2299, Month::December, 31).unwrap(), + time_datetime: max_datetime(), + time_datetime_tz: max_datetime(), + time_datetime64_0: max_datetime64(), + time_datetime64_3: max_datetime64(), + time_datetime64_6: max_datetime64(), + time_datetime64_9: max_datetime64_nanos(), + time_datetime64_9_tz: max_datetime64_nanos(), + + chrono_date: NaiveDate::from_ymd_opt(2149, 6, 6).unwrap(), + chrono_date32: NaiveDate::from_ymd_opt(2299, 12, 31).unwrap(), + chrono_datetime: Utc::now(), + chrono_datetime_tz: Utc::now(), + chrono_datetime64_0: Utc::now(), + chrono_datetime64_3: Utc::now(), + chrono_datetime64_6: Utc::now(), + chrono_datetime64_9: Utc::now(), + chrono_datetime64_9_tz: Utc::now(), } } } diff --git a/src/serde.rs b/src/serde.rs index 81e1307..68ef883 100644 --- a/src/serde.rs +++ b/src/serde.rs @@ -106,6 +106,248 @@ pub mod uuid { } } +#[cfg(feature = "chrono")] +pub mod chrono { + use super::*; + use ::chrono::{DateTime, Utc}; + use serde::{de::Error as _, ser::Error as _}; + + pub mod datetime { + use super::*; + + type DateTimeUtc = DateTime; + + option!( + DateTimeUtc, + "Ser/de `Option>` to/from `Nullable(DateTime)`." + ); + + pub fn serialize(dt: &DateTime, serializer: S) -> Result + where + S: Serializer, + { + let ts = dt.timestamp(); + + u32::try_from(ts) + .map_err(|_| S::Error::custom(format!("{dt} cannot be represented as DateTime")))? + .serialize(serializer) + } + + pub fn deserialize<'de, D>(deserializer: D) -> Result, D::Error> + where + D: Deserializer<'de>, + { + let ts: u32 = Deserialize::deserialize(deserializer)?; + DateTime::::from_timestamp(i64::from(ts), 0).ok_or_else(|| { + D::Error::custom(format!("{ts} cannot be converted to DateTime")) + }) + } + } + + /// Contains modules to ser/de `DateTime` to/from `DateTime64(_)`. + pub mod datetime64 { + use super::*; + type DateTimeUtc = DateTime; + + /// Ser/de `DateTime` to/from `DateTime64(0)` (seconds). + pub mod secs { + use super::*; + + option!( + DateTimeUtc, + "Ser/de `Option` to/from `Nullable(DateTime64(0))`." + ); + + pub fn serialize(dt: &DateTime, serializer: S) -> Result + where + S: Serializer, + { + let ts = dt.timestamp(); + ts.serialize(serializer) + } + + pub fn deserialize<'de, D>(deserializer: D) -> Result, D::Error> + where + D: Deserializer<'de>, + { + let ts: i64 = Deserialize::deserialize(deserializer)?; + DateTime::::from_timestamp(ts, 0).ok_or_else(|| { + D::Error::custom(format!("Can't create DateTime from {ts}")) + }) + } + } + + /// Ser/de `DateTime` to/from `DateTime64(3)` (milliseconds). + pub mod millis { + use super::*; + + option!( + DateTimeUtc, + "Ser/de `Option>` to/from `Nullable(DateTime64(3))`." + ); + + pub fn serialize(dt: &DateTime, serializer: S) -> Result + where + S: Serializer, + { + let ts = dt.timestamp_millis(); + ts.serialize(serializer) + } + + pub fn deserialize<'de, D>(deserializer: D) -> Result, D::Error> + where + D: Deserializer<'de>, + { + let ts: i64 = Deserialize::deserialize(deserializer)?; + DateTime::::from_timestamp_millis(ts).ok_or_else(|| { + D::Error::custom(format!("Can't create DateTime from {ts}")) + }) + } + } + + /// Ser/de `DateTime` to/from `DateTime64(6)` (microseconds). + pub mod micros { + use super::*; + + option!( + DateTimeUtc, + "Ser/de `Option>` to/from `Nullable(DateTime64(6))`." + ); + + pub fn serialize(dt: &DateTime, serializer: S) -> Result + where + S: Serializer, + { + let ts = dt.timestamp_micros(); + ts.serialize(serializer) + } + + pub fn deserialize<'de, D>(deserializer: D) -> Result, D::Error> + where + D: Deserializer<'de>, + { + let ts: i64 = Deserialize::deserialize(deserializer)?; + DateTime::::from_timestamp_micros(ts).ok_or_else(|| { + D::Error::custom(format!("Can't create DateTime from {ts}")) + }) + } + } + + /// Ser/de `DateTime` to/from `DateTime64(9)` (nanoseconds). + pub mod nanos { + use super::*; + + option!( + DateTimeUtc, + "Ser/de `Option>` to/from `Nullable(DateTime64(9))`." + ); + + pub fn serialize(dt: &DateTime, serializer: S) -> Result + where + S: Serializer, + { + let ts = dt.timestamp_nanos_opt().ok_or_else(|| { + S::Error::custom(format!("{dt} cannot be represented as DateTime64")) + })?; + ts.serialize(serializer) + } + + pub fn deserialize<'de, D>(deserializer: D) -> Result, D::Error> + where + D: Deserializer<'de>, + { + let ts: i64 = Deserialize::deserialize(deserializer)?; + Ok(DateTime::::from_timestamp_nanos(ts)) + } + } + } + + /// Ser/de `time::Date` to/from `Date`. + pub mod date { + use super::*; + use ::chrono::{Duration, NaiveDate}; + + option!( + NaiveDate, + "Ser/de `Option` to/from `Nullable(Date)`." + ); + + const ORIGIN: Option = NaiveDate::from_yo_opt(1970, 1); + + pub fn serialize(date: &NaiveDate, serializer: S) -> Result + where + S: Serializer, + { + let origin = ORIGIN.unwrap(); + if *date < origin { + let msg = format!("{date} cannot be represented as Date"); + return Err(S::Error::custom(msg)); + } + + let elapsed = *date - origin; // cannot underflow: checked above + let days = elapsed.num_days(); + + u16::try_from(days) + .map_err(|_| S::Error::custom(format!("{date} cannot be represented as Date")))? + .serialize(serializer) + } + + pub fn deserialize<'de, D>(deserializer: D) -> Result + where + D: Deserializer<'de>, + { + let days: u16 = Deserialize::deserialize(deserializer)?; + Ok(ORIGIN.unwrap() + Duration::days(i64::from(days))) // cannot overflow: always < `Date::MAX` + } + } + + /// Ser/de `time::Date` to/from `Date32`. + pub mod date32 { + use ::chrono::{Duration, NaiveDate}; + + use super::*; + + option!( + NaiveDate, + "Ser/de `Option` to/from `Nullable(Date32)`." + ); + + const ORIGIN: Option = NaiveDate::from_yo_opt(1970, 1); + + // NOTE: actually, it's 1925 and 2283 with a tail for versions before 22.8-lts. + const MIN: Option = NaiveDate::from_yo_opt(1900, 1); + const MAX: Option = NaiveDate::from_yo_opt(2299, 365); + + pub fn serialize(date: &NaiveDate, serializer: S) -> Result + where + S: Serializer, + { + if *date < MIN.unwrap() || *date > MAX.unwrap() { + let msg = format!("{date} cannot be represented as Date"); + return Err(S::Error::custom(msg)); + } + + let elapsed = *date - ORIGIN.unwrap(); // cannot underflow: checked above + let days = elapsed.num_days(); + + i32::try_from(days) + .map_err(|_| S::Error::custom(format!("{date} cannot be represented as Date32")))? + .serialize(serializer) + } + + pub fn deserialize<'de, D>(deserializer: D) -> Result + where + D: Deserializer<'de>, + { + let days: i32 = Deserialize::deserialize(deserializer)?; + + // It shouldn't overflow, because clamped by CH and < `Date::MAX`. + // TODO: ensure CH clamps when an invalid value is inserted in binary format. + Ok(ORIGIN.unwrap() + Duration::days(i64::from(days))) + } + } +} + /// Ser/de [`::time::OffsetDateTime`] and [`::time::Date`]. #[cfg(feature = "time")] pub mod time { diff --git a/tests/it/chrono.rs b/tests/it/chrono.rs new file mode 100644 index 0000000..58a4a8b --- /dev/null +++ b/tests/it/chrono.rs @@ -0,0 +1,262 @@ +#![cfg(feature = "chrono")] + +use std::ops::RangeBounds; + +use chrono::{DateTime, Datelike, NaiveDate, Utc}; +use rand::{ + distributions::{Distribution, Standard}, + Rng, +}; +use serde::{Deserialize, Serialize}; + +use clickhouse::Row; + +#[tokio::test] +async fn datetime() { + let client = prepare_database!(); + + #[derive(Debug, PartialEq, Eq, Serialize, Deserialize, Row)] + struct MyRow { + #[serde(with = "clickhouse::serde::chrono::datetime")] + dt: DateTime, + #[serde(with = "clickhouse::serde::chrono::datetime::option")] + dt_opt: Option>, + #[serde(with = "clickhouse::serde::chrono::datetime64::secs")] + dt64s: DateTime, + #[serde(with = "clickhouse::serde::chrono::datetime64::secs::option")] + dt64s_opt: Option>, + #[serde(with = "clickhouse::serde::chrono::datetime64::millis")] + dt64ms: DateTime, + #[serde(with = "clickhouse::serde::chrono::datetime64::millis::option")] + dt64ms_opt: Option>, + #[serde(with = "clickhouse::serde::chrono::datetime64::micros")] + dt64us: DateTime, + #[serde(with = "clickhouse::serde::chrono::datetime64::micros::option")] + dt64us_opt: Option>, + #[serde(with = "clickhouse::serde::chrono::datetime64::nanos")] + dt64ns: DateTime, + #[serde(with = "clickhouse::serde::chrono::datetime64::nanos::option")] + dt64ns_opt: Option>, + } + + #[derive(Debug, Deserialize, Row)] + struct MyRowStr { + dt: String, + dt64s: String, + dt64ms: String, + dt64us: String, + dt64ns: String, + } + + client + .query( + " + CREATE TABLE test( + dt DateTime, + dt_opt Nullable(DateTime), + dt64s DateTime64(0), + dt64s_opt Nullable(DateTime64(0)), + dt64ms DateTime64(3), + dt64ms_opt Nullable(DateTime64(3)), + dt64us DateTime64(6), + dt64us_opt Nullable(DateTime64(6)), + dt64ns DateTime64(9), + dt64ns_opt Nullable(DateTime64(9)) + ) + ENGINE = MergeTree ORDER BY dt + ", + ) + .execute() + .await + .unwrap(); + let d = NaiveDate::from_ymd_opt(2022, 11, 13).unwrap(); + let dt_s = d.and_hms_opt(15, 27, 42).unwrap().and_utc(); + let dt_ms = d.and_hms_milli_opt(15, 27, 42, 123).unwrap().and_utc(); + let dt_us = d.and_hms_micro_opt(15, 27, 42, 123456).unwrap().and_utc(); + let dt_ns = d.and_hms_nano_opt(15, 27, 42, 123456789).unwrap().and_utc(); + + let original_row = MyRow { + dt: dt_s, + dt_opt: Some(dt_s), + dt64s: dt_s, + dt64s_opt: Some(dt_s), + dt64ms: dt_ms, + dt64ms_opt: Some(dt_ms), + dt64us: dt_us, + dt64us_opt: Some(dt_us), + dt64ns: dt_ns, + dt64ns_opt: Some(dt_ns), + }; + + let mut insert = client.insert("test").unwrap(); + insert.write(&original_row).await.unwrap(); + insert.end().await.unwrap(); + + let row = client + .query("SELECT ?fields FROM test") + .fetch_one::() + .await + .unwrap(); + + let row_str = client + .query( + " + SELECT toString(dt), + toString(dt64s), + toString(dt64ms), + toString(dt64us), + toString(dt64ns) + FROM test + ", + ) + .fetch_one::() + .await + .unwrap(); + + assert_eq!(row, original_row); + assert_eq!(row_str.dt, &original_row.dt.to_string()[..19]); + assert_eq!(row_str.dt64s, &original_row.dt64s.to_string()[..19]); + assert_eq!(row_str.dt64ms, &original_row.dt64ms.to_string()[..23]); + assert_eq!(row_str.dt64us, &original_row.dt64us.to_string()[..26]); + assert_eq!(row_str.dt64ns, &original_row.dt64ns.to_string()[..29]); +} + +#[tokio::test] +async fn date() { + let client = prepare_database!(); + + #[derive(Debug, Serialize, Deserialize, Row)] + struct MyRow { + #[serde(with = "clickhouse::serde::chrono::date")] + date: NaiveDate, + #[serde(with = "clickhouse::serde::chrono::date::option")] + date_opt: Option, + } + + client + .query( + " + CREATE TABLE test( + date Date, + date_opt Nullable(Date) + ) ENGINE = MergeTree ORDER BY date + ", + ) + .execute() + .await + .unwrap(); + + let mut insert = client.insert("test").unwrap(); + + let dates = generate_dates(1970..2149, 100); + for &date in &dates { + let original_row = MyRow { + date, + date_opt: Some(date), + }; + + insert.write(&original_row).await.unwrap(); + } + insert.end().await.unwrap(); + + let actual = client + .query("SELECT ?fields, toString(date) FROM test ORDER BY date") + .fetch_all::<(MyRow, String)>() + .await + .unwrap(); + + assert_eq!(actual.len(), dates.len()); + + for ((row, date_str), expected) in actual.iter().zip(dates) { + assert_eq!(row.date, expected); + assert_eq!(row.date_opt, Some(expected)); + assert_eq!(date_str, &expected.to_string()); + } +} + +#[tokio::test] +async fn date32() { + let client = prepare_database!(); + + #[derive(Debug, Serialize, Deserialize, Row)] + struct MyRow { + #[serde(with = "clickhouse::serde::chrono::date32")] + date: NaiveDate, + #[serde(with = "clickhouse::serde::chrono::date32::option")] + date_opt: Option, + } + + client + .query( + " + CREATE TABLE test( + date Date32, + date_opt Nullable(Date32) + ) ENGINE = MergeTree ORDER BY date + ", + ) + .execute() + .await + .unwrap(); + + let mut insert = client.insert("test").unwrap(); + + let dates = generate_dates(1925..2283, 100); // TODO: 1900..=2299 for newer versions. + for &date in &dates { + let original_row = MyRow { + date, + date_opt: Some(date), + }; + + insert.write(&original_row).await.unwrap(); + } + insert.end().await.unwrap(); + + let actual = client + .query("SELECT ?fields, toString(date) FROM test ORDER BY date") + .fetch_all::<(MyRow, String)>() + .await + .unwrap(); + + assert_eq!(actual.len(), dates.len()); + + for ((row, date_str), expected) in actual.iter().zip(dates) { + assert_eq!(row.date, expected); + assert_eq!(row.date_opt, Some(expected)); + assert_eq!(date_str, &expected.to_string()); + } +} + +// Distribution isn't implemented for `chrono` types, but we can lift the implementation from the `time` crate: https://docs.rs/time/latest/src/time/rand.rs.html#14-20 +struct NaiveDateWrapper(NaiveDate); + +impl Distribution for Standard { + fn sample(&self, rng: &mut R) -> NaiveDateWrapper { + NaiveDateWrapper( + NaiveDate::from_num_days_from_ce_opt( + rng.gen_range( + NaiveDate::MIN.num_days_from_ce()..=NaiveDate::MAX.num_days_from_ce(), + ), + ) + .unwrap(), + ) + } +} + +fn generate_dates(years: impl RangeBounds, count: usize) -> Vec { + let mut rng = rand::thread_rng(); + let mut dates: Vec<_> = (&mut rng) + .sample_iter(Standard) + .filter_map(|date: NaiveDateWrapper| { + if years.contains(&date.0.year()) { + Some(date.0) + } else { + None + } + }) + .take(count) + .collect(); + + dates.sort_unstable(); + dates +} diff --git a/tests/it/main.rs b/tests/it/main.rs index 87995b4..c35a6cc 100644 --- a/tests/it/main.rs +++ b/tests/it/main.rs @@ -54,6 +54,7 @@ async fn flush_query_log(client: &Client) { client.query("SYSTEM FLUSH LOGS").execute().await.unwrap(); } +mod chrono; mod compression; mod cursor_error; mod cursor_stats;