diff --git a/Cargo.toml b/Cargo.toml index 45b09459..d242c7f9 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -158,6 +158,9 @@ criterion = "0.6" serde = { version = "1.0.106", features = ["derive"] } tokio = { version = "1.0.1", features = ["full", "test-util"] } hyper = { version = "1.1", features = ["server"] } +indexmap = { version = "2.10.0", features = ["serde"] } +linked-hash-map = { version = "0.5.6", features = ["serde_impl"] } +fxhash = { version = "0.2.1" } serde_bytes = "0.11.4" serde_json = "1" serde_repr = "0.1.7" diff --git a/README.md b/README.md index de688ea2..2ef75591 100644 --- a/README.md +++ b/README.md @@ -137,7 +137,7 @@ struct MyRow { name: String, } -let mut insert = client.insert("some")?; +let mut insert = client.insert::("some").await?; insert.write(&MyRow { no: 0, name: "foo".into() }).await?; insert.write(&MyRow { no: 1, name: "bar".into() }).await?; insert.end().await?; @@ -158,14 +158,14 @@ insert.end().await?; Requires the `inserter` feature. ```rust,ignore -let mut inserter = client.inserter("some")? +let mut inserter = client.inserter::("some")? .with_timeouts(Some(Duration::from_secs(5)), Some(Duration::from_secs(20))) .with_max_bytes(50_000_000) .with_max_rows(750_000) .with_period(Some(Duration::from_secs(15))); -inserter.write(&MyRow { no: 0, name: "foo".into() })?; -inserter.write(&MyRow { no: 1, name: "bar".into() })?; +inserter.write(&MyRow { no: 0, name: "foo".into() }).await?; +inserter.write(&MyRow { no: 1, name: "bar".into() }).await?; let stats = inserter.commit().await?; if stats.rows > 0 { println!( diff --git a/benches/mocked_insert.rs b/benches/mocked_insert.rs index 8f3106ba..98bf7cde 100644 --- a/benches/mocked_insert.rs +++ b/benches/mocked_insert.rs @@ -1,23 +1,57 @@ use bytes::Bytes; +use clickhouse::{error::Result, Client, Compression, Row}; +use clickhouse_types::{Column, DataTypeNode}; use criterion::{criterion_group, criterion_main, Criterion, Throughput}; -use http_body_util::Empty; +use futures_util::stream; +use http_body_util::StreamBody; +use hyper::body::{Body, Frame}; use hyper::{body::Incoming, Request, Response}; use serde::Serialize; +use std::convert::Infallible; use std::hint::black_box; -use std::net::SocketAddr; +use std::net::{Ipv4Addr, SocketAddr, SocketAddrV4}; use std::{ future::Future, mem, time::{Duration, Instant}, }; -use clickhouse::{error::Result, Client, Compression, Row}; - mod common; -async fn serve(request: Request) -> Response> { +async fn serve( + request: Request, + compression: Compression, + with_validation: bool, +) -> Response> { common::skip_incoming(request).await; - Response::new(Empty::new()) + + let bytes = if with_validation { + let schema = vec![ + Column::new("a".to_string(), DataTypeNode::UInt64), + Column::new("b".to_string(), DataTypeNode::Int64), + Column::new("c".to_string(), DataTypeNode::Int32), + Column::new("d".to_string(), DataTypeNode::UInt32), + Column::new("e".to_string(), DataTypeNode::UInt64), + Column::new("f".to_string(), DataTypeNode::UInt32), + Column::new("g".to_string(), DataTypeNode::UInt64), + Column::new("h".to_string(), DataTypeNode::Int64), + ]; + + let mut buffer = Vec::new(); + clickhouse_types::put_rbwnat_columns_header(&schema, &mut buffer).unwrap(); + + match compression { + Compression::None => Bytes::from(buffer), + #[cfg(feature = "lz4")] + Compression::Lz4 => clickhouse::_priv::lz4_compress(&buffer).unwrap(), + _ => unreachable!(), + } + } else { + Bytes::new() + }; + + let stream = StreamBody::new(stream::once(async { Ok(Frame::data(bytes)) })); + Response::new(stream) } #[derive(Row, Serialize)] @@ -47,11 +81,18 @@ impl SomeRow { } } -async fn run_insert(client: Client, addr: SocketAddr, iters: u64) -> Result { - let _server = common::start_server(addr, serve).await; +const ADDR: SocketAddr = SocketAddr::V4(SocketAddrV4::new(Ipv4Addr::LOCALHOST, 6524)); + +async fn run_insert( + client: Client, + iters: u64, + compression: Compression, + validation: bool, +) -> Result { + let _server = common::start_server(ADDR, move |req| serve(req, compression, validation)).await; let start = Instant::now(); - let mut insert = client.insert::("table")?; + let mut insert = client.insert::("table").await?; for _ in 0..iters { insert.write(&SomeRow::sample()).await?; @@ -64,13 +105,14 @@ async fn run_insert(client: Client, addr: SocketAddr, iters: u64) -> Result( client: Client, - addr: SocketAddr, iters: u64, + compression: Compression, + validation: bool, ) -> Result { - let _server = common::start_server(addr, serve).await; + let _server = common::start_server(ADDR, move |req| serve(req, compression, validation)).await; let start = Instant::now(); - let mut inserter = client.inserter::("table")?.with_max_rows(iters); + let mut inserter = client.inserter::("table").with_max_rows(iters); if WITH_PERIOD { // Just to measure overhead, not to actually use it. @@ -78,7 +120,7 @@ async fn run_inserter( } for _ in 0..iters { - inserter.write(&SomeRow::sample())?; + inserter.write(&SomeRow::sample()).await?; inserter.commit().await?; } @@ -86,43 +128,45 @@ async fn run_inserter( Ok(start.elapsed()) } -fn run(c: &mut Criterion, name: &str, port: u16, f: impl Fn(Client, SocketAddr, u64) -> F) +fn run(c: &mut Criterion, name: &str, f: impl Fn(Client, u64, Compression, bool) -> F) where F: Future> + Send + 'static, { - let addr: SocketAddr = format!("127.0.0.1:{port}").parse().unwrap(); let runner = common::start_runner(); - let mut group = c.benchmark_group(name); group.throughput(Throughput::Bytes(mem::size_of::() as u64)); - group.bench_function("uncompressed", |b| { - b.iter_custom(|iters| { - let client = Client::default() - .with_url(format!("http://{addr}")) - .with_compression(Compression::None); - runner.run((f)(client, addr, iters)) - }) - }); - #[cfg(feature = "lz4")] - group.bench_function("lz4", |b| { - b.iter_custom(|iters| { - let client = Client::default() - .with_url(format!("http://{addr}")) - .with_compression(Compression::Lz4); - runner.run((f)(client, addr, iters)) - }) - }); + for validation in [true, false] { + #[allow(clippy::single_element_loop)] + for compression in [ + Compression::None, + #[cfg(feature = "lz4")] + Compression::Lz4, + ] { + group.bench_function( + format!("validation={validation}/compression={compression:?}"), + |b| { + b.iter_custom(|iters| { + let client = Client::default() + .with_url(format!("http://{ADDR}")) + .with_compression(compression) + .with_validation(validation); + runner.run((f)(client, iters, compression, validation)) + }) + }, + ); + } + } group.finish(); } fn insert(c: &mut Criterion) { - run(c, "insert", 6543, run_insert); + run(c, "insert", run_insert); } #[cfg(feature = "inserter")] fn inserter(c: &mut Criterion) { - run(c, "inserter", 6544, run_inserter::); - run(c, "inserter-period", 6545, run_inserter::); + run(c, "inserter", run_inserter::); + run(c, "inserter-period", run_inserter::); } #[cfg(not(feature = "inserter"))] diff --git a/benches/mocked_select.rs b/benches/mocked_select.rs index dfc3ff6a..3786d35d 100644 --- a/benches/mocked_select.rs +++ b/benches/mocked_select.rs @@ -21,11 +21,11 @@ mod common; async fn serve( request: Request, compression: Compression, - use_rbwnat: bool, + with_validation: bool, ) -> Response> { common::skip_incoming(request).await; - let maybe_schema = if use_rbwnat { + let maybe_schema = if with_validation { let schema = vec![ Column::new("a".to_string(), DataTypeNode::UInt64), Column::new("b".to_string(), DataTypeNode::Int64), @@ -79,8 +79,8 @@ fn prepare_chunk() -> Bytes { const ADDR: SocketAddr = SocketAddr::V4(SocketAddrV4::new(Ipv4Addr::LOCALHOST, 6523)); fn select(c: &mut Criterion) { - async fn start_server(compression: Compression, use_rbwnat: bool) -> common::ServerHandle { - common::start_server(ADDR, move |req| serve(req, compression, use_rbwnat)).await + async fn start_server(compression: Compression, with_validation: bool) -> common::ServerHandle { + common::start_server(ADDR, move |req| serve(req, compression, with_validation)).await } let runner = common::start_runner(); diff --git a/benches/select_market_data.rs b/benches/select_market_data.rs index 21ed7df3..8104c579 100644 --- a/benches/select_market_data.rs +++ b/benches/select_market_data.rs @@ -41,21 +41,21 @@ async fn prepare_data() { client .query( r#" - CREATE TABLE IF NOT EXISTS l2_book_log - ( - `instrument_id` UInt32 CODEC(T64, Default), - `received_time` DateTime64(9) CODEC(DoubleDelta, Default), - `exchange_time` Nullable(DateTime64(9)) CODEC(DoubleDelta, Default), - `sequence_no` Nullable(UInt64) CODEC(DoubleDelta, Default), - `trace_id` UInt64 CODEC(DoubleDelta, Default), - `side` Enum8('Bid' = 0, 'Ask' = 1), - `price` Float64, - `amount` Float64, - `is_eot` Bool - ) - ENGINE = MergeTree - PRIMARY KEY (instrument_id, received_time) - "#, + CREATE TABLE IF NOT EXISTS l2_book_log + ( + `instrument_id` UInt32 CODEC(T64, Default), + `received_time` DateTime64(9) CODEC(DoubleDelta, Default), + `exchange_time` Nullable(DateTime64(9)) CODEC(DoubleDelta, Default), + `sequence_no` Nullable(UInt64) CODEC(DoubleDelta, Default), + `trace_id` UInt64 CODEC(DoubleDelta, Default), + `side` Enum8('Bid' = 0, 'Ask' = 1), + `price` Float64, + `amount` Float64, + `is_eot` Bool + ) + ENGINE = MergeTree + PRIMARY KEY (instrument_id, received_time) + "#, ) .execute() .await @@ -71,7 +71,7 @@ async fn prepare_data() { return; } - let mut insert = client.insert::("l2_book_log").unwrap(); + let mut insert = client.insert::("l2_book_log").await.unwrap(); for i in 0..10_000_000 { insert diff --git a/examples/async_insert.rs b/examples/async_insert.rs index 24722a69..8a5fd2cb 100644 --- a/examples/async_insert.rs +++ b/examples/async_insert.rs @@ -40,7 +40,7 @@ async fn main() -> Result<()> { .execute() .await?; - let mut insert = client.insert::(table_name)?; + let mut insert = client.insert::(table_name).await?; insert .write(&Event { timestamp: now(), diff --git a/examples/clickhouse_cloud.rs b/examples/clickhouse_cloud.rs index 247e6e91..08603913 100644 --- a/examples/clickhouse_cloud.rs +++ b/examples/clickhouse_cloud.rs @@ -42,7 +42,7 @@ async fn main() -> clickhouse::error::Result<()> { .execute() .await?; - let mut insert = client.insert::(table_name)?; + let mut insert = client.insert::(table_name).await?; insert .write(&Data { id: 42, diff --git a/examples/data_types_derive_containers.rs b/examples/data_types_derive_containers.rs index 96004023..cae6acc6 100644 --- a/examples/data_types_derive_containers.rs +++ b/examples/data_types_derive_containers.rs @@ -37,7 +37,7 @@ async fn main() -> Result<()> { .execute() .await?; - let mut insert = client.insert::(table_name)?; + let mut insert = client.insert::(table_name).await?; insert.write(&Row::new()).await?; insert.end().await?; diff --git a/examples/data_types_derive_simple.rs b/examples/data_types_derive_simple.rs index 2e842b17..bc97b6f9 100644 --- a/examples/data_types_derive_simple.rs +++ b/examples/data_types_derive_simple.rs @@ -80,7 +80,7 @@ async fn main() -> Result<()> { .execute() .await?; - let mut insert = client.insert::(table_name)?; + let mut insert = client.insert::(table_name).await?; insert.write(&Row::new()).await?; insert.end().await?; diff --git a/examples/data_types_new_json.rs b/examples/data_types_new_json.rs index b7c7c2ab..3e5e89a3 100644 --- a/examples/data_types_new_json.rs +++ b/examples/data_types_new_json.rs @@ -49,7 +49,7 @@ async fn main() -> Result<()> { .to_string(), }; - let mut insert = client.insert::(table_name)?; + let mut insert = client.insert::(table_name).await?; insert.write(&row).await?; insert.end().await?; diff --git a/examples/data_types_variant.rs b/examples/data_types_variant.rs index 642f26fd..717c9d3c 100644 --- a/examples/data_types_variant.rs +++ b/examples/data_types_variant.rs @@ -42,7 +42,7 @@ async fn main() -> Result<()> { .execute() .await?; - let mut insert = client.insert::(table_name)?; + let mut insert = client.insert::(table_name).await?; let rows_to_insert = get_rows(); for row in rows_to_insert { insert.write(&row).await?; diff --git a/examples/enums.rs b/examples/enums.rs index e85e47da..04dc2c7f 100644 --- a/examples/enums.rs +++ b/examples/enums.rs @@ -50,7 +50,7 @@ async fn main() -> Result<()> { Error = 4, } - let mut insert = client.insert::("event_log")?; + let mut insert = client.insert::("event_log").await?; insert .write(&Event { timestamp: now(), diff --git a/examples/inserter.rs b/examples/inserter.rs index 3b589481..dfd9b702 100644 --- a/examples/inserter.rs +++ b/examples/inserter.rs @@ -22,7 +22,7 @@ struct MyRow { // In other words, this pattern is applicable for ETL-like tasks. async fn dense(client: &Client, mut rx: Receiver) -> Result<()> { let mut inserter = client - .inserter::(TABLE_NAME)? + .inserter::(TABLE_NAME) // We limit the number of rows to be inserted in a single `INSERT` statement. // We use small value (100) for the example only. // See documentation of `with_max_rows` for details. @@ -32,7 +32,7 @@ async fn dense(client: &Client, mut rx: Receiver) -> Result<()> { .with_max_bytes(1_048_576); while let Some(no) = rx.recv().await { - inserter.write(&MyRow { no })?; + inserter.write(&MyRow { no }).await?; inserter.commit().await?; } @@ -47,7 +47,7 @@ async fn dense(client: &Client, mut rx: Receiver) -> Result<()> { // Some rows are arriving one by one with delay, some batched. async fn sparse(client: &Client, mut rx: Receiver) -> Result<()> { let mut inserter = client - .inserter::(TABLE_NAME)? + .inserter::(TABLE_NAME) // Slice the stream into chunks (one `INSERT` per chunk) by time. // See documentation of `with_period` for details. .with_period(Some(Duration::from_millis(100))) @@ -85,7 +85,7 @@ async fn sparse(client: &Client, mut rx: Receiver) -> Result<()> { Err(TryRecvError::Disconnected) => break, }; - inserter.write(&MyRow { no })?; + inserter.write(&MyRow { no }).await?; inserter.commit().await?; // You can use result of `commit()` to get the number of rows inserted. diff --git a/examples/mock.rs b/examples/mock.rs index 9b539a4b..44dec392 100644 --- a/examples/mock.rs +++ b/examples/mock.rs @@ -19,7 +19,7 @@ async fn make_select(client: &Client) -> Result> { } async fn make_insert(client: &Client, data: &[SomeRow]) -> Result<()> { - let mut insert = client.insert::("who cares")?; + let mut insert = client.insert::("who cares").await?; for row in data { insert.write(row).await?; } diff --git a/examples/session_id.rs b/examples/session_id.rs index e0a8b19f..5ac3d187 100644 --- a/examples/session_id.rs +++ b/examples/session_id.rs @@ -41,7 +41,7 @@ async fn main() -> Result<()> { i: i32, } - let mut insert = client.insert::(table_name)?; + let mut insert = client.insert::(table_name).await?; insert.write(&MyRow { i: 42 }).await?; insert.end().await?; diff --git a/examples/time_types_example.rs b/examples/time_types_example.rs index 0be4ea38..dfac616a 100644 --- a/examples/time_types_example.rs +++ b/examples/time_types_example.rs @@ -75,7 +75,7 @@ async fn main() -> Result<(), Box> { + time::Duration::nanoseconds(123_456_789), }; - let mut insert = client.insert::("time_example")?; + let mut insert = client.insert::("time_example").await?; insert.write(&time_example).await?; insert.end().await?; @@ -90,7 +90,7 @@ async fn main() -> Result<(), Box> { + Duration::nanoseconds(987_654_321), }; - let mut insert = client.insert::("time_example")?; + let mut insert = client.insert::("time_example").await?; insert.write(&time_example_chrono).await?; insert.end().await?; @@ -110,7 +110,7 @@ async fn main() -> Result<(), Box> { time64_micros: edge, time64_nanos: edge, }; - let mut insert = client.insert::("time_example")?; + let mut insert = client.insert::("time_example").await?; insert.write(&data).await?; insert.end().await?; println!("Inserted edge case #{i}: {edge:?}"); diff --git a/examples/usage.rs b/examples/usage.rs index 6e78fd52..a0bfbf8c 100644 --- a/examples/usage.rs +++ b/examples/usage.rs @@ -29,7 +29,7 @@ async fn ddl(client: &Client) -> Result<()> { } async fn insert(client: &Client) -> Result<()> { - let mut insert = client.insert::>("some")?; + let mut insert = client.insert::>("some").await?; for i in 0..1000 { insert.write(&MyRow { no: i, name: "foo" }).await?; } @@ -42,12 +42,12 @@ async fn insert(client: &Client) -> Result<()> { #[cfg(feature = "inserter")] async fn inserter(client: &Client) -> Result<()> { let mut inserter = client - .inserter::>("some")? + .inserter::>("some") .with_max_rows(100_000) .with_period(Some(std::time::Duration::from_secs(15))); for i in 0..1000 { - inserter.write(&MyRow { no: i, name: "foo" })?; + inserter.write(&MyRow { no: i, name: "foo" }).await?; inserter.commit().await?; } diff --git a/src/cursors/row.rs b/src/cursors/row.rs index 1b3e1aaf..83ee12f7 100644 --- a/src/cursors/row.rs +++ b/src/cursors/row.rs @@ -45,7 +45,7 @@ impl RowCursor { match parse_rbwnat_columns_header(&mut slice) { Ok(columns) if !columns.is_empty() => { self.bytes.set_remaining(slice.len()); - self.row_metadata = Some(RowMetadata::new::(columns)); + self.row_metadata = Some(RowMetadata::new_for_cursor::(columns)); return Ok(()); } Ok(_) => { diff --git a/src/insert.rs b/src/insert.rs index 6c33c344..89b93e1b 100644 --- a/src/insert.rs +++ b/src/insert.rs @@ -1,23 +1,25 @@ -use std::{future::Future, marker::PhantomData, mem, panic, pin::Pin, time::Duration}; - +use crate::headers::{with_authentication, with_request_headers}; +use crate::row_metadata::RowMetadata; +use crate::rowbinary::{serialize_row_binary, serialize_with_validation}; +use crate::{ + error::{Error, Result}, + request_body::{ChunkSender, RequestBody}, + response::Response, + row::{self, Row}, + Client, Compression, RowWrite, +}; use bytes::{Bytes, BytesMut}; +use clickhouse_types::put_rbwnat_columns_header; use hyper::{self, Request}; use replace_with::replace_with_or_abort; +use std::sync::Arc; +use std::{future::Future, marker::PhantomData, mem, panic, pin::Pin, time::Duration}; use tokio::{ task::JoinHandle, time::{Instant, Sleep}, }; use url::Url; -use crate::headers::{with_authentication, with_request_headers}; -use crate::{ - error::{Error, Result}, - request_body::{ChunkSender, RequestBody}, - response::Response, - row::{self, Row, RowWrite}, - rowbinary, Client, Compression, -}; - // The desired max frame size. const BUFFER_SIZE: usize = 256 * 1024; // Threshold to send a chunk. Should be slightly less than `BUFFER_SIZE` @@ -36,6 +38,7 @@ const_assert!(BUFFER_SIZE.is_power_of_two()); // to use the whole buffer's capac pub struct Insert { state: InsertState, buffer: BytesMut, + row_metadata: Option>, #[cfg(feature = "lz4")] compression: Compression, send_timeout: Option, @@ -118,8 +121,7 @@ macro_rules! timeout { } impl Insert { - // TODO: remove Result - pub(crate) fn new(client: &Client, table: &str) -> Result + pub(crate) fn new(client: &Client, table: &str, row_metadata: Option>) -> Self where T: Row, { @@ -128,9 +130,14 @@ impl Insert { // TODO: what about escaping a table name? // https://clickhouse.com/docs/en/sql-reference/syntax#identifiers - let sql = format!("INSERT INTO {table}({fields}) FORMAT RowBinary"); + let format = if row_metadata.is_some() { + "RowBinaryWithNamesAndTypes" + } else { + "RowBinary" + }; + let sql = format!("INSERT INTO {table}({fields}) FORMAT {format}"); - Ok(Self { + Self { state: InsertState::NotStarted { client: Box::new(client.clone()), sql, @@ -142,7 +149,8 @@ impl Insert { end_timeout: None, sleep: Box::pin(tokio::time::sleep(Duration::new(0, 0))), _marker: PhantomData, - }) + row_metadata, + } } /// Sets timeouts for different operations. @@ -193,6 +201,7 @@ impl Insert { /// socket. /// /// Close to: + /// /// ```ignore /// async fn write(&self, row: &T) -> Result; /// ``` @@ -204,6 +213,7 @@ impl Insert { /// used anymore. /// /// # Panics + /// /// If called after the previous call that returned an error. pub fn write<'a>( &'a mut self, @@ -235,7 +245,10 @@ impl Insert { }?; let old_buf_size = self.buffer.len(); - let result = rowbinary::serialize_into(&mut self.buffer, row); + let result = match &self.row_metadata { + Some(metadata) => serialize_with_validation(&mut self.buffer, row, metadata), + None => serialize_row_binary(&mut self.buffer, row), + }; let written = self.buffer.len() - old_buf_size; if result.is_err() { @@ -266,7 +279,6 @@ impl Insert { // It's difficult to determine when allocations occur. // So, instead we control it manually here and rely on the system allocator. let chunk = self.take_and_prepare_chunk()?; - let sender = self.state.sender().unwrap(); // checked above let is_timed_out = match timeout!(self, send_timeout, sender.send(chunk)) { @@ -368,6 +380,13 @@ impl Insert { let handle = tokio::spawn(async move { Response::new(future, Compression::None).finish().await }); + match self.row_metadata { + None => (), // RowBinary is used, no header is required. + Some(ref metadata) => { + put_rbwnat_columns_header(&metadata.columns, &mut self.buffer)?; + } + } + self.state = InsertState::Active { handle, sender }; Ok(()) } diff --git a/src/inserter.rs b/src/inserter.rs index f0d68a41..6906cbfd 100644 --- a/src/inserter.rs +++ b/src/inserter.rs @@ -58,9 +58,8 @@ impl Inserter where T: Row, { - // TODO: (breaking change) remove `Result`. - pub(crate) fn new(client: &Client, table: &str) -> Result { - Ok(Self { + pub(crate) fn new(client: &Client, table: &str) -> Self { + Self { client: client.clone(), table: table.into(), max_bytes: u64::MAX, @@ -71,7 +70,7 @@ where ticks: Ticks::default(), pending: Quantities::ZERO, in_transaction: false, - }) + } } /// See [`Insert::with_timeouts()`]. @@ -220,12 +219,12 @@ where /// # Panics /// If called after the previous call that returned an error. #[inline] - pub fn write(&mut self, row: &T::Value<'_>) -> Result<()> + pub async fn write(&mut self, row: &T::Value<'_>) -> Result<()> where T: RowWrite, { if self.insert.is_none() { - self.init_insert()?; + self.init_insert().await?; } match self.insert.as_mut().unwrap().do_write(row) { @@ -291,11 +290,11 @@ where #[cold] #[inline(never)] - fn init_insert(&mut self) -> Result<()> { + async fn init_insert(&mut self) -> Result<()> { debug_assert!(self.insert.is_none()); debug_assert_eq!(self.pending, Quantities::ZERO); - let mut new_insert: Insert = self.client.insert(&self.table)?; + let mut new_insert: Insert = self.client.insert(&self.table).await?; new_insert.set_timeouts(self.send_timeout, self.end_timeout); self.insert = Some(new_insert); Ok(()) diff --git a/src/lib.rs b/src/lib.rs index beff1bf8..39401f1c 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -10,11 +10,15 @@ pub use self::{ row::{Row, RowOwned, RowRead, RowWrite}, }; use self::{error::Result, http_client::HttpClient}; +use crate::row_metadata::RowMetadata; +use crate::sql::Identifier; #[doc = include_str!("row_derive.md")] pub use clickhouse_derive::Row; +use clickhouse_types::parse_rbwnat_columns_header; use std::{collections::HashMap, fmt::Display, sync::Arc}; +use tokio::sync::RwLock; pub mod error; pub mod insert; @@ -58,6 +62,7 @@ pub struct Client { headers: HashMap, products_info: Vec, validation: bool, + row_metadata_cache: Arc, #[cfg(feature = "test-util")] mocked: bool, @@ -101,6 +106,16 @@ impl Default for Client { } } +/// Cache for [`RowMetadata`] to avoid allocating it for the same struct more than once +/// during the application lifecycle. Key: fully qualified table name (e.g. `database.table`). +pub(crate) struct RowMetadataCache(RwLock>>); + +impl Default for RowMetadataCache { + fn default() -> Self { + RowMetadataCache(RwLock::new(HashMap::default())) + } +} + impl Client { /// Creates a new client with a specified underlying HTTP client. /// @@ -116,6 +131,7 @@ impl Client { headers: HashMap::new(), products_info: Vec::default(), validation: true, + row_metadata_cache: Arc::new(RowMetadataCache::default()), #[cfg(feature = "test-util")] mocked: false, } @@ -314,15 +330,33 @@ impl Client { /// Starts a new INSERT statement. /// + /// # Validation + /// + /// If validation is enabled (default), `RowBinaryWithNamesAndTypes` input format is used. + /// When [`Client::insert`] method is called for this `table` for the first time, + /// it will fetch the table schema from the server, allowing to validate the serialized rows, + /// as well as write the names and types of the columns in the request header. + /// + /// Fetching the schema will happen only once per `table`, + /// as the schema is cached by the client internally. + /// + /// With disabled validation, the schema is not fetched, + /// and the rows serialized with `RowBinary` input format. + /// /// # Panics + /// /// If `T` has unnamed fields, e.g. tuples. - pub fn insert(&self, table: &str) -> Result> { - insert::Insert::new(self, table) + pub async fn insert(&self, table: &str) -> Result> { + if self.get_validation() { + let metadata = self.get_row_metadata_for_insert::(table).await?; + return Ok(insert::Insert::new(self, table, Some(metadata))); + } + Ok(insert::Insert::new(self, table, None)) } - /// Creates an inserter to perform multiple INSERTs. + /// Creates an inserter to perform multiple INSERT statements. #[cfg(feature = "inserter")] - pub fn inserter(&self, table: &str) -> Result> { + pub fn inserter(&self, table: &str) -> inserter::Inserter { inserter::Inserter::new(self, table) } @@ -384,6 +418,40 @@ impl Client { self.mocked = true; self } + + async fn get_row_metadata_for_insert( + &self, + table_name: &str, + ) -> Result> { + let read_lock = self.row_metadata_cache.0.read().await; + match read_lock.get(table_name) { + Some(metadata) => Ok(metadata.clone()), + None => { + drop(read_lock); + // TODO: should it be moved to a cold function? + let mut write_lock = self.row_metadata_cache.0.write().await; + let db = match self.database { + Some(ref db) => db, + None => "default", + }; + let mut bytes_cursor = self + .query("SELECT * FROM ? LIMIT 0") + .bind(Identifier(table_name)) + // don't allow to override the client database set in the client instance + // with a `.with_option("database", "some_other_db")` call on the app side + .with_option("database", db) + .fetch_bytes("RowBinaryWithNamesAndTypes")?; + let mut buffer = Vec::::new(); + while let Some(chunk) = bytes_cursor.next().await? { + buffer.extend_from_slice(&chunk); + } + let columns = parse_rbwnat_columns_header(&mut buffer.as_slice())?; + let metadata = Arc::new(RowMetadata::new_for_insert::(columns)); + write_lock.insert(table_name.to_string(), metadata.clone()); + Ok(metadata) + } + } + } } /// This is a private API exported only for internal purposes. @@ -400,7 +468,10 @@ pub mod _priv { #[cfg(test)] mod client_tests { - use crate::{Authentication, Client}; + use crate::_priv::RowKind; + use crate::row_metadata::{AccessType, RowMetadata}; + use crate::{Authentication, Client, Row}; + use clickhouse_types::{Column, DataTypeNode}; #[test] fn it_can_use_credentials_auth() { @@ -528,6 +599,77 @@ mod client_tests { assert!(client.validation); } + #[derive(Debug, Clone, PartialEq)] + struct SystemRolesRow { + name: String, + id: uuid::Uuid, + storage: String, + } + + impl SystemRolesRow { + fn columns() -> Vec { + vec![ + Column::new("name".to_string(), DataTypeNode::String), + Column::new("id".to_string(), DataTypeNode::UUID), + Column::new("storage".to_string(), DataTypeNode::String), + ] + } + } + + impl Row for SystemRolesRow { + const NAME: &'static str = "SystemRolesRow"; + const KIND: RowKind = RowKind::Struct; + const COLUMN_COUNT: usize = 3; + const COLUMN_NAMES: &'static [&'static str] = &["name", "id", "storage"]; + type Value<'a> = SystemRolesRow; + } + + #[test] + fn get_row_metadata() { + let metadata = RowMetadata::new_for_cursor::(SystemRolesRow::columns()); + assert_eq!(metadata.columns, SystemRolesRow::columns()); + assert_eq!(metadata.access_type, AccessType::WithSeqAccess); + + // the order is shuffled => map access + let columns = vec![ + Column::new("id".to_string(), DataTypeNode::UUID), + Column::new("storage".to_string(), DataTypeNode::String), + Column::new("name".to_string(), DataTypeNode::String), + ]; + let metadata = RowMetadata::new_for_cursor::(columns.clone()); + assert_eq!(metadata.columns, columns); + assert_eq!( + metadata.access_type, + AccessType::WithMapAccess(vec![1, 2, 0]) // see COLUMN_NAMES above + ); + } + + #[tokio::test] + async fn cache_row_metadata() { + let client = Client::default() + .with_url("http://localhost:8123") + .with_database("system"); + + let metadata = client + .get_row_metadata_for_insert::("roles") + .await + .unwrap(); + + assert_eq!(metadata.columns, SystemRolesRow::columns()); + assert_eq!(metadata.access_type, AccessType::WithSeqAccess); + + // we can now use a dummy client, cause the metadata is cached, + // and no calls to the database will be made + client + .with_url("whatever") + .get_row_metadata_for_insert::("roles") + .await + .unwrap(); + + assert_eq!(metadata.columns, SystemRolesRow::columns()); + assert_eq!(metadata.access_type, AccessType::WithSeqAccess); + } + #[test] fn it_does_follow_previous_configuration() { let client = Client::default().with_option("async_insert", "1"); diff --git a/src/row.rs b/src/row.rs index c0c18e82..a0f0db6c 100644 --- a/src/row.rs +++ b/src/row.rs @@ -131,7 +131,7 @@ impl RowRead for R where R: for<'a> Row: Deserialize<'a>> {} /// client: Client, /// data: &[R], /// ) -> Result<()> { -/// let mut insert = client.insert::(table)?; +/// let mut insert = client.insert::(table).await?; /// for row in data { /// insert.write(row).await?; /// } @@ -160,7 +160,7 @@ impl RowRead for R where R: for<'a> Row: Deserialize<'a>> {} /// data: &[R::Value<'_>], //<<< R::Value instead of R /// ) -> Result<()> { /// /* same code */ -/// # let mut insert = client.insert::(table)?; +/// # let mut insert = client.insert::(table).await?; /// # for row in data { insert.write(row).await?; } /// # insert.end().await /// } diff --git a/src/row_metadata.rs b/src/row_metadata.rs index 1f2a921f..f9390d52 100644 --- a/src/row_metadata.rs +++ b/src/row_metadata.rs @@ -1,23 +1,8 @@ -// FIXME: this is allowed only temporarily, -// before the insert RBWNAT implementation is ready, -// cause otherwise the caches are never used. -#![allow(dead_code)] -#![allow(unreachable_pub)] - use crate::row::RowKind; -use crate::sql::Identifier; -use crate::Result; use crate::Row; -use clickhouse_types::{parse_rbwnat_columns_header, Column}; +use clickhouse_types::Column; use std::collections::HashMap; use std::fmt::Display; -use std::sync::Arc; -use tokio::sync::{OnceCell, RwLock}; - -/// Cache for [`RowMetadata`] to avoid allocating it for the same struct more than once -/// during the application lifecycle. Key: fully qualified table name (e.g. `database.table`). -type LockedRowMetadataCache = RwLock>>; -static ROW_METADATA_CACHE: OnceCell = OnceCell::const_new(); #[derive(Debug, PartialEq)] pub(crate) enum AccessType { @@ -32,17 +17,23 @@ pub(crate) enum AccessType { /// as it is calculated only once per struct. It does not have lifetimes, /// so it does not introduce a breaking change to [`crate::cursors::RowCursor`]. pub(crate) struct RowMetadata { - /// Database schema, or columns, are parsed before the first call to (de)serializer. + /// Database schema, or table columns, are parsed before the first call to deserializer. + /// However, the order here depends on the usage context: + /// * For selects, it is defined in the same order as in the database schema. + /// * For inserts, it is adjusted to the order of fields in the struct definition. pub(crate) columns: Vec, /// This determines whether we can just use [`crate::rowbinary::de::RowBinarySeqAccess`] /// or a more sophisticated approach with [`crate::rowbinary::de::RowBinaryStructAsMapAccess`] /// to support structs defined with different fields order than in the schema. - /// (De)serializing a struct as a map will be approximately 40% slower than as a sequence. - access_type: AccessType, + /// + /// Deserializing a struct as a map can be significantly slower, but that depends + /// on the shape of the data. In some cases, there is no noticeable difference, + /// in others, it could be up to 2-3x slower. + pub(crate) access_type: AccessType, } impl RowMetadata { - pub(crate) fn new(columns: Vec) -> Self { + pub(crate) fn new_for_cursor(columns: Vec) -> Self { let access_type = match T::KIND { RowKind::Primitive => { if columns.len() != 1 { @@ -130,6 +121,60 @@ impl RowMetadata { } } + pub(crate) fn new_for_insert(columns: Vec) -> Self { + if T::KIND != RowKind::Struct { + panic!( + "SerializerRowMetadata can only be created for structs, \ + but got {:?} instead.\n#### All schema columns:\n{}", + T::KIND, + join_panic_schema_hint(&columns), + ); + } + if columns.len() != T::COLUMN_NAMES.len() { + panic!( + "While processing struct {}: database schema has {} columns, \ + but the struct definition has {} fields.\ + \n#### All struct fields:\n{}\n#### All schema columns:\n{}", + T::NAME, + columns.len(), + T::COLUMN_NAMES.len(), + join_panic_schema_hint(T::COLUMN_NAMES), + join_panic_schema_hint(&columns), + ); + } + + let mut result_columns: Vec = Vec::with_capacity(columns.len()); + let db_columns_lookup: HashMap<&str, &Column> = + columns.iter().map(|col| (col.name.as_str(), col)).collect(); + + for struct_column_name in T::COLUMN_NAMES { + match db_columns_lookup.get(*struct_column_name) { + Some(col) => result_columns.push((*col).clone()), + None => { + panic!( + "While processing struct {}: database schema has no column named {}.\ + \n#### All struct fields:\n{}\n#### All schema columns:\n{}", + T::NAME, + struct_column_name, + join_panic_schema_hint(T::COLUMN_NAMES), + join_panic_schema_hint(&db_columns_lookup.values().collect::>()), + ); + } + } + } + + Self { + columns: result_columns, + access_type: AccessType::WithSeqAccess, // ignored + } + } + + /// Returns the index of the column in the database schema + /// that corresponds to the field with the given index in the struct. + /// + /// Only makes sense for selects; for inserts, it is always the same as `struct_idx`, + /// since we write the header with the field order defined in the struct, + /// and ClickHouse server figures out the rest on its own. #[inline] pub(crate) fn get_schema_index(&self, struct_idx: usize) -> usize { match &self.access_type { @@ -138,55 +183,22 @@ impl RowMetadata { mapping[struct_idx] } else { // unreachable - panic!("Struct has more fields than columns in the database schema",) + panic!("Struct has more fields than columns in the database schema") } } AccessType::WithSeqAccess => struct_idx, // should be unreachable } } + /// Returns `true` if the field order in the struct is different from the database schema. + /// + /// Only makes sense for selects; for inserts, it is always `false`. #[inline] pub(crate) fn is_field_order_wrong(&self) -> bool { matches!(self.access_type, AccessType::WithMapAccess(_)) } } -pub(crate) async fn get_row_metadata( - client: &crate::Client, - table_name: &str, -) -> Result> { - let locked_cache = ROW_METADATA_CACHE - .get_or_init(|| async { RwLock::new(HashMap::new()) }) - .await; - let cache_guard = locked_cache.read().await; - match cache_guard.get(table_name) { - Some(metadata) => Ok(metadata.clone()), - None => cache_row_metadata::(client, table_name, locked_cache).await, - } -} - -/// Used internally to introspect and cache the table structure to allow validation -/// of serialized rows before submitting the first [`insert::Insert::write`]. -async fn cache_row_metadata( - client: &crate::Client, - table_name: &str, - locked_cache: &LockedRowMetadataCache, -) -> Result> { - let mut bytes_cursor = client - .query("SELECT * FROM ? LIMIT 0") - .bind(Identifier(table_name)) - .fetch_bytes("RowBinaryWithNamesAndTypes")?; - let mut buffer = Vec::::new(); - while let Some(chunk) = bytes_cursor.next().await? { - buffer.extend_from_slice(&chunk); - } - let columns = parse_rbwnat_columns_header(&mut buffer.as_slice())?; - let mut cache = locked_cache.write().await; - let metadata = Arc::new(RowMetadata::new::(columns)); - cache.insert(table_name.to_string(), metadata.clone()); - Ok(metadata) -} - fn join_panic_schema_hint(col: &[T]) -> String { if col.is_empty() { return String::default(); diff --git a/src/rowbinary/de.rs b/src/rowbinary/de.rs index 88bf9823..e8c8f7f7 100644 --- a/src/rowbinary/de.rs +++ b/src/rowbinary/de.rs @@ -232,7 +232,7 @@ where _variants: &'static [&'static str], visitor: V, ) -> Result { - let deserializer = &mut self.inner(SerdeType::Enum); + let deserializer = &mut self.inner(SerdeType::Variant); visitor.visit_enum(RowBinaryEnumAccess { deserializer }) } diff --git a/src/rowbinary/mod.rs b/src/rowbinary/mod.rs index 0816d52b..997bca1d 100644 --- a/src/rowbinary/mod.rs +++ b/src/rowbinary/mod.rs @@ -1,5 +1,6 @@ pub(crate) use de::deserialize_row; -pub(crate) use ser::serialize_into; +pub(crate) use ser::serialize_row_binary; +pub(crate) use ser::serialize_with_validation; pub(crate) mod validation; diff --git a/src/rowbinary/ser.rs b/src/rowbinary/ser.rs index c644b118..3ae10db6 100644 --- a/src/rowbinary/ser.rs +++ b/src/rowbinary/ser.rs @@ -1,15 +1,38 @@ +use crate::error::Error::SequenceMustHaveLength; +use crate::error::{Error, Result}; +use crate::row_metadata::RowMetadata; +use crate::rowbinary::validation::{DataTypeValidator, SchemaValidator, SerdeType}; +use crate::Row; use bytes::BufMut; use clickhouse_types::put_leb128; +use serde::ser::SerializeMap; use serde::{ ser::{Impossible, SerializeSeq, SerializeStruct, SerializeTuple, Serializer}, Serialize, }; +use std::marker::PhantomData; -use crate::error::{Error, Result}; +/// Serializes `row` using the `RowBinary` format and writes to `buffer`. +pub(crate) fn serialize_row_binary( + buffer: B, + row: &R, +) -> Result<()> { + let mut serializer = RowBinarySerializer::::new(buffer, ()); + row.serialize(&mut serializer)?; + Ok(()) +} -/// Serializes `value` using the RowBinary format and writes to `buffer`. -pub(crate) fn serialize_into(buffer: impl BufMut, value: &impl Serialize) -> Result<()> { - let mut serializer = RowBinarySerializer { buffer }; +/// Serializes `row` using the `RowBinary` format and writes to `buffer`. +/// Additionally, it will perform validation against the provided `row_metadata`, +/// similarly to how [`crate::rowbinary::de::deserialize_row_with_validation`] works. +/// `RowBinaryWithNamesAndTypes` header is expected to be written by [`crate::insert::Insert`]. +pub(crate) fn serialize_with_validation( + buffer: B, + value: &R, + metadata: &RowMetadata, +) -> Result<()> { + let validator = DataTypeValidator::::new(metadata); + let mut serializer = RowBinarySerializer::new(buffer, validator); value.serialize(&mut serializer)?; Ok(()) } @@ -17,46 +40,81 @@ pub(crate) fn serialize_into(buffer: impl BufMut, value: &impl Serialize) -> Res /// A serializer for the RowBinary format. /// /// See https://clickhouse.com/docs/en/interfaces/formats#rowbinary for details. -struct RowBinarySerializer { +struct RowBinarySerializer = ()> { buffer: B, + validator: V, + _marker: PhantomData, +} + +type RowBinaryInnerTypeSerializer<'ser, B, R, V> = RowBinarySerializer<&'ser mut B, R, V>; + +impl> RowBinarySerializer { + fn new(buffer: B, validator: V) -> Self { + Self { + buffer, + validator, + _marker: PhantomData, + } + } } macro_rules! impl_num { - ($ty:ty, $ser_method:ident, $writer_method:ident) => { + ($ty:ty, $ser_method:ident, $writer_method:ident, $serde_type:expr) => { + #[inline] + fn $ser_method(self, v: $ty) -> Result<()> { + self.validator.validate($serde_type); + self.buffer.$writer_method(v); + Ok(()) + } + }; +} + +macro_rules! impl_num_or_enum { + ($ty:ty, $ser_method:ident, $writer_method:ident, $serde_type:expr) => { #[inline] fn $ser_method(self, v: $ty) -> Result<()> { + self.validator + .validate($serde_type) + .validate_identifier::<$ty>(v); self.buffer.$writer_method(v); Ok(()) } }; } -impl Serializer for &'_ mut RowBinarySerializer { +impl<'ser, B: BufMut, R: Row, V: SchemaValidator> Serializer + for &'ser mut RowBinarySerializer +{ type Error = Error; type Ok = (); - type SerializeMap = Impossible<(), Error>; - type SerializeSeq = Self; + type SerializeStruct = Self; + + type SerializeSeq = RowBinaryInnerTypeSerializer<'ser, B, R, V::Inner<'ser>>; + type SerializeTuple = Self::SerializeSeq; + type SerializeMap = Self::SerializeSeq; + type SerializeStructVariant = Impossible<(), Error>; - type SerializeTuple = Self; type SerializeTupleStruct = Impossible<(), Error>; type SerializeTupleVariant = Impossible<(), Error>; - impl_num!(i8, serialize_i8, put_i8); - impl_num!(i16, serialize_i16, put_i16_le); - impl_num!(i32, serialize_i32, put_i32_le); - impl_num!(i64, serialize_i64, put_i64_le); - impl_num!(i128, serialize_i128, put_i128_le); - impl_num!(u8, serialize_u8, put_u8); - impl_num!(u16, serialize_u16, put_u16_le); - impl_num!(u32, serialize_u32, put_u32_le); - impl_num!(u64, serialize_u64, put_u64_le); - impl_num!(u128, serialize_u128, put_u128_le); - impl_num!(f32, serialize_f32, put_f32_le); - impl_num!(f64, serialize_f64, put_f64_le); + impl_num_or_enum!(i8, serialize_i8, put_i8, SerdeType::I8); + impl_num_or_enum!(i16, serialize_i16, put_i16_le, SerdeType::I16); + + impl_num!(i32, serialize_i32, put_i32_le, SerdeType::I32); + impl_num!(i64, serialize_i64, put_i64_le, SerdeType::I64); + impl_num!(i128, serialize_i128, put_i128_le, SerdeType::I128); + impl_num!(u8, serialize_u8, put_u8, SerdeType::U8); + impl_num!(u16, serialize_u16, put_u16_le, SerdeType::U16); + impl_num!(u32, serialize_u32, put_u32_le, SerdeType::U32); + impl_num!(u64, serialize_u64, put_u64_le, SerdeType::U64); + impl_num!(u128, serialize_u128, put_u128_le, SerdeType::U128); + impl_num!(f32, serialize_f32, put_f32_le, SerdeType::F32); + impl_num!(f64, serialize_f64, put_f64_le, SerdeType::F64); #[inline] fn serialize_bool(self, v: bool) -> Result<()> { + self.validator.validate(SerdeType::Bool); self.buffer.put_u8(v as _); Ok(()) } @@ -68,6 +126,7 @@ impl Serializer for &'_ mut RowBinarySerializer { #[inline] fn serialize_str(self, v: &str) -> Result<()> { + self.validator.validate(SerdeType::Str); put_leb128(&mut self.buffer, v.len() as u64); self.buffer.put_slice(v.as_bytes()); Ok(()) @@ -75,21 +134,26 @@ impl Serializer for &'_ mut RowBinarySerializer { #[inline] fn serialize_bytes(self, v: &[u8]) -> Result<()> { - put_leb128(&mut self.buffer, v.len() as u64); + let size = v.len(); + self.validator.validate(SerdeType::Bytes(size)); + put_leb128(&mut self.buffer, size as u64); self.buffer.put_slice(v); Ok(()) } #[inline] fn serialize_none(self) -> Result<()> { + self.validator.validate(SerdeType::Option); self.buffer.put_u8(1); Ok(()) } #[inline] fn serialize_some(self, value: &T) -> Result<()> { - self.buffer.put_u8(0); - value.serialize(self) + let mut inner = + RowBinarySerializer::new(&mut self.buffer, self.validator.validate(SerdeType::Option)); + inner.buffer.put_u8(0); + value.serialize(&mut inner) } #[inline] @@ -124,36 +188,40 @@ impl Serializer for &'_ mut RowBinarySerializer { #[inline] fn serialize_newtype_variant( self, - _name: &'static str, + name: &'static str, variant_index: u32, - _variant: &'static str, + variant: &'static str, value: &T, ) -> Result<()> { - // TODO: - // - Now this code implicitly allows using enums at the top level. - // However, instead of a more descriptive panic, it ends with a "not enough data." error. - // - Also, it produces an unclear message for a forgotten `serde_repr` (Enum8 and Enum16). - // See https://github.com/ClickHouse/clickhouse-rs/pull/170#discussion_r1848549636 - // Max number of types in the Variant data type is 255 // See also: https://github.com/ClickHouse/ClickHouse/issues/54864 if variant_index > 255 { - panic!("max number of types in the Variant data type is 255, got {variant_index}") + panic!( + "max number of types in the Variant data type is 255, \ + got {variant_index} for {name}::{variant}" + ); } - self.buffer.put_u8(variant_index as u8); - value.serialize(self) + let idx = variant_index as u8; // safe cast due to the check above + let mut inner = self.validator.validate(SerdeType::Variant); + inner.validate_identifier(idx); + self.buffer.put_u8(idx); + value.serialize(&mut RowBinarySerializer::new(&mut self.buffer, inner)) } #[inline] fn serialize_seq(self, len: Option) -> Result { - let len = len.ok_or(Error::SequenceMustHaveLength)?; + let len = len.ok_or(SequenceMustHaveLength)?; + let inner = self.validator.validate(SerdeType::Seq(len)); put_leb128(&mut self.buffer, len as u64); - Ok(self) + Ok(RowBinarySerializer::new(&mut self.buffer, inner)) } #[inline] - fn serialize_tuple(self, _len: usize) -> Result { - Ok(self) + fn serialize_tuple(self, len: usize) -> Result { + Ok(RowBinarySerializer::new( + &mut self.buffer, + self.validator.validate(SerdeType::Tuple(len)), + )) } #[inline] @@ -177,8 +245,13 @@ impl Serializer for &'_ mut RowBinarySerializer { } #[inline] - fn serialize_map(self, _len: Option) -> Result { - panic!("maps are unsupported, use `Vec<(A, B)>` instead"); + fn serialize_map(self, len: Option) -> Result { + let len = len.ok_or(SequenceMustHaveLength)?; + put_leb128(&mut self.buffer, len as u64); + Ok(RowBinarySerializer::new( + &mut self.buffer, + self.validator.validate(SerdeType::Map(len)), + )) } #[inline] @@ -203,7 +276,14 @@ impl Serializer for &'_ mut RowBinarySerializer { } } -impl SerializeStruct for &mut RowBinarySerializer { +/// Unlike [`SerializeSeq`], [`SerializeTuple`] and [`SerializeMap`], +/// this is supposed to be the main entry point of serialization. +/// The impl here uses the _reference_ to the serializer with B: BufMut, +/// while the others are implemented over the concrete type, +/// but with [`&mut B: BufMut`] inside. +impl> SerializeStruct + for &'_ mut RowBinarySerializer +{ type Error = Error; type Ok = (); @@ -218,20 +298,29 @@ impl SerializeStruct for &mut RowBinarySerializer { } } -impl SerializeSeq for &'_ mut RowBinarySerializer { +impl> SerializeSeq + for RowBinaryInnerTypeSerializer<'_, B, R, V> +{ type Error = Error; type Ok = (); - fn serialize_element(&mut self, value: &T) -> Result<()> { - value.serialize(&mut **self) + #[inline] + fn serialize_element(&mut self, value: &T) -> std::result::Result<(), Self::Error> + where + T: ?Sized + Serialize, + { + value.serialize(self) } + #[inline] fn end(self) -> Result<()> { Ok(()) } } -impl SerializeTuple for &'_ mut RowBinarySerializer { +impl> SerializeTuple + for RowBinaryInnerTypeSerializer<'_, B, R, V> +{ type Error = Error; type Ok = (); @@ -240,7 +329,36 @@ impl SerializeTuple for &'_ mut RowBinarySerializer { where T: ?Sized + Serialize, { - value.serialize(&mut **self) + value.serialize(self) + } + + #[inline] + fn end(self) -> Result<()> { + Ok(()) + } +} + +/// TODO: usage of [`SerializeMap::serialize_entry`] implies some borrow checker issues. +impl<'ser, B: BufMut, R: Row, V: SchemaValidator> SerializeMap + for RowBinaryInnerTypeSerializer<'ser, B, R, V> +{ + type Error = Error; + type Ok = (); + + #[inline] + fn serialize_key(&mut self, key: &T) -> Result<()> + where + T: ?Sized + Serialize, + { + key.serialize(self) + } + + #[inline] + fn serialize_value(&mut self, value: &T) -> Result<()> + where + T: ?Sized + Serialize, + { + value.serialize(self) } #[inline] diff --git a/src/rowbinary/tests.rs b/src/rowbinary/tests.rs index 12ee127a..282d0e5d 100644 --- a/src/rowbinary/tests.rs +++ b/src/rowbinary/tests.rs @@ -1,3 +1,4 @@ +use crate::row::Primitive; use crate::Row; use serde::{Deserialize, Serialize}; @@ -9,9 +10,13 @@ struct Timestamp64(u64); #[derive(Debug, PartialEq, Serialize, Deserialize)] struct Time32(i32); +impl Primitive for Time32 {} +impl Primitive for Option {} #[derive(Debug, PartialEq, Serialize, Deserialize)] struct Time64(i64); +impl Primitive for Time64 {} +impl Primitive for Option {} #[derive(Debug, PartialEq, Serialize, Deserialize)] struct FixedPoint64(i64); @@ -157,7 +162,7 @@ fn sample_serialized() -> Vec { #[test] fn it_serializes() { let mut actual = Vec::new(); - super::serialize_into(&mut actual, &sample()).unwrap(); + super::serialize_row_binary(&mut actual, &sample()).unwrap(); assert_eq!(actual, sample_serialized()); } @@ -183,7 +188,7 @@ fn it_serializes_time64() { let time64 = Time64(value); println!("Time64 value: {}", time64.0); let mut actual = Vec::new(); - super::serialize_into(&mut actual, &time64).unwrap(); + super::serialize_row_binary(&mut actual, &time64).unwrap(); // Expected: 42000000000 in little-endian let expected = value.to_le_bytes(); @@ -203,7 +208,7 @@ fn it_serializes_time32() { let value = 42_000; let time32 = Time32(value); let mut actual = Vec::new(); - super::serialize_into(&mut actual, &time32).unwrap(); + super::serialize_row_binary(&mut actual, &time32).unwrap(); let expected = value.to_le_bytes(); assert_eq!(actual, expected, "Time32 serialization mismatch"); } @@ -220,7 +225,7 @@ fn it_serializes_option_time32_some() { let value = 42_000; let time: Option = Some(Time32(value)); let mut actual = Vec::new(); - super::serialize_into(&mut actual, &time).unwrap(); + super::serialize_row_binary(&mut actual, &time).unwrap(); // Nullable encoding: 0x00 = not null, followed by value let mut expected = vec![0x00]; diff --git a/src/rowbinary/validation.rs b/src/rowbinary/validation.rs index abecd5e7..6f713950 100644 --- a/src/rowbinary/validation.rs +++ b/src/rowbinary/validation.rs @@ -8,10 +8,10 @@ use std::marker::PhantomData; /// Note that [`SchemaValidator`] is also implemented for `()`, /// which is used to skip validation if the user disabled it. pub(crate) trait SchemaValidator: Sized { - type Inner<'de>: SchemaValidator + type Inner<'serde>: SchemaValidator where - Self: 'de; - /// The main entry point. The validation flow based on the [`crate::Row::KIND`]. + Self: 'serde; + /// The main entry point. The validation flow based on the [`Row::KIND`]. /// For container types (nullable, array, map, tuple, variant, etc.), /// it will return an [`InnerDataTypeValidator`] instance (see [`InnerDataTypeValidatorKind`]), /// which has its own implementation of this method, allowing recursive validation. @@ -30,14 +30,14 @@ pub(crate) trait SchemaValidator: Sized { fn get_schema_index(&self, struct_idx: usize) -> usize; } -pub(crate) struct DataTypeValidator<'cursor, R: Row> { - metadata: &'cursor RowMetadata, +pub(crate) struct DataTypeValidator<'caller, R: Row> { + metadata: &'caller RowMetadata, current_column_idx: usize, _marker: PhantomData, } -impl<'cursor, R: Row> DataTypeValidator<'cursor, R> { - pub(crate) fn new(metadata: &'cursor RowMetadata) -> Self { +impl<'caller, R: Row> DataTypeValidator<'caller, R> { + pub(crate) fn new(metadata: &'caller RowMetadata) -> Self { Self { metadata, current_column_idx: 0, @@ -62,28 +62,28 @@ impl<'cursor, R: Row> DataTypeValidator<'cursor, R> { .unwrap_or(("Struct".to_string(), &DataTypeNode::Bool)) } - fn panic_on_schema_mismatch<'de>( - &'de self, + fn panic_on_schema_mismatch<'serde>( + &'serde self, data_type: &DataTypeNode, serde_type: &SerdeType, is_inner: bool, - ) -> Option> { + ) -> Option> { match R::KIND { RowKind::Primitive => { panic!( - "While processing row as a primitive: attempting to deserialize \ + "While processing row as a primitive: attempting to (de)serialize \ ClickHouse type {data_type} as {serde_type} which is not compatible" ) } RowKind::Vec => { panic!( - "While processing row as a vector: attempting to deserialize \ + "While processing row as a vector: attempting to (de)serialize \ ClickHouse type {data_type} as {serde_type} which is not compatible" ) } RowKind::Tuple => { panic!( - "While processing row as a tuple: attempting to deserialize \ + "While processing row as a tuple: attempting to (de)serialize \ ClickHouse type {data_type} as {serde_type} which is not compatible" ) } @@ -91,12 +91,12 @@ impl<'cursor, R: Row> DataTypeValidator<'cursor, R> { if is_inner { let (full_name, full_data_type) = self.get_current_column_name_and_type(); panic!( - "While processing column {full_name} defined as {full_data_type}: attempting to deserialize \ + "While processing column {full_name} defined as {full_data_type}: attempting to (de)serialize \ nested ClickHouse type {data_type} as {serde_type} which is not compatible" ) } else { panic!( - "While processing column {}: attempting to deserialize \ + "While processing column {}: attempting to (de)serialize \ ClickHouse type {} as {} which is not compatible", self.get_current_column_name_and_type().0, data_type, @@ -108,11 +108,11 @@ impl<'cursor, R: Row> DataTypeValidator<'cursor, R> { } } -impl<'cursor, R: Row> SchemaValidator for DataTypeValidator<'cursor, R> { - type Inner<'de> - = Option> +impl<'caller, R: Row> SchemaValidator for DataTypeValidator<'caller, R> { + type Inner<'serde> + = Option> where - Self: 'de; + Self: 'serde; #[inline] fn validate(&'_ mut self, serde_type: SerdeType) -> Self::Inner<'_> { @@ -212,25 +212,25 @@ pub(crate) enum MapAsSequenceValidatorState { Value, } -pub(crate) struct InnerDataTypeValidator<'de, 'cursor, R: Row> { - root: &'de DataTypeValidator<'cursor, R>, - kind: InnerDataTypeValidatorKind<'cursor>, +pub(crate) struct InnerDataTypeValidator<'serde, 'caller, R: Row> { + root: &'serde DataTypeValidator<'caller, R>, + kind: InnerDataTypeValidatorKind<'caller>, } #[derive(Debug)] -pub(crate) enum InnerDataTypeValidatorKind<'cursor> { - Array(&'cursor DataTypeNode), +pub(crate) enum InnerDataTypeValidatorKind<'caller> { + Array(&'caller DataTypeNode), FixedString(usize), - Map(&'cursor [Box; 2], MapValidatorState), + Map(&'caller [Box; 2], MapValidatorState), /// Allows supporting ClickHouse `Map` defined as `Vec<(K, V)>` in Rust - MapAsSequence(&'cursor [Box; 2], MapAsSequenceValidatorState), - Tuple(&'cursor [DataTypeNode]), + MapAsSequence(&'caller [Box; 2], MapAsSequenceValidatorState), + Tuple(&'caller [DataTypeNode]), /// This is a hack to support deserializing tuples/arrays (and not structs) from fetch calls - RootTuple(&'cursor [Column], usize), - RootArray(&'cursor DataTypeNode), - Enum(&'cursor HashMap), - Variant(&'cursor [DataTypeNode], VariantValidationState), - Nullable(&'cursor DataTypeNode), + RootTuple(&'caller [Column], usize), + RootArray(&'caller DataTypeNode), + Enum(&'caller HashMap), + Variant(&'caller [DataTypeNode], VariantValidationState), + Nullable(&'caller DataTypeNode), } #[derive(Debug)] @@ -239,11 +239,11 @@ pub(crate) enum VariantValidationState { Identifier(u8), } -impl<'cursor, R: Row> SchemaValidator for Option> { - type Inner<'de> +impl<'caller, R: Row> SchemaValidator for Option> { + type Inner<'serde> = Self where - Self: 'de; + Self: 'serde; #[inline] fn validate(&mut self, serde_type: SerdeType) -> Self { @@ -298,7 +298,7 @@ impl<'cursor, R: Row> SchemaValidator for Option SchemaValidator for Option Drop for InnerDataTypeValidator<'_, '_, R> { if !elements_types.is_empty() { let (column_name, column_type) = self.root.get_current_column_name_and_type(); panic!( - "While processing column {} defined as {}: tuple was not fully deserialized; \ + "While processing column {} defined as {}: tuple was not fully (de)serialized; \ remaining elements: {}; likely, the field definition is incomplete", column_name, column_type, @@ -420,12 +420,12 @@ impl Drop for InnerDataTypeValidator<'_, '_, R> { // static/const dispatch? // separate smaller inline functions? #[inline] -fn validate_impl<'de, 'cursor, R: Row>( - root: &'de DataTypeValidator<'cursor, R>, - column_data_type: &'cursor DataTypeNode, +fn validate_impl<'serde, 'caller, R: Row>( + root: &'serde DataTypeValidator<'caller, R>, + column_data_type: &'caller DataTypeNode, serde_type: &SerdeType, is_inner: bool, -) -> Option> { +) -> Option> { let data_type = column_data_type.remove_low_cardinality(); match serde_type { SerdeType::Bool @@ -560,7 +560,7 @@ fn validate_impl<'de, 'cursor, R: Row>( } else { let (full_name, full_data_type) = root.get_current_column_name_and_type(); panic!( - "While processing column {full_name} defined as {full_data_type}: attempting to deserialize \ + "While processing column {full_name} defined as {full_data_type}: attempting to (de)serialize \ nested ClickHouse type {data_type} as {serde_type}", ) } @@ -597,7 +597,7 @@ fn validate_impl<'de, 'cursor, R: Row>( panic!("Expected Map for {serde_type} call, but got {data_type}",) } } - SerdeType::Enum => { + SerdeType::Variant => { if let DataTypeNode::Variant(possible_types) = data_type { Some(InnerDataTypeValidator { root, @@ -620,7 +620,7 @@ fn validate_impl<'de, 'cursor, R: Row>( } impl SchemaValidator for () { - type Inner<'de> = (); + type Inner<'serde> = (); #[inline(always)] fn validate(&mut self, _serde_type: SerdeType) {} @@ -661,7 +661,7 @@ pub(crate) enum SerdeType { Str, String, Option, - Enum, + Variant, Bytes(usize), ByteBuf(usize), Tuple(usize), @@ -698,7 +698,7 @@ impl Display for SerdeType { SerdeType::Bytes(len) => write!(f, "&[u8; {len}]"), SerdeType::ByteBuf(_len) => write!(f, "Vec"), SerdeType::Option => write!(f, "Option"), - SerdeType::Enum => write!(f, "enum"), + SerdeType::Variant => write!(f, "enum"), SerdeType::Seq(_len) => write!(f, "Vec"), SerdeType::Tuple(len) => write!(f, "a tuple or sequence with length {len}"), SerdeType::Map(_len) => write!(f, "Map"), diff --git a/src/test/handlers.rs b/src/test/handlers.rs index 925a50e7..0a9fd2b1 100644 --- a/src/test/handlers.rs +++ b/src/test/handlers.rs @@ -7,7 +7,7 @@ use sealed::sealed; use serde::Serialize; use super::{Handler, HandlerFn}; -use crate::{rowbinary, RowOwned, RowRead}; +use crate::{rowbinary, Row, RowOwned, RowRead}; const BUFFER_INITIAL_CAPACITY: usize = 1024; @@ -51,11 +51,11 @@ pub fn exception(code: u8) -> impl Handler { #[track_caller] pub fn provide(rows: impl IntoIterator) -> impl Handler where - T: Serialize, + T: Serialize + Row, { let mut buffer = Vec::with_capacity(BUFFER_INITIAL_CAPACITY); for row in rows { - rowbinary::serialize_into(&mut buffer, &row).expect("failed to serialize"); + rowbinary::serialize_row_binary(&mut buffer, &row).expect("failed to serialize"); } Thunk(Response::new(buffer.into())) } diff --git a/tests/it/chrono.rs b/tests/it/chrono.rs index 69b03e49..be784eea 100644 --- a/tests/it/chrono.rs +++ b/tests/it/chrono.rs @@ -88,7 +88,7 @@ async fn datetime() { dt64ns_opt: Some(dt_ns), }; - let mut insert = client.insert::("test").unwrap(); + let mut insert = client.insert::("test").await.unwrap(); insert.write(&original_row).await.unwrap(); insert.end().await.unwrap(); @@ -146,7 +146,7 @@ async fn date() { .await .unwrap(); - let mut insert = client.insert::("test").unwrap(); + let mut insert = client.insert::("test").await.unwrap(); let dates = generate_dates(1970..2149, 100); for &date in &dates { @@ -199,7 +199,7 @@ async fn date32() { .await .unwrap(); - let mut insert = client.insert::("test").unwrap(); + let mut insert = client.insert::("test").await.unwrap(); let dates = generate_dates(1925..2283, 100); // TODO: 1900..=2299 for newer versions. for &date in &dates { @@ -260,7 +260,7 @@ fn generate_dates(years: impl RangeBounds, count: usize) -> Vec } #[tokio::test] -async fn time_roundtrip() { +async fn time_round_trip() { let client = prepare_database!(); client @@ -293,7 +293,7 @@ async fn time_roundtrip() { t1: Some(duration), }; - let mut insert = client.insert::("test_time").unwrap(); + let mut insert = client.insert::("test_time").await.unwrap(); insert.write(&row).await.unwrap(); insert.end().await.unwrap(); @@ -307,7 +307,7 @@ async fn time_roundtrip() { } #[tokio::test] -async fn time_negative_roundtrip() { +async fn time_negative_round_trip() { let client = prepare_database!(); client @@ -342,6 +342,7 @@ async fn time_negative_roundtrip() { let mut insert = client .insert::("test_time_chrono_negative") + .await .unwrap(); insert.write(&row).await.unwrap(); insert.end().await.unwrap(); @@ -356,7 +357,7 @@ async fn time_negative_roundtrip() { } #[tokio::test] -async fn time64_roundtrip() { +async fn time64_round_trip() { let client = prepare_database!(); client @@ -427,7 +428,7 @@ async fn time64_roundtrip() { t9_opt: Some(dur_ns), }; - let mut insert = client.insert::("test_time64").unwrap(); + let mut insert = client.insert::("test_time64").await.unwrap(); insert.write(&original_row).await.unwrap(); insert.end().await.unwrap(); @@ -441,7 +442,7 @@ async fn time64_roundtrip() { } #[tokio::test] -async fn time64_negative_roundtrip() { +async fn time64_negative_round_trip() { let client = prepare_database!(); client @@ -507,7 +508,10 @@ async fn time64_negative_roundtrip() { t9_opt: Some(dur_ns), }; - let mut insert = client.insert::("test_time64_negative").unwrap(); + let mut insert = client + .insert::("test_time64_negative") + .await + .unwrap(); insert.write(&negative_row).await.unwrap(); insert.end().await.unwrap(); diff --git a/tests/it/compression.rs b/tests/it/compression.rs index ba6c6429..ce7ef1dd 100644 --- a/tests/it/compression.rs +++ b/tests/it/compression.rs @@ -5,7 +5,7 @@ use crate::{create_simple_table, SimpleRow}; async fn check(client: Client) { create_simple_table(&client, "test").await; - let mut insert = client.insert::("test").unwrap(); + let mut insert = client.insert::("test").await.unwrap(); for i in 0..200_000 { insert.write(&SimpleRow::new(i, "foo")).await.unwrap(); } diff --git a/tests/it/cursor_error.rs b/tests/it/cursor_error.rs index 1d0f2001..d58128f0 100644 --- a/tests/it/cursor_error.rs +++ b/tests/it/cursor_error.rs @@ -66,7 +66,7 @@ async fn deferred_lz4() { // Due to compression we need more complex test here: write a lot of big parts. for i in 0..part_count { - let mut insert = client.insert::("test").unwrap(); + let mut insert = client.insert::("test").await.unwrap(); for j in 0..part_size { let row = Row { diff --git a/tests/it/cursor_stats.rs b/tests/it/cursor_stats.rs index d7d3f639..db6157e8 100644 --- a/tests/it/cursor_stats.rs +++ b/tests/it/cursor_stats.rs @@ -5,7 +5,7 @@ use crate::{create_simple_table, SimpleRow}; async fn check(client: Client, expected_ratio: f64) { create_simple_table(&client, "test").await; - let mut insert = client.insert::("test").unwrap(); + let mut insert = client.insert::("test").await.unwrap(); for i in 0..1_000 { insert.write(&SimpleRow::new(i, "foobar")).await.unwrap(); } diff --git a/tests/it/insert.rs b/tests/it/insert.rs index 562f63b0..81da1cce 100644 --- a/tests/it/insert.rs +++ b/tests/it/insert.rs @@ -16,6 +16,7 @@ async fn keeps_client_options() { let mut insert = client .insert::(table_name) + .await .unwrap() .with_option(insert_setting_name, insert_setting_value) .with_option("query_id", &query_id); @@ -68,6 +69,7 @@ async fn overrides_client_options() { let mut insert = client .insert::(table_name) + .await .unwrap() .with_option(setting_name, override_value) .with_option("query_id", &query_id); @@ -114,9 +116,9 @@ async fn empty_insert() { let insert = client .insert::(table_name) + .await .unwrap() .with_option("query_id", &query_id); - insert.end().await.unwrap(); let rows = fetch_rows::(&client, table_name).await; @@ -164,6 +166,7 @@ async fn rename_insert() { let mut insert = client .insert::(table_name) + .await .unwrap() .with_option("query_id", &query_id); @@ -188,7 +191,7 @@ async fn insert_from_cursor() { create_simple_table(&client, "test").await; // Fill the table with initial data. - let mut insert = client.insert::>("test").unwrap(); + let mut insert = client.insert::>("test").await.unwrap(); for (i, data) in ["foo", "bar"].iter().enumerate() { let row = BorrowedRow { id: i as _, data }; insert.write(&row).await.unwrap(); @@ -201,7 +204,7 @@ async fn insert_from_cursor() { .fetch::>() .unwrap(); - let mut insert = client.insert::>("test").unwrap(); + let mut insert = client.insert::>("test").await.unwrap(); while let Some(row) = cursor.next().await.unwrap() { insert.write(&row).await.unwrap(); } diff --git a/tests/it/inserter.rs b/tests/it/inserter.rs index 350dbe1d..89a80d44 100644 --- a/tests/it/inserter.rs +++ b/tests/it/inserter.rs @@ -34,11 +34,11 @@ async fn force_commit() { let client = prepare_database!(); create_table(&client).await; - let mut inserter = client.inserter::("test").unwrap(); + let mut inserter = client.inserter::("test"); let rows = 100; for i in 1..=rows { - inserter.write(&MyRow::new(i)).unwrap(); + inserter.write(&MyRow::new(i)).await.unwrap(); assert_eq!(inserter.commit().await.unwrap(), Quantities::ZERO); if i % 10 == 0 { @@ -63,14 +63,14 @@ async fn limited_by_rows() { let client = prepare_database!(); create_table(&client).await; - let mut inserter = client.inserter::("test").unwrap().with_max_rows(10); + let mut inserter = client.inserter::("test").with_max_rows(10); let rows = 100; for i in (2..=rows).step_by(2) { let row = MyRow::new(i - 1); - inserter.write(&row).unwrap(); + inserter.write(&row).await.unwrap(); let row = MyRow::new(i); - inserter.write(&row).unwrap(); + inserter.write(&row).await.unwrap(); let inserted = inserter.commit().await.unwrap(); let pending = inserter.pending(); @@ -105,16 +105,13 @@ async fn limited_by_bytes() { let client = prepare_database!(); create_table(&client).await; - let mut inserter = client - .inserter::("test") - .unwrap() - .with_max_bytes(100); + let mut inserter = client.inserter::("test").with_max_bytes(100); let rows = 100; let row = MyRow::new("x".repeat(9)); for i in 1..=rows { - inserter.write(&row).unwrap(); + inserter.write(&row).await.unwrap(); let inserted = inserter.commit().await.unwrap(); let pending = inserter.pending(); @@ -152,15 +149,12 @@ async fn limited_by_time() { create_table(&client).await; let period = Duration::from_secs(1); - let mut inserter = client - .inserter::("test") - .unwrap() - .with_period(Some(period)); + let mut inserter = client.inserter::("test").with_period(Some(period)); let rows = 100; for i in 1..=rows { let row = MyRow::new(i); - inserter.write(&row).unwrap(); + inserter.write(&row).await.unwrap(); tokio::time::sleep(period / 10).await; @@ -207,11 +201,10 @@ async fn keeps_client_options() { let mut inserter = client .inserter::(table_name) - .unwrap() .with_option("async_insert", "1") .with_option("query_id", &query_id); - inserter.write(&row).unwrap(); + inserter.write(&row).await.unwrap(); inserter.end().await.unwrap(); flush_query_log(&client).await; @@ -260,11 +253,10 @@ async fn overrides_client_options() { let mut inserter = client .inserter::(table_name) - .unwrap() .with_option("async_insert", override_value) .with_option("query_id", &query_id); - inserter.write(&row).unwrap(); + inserter.write(&row).await.unwrap(); inserter.end().await.unwrap(); flush_query_log(&client).await; diff --git a/tests/it/int128.rs b/tests/it/int128.rs index 1ea0e6b3..c273c3a8 100644 --- a/tests/it/int128.rs +++ b/tests/it/int128.rs @@ -44,7 +44,7 @@ async fn u128() { }, ]; - let mut insert = client.insert::("test").unwrap(); + let mut insert = client.insert::("test").await.unwrap(); for row in &original_rows { insert.write(row).await.unwrap(); } @@ -103,7 +103,7 @@ async fn i128() { }, ]; - let mut insert = client.insert::("test").unwrap(); + let mut insert = client.insert::("test").await.unwrap(); for row in &original_rows { insert.write(row).await.unwrap(); } diff --git a/tests/it/ip.rs b/tests/it/ip.rs index 54b97549..ff9ddd44 100644 --- a/tests/it/ip.rs +++ b/tests/it/ip.rs @@ -40,7 +40,7 @@ async fn smoke() { ipv6_opt: Some(Ipv6Addr::new(0x2001, 0xdb8, 0, 0, 0, 0xafc8, 0x10, 0x1)), }; - let mut insert = client.insert::("test").unwrap(); + let mut insert = client.insert::("test").await.unwrap(); insert.write(&original_row).await.unwrap(); insert.end().await.unwrap(); diff --git a/tests/it/main.rs b/tests/it/main.rs index 0b75f05c..48167acf 100644 --- a/tests/it/main.rs +++ b/tests/it/main.rs @@ -25,7 +25,7 @@ //! the "cloud" environment, it appends the current timestamp to allow //! clean up outdated databases based on its creation time. -use clickhouse::{sql::Identifier, Client, Row, RowOwned, RowRead}; +use clickhouse::{sql::Identifier, Client, Row, RowOwned, RowRead, RowWrite}; use serde::{Deserialize, Serialize}; use std::sync::LazyLock; @@ -68,6 +68,25 @@ macro_rules! assert_panic_on_fetch { }; } +macro_rules! assert_panic_msg { + ($unwinded:ident, $msg_parts:expr) => { + use futures_util::FutureExt; + let result = $unwinded.catch_unwind().await; + assert!( + result.is_err(), + "expected a panic, but got a result instead: {:?}", + result.unwrap() + ); + let panic_msg = *result.unwrap_err().downcast::().unwrap(); + for &msg in $msg_parts { + assert!( + panic_msg.contains(msg), + "panic message:\n{panic_msg}\ndid not contain the expected part:\n{msg}" + ); + } + }; +} + macro_rules! prepare_database { () => { crate::_priv::prepare_database({ @@ -111,7 +130,7 @@ pub(crate) fn get_cloud_url() -> String { format!("https://{hostname}:8443") } -#[derive(Debug, Row, Serialize, Deserialize, PartialEq)] +#[derive(Clone, Debug, Row, Serialize, Deserialize, PartialEq)] struct SimpleRow { id: u64, data: String, @@ -163,6 +182,48 @@ pub(crate) async fn execute_statements(client: &Client, statements: &[&str]) { } } +pub(crate) async fn insert_and_select( + client: &Client, + table_name: &str, + data: impl IntoIterator, +) -> Vec +where + T: RowOwned + RowRead + RowWrite, +{ + let mut insert = client.insert::(table_name).await.unwrap(); + for row in data.into_iter() { + insert.write(&row).await.unwrap(); + } + insert.end().await.unwrap(); + + client + .query("SELECT ?fields FROM ? ORDER BY () ASC") + .bind(Identifier(table_name)) + .fetch_all::() + .await + .unwrap() +} + +pub(crate) mod geo_types { + // See https://clickhouse.com/docs/en/sql-reference/data-types/geo + pub(crate) type Point = (f64, f64); + pub(crate) type Ring = Vec; + pub(crate) type Polygon = Vec; + pub(crate) type MultiPolygon = Vec; + pub(crate) type LineString = Vec; + pub(crate) type MultiLineString = Vec; +} + +pub(crate) mod decimals { + use fixnum::typenum::{U12, U4, U8}; + use fixnum::FixedPoint; + + // See ClickHouse decimal sizes: https://clickhouse.com/docs/en/sql-reference/data-types/decimal + pub(crate) type Decimal32 = FixedPoint; // Decimal(9, 4) = Decimal32(4) + pub(crate) type Decimal64 = FixedPoint; // Decimal(18, 8) = Decimal64(8) + pub(crate) type Decimal128 = FixedPoint; // Decimal(38, 12) = Decimal128(12) +} + mod chrono; mod cloud_jwt; mod compression; @@ -177,7 +238,9 @@ mod ip; mod mock; mod nested; mod query; -mod rbwnat; +mod rbwnat_header; +mod rbwnat_smoke; +mod rbwnat_validation; mod time; mod user_agent; mod uuid; diff --git a/tests/it/mock.rs b/tests/it/mock.rs index 3cc92481..27bc032e 100644 --- a/tests/it/mock.rs +++ b/tests/it/mock.rs @@ -9,7 +9,8 @@ async fn test_provide() { let client = Client::default().with_mock(&mock); let expected = vec![SimpleRow::new(1, "one"), SimpleRow::new(2, "two")]; - mock.add(test::handlers::provide(&expected)); + // FIXME: &expected is not allowed due to new trait bounds + mock.add(test::handlers::provide(expected.clone())); let actual = crate::fetch_rows::(&client, "doesn't matter").await; assert_eq!(actual, expected); diff --git a/tests/it/nested.rs b/tests/it/nested.rs index de80a311..7a78eb58 100644 --- a/tests/it/nested.rs +++ b/tests/it/nested.rs @@ -38,7 +38,7 @@ async fn smoke() { items_count: vec![1, 5], }; - let mut insert = client.insert::("test").unwrap(); + let mut insert = client.insert::("test").await.unwrap(); insert.write(&original_row).await.unwrap(); insert.end().await.unwrap(); diff --git a/tests/it/query.rs b/tests/it/query.rs index f7e1c01d..aac0cf29 100644 --- a/tests/it/query.rs +++ b/tests/it/query.rs @@ -26,7 +26,7 @@ async fn smoke() { .unwrap(); // Write to the table. - let mut insert = client.insert::>("test").unwrap(); + let mut insert = client.insert::>("test").await.unwrap(); for i in 0..1000 { insert.write(&MyRow { no: i, name: "foo" }).await.unwrap(); } @@ -73,7 +73,7 @@ async fn fetch_one_and_optional() { n: String, } - let mut insert = client.insert::("test").unwrap(); + let mut insert = client.insert::("test").await.unwrap(); insert.write(&Row { n: "foo".into() }).await.unwrap(); insert.write(&Row { n: "bar".into() }).await.unwrap(); insert.end().await.unwrap(); @@ -165,7 +165,7 @@ async fn big_borrowed_str() { let long_string = "A".repeat(10000); - let mut insert = client.insert::>("test").unwrap(); + let mut insert = client.insert::>("test").await.unwrap(); insert .write(&MyRow { no: 0, @@ -201,7 +201,7 @@ async fn all_floats() { f: f64, } - let mut insert = client.insert::("test").unwrap(); + let mut insert = client.insert::("test").await.unwrap(); insert.write(&Row { no: 0, f: 42.5 }).await.unwrap(); insert.write(&Row { no: 1, f: 43.5 }).await.unwrap(); insert.end().await.unwrap(); diff --git a/tests/it/rbwnat.rs b/tests/it/rbwnat.rs index 4200b62b..e69de29b 100644 --- a/tests/it/rbwnat.rs +++ b/tests/it/rbwnat.rs @@ -1,1843 +0,0 @@ -use crate::{execute_statements, get_client}; -use clickhouse::sql::Identifier; -use clickhouse_derive::Row; -use clickhouse_types::data_types::{Column, DataTypeNode}; -use clickhouse_types::parse_rbwnat_columns_header; -use fixnum::typenum::{U12, U4, U8}; -use fixnum::FixedPoint; -use serde::{Deserialize, Serialize}; -use serde_repr::{Deserialize_repr, Serialize_repr}; -use std::collections::HashMap; -use std::str::FromStr; - -#[tokio::test] -async fn header_parsing() { - let client = prepare_database!(); - client - .query( - " - CREATE OR REPLACE TABLE visits - ( - CounterID UInt32, - StartDate Date, - Sign Int8, - IsNew UInt8, - VisitID UInt64, - UserID UInt64, - Goals Nested - ( - ID UInt32, - Serial UInt32, - EventTime DateTime, - Price Int64, - OrderID String, - CurrencyID UInt32 - ) - ) ENGINE = MergeTree ORDER BY () - ", - ) - .execute() - .await - .unwrap(); - - let mut cursor = client - .query("SELECT * FROM visits LIMIT 0") - .fetch_bytes("RowBinaryWithNamesAndTypes") - .unwrap(); - - let data = cursor.collect().await.unwrap(); - let result = parse_rbwnat_columns_header(&mut &data[..]).unwrap(); - assert_eq!( - result, - vec![ - Column { - name: "CounterID".to_string(), - data_type: DataTypeNode::UInt32, - }, - Column { - name: "StartDate".to_string(), - data_type: DataTypeNode::Date, - }, - Column { - name: "Sign".to_string(), - data_type: DataTypeNode::Int8, - }, - Column { - name: "IsNew".to_string(), - data_type: DataTypeNode::UInt8, - }, - Column { - name: "VisitID".to_string(), - data_type: DataTypeNode::UInt64, - }, - Column { - name: "UserID".to_string(), - data_type: DataTypeNode::UInt64, - }, - Column { - name: "Goals.ID".to_string(), - data_type: DataTypeNode::Array(Box::new(DataTypeNode::UInt32)), - }, - Column { - name: "Goals.Serial".to_string(), - data_type: DataTypeNode::Array(Box::new(DataTypeNode::UInt32)), - }, - Column { - name: "Goals.EventTime".to_string(), - data_type: DataTypeNode::Array(Box::new(DataTypeNode::DateTime(None))), - }, - Column { - name: "Goals.Price".to_string(), - data_type: DataTypeNode::Array(Box::new(DataTypeNode::Int64)), - }, - Column { - name: "Goals.OrderID".to_string(), - data_type: DataTypeNode::Array(Box::new(DataTypeNode::String)), - }, - Column { - name: "Goals.CurrencyID".to_string(), - data_type: DataTypeNode::Array(Box::new(DataTypeNode::UInt32)), - } - ] - ); -} - -#[tokio::test] -async fn fetch_primitive_row() { - let client = get_client(); - let result = client - .query("SELECT count() FROM (SELECT * FROM system.numbers LIMIT 3)") - .fetch_one::() - .await; - assert_eq!(result.unwrap(), 3); -} - -#[tokio::test] -async fn fetch_primitive_row_schema_mismatch() { - type Data = i32; // expected type is UInt64 - assert_panic_on_fetch!( - &["primitive", "UInt64", "i32"], - "SELECT count() FROM (SELECT * FROM system.numbers LIMIT 3)" - ); -} - -#[tokio::test] -async fn fetch_vector_row() { - let client = get_client(); - let result = client - .query("SELECT [1, 2, 3] :: Array(UInt32)") - .fetch_one::>() - .await; - assert_eq!(result.unwrap(), vec![1, 2, 3]); -} - -#[tokio::test] -async fn fetch_vector_row_schema_mismatch_nested_type() { - type Data = Vec; // expected type for Array(UInt32) is Vec - assert_panic_on_fetch!( - &["vector", "UInt32", "i128"], - "SELECT [1, 2, 3] :: Array(UInt32)" - ); -} - -#[tokio::test] -async fn fetch_tuple_row() { - let client = get_client(); - let result = client - .query("SELECT 42 :: UInt32 AS a, 'foo' :: String AS b") - .fetch_one::<(u32, String)>() - .await; - assert_eq!(result.unwrap(), (42, "foo".to_string())); -} - -#[tokio::test] -async fn fetch_tuple_row_schema_mismatch_first_element() { - type Data = (i128, String); // expected u32 instead of i128 - assert_panic_on_fetch!( - &["tuple", "UInt32", "i128"], - "SELECT 42 :: UInt32 AS a, 'foo' :: String AS b" - ); -} - -#[tokio::test] -async fn fetch_tuple_row_schema_mismatch_second_element() { - type Data = (u32, i64); // expected String instead of i64 - assert_panic_on_fetch!( - &["tuple", "String", "i64"], - "SELECT 42 :: UInt32 AS a, 'foo' :: String AS b" - ); -} - -#[tokio::test] -async fn fetch_tuple_row_schema_mismatch_missing_element() { - type Data = (u32, String); // expected to have the third element as i64 - assert_panic_on_fetch!( - &[ - "database schema has 3 columns", - "tuple definition has 2 fields" - ], - "SELECT 42 :: UInt32 AS a, 'foo' :: String AS b, 144 :: Int64 AS c" - ); -} - -#[tokio::test] -async fn fetch_tuple_row_schema_mismatch_too_many_elements() { - type Data = (u32, String, i128); // i128 should not be there - assert_panic_on_fetch!( - &[ - "database schema has 2 columns", - "tuple definition has 3 fields" - ], - "SELECT 42 :: UInt32 AS a, 'foo' :: String AS b" - ); -} - -#[tokio::test] -async fn fetch_tuple_row_with_struct() { - #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] - struct Data { - a: u32, - b: String, - } - - let client = get_client(); - let result = client - .query("SELECT 42 :: UInt32 AS a, 'foo' :: String AS b, 144 :: UInt64 AS c") - .fetch_one::<(Data, u64)>() - .await; - assert_eq!( - result.unwrap(), - ( - Data { - a: 42, - b: "foo".to_string() - }, - 144 - ) - ); -} - -#[tokio::test] -async fn fetch_tuple_row_with_struct_schema_mismatch() { - #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] - struct _Data { - a: u64, // expected type is u32 - b: String, - } - type Data = (_Data, u64); - assert_panic_on_fetch!( - &["tuple", "UInt32", "u64"], - "SELECT 42 :: UInt32 AS a, 'foo' :: String AS b, 144 :: UInt64 AS c" - ); -} - -#[tokio::test] -async fn fetch_tuple_row_with_struct_schema_mismatch_too_many_struct_fields() { - #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] - struct _Data { - a: u32, - b: String, - c: u64, // this field should not be here - } - type Data = (_Data, u64); - assert_panic_on_fetch!( - &["3 columns", "4 fields"], - "SELECT 42 :: UInt32 AS a, 'foo' :: String AS b, 144 :: UInt64 AS c" - ); -} - -#[tokio::test] -async fn fetch_tuple_row_with_struct_schema_mismatch_too_many_fields() { - #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] - struct _Data { - a: u32, - b: String, - } - type Data = (_Data, u64, u64); // one too many u64 - assert_panic_on_fetch!( - &["3 columns", "4 fields"], - "SELECT 42 :: UInt32 AS a, 'foo' :: String AS b, 144 :: UInt64 AS c" - ); -} - -#[tokio::test] -async fn fetch_tuple_row_with_struct_schema_mismatch_too_few_struct_fields() { - #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] - struct _Data { - a: u32, // the second field is missing now - } - type Data = (_Data, u64); - assert_panic_on_fetch!( - &["3 columns", "2 fields"], - "SELECT 42 :: UInt32 AS a, 'foo' :: String AS b, 144 :: UInt64 AS c" - ); -} - -#[tokio::test] -async fn fetch_tuple_row_with_struct_schema_mismatch_too_few_fields() { - #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] - struct _Data { - a: u32, - b: String, - } - type Data = (_Data, u64); // another u64 is missing here - assert_panic_on_fetch!( - &["4 columns", "3 fields"], - "SELECT 42 :: UInt32 AS a, 'foo' :: String AS b, 144 :: UInt64 AS c, 255 :: UInt64 AS d" - ); -} - -#[tokio::test] -async fn basic_types() { - #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] - struct Data { - uint8_val: u8, - uint16_val: u16, - uint32_val: u32, - uint64_val: u64, - uint128_val: u128, - int8_val: i8, - int16_val: i16, - int32_val: i32, - int64_val: i64, - int128_val: i128, - float32_val: f32, - float64_val: f64, - string_val: String, - } - - let client = get_client(); - let result = client - .query( - " - SELECT - 255 :: UInt8 AS uint8_val, - 65535 :: UInt16 AS uint16_val, - 4294967295 :: UInt32 AS uint32_val, - 18446744073709551615 :: UInt64 AS uint64_val, - 340282366920938463463374607431768211455 :: UInt128 AS uint128_val, - -128 :: Int8 AS int8_val, - -32768 :: Int16 AS int16_val, - -2147483648 :: Int32 AS int32_val, - -9223372036854775808 :: Int64 AS int64_val, - -170141183460469231731687303715884105728 :: Int128 AS int128_val, - 42.0 :: Float32 AS float32_val, - 144.0 :: Float64 AS float64_val, - 'test' :: String AS string_val - ", - ) - .fetch_one::() - .await; - - assert_eq!( - result.unwrap(), - Data { - uint8_val: 255, - uint16_val: 65535, - uint32_val: 4294967295, - uint64_val: 18446744073709551615, - uint128_val: 340282366920938463463374607431768211455, - int8_val: -128, - int16_val: -32768, - int32_val: -2147483648, - int64_val: -9223372036854775808, - int128_val: -170141183460469231731687303715884105728, - float32_val: 42.0, - float64_val: 144.0, - string_val: "test".to_string(), - } - ); -} - -#[tokio::test] -async fn borrowed_data() { - #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] - struct Data<'a> { - str: &'a str, - array: Vec<&'a str>, - tuple: (&'a str, &'a str), - str_opt: Option<&'a str>, - vec_map_str: Vec<(&'a str, &'a str)>, - vec_map_f32: Vec<(&'a str, f32)>, - vec_map_nested: Vec<(&'a str, Vec<(&'a str, &'a str)>)>, - hash_map_str: HashMap<&'a str, &'a str>, - hash_map_f32: HashMap<&'a str, f32>, - hash_map_nested: HashMap<&'a str, HashMap<&'a str, &'a str>>, - } - - let client = get_client(); - let mut cursor = client - .query( - " - SELECT * FROM - ( - SELECT - 'a' :: String AS str, - ['b', 'c'] :: Array(String) AS array, - ('d', 'e') :: Tuple(String, String) AS tuple, - NULL :: Nullable(String) AS str_opt, - map('key1', 'value1', 'key2', 'value2') :: Map(String, String) AS hash_map_str, - map('key3', 100, 'key4', 200) :: Map(String, Float32) AS hash_map_f32, - map('n1', hash_map_str) :: Map(String, Map(String, String)) AS hash_map_nested, - hash_map_str AS vec_map_str, - hash_map_f32 AS vec_map_f32, - hash_map_nested AS vec_map_nested - UNION ALL - SELECT - 'f' :: String AS str, - ['g', 'h'] :: Array(String) AS array, - ('i', 'j') :: Tuple(String, String) AS tuple, - 'k' :: Nullable(String) AS str_opt, - map('key4', 'value4', 'key5', 'value5') :: Map(String, String) AS hash_map_str, - map('key6', 300, 'key7', 400) :: Map(String, Float32) AS hash_map_f32, - map('n2', hash_map_str) :: Map(String, Map(String, String)) AS hash_map_nested, - hash_map_str AS vec_map_str, - hash_map_f32 AS vec_map_f32, - hash_map_nested AS vec_map_nested - ) - ORDER BY str - ", - ) - .fetch::>() - .unwrap(); - - assert_eq!( - cursor.next().await.unwrap().unwrap(), - Data { - str: "a", - array: vec!["b", "c"], - tuple: ("d", "e"), - str_opt: None, - vec_map_str: vec![("key1", "value1"), ("key2", "value2")], - vec_map_f32: vec![("key3", 100.0), ("key4", 200.0)], - vec_map_nested: vec![("n1", vec![("key1", "value1"), ("key2", "value2")])], - hash_map_str: HashMap::from([("key1", "value1"), ("key2", "value2")]), - hash_map_f32: HashMap::from([("key3", 100.0), ("key4", 200.0)]), - hash_map_nested: HashMap::from([( - "n1", - HashMap::from([("key1", "value1"), ("key2", "value2")]), - )]), - } - ); - - assert_eq!( - cursor.next().await.unwrap().unwrap(), - Data { - str: "f", - array: vec!["g", "h"], - tuple: ("i", "j"), - str_opt: Some("k"), - vec_map_str: vec![("key4", "value4"), ("key5", "value5")], - vec_map_f32: vec![("key6", 300.0), ("key7", 400.0)], - vec_map_nested: vec![("n2", vec![("key4", "value4"), ("key5", "value5")])], - hash_map_str: HashMap::from([("key4", "value4"), ("key5", "value5")]), - hash_map_f32: HashMap::from([("key6", 300.0), ("key7", 400.0)]), - hash_map_nested: HashMap::from([( - "n2", - HashMap::from([("key4", "value4"), ("key5", "value5")]), - )]), - }, - ); - - assert!(cursor.next().await.unwrap().is_none()); -} - -#[tokio::test] -async fn several_simple_rows() { - #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] - struct Data { - num: u64, - str: String, - } - - let client = get_client(); - let result = client - .query("SELECT number AS num, toString(number) AS str FROM system.numbers LIMIT 3") - .fetch_all::() - .await; - - assert_eq!( - result.unwrap(), - vec![ - Data { - num: 0, - str: "0".to_string(), - }, - Data { - num: 1, - str: "1".to_string(), - }, - Data { - num: 2, - str: "2".to_string(), - }, - ] - ); -} - -#[tokio::test] -async fn many_numbers() { - #[derive(Row, Deserialize)] - struct Data { - number: u64, - } - - let client = get_client(); - let mut cursor = client - .query("SELECT number FROM system.numbers_mt LIMIT 2000") - .fetch::() - .unwrap(); - - let mut sum = 0; - while let Some(row) = cursor.next().await.unwrap() { - sum += row.number; - } - assert_eq!(sum, (0..2000).sum::()); -} - -#[tokio::test] -async fn blob_string_with_serde_bytes() { - #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] - struct Data { - #[serde(with = "serde_bytes")] - blob: Vec, - } - - let client = get_client(); - let result = client - .query("SELECT 'foo' :: String AS blob") - .fetch_one::() - .await; - - assert_eq!( - result.unwrap(), - Data { - blob: "foo".as_bytes().to_vec(), - } - ); -} - -#[tokio::test] -async fn arrays() { - #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] - struct Data { - id: u16, - one_dim_array: Vec, - two_dim_array: Vec>, - three_dim_array: Vec>>, - description: String, - } - - let client = get_client(); - let result = client - .query( - " - SELECT - 42 :: UInt16 AS id, - [1, 2] :: Array(UInt32) AS one_dim_array, - [[1, 2], [3, 4]] :: Array(Array(Int64)) AS two_dim_array, - [[[1.1, 2.2], [3.3, 4.4]], [], [[5.5, 6.6], [7.7, 8.8]]] :: Array(Array(Array(Float64))) AS three_dim_array, - 'foobar' :: String AS description - ", - ) - .fetch_one::() - .await; - - assert_eq!( - result.unwrap(), - Data { - id: 42, - one_dim_array: vec![1, 2], - two_dim_array: vec![vec![1, 2], vec![3, 4]], - three_dim_array: vec![ - vec![vec![1.1, 2.2], vec![3.3, 4.4]], - vec![], - vec![vec![5.5, 6.6], vec![7.7, 8.8]] - ], - description: "foobar".to_string(), - } - ); -} - -#[tokio::test] -async fn maps() { - #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] - struct Data { - m1: HashMap, - m2: HashMap>, - } - - let client = get_client(); - let result = client - .query( - " - SELECT - map('key1', 'value1', 'key2', 'value2') :: Map(String, String) AS m1, - map(42, map('foo', 100, 'bar', 200), - 144, map('qaz', 300, 'qux', 400)) :: Map(UInt16, Map(String, Int32)) AS m2 - ", - ) - .fetch_one::() - .await; - - assert_eq!( - result.unwrap(), - Data { - m1: vec![ - ("key1".to_string(), "value1".to_string()), - ("key2".to_string(), "value2".to_string()), - ] - .into_iter() - .collect(), - m2: vec![ - ( - 42, - vec![("foo".to_string(), 100), ("bar".to_string(), 200)] - .into_iter() - .collect() - ), - ( - 144, - vec![("qaz".to_string(), 300), ("qux".to_string(), 400)] - .into_iter() - .collect() - ) - ] - .into_iter() - .collect::>>(), - } - ); -} - -#[tokio::test] -async fn map_as_vec_of_tuples() { - #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] - struct Data { - m1: Vec<(i128, String)>, - m2: Vec<(u16, Vec<(String, i32)>)>, - } - - let client = get_client(); - let result = client - .query( - " - SELECT - map(100, 'value1', 200, 'value2') :: Map(Int128, String) AS m1, - map(42, map('foo', 100, 'bar', 200), - 144, map('qaz', 300, 'qux', 400)) :: Map(UInt16, Map(String, Int32)) AS m2 - ", - ) - .fetch_one::() - .await; - - assert_eq!( - result.unwrap(), - Data { - m1: vec![(100, "value1".to_string()), (200, "value2".to_string()),], - m2: vec![ - ( - 42, - vec![("foo".to_string(), 100), ("bar".to_string(), 200)] - .into_iter() - .collect() - ), - ( - 144, - vec![("qaz".to_string(), 300), ("qux".to_string(), 400)] - .into_iter() - .collect() - ) - ], - } - ) -} - -#[tokio::test] -async fn map_as_vec_of_tuples_schema_mismatch() { - #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] - struct Data { - m: Vec<(u16, Vec<(String, i32)>)>, - } - - assert_panic_on_fetch!( - &["Data.m", "Map(Int64, String)", "Int64", "u16"], - "SELECT map(100, 'value1', 200, 'value2') :: Map(Int64, String) AS m" - ); -} - -#[tokio::test] -async fn map_as_vec_of_tuples_schema_mismatch_nested() { - type Inner = Vec<(i32, i64)>; // the value should be i128 instead of i64 - - #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] - struct Data { - m: Vec<(u16, Vec<(String, Inner)>)>, - } - - assert_panic_on_fetch!( - &[ - "Data.m", - "Map(UInt16, Map(String, Map(Int32, Int128)))", - "Int128", - "i64" - ], - "SELECT map(42, map('foo', map(144, 255))) - :: Map(UInt16, Map(String, Map(Int32, Int128))) AS m" - ); -} - -#[tokio::test] -async fn enums() { - #[derive(Debug, PartialEq, Serialize_repr, Deserialize_repr)] - #[repr(i8)] - enum MyEnum8 { - Winter = -128, - Spring = 0, - Summer = 100, - Autumn = 127, - } - - #[derive(Debug, PartialEq, Serialize_repr, Deserialize_repr)] - #[repr(i16)] - enum MyEnum16 { - North = -32768, - East = 0, - South = 144, - West = 32767, - } - - #[derive(Debug, PartialEq, Row, Serialize, Deserialize)] - struct Data { - id: u16, - enum8: MyEnum8, - enum16: MyEnum16, - } - - let table_name = "test_rbwnat_enum"; - - let client = prepare_database!(); - client - .query( - " - CREATE OR REPLACE TABLE ? - ( - id UInt16, - enum8 Enum8 ('Winter' = -128, 'Spring' = 0, 'Summer' = 100, 'Autumn' = 127), - enum16 Enum16('North' = -32768, 'East' = 0, 'South' = 144, 'West' = 32767) - ) ENGINE MergeTree ORDER BY id - ", - ) - .bind(Identifier(table_name)) - .execute() - .await - .unwrap(); - - let expected = vec![ - Data { - id: 1, - enum8: MyEnum8::Spring, - enum16: MyEnum16::East, - }, - Data { - id: 2, - enum8: MyEnum8::Autumn, - enum16: MyEnum16::North, - }, - Data { - id: 3, - enum8: MyEnum8::Winter, - enum16: MyEnum16::South, - }, - Data { - id: 4, - enum8: MyEnum8::Summer, - enum16: MyEnum16::West, - }, - ]; - - let mut insert = client.insert::(table_name).unwrap(); - for row in &expected { - insert.write(row).await.unwrap() - } - insert.end().await.unwrap(); - - let result = client - .query("SELECT * FROM ? ORDER BY id ASC") - .bind(Identifier(table_name)) - .fetch_all::() - .await - .unwrap(); - - assert_eq!(result, expected); -} - -#[tokio::test] -async fn nullable() { - #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] - struct Data { - a: u32, - b: Option, - } - - let client = get_client(); - let result = client - .query( - " - SELECT * FROM ( - SELECT 1 :: UInt32 AS a, 2 :: Nullable(Int64) AS b - UNION ALL - SELECT 3 :: UInt32 AS a, NULL :: Nullable(Int64) AS b - UNION ALL - SELECT 4 :: UInt32 AS a, 5 :: Nullable(Int64) AS b - ) - ORDER BY a ASC - ", - ) - .fetch_all::() - .await; - - assert_eq!( - result.unwrap(), - vec![ - Data { a: 1, b: Some(2) }, - Data { a: 3, b: None }, - Data { a: 4, b: Some(5) }, - ] - ); -} - -#[tokio::test] -async fn invalid_nullable() { - #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] - struct Data { - n: Option, - } - assert_panic_on_fetch!( - &["Data.n", "Array(UInt32)", "Option"], - "SELECT array(42) :: Array(UInt32) AS n" - ); -} - -#[tokio::test] -async fn low_cardinality() { - #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] - struct Data { - a: u32, - b: Option, - } - - let client = get_client(); - let result = client - .query( - " - SELECT * FROM ( - SELECT 1 :: LowCardinality(UInt32) AS a, 2 :: LowCardinality(Nullable(Int64)) AS b - UNION ALL - SELECT 3 :: LowCardinality(UInt32) AS a, NULL :: LowCardinality(Nullable(Int64)) AS b - UNION ALL - SELECT 4 :: LowCardinality(UInt32) AS a, 5 :: LowCardinality(Nullable(Int64)) AS b - ) - ORDER BY a ASC - ", - ) - .with_option("allow_suspicious_low_cardinality_types", "1") - .fetch_all::() - .await; - - assert_eq!( - result.unwrap(), - vec![ - Data { a: 1, b: Some(2) }, - Data { a: 3, b: None }, - Data { a: 4, b: Some(5) }, - ] - ); -} - -#[tokio::test] -async fn invalid_low_cardinality() { - #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] - struct Data { - a: u32, - } - let client = get_client().with_option("allow_suspicious_low_cardinality_types", "1"); - assert_panic_on_fetch_with_client!( - client, - &["Data.a", "LowCardinality(Int32)", "u32"], - "SELECT 144 :: LowCardinality(Int32) AS a" - ); -} - -#[tokio::test] -async fn invalid_nullable_low_cardinality() { - #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] - struct Data { - a: Option, - } - let client = get_client().with_option("allow_suspicious_low_cardinality_types", "1"); - assert_panic_on_fetch_with_client!( - client, - &["Data.a", "LowCardinality(Nullable(Int32))", "u32"], - "SELECT 144 :: LowCardinality(Nullable(Int32)) AS a" - ); -} - -#[tokio::test] -#[cfg(feature = "time")] -async fn invalid_serde_with() { - #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] - struct Data { - #[serde(with = "clickhouse::serde::time::datetime64::millis")] - n1: time::OffsetDateTime, // underlying is still Int64; should not compose it from two (U)Int32 - } - assert_panic_on_fetch!(&["Data.n1", "UInt32", "i64"], "SELECT 42 :: UInt32 AS n1"); -} - -#[tokio::test] -async fn too_many_struct_fields() { - #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] - struct Data { - a: u32, - b: u32, - c: u32, - } - assert_panic_on_fetch!( - &["2 columns", "3 fields"], - "SELECT 42 :: UInt32 AS a, 144 :: UInt32 AS b" - ); -} - -#[tokio::test] -async fn serde_skip_deserializing() { - #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] - struct Data { - a: u32, - #[serde(skip_deserializing)] - b: u32, - c: u32, - } - - let client = get_client(); - let result = client - .query("SELECT 42 :: UInt32 AS a, 144 :: UInt32 AS c") - .fetch_one::() - .await; - - assert_eq!( - result.unwrap(), - Data { - a: 42, - b: 0, // default value - c: 144, - } - ); -} - -#[tokio::test] -#[cfg(feature = "time")] -async fn date_and_time() { - use time::format_description::well_known::Iso8601; - use time::Month::{February, January}; - use time::OffsetDateTime; - - #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] - struct Data { - #[serde(with = "clickhouse::serde::time::date")] - date: time::Date, - #[serde(with = "clickhouse::serde::time::date32")] - date32: time::Date, - #[serde(with = "clickhouse::serde::time::datetime")] - date_time: OffsetDateTime, - #[serde(with = "clickhouse::serde::time::datetime64::secs")] - date_time64_0: OffsetDateTime, - #[serde(with = "clickhouse::serde::time::datetime64::millis")] - date_time64_3: OffsetDateTime, - #[serde(with = "clickhouse::serde::time::datetime64::micros")] - date_time64_6: OffsetDateTime, - #[serde(with = "clickhouse::serde::time::datetime64::nanos")] - date_time64_9: OffsetDateTime, - } - - let client = get_client(); - let result = client - .query( - " - SELECT - '2023-01-01' :: Date AS date, - '2023-02-02' :: Date32 AS date32, - '2023-01-03 12:00:00' :: DateTime AS date_time, - '2023-01-04 13:00:00' :: DateTime64(0) AS date_time64_0, - '2023-01-05 14:00:00.123' :: DateTime64(3) AS date_time64_3, - '2023-01-06 15:00:00.123456' :: DateTime64(6) AS date_time64_6, - '2023-01-07 16:00:00.123456789' :: DateTime64(9) AS date_time64_9 - ", - ) - .fetch_one::() - .await; - - assert_eq!( - result.unwrap(), - Data { - date: time::Date::from_calendar_date(2023, January, 1).unwrap(), - date32: time::Date::from_calendar_date(2023, February, 2).unwrap(), - date_time: OffsetDateTime::parse("2023-01-03T12:00:00Z", &Iso8601::DEFAULT).unwrap(), - date_time64_0: OffsetDateTime::parse("2023-01-04T13:00:00Z", &Iso8601::DEFAULT) - .unwrap(), - date_time64_3: OffsetDateTime::parse("2023-01-05T14:00:00.123Z", &Iso8601::DEFAULT) - .unwrap(), - date_time64_6: OffsetDateTime::parse("2023-01-06T15:00:00.123456Z", &Iso8601::DEFAULT) - .unwrap(), - date_time64_9: OffsetDateTime::parse( - "2023-01-07T16:00:00.123456789Z", - &Iso8601::DEFAULT - ) - .unwrap(), - } - ); -} - -#[tokio::test] -async fn interval() { - #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] - struct Data { - id: u32, - interval_nanosecond: i64, - interval_microsecond: i64, - interval_millisecond: i64, - interval_second: i64, - interval_minute: i64, - interval_hour: i64, - interval_day: i64, - interval_week: i64, - interval_month: i64, - interval_quarter: i64, - interval_year: i64, - } - - let client = get_client(); - let mut cursor = client - .query( - " - SELECT * FROM ( - SELECT - 0 :: UInt32 AS id, - toIntervalNanosecond (0) AS interval_nanosecond, - toIntervalMicrosecond (0) AS interval_microsecond, - toIntervalMillisecond (0) AS interval_millisecond, - toIntervalSecond (0) AS interval_second, - toIntervalMinute (0) AS interval_minute, - toIntervalHour (0) AS interval_hour, - toIntervalDay (0) AS interval_day, - toIntervalWeek (0) AS interval_week, - toIntervalMonth (0) AS interval_month, - toIntervalQuarter (0) AS interval_quarter, - toIntervalYear (0) AS interval_year - UNION ALL - SELECT - 1 :: UInt32 AS id, - toIntervalNanosecond (-9223372036854775808) AS interval_nanosecond, - toIntervalMicrosecond (-9223372036854775808) AS interval_microsecond, - toIntervalMillisecond (-9223372036854775808) AS interval_millisecond, - toIntervalSecond (-9223372036854775808) AS interval_second, - toIntervalMinute (-9223372036854775808) AS interval_minute, - toIntervalHour (-9223372036854775808) AS interval_hour, - toIntervalDay (-9223372036854775808) AS interval_day, - toIntervalWeek (-9223372036854775808) AS interval_week, - toIntervalMonth (-9223372036854775808) AS interval_month, - toIntervalQuarter (-9223372036854775808) AS interval_quarter, - toIntervalYear (-9223372036854775808) AS interval_year - UNION ALL - SELECT - 2 :: UInt32 AS id, - toIntervalNanosecond (9223372036854775807) AS interval_nanosecond, - toIntervalMicrosecond (9223372036854775807) AS interval_microsecond, - toIntervalMillisecond (9223372036854775807) AS interval_millisecond, - toIntervalSecond (9223372036854775807) AS interval_second, - toIntervalMinute (9223372036854775807) AS interval_minute, - toIntervalHour (9223372036854775807) AS interval_hour, - toIntervalDay (9223372036854775807) AS interval_day, - toIntervalWeek (9223372036854775807) AS interval_week, - toIntervalMonth (9223372036854775807) AS interval_month, - toIntervalQuarter (9223372036854775807) AS interval_quarter, - toIntervalYear (9223372036854775807) AS interval_year - ) ORDER BY id ASC - ", - ) - .fetch::() - .unwrap(); - - assert_eq!( - cursor.next().await.unwrap().unwrap(), - Data { - id: 0, - interval_nanosecond: 0, - interval_microsecond: 0, - interval_millisecond: 0, - interval_second: 0, - interval_minute: 0, - interval_hour: 0, - interval_day: 0, - interval_week: 0, - interval_month: 0, - interval_quarter: 0, - interval_year: 0, - } - ); - - assert_eq!( - cursor.next().await.unwrap().unwrap(), - Data { - id: 1, - interval_nanosecond: i64::MIN, - interval_microsecond: i64::MIN, - interval_millisecond: i64::MIN, - interval_second: i64::MIN, - interval_minute: i64::MIN, - interval_hour: i64::MIN, - interval_day: i64::MIN, - interval_week: i64::MIN, - interval_month: i64::MIN, - interval_quarter: i64::MIN, - interval_year: i64::MIN, - } - ); - - assert_eq!( - cursor.next().await.unwrap().unwrap(), - Data { - id: 2, - interval_nanosecond: i64::MAX, - interval_microsecond: i64::MAX, - interval_millisecond: i64::MAX, - interval_second: i64::MAX, - interval_minute: i64::MAX, - interval_hour: i64::MAX, - interval_day: i64::MAX, - interval_week: i64::MAX, - interval_month: i64::MAX, - interval_quarter: i64::MAX, - interval_year: i64::MAX, - } - ); -} - -#[tokio::test] -#[cfg(feature = "uuid")] -async fn uuid() { - #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] - struct Data { - id: u16, - #[serde(with = "clickhouse::serde::uuid")] - uuid: uuid::Uuid, - } - - let client = get_client(); - let result = client - .query( - " - SELECT - 42 :: UInt16 AS id, - '550e8400-e29b-41d4-a716-446655440000' :: UUID AS uuid - ", - ) - .fetch_one::() - .await; - - assert_eq!( - result.unwrap(), - Data { - id: 42, - uuid: uuid::Uuid::from_str("550e8400-e29b-41d4-a716-446655440000").unwrap(), - } - ); -} - -#[tokio::test] -async fn ipv4_ipv6() { - #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] - struct Data { - id: u16, - #[serde(with = "clickhouse::serde::ipv4")] - ipv4: std::net::Ipv4Addr, - ipv6: std::net::Ipv6Addr, - } - - let client = get_client(); - let result = client - .query( - " - SELECT - 42 :: UInt16 AS id, - '192.168.0.1' :: IPv4 AS ipv4, - '2001:db8:3333:4444:5555:6666:7777:8888' :: IPv6 AS ipv6 - ", - ) - .fetch_all::() - .await; - - assert_eq!( - result.unwrap(), - vec![Data { - id: 42, - ipv4: std::net::Ipv4Addr::new(192, 168, 0, 1), - ipv6: std::net::Ipv6Addr::from_str("2001:db8:3333:4444:5555:6666:7777:8888").unwrap(), - }] - ) -} - -#[tokio::test] -async fn fixed_str() { - #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] - struct Data { - a: [u8; 4], - b: [u8; 3], - } - - let client = get_client(); - let result = client - .query("SELECT '1234' :: FixedString(4) AS a, '777' :: FixedString(3) AS b") - .fetch_one::() - .await; - - let data = result.unwrap(); - assert_eq!(String::from_utf8_lossy(&data.a), "1234"); - assert_eq!(String::from_utf8_lossy(&data.b), "777"); -} - -#[tokio::test] -async fn fixed_str_too_long() { - #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] - struct Data { - a: [u8; 4], - b: [u8; 3], - } - assert_panic_on_fetch!( - &["Data.a", "FixedString(5)", "with length 4"], - "SELECT '12345' :: FixedString(5) AS a, '777' :: FixedString(3) AS b" - ); -} - -#[tokio::test] -async fn tuple() { - #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] - struct Data { - a: (u32, String), - b: (i128, HashMap), - } - - let client = get_client(); - let result = client - .query( - " - SELECT - (42, 'foo') :: Tuple(UInt32, String) AS a, - (144, map(255, 'bar')) :: Tuple(Int128, Map(UInt16, String)) AS b - ", - ) - .fetch_one::() - .await; - - assert_eq!( - result.unwrap(), - Data { - a: (42, "foo".to_string()), - b: (144, vec![(255, "bar".to_string())].into_iter().collect()), - } - ); -} - -#[tokio::test] -async fn tuple_invalid_definition() { - #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] - struct Data { - a: (u32, String), - b: (i128, HashMap), - } - // Map key is UInt64 instead of UInt16 requested in the struct - assert_panic_on_fetch!( - &[ - "Data.b", - "Tuple(Int128, Map(UInt64, String))", - "UInt64 as u16" - ], - " - SELECT - (42, 'foo') :: Tuple(UInt32, String) AS a, - (144, map(255, 'bar')) :: Tuple(Int128, Map(UInt64, String)) AS b - " - ); -} - -#[tokio::test] -async fn tuple_too_many_elements_in_the_schema() { - #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] - struct Data { - a: (u32, String), - b: (i128, HashMap), - } - // too many elements in the db type definition - assert_panic_on_fetch!( - &[ - "Data.a", - "Tuple(UInt32, String, Bool)", - "remaining elements: Bool" - ], - " - SELECT - (42, 'foo', true) :: Tuple(UInt32, String, Bool) AS a, - (144, map(255, 'bar')) :: Tuple(Int128, Map(UInt16, String)) AS b - " - ); -} - -#[tokio::test] -async fn tuple_too_many_elements_in_the_struct() { - #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] - struct Data { - a: (u32, String, bool), - b: (i128, HashMap), - } - // too many elements in the struct enum - assert_panic_on_fetch!( - &["Data.a", "Tuple(UInt32, String)", "deserialize bool"], - " - SELECT - (42, 'foo') :: Tuple(UInt32, String) AS a, - (144, map(255, 'bar')) :: Tuple(Int128, Map(UInt16, String)) AS b - " - ); -} - -#[tokio::test] -async fn deeply_nested_validation_incorrect_fixed_string() { - #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] - struct Data { - id: u32, - col: Vec>>>, - } - // Struct has FixedString(2) instead of FixedString(1) - assert_panic_on_fetch!( - &["Data.col", "FixedString(1)", "with length 2"], - " - SELECT - 42 :: UInt32 AS id, - array(array(map(42, array('1', '2')))) :: Array(Array(Map(UInt32, Array(FixedString(1))))) AS col - " - ); -} - -#[tokio::test] -async fn geo() { - #[derive(Clone, Debug, PartialEq)] - #[derive(Row, serde::Serialize, serde::Deserialize)] - struct Data { - id: u32, - point: Point, - ring: Ring, - polygon: Polygon, - multi_polygon: MultiPolygon, - line_string: LineString, - multi_line_string: MultiLineString, - } - - let client = get_client(); - let result = client - .query( - " - SELECT - 42 :: UInt32 AS id, - (1.0, 2.0) :: Point AS point, - [(3.0, 4.0), (5.0, 6.0)] :: Ring AS ring, - [[(7.0, 8.0), (9.0, 10.0)], [(11.0, 12.0)]] :: Polygon AS polygon, - [[[(13.0, 14.0), (15.0, 16.0)], [(17.0, 18.0)]]] :: MultiPolygon AS multi_polygon, - [(19.0, 20.0), (21.0, 22.0)] :: LineString AS line_string, - [[(23.0, 24.0), (25.0, 26.0)], [(27.0, 28.0)]] :: MultiLineString AS multi_line_string - ", - ) - .fetch_one::() - .await; - - assert_eq!( - result.unwrap(), - Data { - id: 42, - point: (1.0, 2.0), - ring: vec![(3.0, 4.0), (5.0, 6.0)], - polygon: vec![vec![(7.0, 8.0), (9.0, 10.0)], vec![(11.0, 12.0)]], - multi_polygon: vec![vec![vec![(13.0, 14.0), (15.0, 16.0)], vec![(17.0, 18.0)]]], - line_string: vec![(19.0, 20.0), (21.0, 22.0)], - multi_line_string: vec![vec![(23.0, 24.0), (25.0, 26.0)], vec![(27.0, 28.0)]], - } - ); -} - -// TODO: there are two panics; one about schema mismatch, -// another about not all Tuple elements being deserialized -// not easy to assert, same applies to the other Geo types -#[ignore] -#[tokio::test] -async fn geo_invalid_point() { - #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] - struct Data { - id: u32, - pt: (i32, i32), - } - assert_panic_on_fetch!( - &["Data.pt", "Point", "Float64 as i32"], - " - SELECT - 42 :: UInt32 AS id, - (1.0, 2.0) :: Point AS pt - " - ); -} - -#[tokio::test] -/// See https://github.com/ClickHouse/clickhouse-rs/issues/100 -async fn issue_100() { - { - #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] - struct Data { - n: i8, - } - assert_panic_on_fetch!( - &["Data.n", "Nullable(Bool)", "i8"], - "SELECT NULL :: Nullable(Bool) AS n" - ); - } - - { - #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] - struct Data { - n: u8, - } - assert_panic_on_fetch!( - &["Data.n", "Nullable(Bool)", "u8"], - "SELECT NULL :: Nullable(Bool) AS n" - ); - } - - { - #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] - struct Data { - n: bool, - } - assert_panic_on_fetch!( - &["Data.n", "Nullable(Bool)", "bool"], - "SELECT NULL :: Nullable(Bool) AS n" - ); - } -} - -// TODO: unignore after insert implementation uses RBWNAT, too -#[ignore] -#[tokio::test] -/// See https://github.com/ClickHouse/clickhouse-rs/issues/109#issuecomment-2243197221 -async fn issue_109_1() { - #[derive(Debug, Serialize, Deserialize, Row)] - struct Data { - #[serde(skip_deserializing)] - en_id: String, - journey: u32, - drone_id: String, - call_sign: String, - } - let client = prepare_database!(); - execute_statements( - &client, - &[ - " - CREATE TABLE issue_109 ( - drone_id String, - call_sign String, - journey UInt32, - en_id String, - ) - ENGINE = MergeTree - ORDER BY (drone_id) - ", - " - INSERT INTO issue_109 VALUES - ('drone_1', 'call_sign_1', 1, 'en_id_1'), - ('drone_2', 'call_sign_2', 2, 'en_id_2'), - ('drone_3', 'call_sign_3', 3, 'en_id_3') - ", - ], - ) - .await; - - let data = client - .query("SELECT journey, drone_id, call_sign FROM issue_109") - .fetch_all::() - .await - .unwrap(); - let mut insert = client.insert::("issue_109").unwrap(); - for (id, elem) in data.iter().enumerate() { - let elem = Data { - en_id: format!("ABC-{id}"), - journey: elem.journey, - drone_id: elem.drone_id.clone(), - call_sign: elem.call_sign.clone(), - }; - insert.write(&elem).await.unwrap(); - } - insert.end().await.unwrap(); -} - -#[tokio::test] -async fn issue_112() { - #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] - struct Data { - a: bool, - b: bool, - } - - assert_panic_on_fetch!( - &["Data.a", "Nullable(Bool)", "bool"], - "WITH (SELECT true) AS a, (SELECT true) AS b SELECT ?fields" - ); -} - -#[tokio::test] -/// See https://github.com/ClickHouse/clickhouse-rs/issues/113 -async fn issue_113() { - #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] - struct Data { - a: u64, - b: f64, - c: f64, - } - let client = prepare_database!(); - execute_statements(&client, &[ - " - CREATE TABLE issue_113_1( - id UInt32 - ) - ENGINE MergeTree - ORDER BY id - ", - " - CREATE TABLE issue_113_2( - id UInt32, - pos Float64 - ) - ENGINE MergeTree - ORDER BY id - ", - "INSERT INTO issue_113_1 VALUES (1), (2), (3), (4), (5), (6), (7), (8), (9), (10)", - "INSERT INTO issue_113_2 VALUES (1, 100.5), (2, 200.2), (3, 300.3), (4, 444.4), (5, 555.5)", - ]).await; - - // Struct should have had Option instead of f64 - assert_panic_on_fetch_with_client!( - client, - &["Data.b", "Nullable(Float64)", "f64"], - " - SELECT - COUNT(*) AS a, - (COUNT(*) / (SELECT COUNT(*) FROM issue_113_1)) * 100.0 AS b, - AVG(pos) AS c - FROM issue_113_2 - " - ); -} - -#[tokio::test] -#[cfg(feature = "time")] -/// See https://github.com/ClickHouse/clickhouse-rs/issues/114 -async fn issue_114() { - #[derive(Row, Deserialize, Debug, PartialEq)] - struct Data { - #[serde(with = "clickhouse::serde::time::date")] - date: time::Date, - arr: Vec>, - } - - let client = get_client(); - let result = client - .query( - " - SELECT - '2023-05-01' :: Date AS date, - array(map('k1', 'v1'), map('k2', 'v2')) :: Array(Map(String, String)) AS arr - ", - ) - .fetch_one::() - .await; - - assert_eq!( - result.unwrap(), - Data { - date: time::Date::from_calendar_date(2023, time::Month::May, 1).unwrap(), - arr: vec![ - HashMap::from([("k1".to_owned(), "v1".to_owned())]), - HashMap::from([("k2".to_owned(), "v2".to_owned())]), - ], - } - ); -} - -#[tokio::test] -#[cfg(feature = "time")] -/// See https://github.com/ClickHouse/clickhouse-rs/issues/173 -async fn issue_173() { - #[derive(Debug, Serialize, Deserialize, Row)] - struct Data { - log_id: String, - #[serde(with = "clickhouse::serde::time::datetime")] - ts: time::OffsetDateTime, - } - - let client = prepare_database!().with_option("date_time_input_format", "best_effort"); - - execute_statements(&client, &[ - " - CREATE OR REPLACE TABLE logs ( - log_id String, - timestamp DateTime('Europe/Berlin') - ) - ENGINE = MergeTree() - PRIMARY KEY (log_id, timestamp) - ", - "INSERT INTO logs VALUES ('56cde52f-5f34-45e0-9f08-79d6f582e913', '2024-11-05T11:52:52+01:00')", - "INSERT INTO logs VALUES ('0e967129-6271-44f2-967b-0c8d11a60fdc', '2024-11-05T11:59:21+01:00')", - ]).await; - - // panics as we fetch `ts` two times: one from `?fields` macro, and the second time explicitly - // the resulting dataset will, in fact, contain 3 columns instead of 2: - assert_panic_on_fetch_with_client!( - client, - &["3 columns", "2 fields"], - "SELECT ?fields, toUnixTimestamp(timestamp) AS ts FROM logs ORDER by ts DESC" - ); -} - -#[tokio::test] -/// See https://github.com/ClickHouse/clickhouse-rs/issues/185 -async fn issue_185() { - #[derive(Row, Deserialize, Debug, PartialEq)] - struct Data { - pk: u32, - decimal_col: Option, - } - - let client = prepare_database!(); - execute_statements( - &client, - &[ - " - CREATE TABLE issue_185( - pk UInt32, - decimal_col Nullable(Decimal(10, 4))) - ENGINE MergeTree - ORDER BY pk - ", - "INSERT INTO issue_185 VALUES (1, 1.1), (2, 2.2), (3, 3.3)", - ], - ) - .await; - - assert_panic_on_fetch_with_client!( - client, - &["Data.decimal_col", "Decimal(10, 4)", "String"], - "SELECT ?fields FROM issue_185" - ); -} - -#[tokio::test] -#[cfg(feature = "chrono")] -async fn issue_218() { - #[derive(Row, Serialize, Deserialize, Debug)] - struct Data { - max_time: chrono::DateTime, - } - - let client = prepare_database!(); - execute_statements( - &client, - &[" - CREATE TABLE IF NOT EXISTS issue_218 ( - my_time DateTime64(3, 'UTC') CODEC(Delta, ZSTD), - ) ENGINE = MergeTree - ORDER BY my_time - "], - ) - .await; - - // FIXME: It is not a super clear panic as it hints about `&str`, - // and not about the missing attribute for `chrono::DateTime`. - // Still better than a `premature end of input` error, though. - assert_panic_on_fetch_with_client!( - client, - &["Data.max_time", "DateTime64(3, 'UTC')", "&str"], - "SELECT max(my_time) AS max_time FROM issue_218" - ); -} - -#[tokio::test] -async fn variant_wrong_definition() { - #[derive(Debug, Deserialize, PartialEq)] - enum MyVariant { - Str(String), - U32(u32), - } - - #[derive(Debug, Row, Deserialize, PartialEq)] - struct Data { - id: u8, - var: MyVariant, - } - - let client = get_client().with_option("allow_experimental_variant_type", "1"); - - assert_panic_on_fetch_with_client!( - client, - &["Data.var", "Variant(String, UInt16)", "u32"], - " - SELECT * FROM ( - SELECT 0 :: UInt8 AS id, 'foo' :: Variant(String, UInt16) AS var - UNION ALL - SELECT 1 :: UInt8 AS id, 144 :: Variant(String, UInt16) AS var - ) ORDER BY id ASC - " - ); -} - -#[tokio::test] -async fn decimals() { - #[derive(Row, Deserialize, Debug, PartialEq)] - struct Data { - decimal32_9_4: Decimal32, - decimal64_18_8: Decimal64, - decimal128_38_12: Decimal128, - } - - let client = get_client(); - let result = client - .query( - " - SELECT - 42.1234 :: Decimal32(4) AS decimal32_9_4, - 144.56789012 :: Decimal64(8) AS decimal64_18_8, - -17014118346046923173168730.37158841057 :: Decimal128(12) AS decimal128_38_12 - ", - ) - .fetch_one::() - .await; - - assert_eq!( - result.unwrap(), - Data { - decimal32_9_4: Decimal32::from_str("42.1234").unwrap(), - decimal64_18_8: Decimal64::from_str("144.56789012").unwrap(), - decimal128_38_12: Decimal128::from_str("-17014118346046923173168730.37158841057") - .unwrap(), - } - ); -} - -#[tokio::test] -async fn decimal32_wrong_size() { - #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] - struct Data { - decimal32: i16, - } - - assert_panic_on_fetch!( - &["Data.decimal32", "Decimal(9, 4)", "i16"], - "SELECT 42 :: Decimal32(4) AS decimal32" - ); -} - -#[tokio::test] -async fn decimal64_wrong_size() { - #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] - struct Data { - decimal64: i32, - } - - assert_panic_on_fetch!( - &["Data.decimal64", "Decimal(18, 8)", "i32"], - "SELECT 144 :: Decimal64(8) AS decimal64" - ); -} - -#[tokio::test] -async fn decimal128_wrong_size() { - #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] - struct Data { - decimal128: i64, - } - - assert_panic_on_fetch!( - &["Data.decimal128", "Decimal(38, 12)", "i64"], - "SELECT -17014118346046923173168730.37158841057 :: Decimal128(12) AS decimal128" - ); -} - -#[tokio::test] -async fn different_struct_field_order_same_types() { - #[derive(Debug, Row, Deserialize, PartialEq)] - struct Data { - c: String, - a: String, - } - - let client = get_client(); - let result = client - .query("SELECT 'foo' AS a, 'bar' :: String AS c") - .fetch_one::() - .await; - - assert_eq!( - result.unwrap(), - Data { - c: "bar".to_string(), - a: "foo".to_string(), - } - ); -} - -#[tokio::test] -async fn different_struct_field_order_different_types() { - #[derive(Debug, Row, Deserialize, PartialEq)] - struct Data { - b: u32, - a: String, - c: Vec, - } - - let client = get_client(); - let result = client - .query( - " - SELECT array(true, false, true) AS c, - 42 :: UInt32 AS b, - 'foo' AS a - ", - ) - .fetch_one::() - .await; - - assert_eq!( - result.unwrap(), - Data { - c: vec![true, false, true], - b: 42, - a: "foo".to_string(), - } - ); -} - -// See https://clickhouse.com/docs/en/sql-reference/data-types/geo -type Point = (f64, f64); -type Ring = Vec; -type Polygon = Vec; -type MultiPolygon = Vec; -type LineString = Vec; -type MultiLineString = Vec; - -// See ClickHouse decimal sizes: https://clickhouse.com/docs/en/sql-reference/data-types/decimal -type Decimal32 = FixedPoint; // Decimal(9, 4) = Decimal32(4) -type Decimal64 = FixedPoint; // Decimal(18, 8) = Decimal64(8) -type Decimal128 = FixedPoint; // Decimal(38, 12) = Decimal128(12) diff --git a/tests/it/rbwnat_header.rs b/tests/it/rbwnat_header.rs new file mode 100644 index 00000000..b05408ba --- /dev/null +++ b/tests/it/rbwnat_header.rs @@ -0,0 +1,93 @@ +use clickhouse_types::{parse_rbwnat_columns_header, Column, DataTypeNode}; + +#[tokio::test] +async fn header_parsing() { + let client = prepare_database!(); + client + .query( + " + CREATE OR REPLACE TABLE visits + ( + CounterID UInt32, + StartDate Date, + Sign Int8, + IsNew UInt8, + VisitID UInt64, + UserID UInt64, + Goals Nested + ( + ID UInt32, + Serial UInt32, + EventTime DateTime, + Price Int64, + OrderID String, + CurrencyID UInt32 + ) + ) ENGINE = MergeTree ORDER BY () + ", + ) + .execute() + .await + .unwrap(); + + let mut cursor = client + .query("SELECT * FROM visits LIMIT 0") + .fetch_bytes("RowBinaryWithNamesAndTypes") + .unwrap(); + + let data = cursor.collect().await.unwrap(); + let result = parse_rbwnat_columns_header(&mut &data[..]).unwrap(); + assert_eq!( + result, + vec![ + Column { + name: "CounterID".to_string(), + data_type: DataTypeNode::UInt32, + }, + Column { + name: "StartDate".to_string(), + data_type: DataTypeNode::Date, + }, + Column { + name: "Sign".to_string(), + data_type: DataTypeNode::Int8, + }, + Column { + name: "IsNew".to_string(), + data_type: DataTypeNode::UInt8, + }, + Column { + name: "VisitID".to_string(), + data_type: DataTypeNode::UInt64, + }, + Column { + name: "UserID".to_string(), + data_type: DataTypeNode::UInt64, + }, + Column { + name: "Goals.ID".to_string(), + data_type: DataTypeNode::Array(Box::new(DataTypeNode::UInt32)), + }, + Column { + name: "Goals.Serial".to_string(), + data_type: DataTypeNode::Array(Box::new(DataTypeNode::UInt32)), + }, + Column { + name: "Goals.EventTime".to_string(), + data_type: DataTypeNode::Array(Box::new(DataTypeNode::DateTime(None))), + }, + Column { + name: "Goals.Price".to_string(), + data_type: DataTypeNode::Array(Box::new(DataTypeNode::Int64)), + }, + Column { + name: "Goals.OrderID".to_string(), + data_type: DataTypeNode::Array(Box::new(DataTypeNode::String)), + }, + Column { + name: "Goals.CurrencyID".to_string(), + data_type: DataTypeNode::Array(Box::new(DataTypeNode::UInt32)), + } + ] + ); +} diff --git a/tests/it/rbwnat_smoke.rs b/tests/it/rbwnat_smoke.rs new file mode 100644 index 00000000..0868efa1 --- /dev/null +++ b/tests/it/rbwnat_smoke.rs @@ -0,0 +1,1414 @@ +use crate::decimals::*; +use crate::geo_types::{LineString, MultiLineString, MultiPolygon, Point, Polygon, Ring}; +use crate::{create_simple_table, get_client, insert_and_select, SimpleRow}; +use clickhouse::sql::Identifier; +use clickhouse::Row; +use fxhash::FxHashMap; +use indexmap::IndexMap; +use linked_hash_map::LinkedHashMap; +use serde::{Deserialize, Serialize}; +use serde_repr::{Deserialize_repr, Serialize_repr}; +use std::collections::HashMap; +use std::str::FromStr; + +#[tokio::test] +async fn basic_types() { + #[derive(Clone, Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + uint8_val: u8, + uint16_val: u16, + uint32_val: u32, + uint64_val: u64, + uint128_val: u128, + int8_val: i8, + int16_val: i16, + int32_val: i32, + int64_val: i64, + int128_val: i128, + float32_val: f32, + float64_val: f64, + string_val: String, + } + + let client = prepare_database!(); + client + .query( + " + CREATE TABLE IF NOT EXISTS test ( + uint8_val UInt8, + uint16_val UInt16, + uint32_val UInt32, + uint64_val UInt64, + uint128_val UInt128, + int8_val Int8, + int16_val Int16, + int32_val Int32, + int64_val Int64, + int128_val Int128, + float32_val Float32, + float64_val Float64, + string_val String + ) + ENGINE = MergeTree + ORDER BY () + ", + ) + .execute() + .await + .unwrap(); + + let rows = vec![Data { + uint8_val: 255, + uint16_val: 65535, + uint32_val: 4294967295, + uint64_val: 18446744073709551615, + uint128_val: 340282366920938463463374607431768211455, + int8_val: -128, + int16_val: -32768, + int32_val: -2147483648, + int64_val: -9223372036854775808, + int128_val: -170141183460469231731687303715884105728, + float32_val: 42.0, + float64_val: 144.0, + string_val: "test".to_string(), + }]; + + let result = insert_and_select(&client, "test", rows.clone()).await; + assert_eq!(result, rows); +} + +#[tokio::test] +async fn several_simple_rows() { + let client = prepare_database!(); + create_simple_table(&client, "test").await; + + let rows = vec![ + SimpleRow::new(42, "foo".to_string()), + SimpleRow::new(144, "bar".to_string()), + SimpleRow::new(222, "baz".to_string()), + ]; + + let result = insert_and_select(&client, "test", rows.clone()).await; + assert_eq!(result, rows); +} + +#[tokio::test] +async fn many_numbers() { + #[derive(Row, Serialize, Deserialize)] + struct Data { + number: u64, + } + + let client = prepare_database!(); + client + .query( + " + CREATE TABLE IF NOT EXISTS test ( + number UInt64 + ) + ENGINE = MergeTree + ORDER BY number + ", + ) + .execute() + .await + .unwrap(); + + let mut insert = client.insert::("test").await.unwrap(); + for i in 1..=20_000 { + insert.write(&Data { number: i }).await.unwrap(); + } + + insert.end().await.unwrap(); + + let mut cursor = client + .query("SELECT number FROM test ORDER BY number") + .fetch::() + .unwrap(); + + let mut sum: u64 = 0; + for i in 1..=20_000 { + let row = cursor.next().await.unwrap().unwrap(); + assert_eq!(row.number, i); + sum += row.number; + } + + assert!(cursor.next().await.unwrap().is_none()); + assert_eq!(sum, 200_010_000); +} + +#[tokio::test] +async fn arrays() { + #[derive(Clone, Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + id: u16, + one_dim_array: Vec, + two_dim_array: Vec>, + three_dim_array: Vec>>, + description: String, + } + + let client = prepare_database!(); + client + .query( + " + CREATE TABLE IF NOT EXISTS test ( + id UInt16, + one_dim_array Array(UInt32), + two_dim_array Array(Array(Int64)), + three_dim_array Array(Array(Array(Float64))), + description String + ) + ENGINE = MergeTree + ORDER BY () + ", + ) + .execute() + .await + .unwrap(); + + let rows = vec![Data { + id: 42, + one_dim_array: vec![1, 2], + two_dim_array: vec![vec![1, 2], vec![3, 4]], + three_dim_array: vec![ + vec![vec![1.1, 2.2], vec![3.3, 4.4]], + vec![], + vec![vec![5.5, 6.6], vec![7.7, 8.8]], + ], + description: "foobar".to_string(), + }]; + + let result = insert_and_select(&client, "test", rows.clone()).await; + assert_eq!(result, rows); +} + +#[tokio::test] +async fn tuples() { + #[derive(Clone, Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + a: (u32, String), + b: (i128, HashMap), + } + + let client = prepare_database!(); + client + .query( + " + CREATE TABLE IF NOT EXISTS test ( + a Tuple(UInt32, String), + b Tuple(Int128, Map(UInt16, String)) + ) + ENGINE = MergeTree + ORDER BY () + ", + ) + .execute() + .await + .unwrap(); + + let rows = vec![ + Data { + a: (42, "foo".to_string()), + b: (144, vec![(255, "bar".to_string())].into_iter().collect()), + }, + Data { + a: (100, "qaz".to_string()), + b: ( + 222, + vec![(1, "qux".to_string()), (2, "quux".to_string())] + .into_iter() + .collect(), + ), + }, + ]; + + let result = insert_and_select(&client, "test", rows.clone()).await; + assert_eq!(result, rows); +} + +#[tokio::test] +async fn geo() { + #[derive(Clone, Debug, PartialEq)] + #[derive(Row, serde::Serialize, serde::Deserialize)] + struct Data { + id: u32, + point: Point, + ring: Ring, + polygon: Polygon, + multi_polygon: MultiPolygon, + line_string: LineString, + multi_line_string: MultiLineString, + } + + let client = prepare_database!(); + client + .query( + " + CREATE TABLE IF NOT EXISTS test ( + id UInt32, + point Point, + ring Ring, + polygon Polygon, + multi_polygon MultiPolygon, + line_string LineString, + multi_line_string MultiLineString + ) + ENGINE = MergeTree + ORDER BY id + ", + ) + .execute() + .await + .unwrap(); + + let rows = vec![ + Data { + id: 42, + point: (1.0, 2.0), + ring: vec![(3.0, 4.0), (5.0, 6.0)], + polygon: vec![vec![(7.0, 8.0), (9.0, 10.0)], vec![(11.0, 12.0)]], + multi_polygon: vec![vec![vec![(13.0, 14.0), (15.0, 16.0)], vec![(17.0, 18.0)]]], + line_string: vec![(19.0, 20.0), (21.0, 22.0)], + multi_line_string: vec![vec![(23.0, 24.0), (25.0, 26.0)], vec![(27.0, 28.0)]], + }, + Data { + id: 144, + point: (29.0, 30.0), + ring: vec![(31.0, 32.0), (33.0, 34.0)], + polygon: vec![vec![(35.0, 36.0), (37.0, 38.0)], vec![(39.0, 40.0)]], + multi_polygon: vec![vec![vec![(41.0, 42.0), (43.0, 44.0)], vec![(45.0, 46.0)]]], + line_string: vec![(47.0, 48.0), (49.0, 50.0)], + multi_line_string: vec![vec![(51.0, 52.0), (53.0, 54.0)], vec![(55.0, 56.0)]], + }, + ]; + + let result = insert_and_select(&client, "test", rows.clone()).await; + assert_eq!(result, rows); +} + +#[tokio::test] +async fn maps() { + #[derive(Clone, Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + m1: HashMap, + m2: HashMap>, + } + + let client = prepare_database!(); + client + .query( + " + CREATE TABLE IF NOT EXISTS test ( + m1 Map(String, String), + m2 Map(UInt16, Map(String, Int32)) + ) + ENGINE = MergeTree + ORDER BY () + ", + ) + .execute() + .await + .unwrap(); + + let rows = vec![Data { + m1: vec![ + ("key1".to_string(), "value1".to_string()), + ("key2".to_string(), "value2".to_string()), + ] + .into_iter() + .collect(), + m2: vec![ + ( + 42, + vec![("foo".to_string(), 100), ("bar".to_string(), 200)] + .into_iter() + .collect(), + ), + ( + 144, + vec![("qaz".to_string(), 300), ("qux".to_string(), 400)] + .into_iter() + .collect(), + ), + ] + .into_iter() + .collect::>>(), + }]; + + let result = insert_and_select(&client, "test", rows.clone()).await; + assert_eq!(result, rows); +} + +#[tokio::test] +async fn map_as_vec_of_tuples() { + #[derive(Clone, Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + m1: Vec<(i128, String)>, + m2: Vec<(u16, Vec<(String, i32)>)>, + } + + let client = prepare_database!(); + client + .query( + " + CREATE TABLE IF NOT EXISTS test ( + m1 Map(Int128, String), + m2 Map(UInt16, Map(String, Int32)) + ) + ENGINE = MergeTree + ORDER BY () + ", + ) + .execute() + .await + .unwrap(); + + let rows = vec![Data { + m1: vec![(100, "value1".to_string()), (200, "value2".to_string())], + m2: vec![ + ( + 42, + vec![("foo".to_string(), 100), ("bar".to_string(), 200)] + .into_iter() + .collect(), + ), + ( + 144, + vec![("qaz".to_string(), 300), ("qux".to_string(), 400)] + .into_iter() + .collect(), + ), + ], + }]; + + let result = insert_and_select(&client, "test", rows.clone()).await; + assert_eq!(result, rows) +} + +#[tokio::test] +async fn maps_third_party() { + #[derive(Clone, Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + im: IndexMap, + lhm: LinkedHashMap, + fx: FxHashMap, + weird_but_ok: LinkedHashMap>>>, + } + + let client = prepare_database!(); + client + .query( + " + CREATE TABLE IF NOT EXISTS test ( + im Map(UInt16, String), + lhm Map(UInt32, String), + fx Map(UInt64, String), + weird_but_ok Map(UInt128, Map(Int8, Map(Int16, Array(Bool)))) + ) + ENGINE = MergeTree + ORDER BY () + ", + ) + .execute() + .await + .unwrap(); + + let rows = vec![Data { + im: IndexMap::from_iter(vec![(1, "one".to_string()), (2, "two".to_string())]), + lhm: LinkedHashMap::from_iter(vec![(3, "three".to_string()), (4, "four".to_string())]), + fx: FxHashMap::from_iter(vec![(5, "five".to_string()), (6, "six".to_string())]), + weird_but_ok: LinkedHashMap::from_iter(vec![( + 7u128, + IndexMap::from_iter(vec![( + -8i8, + FxHashMap::from_iter(vec![(9i16, vec![true, false]), (10i16, vec![false])]), + )]), + )]), + }]; + + let result = insert_and_select(&client, "test", rows.clone()).await; + assert_eq!(result, rows); +} + +#[tokio::test] +async fn enums() { + #[derive(Clone, Debug, PartialEq, Serialize_repr, Deserialize_repr)] + #[repr(i8)] + enum MyEnum8 { + Winter = -128, + Spring = 0, + Summer = 100, + Autumn = 127, + } + + #[derive(Clone, Debug, PartialEq, Serialize_repr, Deserialize_repr)] + #[repr(i16)] + enum MyEnum16 { + North = -32768, + East = 0, + South = 144, + West = 32767, + } + + #[derive(Clone, Debug, PartialEq, Row, Serialize, Deserialize)] + struct Data { + id: u16, + enum8: MyEnum8, + enum16: MyEnum16, + } + + let table_name = "test_rbwnat_enum"; + + let client = prepare_database!(); + client + .query( + " + CREATE OR REPLACE TABLE ? + ( + id UInt16, + enum8 Enum8 ('Winter' = -128, 'Spring' = 0, 'Summer' = 100, 'Autumn' = 127), + enum16 Enum16('North' = -32768, 'East' = 0, 'South' = 144, 'West' = 32767) + ) ENGINE MergeTree ORDER BY id + ", + ) + .bind(Identifier(table_name)) + .execute() + .await + .unwrap(); + + let rows = vec![ + Data { + id: 1, + enum8: MyEnum8::Spring, + enum16: MyEnum16::East, + }, + Data { + id: 2, + enum8: MyEnum8::Autumn, + enum16: MyEnum16::North, + }, + Data { + id: 3, + enum8: MyEnum8::Winter, + enum16: MyEnum16::South, + }, + Data { + id: 4, + enum8: MyEnum8::Summer, + enum16: MyEnum16::West, + }, + ]; + + let result = insert_and_select(&client, table_name, rows.clone()).await; + assert_eq!(result, rows); +} + +#[tokio::test] +async fn nullable() { + #[derive(Clone, Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + a: u32, + b: Option, + } + + let client = prepare_database!(); + client + .query( + " + CREATE TABLE IF NOT EXISTS test ( + a UInt32, + b Nullable(Int64) + ) + ENGINE = MergeTree + ORDER BY a + ", + ) + .execute() + .await + .unwrap(); + + let rows = vec![ + Data { a: 1, b: Some(2) }, + Data { a: 3, b: None }, + Data { a: 4, b: Some(5) }, + ]; + + let result = insert_and_select(&client, "test", rows.clone()).await; + assert_eq!(result, rows); +} + +#[tokio::test] +async fn blob_string_with_serde_bytes() { + #[derive(Clone, Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + #[serde(with = "serde_bytes")] + blob: Vec, + } + + let client = prepare_database!(); + client + .query( + " + CREATE TABLE IF NOT EXISTS test ( + blob String + ) + ENGINE = MergeTree + ORDER BY tuple() + ", + ) + .execute() + .await + .unwrap(); + + let rows = vec![ + Data { + blob: "foo".as_bytes().to_vec(), + }, + Data { + blob: "bar".as_bytes().to_vec(), + }, + Data { + blob: "qaz".as_bytes().to_vec(), + }, + ]; + + let result = insert_and_select(&client, "test", rows.clone()).await; + assert_eq!(result, rows); +} + +#[tokio::test] +async fn low_cardinality() { + #[derive(Clone, Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + a: u32, + b: Option, + } + + let client = prepare_database!(); + client + .query( + " + CREATE TABLE IF NOT EXISTS test ( + a LowCardinality(UInt32), + b LowCardinality(Nullable(Int64)) + ) + ENGINE = MergeTree + ORDER BY a + ", + ) + .with_option("allow_suspicious_low_cardinality_types", "1") + .execute() + .await + .unwrap(); + + let rows = vec![ + Data { a: 1, b: Some(2) }, + Data { a: 3, b: None }, + Data { a: 4, b: Some(5) }, + ]; + + let result = insert_and_select(&client, "test", rows.clone()).await; + assert_eq!(result, rows); +} + +#[tokio::test] +async fn serde_skip_struct_field() { + #[derive(Clone, Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + a: u32, + #[serde(skip_serializing)] + #[serde(skip_deserializing)] + b: u32, + c: u32, + } + + let client = prepare_database!(); + client + .query( + " + CREATE TABLE IF NOT EXISTS test ( + a UInt32, + c UInt32 + ) + ENGINE = MergeTree + ORDER BY a ASC + ", + ) + .execute() + .await + .unwrap(); + + let result = insert_and_select( + &client, + "test", + vec![ + Data { + a: 42, + b: 111, // b will be ignored + c: 144, + }, + Data { + a: 100, + b: 222, + c: 200, + }, + ], + ) + .await; + + assert_eq!( + result, + vec![ + Data { + a: 42, + b: 0, // default value for u32 + c: 144 + }, + Data { + a: 100, + b: 0, + c: 200 + }, + ] + ); +} + +#[tokio::test] +#[cfg(feature = "time")] +async fn date_and_time() { + use time::format_description::well_known::Iso8601; + use time::Month::{February, January}; + use time::OffsetDateTime; + + #[derive(Clone, Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + #[serde(with = "clickhouse::serde::time::date")] + date: time::Date, + #[serde(with = "clickhouse::serde::time::date32")] + date32: time::Date, + #[serde(with = "clickhouse::serde::time::datetime")] + date_time: OffsetDateTime, + #[serde(with = "clickhouse::serde::time::datetime64::secs")] + date_time64_0: OffsetDateTime, + #[serde(with = "clickhouse::serde::time::datetime64::millis")] + date_time64_3: OffsetDateTime, + #[serde(with = "clickhouse::serde::time::datetime64::micros")] + date_time64_6: OffsetDateTime, + #[serde(with = "clickhouse::serde::time::datetime64::nanos")] + date_time64_9: OffsetDateTime, + } + + let client = prepare_database!(); + client + .query( + " + CREATE TABLE IF NOT EXISTS test ( + date Date, + date32 Date32, + date_time DateTime, + date_time64_0 DateTime64(0), + date_time64_3 DateTime64(3), + date_time64_6 DateTime64(6), + date_time64_9 DateTime64(9) + ) + ENGINE = MergeTree + ORDER BY tuple() + ", + ) + .execute() + .await + .unwrap(); + + let data = vec![Data { + date: time::Date::from_calendar_date(2023, January, 1).unwrap(), + date32: time::Date::from_calendar_date(2023, February, 2).unwrap(), + date_time: OffsetDateTime::parse("2023-01-03T12:00:00Z", &Iso8601::DEFAULT).unwrap(), + date_time64_0: OffsetDateTime::parse("2023-01-04T13:00:00Z", &Iso8601::DEFAULT).unwrap(), + date_time64_3: OffsetDateTime::parse("2023-01-05T14:00:00.123Z", &Iso8601::DEFAULT) + .unwrap(), + date_time64_6: OffsetDateTime::parse("2023-01-06T15:00:00.123456Z", &Iso8601::DEFAULT) + .unwrap(), + date_time64_9: OffsetDateTime::parse("2023-01-07T16:00:00.123456789Z", &Iso8601::DEFAULT) + .unwrap(), + }]; + + let result = insert_and_select(&client, "test", data.clone()).await; + assert_eq!(result, data); +} + +#[tokio::test] +#[cfg(feature = "uuid")] +async fn uuid() { + #[derive(Clone, Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + id: u16, + #[serde(with = "clickhouse::serde::uuid")] + uuid: uuid::Uuid, + } + + let client = prepare_database!(); + client + .query( + " + CREATE TABLE IF NOT EXISTS test ( + id UInt16, + uuid UUID + ) + ENGINE = MergeTree + ORDER BY id + ", + ) + .execute() + .await + .unwrap(); + + let rows = vec![ + Data { + id: 42, + uuid: uuid::Uuid::from_str("550e8400-e29b-41d4-a716-446655440000").unwrap(), + }, + Data { + id: 144, + uuid: uuid::Uuid::from_str("12345678-1234-5678-1234-567812345678").unwrap(), + }, + Data { + id: 255, + uuid: uuid::Uuid::from_str("00000000-0000-0000-0000-000000000000").unwrap(), + }, + ]; + + let result = insert_and_select(&client, "test", rows.clone()).await; + assert_eq!(result, rows); +} + +#[tokio::test] +async fn ipv4_ipv6() { + #[derive(Clone, Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + id: u16, + #[serde(with = "clickhouse::serde::ipv4")] + ipv4: std::net::Ipv4Addr, + ipv6: std::net::Ipv6Addr, + } + + let client = prepare_database!(); + client + .query( + " + CREATE TABLE IF NOT EXISTS test ( + id UInt16, + ipv4 IPv4, + ipv6 IPv6 + ) + ENGINE = MergeTree + ORDER BY id + ", + ) + .execute() + .await + .unwrap(); + + let rows = vec![Data { + id: 42, + ipv4: std::net::Ipv4Addr::new(192, 168, 0, 1), + ipv6: std::net::Ipv6Addr::from_str("2001:db8:3333:4444:5555:6666:7777:8888").unwrap(), + }]; + + let result = insert_and_select(&client, "test", rows.clone()).await; + assert_eq!(result, rows) +} + +#[tokio::test] +async fn fixed_str() { + #[derive(Clone, Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + a: [u8; 4], + b: [u8; 3], + } + + let client = prepare_database!(); + client + .query( + " + CREATE TABLE IF NOT EXISTS test ( + a FixedString(4), + b FixedString(3) + ) + ENGINE = MergeTree + ORDER BY tuple() + ", + ) + .execute() + .await + .unwrap(); + + let rows = vec![Data { + a: [49, 50, 51, 52], // '1234' + b: [55, 55, 55], // '777' + }]; + + let result = insert_and_select(&client, "test", rows.clone()).await; + assert_eq!(result, rows); + assert_eq!(String::from_utf8_lossy(&result[0].a), "1234"); + assert_eq!(String::from_utf8_lossy(&result[0].b), "777"); +} + +#[tokio::test] +async fn decimals() { + #[derive(Clone, Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + decimal32_9_4: Decimal32, + decimal64_18_8: Decimal64, + decimal128_38_12: Decimal128, + } + + let client = prepare_database!(); + client + .query( + " + CREATE TABLE IF NOT EXISTS test ( + decimal32_9_4 Decimal32(4), + decimal64_18_8 Decimal64(8), + decimal128_38_12 Decimal128(12) + ) + ENGINE = MergeTree + ORDER BY tuple() + ", + ) + .execute() + .await + .unwrap(); + + let rows = vec![Data { + decimal32_9_4: Decimal32::from_str("42.1234").unwrap(), + decimal64_18_8: Decimal64::from_str("144.56789012").unwrap(), + decimal128_38_12: Decimal128::from_str("-17014118346046923173168730.37158841057").unwrap(), + }]; + + let result = insert_and_select(&client, "test", rows.clone()).await; + assert_eq!(result, rows); +} + +#[tokio::test] +async fn different_struct_field_order_same_types() { + #[derive(Clone, Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + c: String, + a: String, + } + + let client = prepare_database!(); + client + .query( + " + CREATE OR REPLACE TABLE test ( + a String, + c String + ) ENGINE MergeTree ORDER BY a + ", + ) + .execute() + .await + .unwrap(); + + let rows = vec![ + Data { + c: "foo".to_string(), + a: "bar".to_string(), + }, + Data { + c: "baz".to_string(), + a: "qux".to_string(), + }, + ]; + + let result = insert_and_select(&client, "test", rows.clone()).await; + assert_eq!(result, rows); +} + +#[tokio::test] +async fn different_struct_field_order_different_types() { + #[derive(Clone, Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + b: u32, + a: String, + c: Vec, + } + + let client = prepare_database!(); + client + .query( + " + CREATE OR REPLACE TABLE test ( + a String, + b UInt32, + c Array(Bool) + ) ENGINE MergeTree ORDER BY a + ", + ) + .execute() + .await + .unwrap(); + + let rows = vec![ + Data { + b: 42, + a: "bar".to_string(), + c: vec![false, true], + }, + Data { + b: 144, + a: "foo".to_string(), + c: vec![true, false, true], + }, + ]; + + let result = insert_and_select(&client, "test", rows.clone()).await; + assert_eq!(result, rows); +} + +#[tokio::test] +async fn different_struct_field_order_mixed_usage() { + #[derive(Clone, Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + c: String, + a: String, + sku: u32, + id: u32, + #[serde(skip_serializing)] + #[serde(skip_deserializing)] + ignored: u64, + #[serde(rename = "b")] + x: u64, + } + + let client = prepare_database!(); + client + .query( + " + CREATE OR REPLACE TABLE test ( + id UInt32, + a String, + b UInt64, + c String, + sku UInt32 + ) ENGINE MergeTree ORDER BY id + ", + ) + .execute() + .await + .unwrap(); + + let rows = vec![ + Data { + c: "foo".to_string(), + a: "bar".to_string(), + sku: 42, + id: 1, + ignored: 123, // skipped + x: 100, // serialized as 'b' + }, + Data { + c: "baz".to_string(), + a: "qux".to_string(), + sku: 144, + id: 2, + ignored: 777, // skipped + x: 200, // serialized as 'b' + }, + ]; + + let result = insert_and_select(&client, "test", rows.clone()).await; + assert_eq!( + result, + vec![ + Data { + c: "foo".to_string(), + a: "bar".to_string(), + sku: 42, + id: 1, + ignored: 0, // not deserialized, default value + x: 100, // deserialized from the db field 'b' + }, + Data { + c: "baz".to_string(), + a: "qux".to_string(), + sku: 144, + id: 2, + ignored: 0, // not deserialized, default value + x: 200, // deserialized from the db field 'b' + }, + ] + ); +} + +#[tokio::test] +async fn borrowed_data() { + #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data<'a> { + str: &'a str, + array: Vec<&'a str>, + tuple: (&'a str, &'a str), + str_opt: Option<&'a str>, + vec_map_str: Vec<(&'a str, &'a str)>, + vec_map_f32: Vec<(&'a str, f32)>, + vec_map_nested: Vec<(&'a str, Vec<(&'a str, &'a str)>)>, + hash_map_str: HashMap<&'a str, &'a str>, + hash_map_f32: HashMap<&'a str, f32>, + hash_map_nested: HashMap<&'a str, HashMap<&'a str, &'a str>>, + } + + let client = prepare_database!(); + client + .query( + " + CREATE OR REPLACE TABLE test ( + str String, + array Array(String), + tuple Tuple(String, String), + str_opt Nullable(String), + vec_map_str Map(String, String), + vec_map_f32 Map(String, Float32), + vec_map_nested Map(String, Map(String, String)), + hash_map_str Map(String, String), + hash_map_f32 Map(String, Float32), + hash_map_nested Map(String, Map(String, String)) + ) ENGINE MergeTree ORDER BY str + ", + ) + .execute() + .await + .unwrap(); + + let row1 = Data { + str: "a", + array: vec!["b", "c"], + tuple: ("d", "e"), + str_opt: None, + vec_map_str: vec![("key1", "value1"), ("key2", "value2")], + vec_map_f32: vec![("key3", 100.0), ("key4", 200.0)], + vec_map_nested: vec![("n1", vec![("key1", "value1"), ("key2", "value2")])], + hash_map_str: HashMap::from([("key1", "value1"), ("key2", "value2")]), + hash_map_f32: HashMap::from([("key3", 100.0), ("key4", 200.0)]), + hash_map_nested: HashMap::from([( + "n1", + HashMap::from([("key1", "value1"), ("key2", "value2")]), + )]), + }; + + let row2 = Data { + str: "f", + array: vec!["g", "h"], + tuple: ("i", "j"), + str_opt: Some("k"), + vec_map_str: vec![("key4", "value4"), ("key5", "value5")], + vec_map_f32: vec![("key6", 300.0), ("key7", 400.0)], + vec_map_nested: vec![("n2", vec![("key4", "value4"), ("key5", "value5")])], + hash_map_str: HashMap::from([("key4", "value4"), ("key5", "value5")]), + hash_map_f32: HashMap::from([("key6", 300.0), ("key7", 400.0)]), + hash_map_nested: HashMap::from([( + "n2", + HashMap::from([("key4", "value4"), ("key5", "value5")]), + )]), + }; + + let mut insert = client.insert::>("test").await.unwrap(); + insert.write(&row1).await.unwrap(); + insert.write(&row2).await.unwrap(); + insert.end().await.unwrap(); + + let mut cursor = client + .query("SELECT ?fields FROM test") + .fetch::>() + .unwrap(); + + assert_eq!(cursor.next().await.unwrap().unwrap(), row1); + assert_eq!(cursor.next().await.unwrap().unwrap(), row2); + assert!(cursor.next().await.unwrap().is_none()); +} + +#[tokio::test] +async fn nested_data_type() { + #[derive(Clone, Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + id: u16, + #[serde(rename = "nested.id")] + nested_id: Vec, + #[serde(rename = "nested.value")] + nested_value: Vec, + } + + let client = prepare_database!(); + client + .query( + " + CREATE TABLE IF NOT EXISTS test ( + id UInt16, + nested Nested(id UInt32, value String) + ) + ENGINE = MergeTree + ORDER BY id + ", + ) + .execute() + .await + .unwrap(); + + let rows = vec![ + Data { + id: 42, + nested_id: vec![1, 2, 3], + nested_value: vec!["one".to_string(), "two".to_string(), "three".to_string()], + }, + Data { + id: 144, + nested_id: vec![4, 5], + nested_value: vec!["four".to_string(), "five".to_string()], + }, + ]; + + let result = insert_and_select(&client, "test", rows.clone()).await; + assert_eq!(result, rows); +} + +/// See https://github.com/ClickHouse/clickhouse-rs/issues/99 +#[tokio::test] +#[ignore] // FIXME: requires https://github.com/ClickHouse/clickhouse-rs/issues/264 +async fn issue_99_flatten_maps() { + #[derive(Clone, Debug, Row, Serialize, Deserialize, PartialEq)] + struct Metadata { + foo: String, + bar: String, + } + #[derive(Clone, Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + #[serde(flatten)] + metadata: Metadata, + data: String, + } + + let client = prepare_database!(); + client + .query( + " + CREATE TABLE IF NOT EXISTS test ( + foo String, + bar String, + data String + ) + ENGINE = MergeTree + ORDER BY tuple() + ", + ) + .execute() + .await + .unwrap(); + + let rows = vec![Data { + metadata: Metadata { + foo: "foo_value".to_string(), + bar: "bar_value".to_string(), + }, + data: "data_value".to_string(), + }]; + + let result = insert_and_select(&client, "test", rows.clone()).await; + assert_eq!(result, rows); +} + +#[tokio::test] +async fn struct_and_primitive_in_a_tuple() { + #[derive(Clone, Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + id: u16, + value: String, + } + + let client = prepare_database!(); + client + .query( + " + CREATE TABLE IF NOT EXISTS test ( + id UInt16, + value String + ) + ENGINE = MergeTree + ORDER BY id + ", + ) + .execute() + .await + .unwrap(); + + let row1 = Data { + id: 42, + value: "forty-two".to_string(), + }; + let row2 = Data { + id: 144, + value: "one four four".to_string(), + }; + let rows = vec![&row1, &row2]; + + let mut insert = client.insert::("test").await.unwrap(); + for data in rows { + insert.write(data).await.unwrap(); + } + insert.end().await.unwrap(); + + let mut cursor = client + .query("SELECT ?fields, count() FROM test GROUP BY ?fields") + .fetch::<(Data, u64)>() + .unwrap(); + + let mut results = Vec::new(); + while let Some(row) = cursor.next().await.unwrap() { + results.push(row); + } + + assert_eq!(results, vec![(row1, 1), (row2, 1)]); +} + +#[tokio::test] +async fn several_primitives_in_a_tuple() { + let client = get_client(); + let mut cursor = client + .query("SELECT number, number * 2 FROM system.numbers LIMIT 3") + .fetch::<(u64, u64)>() + .unwrap(); + let mut results = Vec::new(); + while let Some(row) = cursor.next().await.unwrap() { + results.push(row); + } + assert_eq!( + results, + vec![(0, 0), (1, 2), (2, 4)], + "Expected tuples with two u64 values" + ); +} + +#[tokio::test] +async fn interval() { + #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + id: u32, + interval_nanosecond: i64, + interval_microsecond: i64, + interval_millisecond: i64, + interval_second: i64, + interval_minute: i64, + interval_hour: i64, + interval_day: i64, + interval_week: i64, + interval_month: i64, + interval_quarter: i64, + interval_year: i64, + } + + let client = get_client(); + let mut cursor = client + .query( + " + SELECT * FROM ( + SELECT + 0 :: UInt32 AS id, + toIntervalNanosecond (0) AS interval_nanosecond, + toIntervalMicrosecond (0) AS interval_microsecond, + toIntervalMillisecond (0) AS interval_millisecond, + toIntervalSecond (0) AS interval_second, + toIntervalMinute (0) AS interval_minute, + toIntervalHour (0) AS interval_hour, + toIntervalDay (0) AS interval_day, + toIntervalWeek (0) AS interval_week, + toIntervalMonth (0) AS interval_month, + toIntervalQuarter (0) AS interval_quarter, + toIntervalYear (0) AS interval_year + UNION ALL + SELECT + 1 :: UInt32 AS id, + toIntervalNanosecond (-9223372036854775808) AS interval_nanosecond, + toIntervalMicrosecond (-9223372036854775808) AS interval_microsecond, + toIntervalMillisecond (-9223372036854775808) AS interval_millisecond, + toIntervalSecond (-9223372036854775808) AS interval_second, + toIntervalMinute (-9223372036854775808) AS interval_minute, + toIntervalHour (-9223372036854775808) AS interval_hour, + toIntervalDay (-9223372036854775808) AS interval_day, + toIntervalWeek (-9223372036854775808) AS interval_week, + toIntervalMonth (-9223372036854775808) AS interval_month, + toIntervalQuarter (-9223372036854775808) AS interval_quarter, + toIntervalYear (-9223372036854775808) AS interval_year + UNION ALL + SELECT + 2 :: UInt32 AS id, + toIntervalNanosecond (9223372036854775807) AS interval_nanosecond, + toIntervalMicrosecond (9223372036854775807) AS interval_microsecond, + toIntervalMillisecond (9223372036854775807) AS interval_millisecond, + toIntervalSecond (9223372036854775807) AS interval_second, + toIntervalMinute (9223372036854775807) AS interval_minute, + toIntervalHour (9223372036854775807) AS interval_hour, + toIntervalDay (9223372036854775807) AS interval_day, + toIntervalWeek (9223372036854775807) AS interval_week, + toIntervalMonth (9223372036854775807) AS interval_month, + toIntervalQuarter (9223372036854775807) AS interval_quarter, + toIntervalYear (9223372036854775807) AS interval_year + ) ORDER BY id ASC + ", + ) + .fetch::() + .unwrap(); + + assert_eq!( + cursor.next().await.unwrap().unwrap(), + Data { + id: 0, + interval_nanosecond: 0, + interval_microsecond: 0, + interval_millisecond: 0, + interval_second: 0, + interval_minute: 0, + interval_hour: 0, + interval_day: 0, + interval_week: 0, + interval_month: 0, + interval_quarter: 0, + interval_year: 0, + } + ); + + assert_eq!( + cursor.next().await.unwrap().unwrap(), + Data { + id: 1, + interval_nanosecond: i64::MIN, + interval_microsecond: i64::MIN, + interval_millisecond: i64::MIN, + interval_second: i64::MIN, + interval_minute: i64::MIN, + interval_hour: i64::MIN, + interval_day: i64::MIN, + interval_week: i64::MIN, + interval_month: i64::MIN, + interval_quarter: i64::MIN, + interval_year: i64::MIN, + } + ); + + assert_eq!( + cursor.next().await.unwrap().unwrap(), + Data { + id: 2, + interval_nanosecond: i64::MAX, + interval_microsecond: i64::MAX, + interval_millisecond: i64::MAX, + interval_second: i64::MAX, + interval_minute: i64::MAX, + interval_hour: i64::MAX, + interval_day: i64::MAX, + interval_week: i64::MAX, + interval_month: i64::MAX, + interval_quarter: i64::MAX, + interval_year: i64::MAX, + } + ); +} diff --git a/tests/it/rbwnat_validation.rs b/tests/it/rbwnat_validation.rs new file mode 100644 index 00000000..b90f87d1 --- /dev/null +++ b/tests/it/rbwnat_validation.rs @@ -0,0 +1,793 @@ +use crate::{execute_statements, get_client}; +use clickhouse_derive::Row; +use serde::{Deserialize, Serialize}; +use std::collections::HashMap; + +#[tokio::test] +async fn fetch_primitive_row() { + let client = get_client(); + let result = client + .query("SELECT count() FROM (SELECT * FROM system.numbers LIMIT 3)") + .fetch_one::() + .await; + assert_eq!(result.unwrap(), 3); +} + +#[tokio::test] +async fn fetch_primitive_row_schema_mismatch() { + type Data = i32; // expected type is UInt64 + assert_panic_on_fetch!( + &["primitive", "UInt64", "i32"], + "SELECT count() FROM (SELECT * FROM system.numbers LIMIT 3)" + ); +} + +#[tokio::test] +async fn fetch_vector_row() { + let client = get_client(); + let result = client + .query("SELECT [1, 2, 3] :: Array(UInt32)") + .fetch_one::>() + .await; + assert_eq!(result.unwrap(), vec![1, 2, 3]); +} + +#[tokio::test] +async fn fetch_vector_row_schema_mismatch_nested_type() { + type Data = Vec; // expected type for Array(UInt32) is Vec + assert_panic_on_fetch!( + &["vector", "UInt32", "i128"], + "SELECT [1, 2, 3] :: Array(UInt32)" + ); +} + +#[tokio::test] +async fn fetch_tuple_row() { + let client = get_client(); + let result = client + .query("SELECT 42 :: UInt32 AS a, 'foo' :: String AS b") + .fetch_one::<(u32, String)>() + .await; + assert_eq!(result.unwrap(), (42, "foo".to_string())); +} + +#[tokio::test] +async fn fetch_tuple_row_schema_mismatch_first_element() { + type Data = (i128, String); // expected u32 instead of i128 + assert_panic_on_fetch!( + &["tuple", "UInt32", "i128"], + "SELECT 42 :: UInt32 AS a, 'foo' :: String AS b" + ); +} + +#[tokio::test] +async fn fetch_tuple_row_schema_mismatch_second_element() { + type Data = (u32, i64); // expected String instead of i64 + assert_panic_on_fetch!( + &["tuple", "String", "i64"], + "SELECT 42 :: UInt32 AS a, 'foo' :: String AS b" + ); +} + +#[tokio::test] +async fn fetch_tuple_row_schema_mismatch_missing_element() { + type Data = (u32, String); // expected to have the third element as i64 + assert_panic_on_fetch!( + &[ + "database schema has 3 columns", + "tuple definition has 2 fields" + ], + "SELECT 42 :: UInt32 AS a, 'foo' :: String AS b, 144 :: Int64 AS c" + ); +} + +#[tokio::test] +async fn fetch_tuple_row_schema_mismatch_too_many_elements() { + type Data = (u32, String, i128); // i128 should not be there + assert_panic_on_fetch!( + &[ + "database schema has 2 columns", + "tuple definition has 3 fields" + ], + "SELECT 42 :: UInt32 AS a, 'foo' :: String AS b" + ); +} + +#[tokio::test] +async fn fetch_tuple_row_with_struct() { + #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + a: u32, + b: String, + } + + let client = get_client(); + let result = client + .query("SELECT 42 :: UInt32 AS a, 'foo' :: String AS b, 144 :: UInt64 AS c") + .fetch_one::<(Data, u64)>() + .await; + assert_eq!( + result.unwrap(), + ( + Data { + a: 42, + b: "foo".to_string() + }, + 144 + ) + ); +} + +#[tokio::test] +async fn fetch_tuple_row_with_struct_schema_mismatch() { + #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] + struct _Data { + a: u64, // expected type is u32 + b: String, + } + type Data = (_Data, u64); + assert_panic_on_fetch!( + &["tuple", "UInt32", "u64"], + "SELECT 42 :: UInt32 AS a, 'foo' :: String AS b, 144 :: UInt64 AS c" + ); +} + +#[tokio::test] +async fn fetch_tuple_row_with_struct_schema_mismatch_too_many_struct_fields() { + #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] + struct _Data { + a: u32, + b: String, + c: u64, // this field should not be here + } + type Data = (_Data, u64); + assert_panic_on_fetch!( + &["3 columns", "4 fields"], + "SELECT 42 :: UInt32 AS a, 'foo' :: String AS b, 144 :: UInt64 AS c" + ); +} + +#[tokio::test] +async fn fetch_tuple_row_with_struct_schema_mismatch_too_many_fields() { + #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] + struct _Data { + a: u32, + b: String, + } + type Data = (_Data, u64, u64); // one too many u64 + assert_panic_on_fetch!( + &["3 columns", "4 fields"], + "SELECT 42 :: UInt32 AS a, 'foo' :: String AS b, 144 :: UInt64 AS c" + ); +} + +#[tokio::test] +async fn fetch_tuple_row_with_struct_schema_mismatch_too_few_struct_fields() { + #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] + struct _Data { + a: u32, // the second field is missing now + } + type Data = (_Data, u64); + assert_panic_on_fetch!( + &["3 columns", "2 fields"], + "SELECT 42 :: UInt32 AS a, 'foo' :: String AS b, 144 :: UInt64 AS c" + ); +} + +#[tokio::test] +async fn fetch_tuple_row_with_struct_schema_mismatch_too_few_fields() { + #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] + struct _Data { + a: u32, + b: String, + } + type Data = (_Data, u64); // another u64 is missing here + assert_panic_on_fetch!( + &["4 columns", "3 fields"], + "SELECT 42 :: UInt32 AS a, 'foo' :: String AS b, 144 :: UInt64 AS c, 255 :: UInt64 AS d" + ); +} + +#[tokio::test] +async fn map_as_vec_of_tuples_schema_mismatch() { + #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + m: Vec<(u16, Vec<(String, i32)>)>, + } + + assert_panic_on_fetch!( + &["Data.m", "Map(Int64, String)", "Int64", "u16"], + "SELECT map(100, 'value1', 200, 'value2') :: Map(Int64, String) AS m" + ); +} + +#[tokio::test] +async fn map_as_vec_of_tuples_schema_mismatch_nested() { + type Inner = Vec<(i32, i64)>; // the value should be i128 instead of i64 + + #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + m: Vec<(u16, Vec<(String, Inner)>)>, + } + + assert_panic_on_fetch!( + &[ + "Data.m", + "Map(UInt16, Map(String, Map(Int32, Int128)))", + "Int128", + "i64" + ], + "SELECT map(42, map('foo', map(144, 255))) + :: Map(UInt16, Map(String, Map(Int32, Int128))) AS m" + ); +} + +#[tokio::test] +async fn invalid_nullable() { + #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + n: Option, + } + assert_panic_on_fetch!( + &["Data.n", "Array(UInt32)", "Option"], + "SELECT array(42) :: Array(UInt32) AS n" + ); +} + +#[tokio::test] +async fn invalid_low_cardinality() { + #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + a: u32, + } + let client = get_client().with_option("allow_suspicious_low_cardinality_types", "1"); + assert_panic_on_fetch_with_client!( + client, + &["Data.a", "LowCardinality(Int32)", "u32"], + "SELECT 144 :: LowCardinality(Int32) AS a" + ); +} + +#[tokio::test] +async fn invalid_nullable_low_cardinality() { + #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + a: Option, + } + let client = get_client().with_option("allow_suspicious_low_cardinality_types", "1"); + assert_panic_on_fetch_with_client!( + client, + &["Data.a", "LowCardinality(Nullable(Int32))", "u32"], + "SELECT 144 :: LowCardinality(Nullable(Int32)) AS a" + ); +} + +#[tokio::test] +#[cfg(feature = "time")] +async fn invalid_serde_with() { + #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + #[serde(with = "clickhouse::serde::time::datetime64::millis")] + n1: time::OffsetDateTime, // underlying is still Int64; should not compose it from two (U)Int32 + } + assert_panic_on_fetch!(&["Data.n1", "UInt32", "i64"], "SELECT 42 :: UInt32 AS n1"); +} + +#[tokio::test] +async fn too_many_struct_fields() { + #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + a: u32, + b: u32, + c: u32, + } + assert_panic_on_fetch!( + &["2 columns", "3 fields"], + "SELECT 42 :: UInt32 AS a, 144 :: UInt32 AS b" + ); +} + +#[tokio::test] +async fn fixed_str_too_long() { + #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + a: [u8; 4], + b: [u8; 3], + } + assert_panic_on_fetch!( + &["Data.a", "FixedString(5)", "with length 4"], + "SELECT '12345' :: FixedString(5) AS a, '777' :: FixedString(3) AS b" + ); +} + +#[tokio::test] +async fn tuple_invalid_definition() { + #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + a: (u32, String), + b: (i128, HashMap), + } + // Map key is UInt64 instead of UInt16 requested in the struct + assert_panic_on_fetch!( + &[ + "Data.b", + "Tuple(Int128, Map(UInt64, String))", + "UInt64 as u16" + ], + " + SELECT + (42, 'foo') :: Tuple(UInt32, String) AS a, + (144, map(255, 'bar')) :: Tuple(Int128, Map(UInt64, String)) AS b + " + ); +} + +#[tokio::test] +async fn tuple_too_many_elements_in_the_schema() { + #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + a: (u32, String), + b: (i128, HashMap), + } + // too many elements in the db type definition + assert_panic_on_fetch!( + &[ + "Data.a", + "Tuple(UInt32, String, Bool)", + "remaining elements: Bool" + ], + " + SELECT + (42, 'foo', true) :: Tuple(UInt32, String, Bool) AS a, + (144, map(255, 'bar')) :: Tuple(Int128, Map(UInt16, String)) AS b + " + ); +} + +#[tokio::test] +async fn tuple_too_many_elements_in_the_struct() { + #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + a: (u32, String, bool), + b: (i128, HashMap), + } + // too many elements in the struct enum + assert_panic_on_fetch!( + &["Data.a", "Tuple(UInt32, String)", "bool"], + " + SELECT + (42, 'foo') :: Tuple(UInt32, String) AS a, + (144, map(255, 'bar')) :: Tuple(Int128, Map(UInt16, String)) AS b + " + ); +} + +#[tokio::test] +async fn deeply_nested_validation_incorrect_fixed_string() { + #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + id: u32, + col: Vec>>>, + } + // Struct has FixedString(2) instead of FixedString(1) + assert_panic_on_fetch!( + &["Data.col", "FixedString(1)", "with length 2"], + " + SELECT + 42 :: UInt32 AS id, + array(array(map(42, array('1', '2')))) :: Array(Array(Map(UInt32, Array(FixedString(1))))) AS col + " + ); +} + +// TODO: there are two panics; one about schema mismatch, +// another about not all Tuple elements being deserialized +// not easy to assert, same applies to the other Geo types +#[ignore] +#[tokio::test] +async fn geo_invalid_point() { + #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + id: u32, + pt: (i32, i32), + } + assert_panic_on_fetch!( + &["Data.pt", "Point", "Float64 as i32"], + " + SELECT + 42 :: UInt32 AS id, + (1.0, 2.0) :: Point AS pt + " + ); +} + +/// See https://github.com/ClickHouse/clickhouse-rs/issues/57 +#[tokio::test] +async fn issue_57() { + #[derive(Debug, Row, Deserialize, Serialize)] + struct Data { + pub metadata_id: String, + pub start_time: i64, + pub end_time: Option, + pub double_value: f64, + pub string_value: String, + pub long_value: i64, + pub write_time: i64, + pub sign: i8, + pub version: u64, + } + + let client = prepare_database!(); + client + .query( + " + CREATE TABLE test ( + metadata_id String, + start_time DateTime64(3, 'UTC') CODEC(DoubleDelta, ZSTD(22)), + end_time DateTime64(3, 'UTC') CODEC(DoubleDelta, ZSTD(22)), + double_value Float64 CODEC(DoubleDelta, ZSTD(22)), + string_value String CODEC(ZSTD(22)), + long_value Int64, + write_time DateTime64(3, 'UTC') CODEC(DoubleDelta, ZSTD(22)), + sign Int8 DEFAULT 1, + version UInt64 + ) Engine = MergeTree + ORDER BY (metadata_id, start_time, end_time); + ", + ) + .execute() + .await + .unwrap(); + + let data = Data { + metadata_id: "test_id".to_string(), + start_time: 1622548800, + end_time: Some(1622552400), + double_value: 4.22, + string_value: "test_value".to_string(), + long_value: 42, + write_time: 1622556000, + sign: -1, + version: 1, + }; + + let mut insert = client.insert::("test").await.unwrap(); + let unwind = std::panic::AssertUnwindSafe(async { + insert.write(&data).await.unwrap(); + }); + assert_panic_msg!( + unwind, + &["Data.end_time", "DateTime64(3, 'UTC')", "Option"] + ); +} + +/// See https://github.com/ClickHouse/clickhouse-rs/issues/100 +#[tokio::test] +async fn issue_100() { + { + #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + n: i8, + } + assert_panic_on_fetch!( + &["Data.n", "Nullable(Bool)", "i8"], + "SELECT NULL :: Nullable(Bool) AS n" + ); + } + + { + #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + n: u8, + } + assert_panic_on_fetch!( + &["Data.n", "Nullable(Bool)", "u8"], + "SELECT NULL :: Nullable(Bool) AS n" + ); + } + + { + #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + n: bool, + } + assert_panic_on_fetch!( + &["Data.n", "Nullable(Bool)", "bool"], + "SELECT NULL :: Nullable(Bool) AS n" + ); + } +} + +/// See https://github.com/ClickHouse/clickhouse-rs/issues/109#issuecomment-2243197221 +#[tokio::test] +async fn issue_109_1() { + #[derive(Debug, Serialize, Deserialize, Row)] + struct Data { + #[serde(skip_deserializing)] + en_id: String, + journey: u32, + drone_id: String, + call_sign: String, + } + + let client = prepare_database!(); + client + .query( + " + CREATE TABLE issue_109 ( + drone_id String, + call_sign String, + journey UInt32, + en_id String + ) + ENGINE = MergeTree + ORDER BY (drone_id) + ", + ) + .execute() + .await + .unwrap(); + + let unwind = std::panic::AssertUnwindSafe(async { + let _ = client.insert::("issue_109").await.unwrap(); + }); + assert_panic_msg!(unwind, &["Data", "4 columns", "3 fields"]); +} + +#[tokio::test] +async fn issue_112() { + #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + a: bool, + b: bool, + } + + assert_panic_on_fetch!( + &["Data.a", "Nullable(Bool)", "bool"], + "WITH (SELECT true) AS a, (SELECT true) AS b SELECT ?fields" + ); +} + +/// See https://github.com/ClickHouse/clickhouse-rs/issues/113 +#[tokio::test] +async fn issue_113() { + #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + a: u64, + b: f64, + c: f64, + } + let client = prepare_database!(); + execute_statements(&client, &[ + " + CREATE TABLE issue_113_1( + id UInt32 + ) + ENGINE MergeTree + ORDER BY id + ", + " + CREATE TABLE issue_113_2( + id UInt32, + pos Float64 + ) + ENGINE MergeTree + ORDER BY id + ", + "INSERT INTO issue_113_1 VALUES (1), (2), (3), (4), (5), (6), (7), (8), (9), (10)", + "INSERT INTO issue_113_2 VALUES (1, 100.5), (2, 200.2), (3, 300.3), (4, 444.4), (5, 555.5)", + ]).await; + + // Struct should have had Option instead of f64 + assert_panic_on_fetch_with_client!( + client, + &["Data.b", "Nullable(Float64)", "f64"], + " + SELECT + COUNT(*) AS a, + (COUNT(*) / (SELECT COUNT(*) FROM issue_113_1)) * 100.0 AS b, + AVG(pos) AS c + FROM issue_113_2 + " + ); +} + +/// See https://github.com/ClickHouse/clickhouse-rs/issues/114 +#[tokio::test] +#[cfg(feature = "time")] +async fn issue_114() { + #[derive(Row, Deserialize, Debug, PartialEq)] + struct Data { + #[serde(with = "clickhouse::serde::time::date")] + date: time::Date, + arr: Vec>, + } + + let client = get_client(); + let result = client + .query( + " + SELECT + '2023-05-01' :: Date AS date, + array(map('k1', 'v1'), map('k2', 'v2')) :: Array(Map(String, String)) AS arr + ", + ) + .fetch_one::() + .await; + + assert_eq!( + result.unwrap(), + Data { + date: time::Date::from_calendar_date(2023, time::Month::May, 1).unwrap(), + arr: vec![ + HashMap::from([("k1".to_owned(), "v1".to_owned())]), + HashMap::from([("k2".to_owned(), "v2".to_owned())]), + ], + } + ); +} + +/// See https://github.com/ClickHouse/clickhouse-rs/issues/173 +#[tokio::test] +#[cfg(feature = "time")] +async fn issue_173() { + #[derive(Debug, Serialize, Deserialize, Row)] + struct Data { + log_id: String, + #[serde(with = "clickhouse::serde::time::datetime")] + ts: time::OffsetDateTime, + } + + let client = prepare_database!().with_option("date_time_input_format", "best_effort"); + + execute_statements(&client, &[ + " + CREATE OR REPLACE TABLE logs ( + log_id String, + timestamp DateTime('Europe/Berlin') + ) + ENGINE = MergeTree() + PRIMARY KEY (log_id, timestamp) + ", + "INSERT INTO logs VALUES ('56cde52f-5f34-45e0-9f08-79d6f582e913', '2024-11-05T11:52:52+01:00')", + "INSERT INTO logs VALUES ('0e967129-6271-44f2-967b-0c8d11a60fdc', '2024-11-05T11:59:21+01:00')", + ]).await; + + // panics as we fetch `ts` two times: one from `?fields` macro, and the second time explicitly + // the resulting dataset will, in fact, contain 3 columns instead of 2: + assert_panic_on_fetch_with_client!( + client, + &["3 columns", "2 fields"], + "SELECT ?fields, toUnixTimestamp(timestamp) AS ts FROM logs ORDER by ts DESC" + ); +} + +/// See https://github.com/ClickHouse/clickhouse-rs/issues/185 +#[tokio::test] +async fn issue_185() { + #[derive(Row, Deserialize, Debug, PartialEq)] + struct Data { + pk: u32, + decimal_col: Option, + } + + let client = prepare_database!(); + execute_statements( + &client, + &[ + " + CREATE TABLE issue_185( + pk UInt32, + decimal_col Nullable(Decimal(10, 4))) + ENGINE MergeTree + ORDER BY pk + ", + "INSERT INTO issue_185 VALUES (1, 1.1), (2, 2.2), (3, 3.3)", + ], + ) + .await; + + assert_panic_on_fetch_with_client!( + client, + &["Data.decimal_col", "Decimal(10, 4)", "String"], + "SELECT ?fields FROM issue_185" + ); +} + +#[tokio::test] +#[cfg(feature = "chrono")] +async fn issue_218() { + #[derive(Row, Serialize, Deserialize, Debug)] + struct Data { + max_time: chrono::DateTime, + } + + let client = prepare_database!(); + execute_statements( + &client, + &[" + CREATE TABLE IF NOT EXISTS issue_218 ( + my_time DateTime64(3, 'UTC') CODEC(Delta, ZSTD), + ) ENGINE = MergeTree + ORDER BY my_time + "], + ) + .await; + + // FIXME: It is not a super clear panic as it hints about `&str`, + // and not about the missing attribute for `chrono::DateTime`. + // Still better than a `premature end of input` error, though. + assert_panic_on_fetch_with_client!( + client, + &["Data.max_time", "DateTime64(3, 'UTC')", "&str"], + "SELECT max(my_time) AS max_time FROM issue_218" + ); +} + +#[tokio::test] +async fn variant_wrong_definition() { + #[derive(Debug, Deserialize, PartialEq)] + enum MyVariant { + Str(String), + U32(u32), + } + + #[derive(Debug, Row, Deserialize, PartialEq)] + struct Data { + id: u8, + var: MyVariant, + } + + let client = get_client().with_option("allow_experimental_variant_type", "1"); + + assert_panic_on_fetch_with_client!( + client, + &["Data.var", "Variant(String, UInt16)", "u32"], + " + SELECT * FROM ( + SELECT 0 :: UInt8 AS id, 'foo' :: Variant(String, UInt16) AS var + UNION ALL + SELECT 1 :: UInt8 AS id, 144 :: Variant(String, UInt16) AS var + ) ORDER BY id ASC + " + ); +} + +#[tokio::test] +async fn decimal32_wrong_size() { + #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + decimal32: i16, + } + + assert_panic_on_fetch!( + &["Data.decimal32", "Decimal(9, 4)", "i16"], + "SELECT 42 :: Decimal32(4) AS decimal32" + ); +} + +#[tokio::test] +async fn decimal64_wrong_size() { + #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + decimal64: i32, + } + + assert_panic_on_fetch!( + &["Data.decimal64", "Decimal(18, 8)", "i32"], + "SELECT 144 :: Decimal64(8) AS decimal64" + ); +} + +#[tokio::test] +async fn decimal128_wrong_size() { + #[derive(Debug, Row, Serialize, Deserialize, PartialEq)] + struct Data { + decimal128: i64, + } + + assert_panic_on_fetch!( + &["Data.decimal128", "Decimal(38, 12)", "i64"], + "SELECT -17014118346046923173168730.37158841057 :: Decimal128(12) AS decimal128" + ); +} diff --git a/tests/it/time.rs b/tests/it/time.rs index 6f729b24..975d2a19 100644 --- a/tests/it/time.rs +++ b/tests/it/time.rs @@ -80,7 +80,7 @@ async fn datetime() { dt64ns_opt: Some(datetime!(2022-11-13 15:27:42.123456789 UTC)), }; - let mut insert = client.insert::("test").unwrap(); + let mut insert = client.insert::("test").await.unwrap(); insert.write(&original_row).await.unwrap(); insert.end().await.unwrap(); @@ -138,7 +138,7 @@ async fn date() { .await .unwrap(); - let mut insert = client.insert::("test").unwrap(); + let mut insert = client.insert::("test").await.unwrap(); let dates = generate_dates(1970..2149, 100); for &date in &dates { @@ -191,7 +191,7 @@ async fn date32() { .await .unwrap(); - let mut insert = client.insert::("test").unwrap(); + let mut insert = client.insert::("test").await.unwrap(); let dates = generate_dates(1925..2283, 100); // TODO: 1900..=2299 for newer versions. for &date in &dates { @@ -267,7 +267,7 @@ async fn time_roundtrip() { t1: Some(positive_value), }; - let mut insert = client.insert::("test_time").unwrap(); + let mut insert = client.insert::("test_time").await.unwrap(); insert.write(&row).await.unwrap(); insert.end().await.unwrap(); @@ -313,7 +313,7 @@ async fn time_negative_roundtrip() { t1: Some(negative_value), }; - let mut insert = client.insert::("test_time_negative").unwrap(); + let mut insert = client.insert::("test_time_negative").await.unwrap(); insert.write(&row).await.unwrap(); insert.end().await.unwrap(); @@ -372,7 +372,7 @@ async fn time64_roundtrip() { t9: time::Duration::nanoseconds(base_seconds * 1_000_000_000 + 789_123_456), }; - let mut insert = client.insert::("test_time64").unwrap(); + let mut insert = client.insert::("test_time64").await.unwrap(); insert.write(&row).await.unwrap(); insert.end().await.unwrap(); @@ -431,7 +431,10 @@ async fn time64_negative_roundtrip() { t9: time::Duration::nanoseconds(neg_base_seconds * 1_000_000_000 - 123_456_789), }; - let mut insert = client.insert::("test_time64_negative").unwrap(); + let mut insert = client + .insert::("test_time64_negative") + .await + .unwrap(); insert.write(&row).await.unwrap(); insert.end().await.unwrap(); diff --git a/tests/it/user_agent.rs b/tests/it/user_agent.rs index 125edc2c..4b439ffc 100644 --- a/tests/it/user_agent.rs +++ b/tests/it/user_agent.rs @@ -40,7 +40,7 @@ async fn assert_queries_user_agents(client: &Client, table_name: &str, expected_ create_simple_table(client, table_name).await; - let mut insert = client.insert::(table_name).unwrap(); + let mut insert = client.insert::(table_name).await.unwrap(); insert.write(&row).await.unwrap(); insert.end().await.unwrap(); diff --git a/tests/it/uuid.rs b/tests/it/uuid.rs index 154f2e91..a6f08652 100644 --- a/tests/it/uuid.rs +++ b/tests/it/uuid.rs @@ -38,7 +38,7 @@ async fn smoke() { uuid_opt: Some(uuid), }; - let mut insert = client.insert::("test").unwrap(); + let mut insert = client.insert::("test").await.unwrap(); insert.write(&original_row).await.unwrap(); insert.end().await.unwrap(); diff --git a/tests/it/variant.rs b/tests/it/variant.rs index 235e3b4e..3ceeaa76 100644 --- a/tests/it/variant.rs +++ b/tests/it/variant.rs @@ -90,7 +90,7 @@ async fn variant_data_type() { let rows = vars.map(|var| MyRow { var }); // Write to the table. - let mut insert = client.insert::("test_var").unwrap(); + let mut insert = client.insert::("test_var").await.unwrap(); for row in &rows { insert.write(row).await.unwrap(); } diff --git a/types/src/lib.rs b/types/src/lib.rs index 9271783b..ced28698 100644 --- a/types/src/lib.rs +++ b/types/src/lib.rs @@ -70,8 +70,9 @@ pub fn put_rbwnat_columns_header( buffer.put_slice(column.name.as_bytes()); } for column in columns.iter() { - put_leb128(&mut buffer, column.data_type.to_string().len() as u64); - buffer.put_slice(column.data_type.to_string().as_bytes()); + let data_type = column.data_type.to_string(); + put_leb128(&mut buffer, data_type.len() as u64); + buffer.put_slice(data_type.as_bytes()); } Ok(()) }