diff --git a/Cargo.toml b/Cargo.toml index 1bf6b88..fada7c1 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -57,6 +57,14 @@ required-features = ["time", "uuid", "chrono"] name = "data_types_variant" required-features = ["time"] +[[example]] +name = "stream_into_file" +required-features = ["futures03"] + +[[example]] +name = "stream_arbitrary_format_rows" +required-features = ["futures03"] + [profile.release] debug = true @@ -70,6 +78,8 @@ uuid = ["dep:uuid"] time = ["dep:time"] lz4 = ["dep:lz4_flex", "dep:cityhash-rs"] chrono = ["dep:chrono"] +futures03 = [] + ## TLS native-tls = ["dep:hyper-tls"] # ext: native-tls-alpn diff --git a/examples/README.md b/examples/README.md index f3724fc..67f1783 100644 --- a/examples/README.md +++ b/examples/README.md @@ -28,6 +28,8 @@ If something is missing, or you found a mistake in one of these examples, please - [custom_http_headers.rs](custom_http_headers.rs) - setting additional HTTP headers to the client, or overriding the generated ones - [query_id.rs](query_id.rs) - setting a specific `query_id` on the query level - [session_id.rs](session_id.rs) - using the client in the session context with temporary tables +- [stream_into_file.rs](stream_into_file.rs) - streaming the query result as raw bytes into a file in an arbitrary format. Required cargo features: `futures03`. +- [stream_arbitrary_format_rows.rs](stream_arbitrary_format_rows.rs) - streaming the query result in an arbitrary format, row by row. Required cargo features: `futures03`. ## How to run diff --git a/examples/stream_arbitrary_format_rows.rs b/examples/stream_arbitrary_format_rows.rs new file mode 100644 index 0000000..46d4ce3 --- /dev/null +++ b/examples/stream_arbitrary_format_rows.rs @@ -0,0 +1,38 @@ +use futures::{AsyncBufReadExt, StreamExt}; + +use clickhouse::Client; + +/// An example of streaming raw data row-by-row in an arbitrary format +/// leveraging the [`AsyncBufReadExt`] and [`StreamExt`] helpers. +/// In this case, the format is JSONEachRow. +/// Similarly, it can be used with other formats such as CSV, TSV, +/// and others that produce each row on a new line; +/// the only difference will be in how the data is parsed. +/// See also: https://clickhouse.com/docs/en/interfaces/formats + +#[tokio::main] +async fn main() -> clickhouse::error::Result<()> { + let client = Client::default().with_url("http://localhost:8123"); + let mut lines_cursor = client + .query( + " + SELECT number, hex(randomPrintableASCII(20)) AS hex_str + FROM system.numbers + LIMIT 100 + ", + ) + .fetch_bytes("JSONEachRow")? + .lines(); + + while let Some(data) = lines_cursor.next().await { + match data { + Ok(line_bytes) => { + let value = serde_json::json!(&line_bytes); + println!("JSONEachRow value: {}", value); + } + Err(err) => return Err(clickhouse::error::Error::Custom(err.to_string())), + } + } + + Ok(()) +} diff --git a/examples/stream_into_file.rs b/examples/stream_into_file.rs new file mode 100644 index 0000000..ff03f3b --- /dev/null +++ b/examples/stream_into_file.rs @@ -0,0 +1,37 @@ +use clickhouse::Client; +use futures::StreamExt; +use tokio::fs::File; +use tokio::io::AsyncWriteExt; + +/// An example of streaming the result of a query in an arbitrary format into a file +/// leveraging the [`StreamExt`] helper. In this case, `CSVWithNamesAndTypes` format is used. +/// See also: https://clickhouse.com/docs/en/interfaces/formats + +#[tokio::main] +async fn main() -> clickhouse::error::Result<()> { + let client = Client::default().with_url("http://localhost:8123"); + let filename = "output.csv"; + let mut bytes_cursor = client + .query( + " + SELECT number, hex(randomPrintableASCII(20)) AS hex_str + FROM system.numbers + LIMIT 10000 + ", + ) + .fetch_bytes("CSVWithNamesAndTypes")?; + + let mut file = File::create(filename).await.unwrap(); + while let Some(data) = bytes_cursor.next().await { + match data { + Ok(bytes) => { + file.write_all(&bytes).await.unwrap(); + println!("Bytes written: {}", bytes.len()); + } + Err(err) => return Err(err), + } + } + + println!("Raw data is written to {filename}"); + Ok(()) +} diff --git a/src/cursor.rs b/src/cursor.rs index b207433..b906ac7 100644 --- a/src/cursor.rs +++ b/src/cursor.rs @@ -1,21 +1,22 @@ -use std::marker::PhantomData; - -use bytes::Bytes; -use futures::TryStreamExt; -use serde::Deserialize; - use crate::{ bytes_ext::BytesExt, error::{Error, Result}, response::{Chunks, Response, ResponseFuture}, rowbinary, }; +use bytes::Bytes; +use futures::TryStreamExt; +use serde::Deserialize; +use std::marker::PhantomData; + +#[cfg(feature = "futures03")] +const INITIAL_BYTES_CURSOR_BUFFER_SIZE: usize = 1024; // === RawCursor === -struct RawCursor(RawCursorInner); +struct RawCursor(RawCursorState); -enum RawCursorInner { +enum RawCursorState { Waiting(ResponseFuture), Loading(RawCursorLoading), } @@ -27,18 +28,18 @@ struct RawCursorLoading { } impl RawCursor { - fn new(response: Response) -> Self { - Self(RawCursorInner::Waiting(response.into_future())) + pub(crate) fn new(response: Response) -> Self { + Self(RawCursorState::Waiting(response.into_future())) } async fn next(&mut self) -> Result> { - if matches!(self.0, RawCursorInner::Waiting(_)) { + if matches!(self.0, RawCursorState::Waiting(_)) { self.resolve().await?; } let state = match &mut self.0 { - RawCursorInner::Loading(state) => state, - RawCursorInner::Waiting(_) => unreachable!(), + RawCursorState::Loading(state) => state, + RawCursorState::Waiting(_) => unreachable!(), }; match state.chunks.try_next().await { @@ -53,9 +54,9 @@ impl RawCursor { } async fn resolve(&mut self) -> Result<()> { - if let RawCursorInner::Waiting(future) = &mut self.0 { + if let RawCursorState::Waiting(future) = &mut self.0 { let chunks = future.await; - self.0 = RawCursorInner::Loading(RawCursorLoading { + self.0 = RawCursorState::Loading(RawCursorLoading { chunks: chunks?, net_size: 0, data_size: 0, @@ -66,15 +67,124 @@ impl RawCursor { fn received_bytes(&self) -> u64 { match &self.0 { - RawCursorInner::Waiting(_) => 0, - RawCursorInner::Loading(state) => state.net_size, + RawCursorState::Waiting(_) => 0, + RawCursorState::Loading(state) => state.net_size, } } fn decoded_bytes(&self) -> u64 { match &self.0 { - RawCursorInner::Waiting(_) => 0, - RawCursorInner::Loading(state) => state.data_size, + RawCursorState::Waiting(_) => 0, + RawCursorState::Loading(state) => state.data_size, + } + } +} + +// === BytesCursor === + +/// Unlike [`RowCursor`] which emits rows deserialized as structures from RowBinary, +/// this cursor emits raw bytes without deserialization. +/// It can be iterated over using [`futures::StreamExt::next`]. +/// Additionally, if the requested format emits each row on a newline +/// (e.g. `JSONEachRow`, `CSV`, `TSV`, etc.), the cursor can be converted into a stream of lines +/// using [`futures::AsyncBufReadExt::lines`]`, which will allow for more convenient processing. +#[cfg(feature = "futures03")] +pub struct BytesCursor { + raw: RawCursor, + buffer: Vec, + filled: usize, + consumed: usize, +} + +#[cfg(feature = "futures03")] +impl BytesCursor { + pub(crate) fn new(response: Response) -> Self { + Self { + raw: RawCursor::new(response), + buffer: Vec::with_capacity(INITIAL_BYTES_CURSOR_BUFFER_SIZE), + filled: 0, + consumed: 0, + } + } +} + +#[cfg(feature = "futures03")] +impl futures::stream::Stream for BytesCursor { + type Item = Result; + + fn poll_next( + mut self: std::pin::Pin<&mut Self>, + cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + use futures::FutureExt; + match self.raw.next().boxed().poll_unpin(cx) { + std::task::Poll::Ready(result) => match result { + Ok(Some(bytes)) => std::task::Poll::Ready(Some(Ok(bytes))), + Ok(None) => std::task::Poll::Ready(None), + Err(err) => std::task::Poll::Ready(Some(Err(err))), + }, + std::task::Poll::Pending => std::task::Poll::Pending, + } + } +} + +#[cfg(feature = "futures03")] +impl futures::AsyncRead for BytesCursor { + fn poll_read( + mut self: std::pin::Pin<&mut Self>, + cx: &mut std::task::Context<'_>, + buf: &mut [u8], + ) -> std::task::Poll> { + use futures::FutureExt; + let buf = match buf.len() { + 0 => return std::task::Poll::Ready(Ok(0)), + len => &mut buf[..len], + }; + + match self.raw.next().boxed().poll_unpin(cx) { + std::task::Poll::Ready(Ok(Some(bytes))) => { + let len = std::cmp::min(bytes.len(), buf.len()); + buf[..len].copy_from_slice(&bytes[..len]); + std::task::Poll::Ready(Ok(len)) + } + std::task::Poll::Ready(Ok(None)) => std::task::Poll::Ready(Ok(0)), + std::task::Poll::Ready(Err(err)) => std::task::Poll::Ready(Err(err.into_io())), + std::task::Poll::Pending => std::task::Poll::Pending, + } + } +} + +#[cfg(feature = "futures03")] +impl futures::AsyncBufRead for BytesCursor { + fn poll_fill_buf( + self: std::pin::Pin<&mut Self>, + cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + use futures::FutureExt; + let this = self.get_mut(); + if this.filled == this.consumed { + let poll_result = this.raw.next().boxed().poll_unpin(cx); + match poll_result { + std::task::Poll::Ready(Ok(Some(bytes))) => { + this.filled = bytes.len(); + this.buffer.extend(bytes); + } + std::task::Poll::Ready(Ok(None)) => return std::task::Poll::Ready(Ok(&[])), + std::task::Poll::Ready(Err(err)) => { + return std::task::Poll::Ready(Err(err.into_io())) + } + std::task::Poll::Pending => return std::task::Poll::Pending, + } + } + std::task::Poll::Ready(Ok(&this.buffer[this.consumed..])) + } + + fn consume(mut self: std::pin::Pin<&mut Self>, amt: usize) { + self.consumed += amt; + if self.consumed == self.filled { + self.filled = 0; + self.consumed = 0; + self.buffer.clear(); } } } @@ -88,7 +198,7 @@ fn workaround_51132<'a, T: ?Sized>(ptr: &T) -> &'a T { // === RowCursor === -/// A cursor that emits rows. +/// A cursor that emits rows deserialized as structures from RowBinary. #[must_use] pub struct RowCursor { raw: RawCursor, @@ -107,7 +217,7 @@ impl RowCursor { /// Emits the next row. /// - /// An result is unspecified if it's called after `Err` is returned. + /// The result is unspecified if it's called after `Err` is returned. pub async fn next<'a, 'b: 'a>(&'a mut self) -> Result> where T: Deserialize<'b>, diff --git a/src/query.rs b/src/query.rs index eba1915..a196d95 100644 --- a/src/query.rs +++ b/src/query.rs @@ -15,6 +15,8 @@ use crate::{ const MAX_QUERY_LEN_TO_USE_GET: usize = 8192; +#[cfg(feature = "futures03")] +pub use crate::cursor::BytesCursor; pub use crate::cursor::RowCursor; #[must_use] @@ -84,12 +86,22 @@ impl Query { /// ``` pub fn fetch(mut self) -> Result> { self.sql.bind_fields::(); - self.sql.append(" FORMAT RowBinary"); + self.sql.set_output_format("RowBinary"); let response = self.do_execute(true)?; Ok(RowCursor::new(response)) } + /// Executes the query, returning a [`crate::cursor::BytesCursor`] + /// to obtain results as raw bytes. + /// For available formats, see + #[cfg(feature = "futures03")] + pub fn fetch_bytes(mut self, format: impl Into) -> Result { + self.sql.set_output_format(format); + let response = self.do_execute(true)?; + Ok(crate::cursor::BytesCursor::new(response)) + } + /// Executes the query and returns just a single row. /// /// Note that `T` must be owned. diff --git a/src/sql/mod.rs b/src/sql/mod.rs index 66330f6..d4a7e3b 100644 --- a/src/sql/mod.rs +++ b/src/sql/mod.rs @@ -13,7 +13,7 @@ pub(crate) mod ser; #[derive(Debug, Clone)] pub(crate) enum SqlBuilder { - InProgress(Vec), + InProgress(Vec, Option), Failed(String), } @@ -21,7 +21,6 @@ pub(crate) enum SqlBuilder { pub(crate) enum Part { Arg, Fields, - Str(&'static str), Text(String), } @@ -29,15 +28,17 @@ pub(crate) enum Part { impl fmt::Display for SqlBuilder { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { match self { - SqlBuilder::InProgress(parts) => { + SqlBuilder::InProgress(parts, output_format_opt) => { for part in parts { match part { Part::Arg => f.write_char('?')?, Part::Fields => f.write_str("?fields")?, - Part::Str(text) => f.write_str(text)?, Part::Text(text) => f.write_str(text)?, } } + if let Some(output_format) = output_format_opt { + f.write_str(&format!(" FORMAT {output_format}"))? + } } SqlBuilder::Failed(err) => f.write_str(err)?, } @@ -71,11 +72,17 @@ impl SqlBuilder { parts.push(Part::Text(rest.to_string())); } - SqlBuilder::InProgress(parts) + SqlBuilder::InProgress(parts, None) + } + + pub(crate) fn set_output_format(&mut self, format: impl Into) { + if let Self::InProgress(_, format_opt) = self { + *format_opt = Some(format.into()); + } } pub(crate) fn bind_arg(&mut self, value: impl Bind) { - let Self::InProgress(parts) = self else { + let Self::InProgress(parts, _) = self else { return; }; @@ -93,7 +100,7 @@ impl SqlBuilder { } pub(crate) fn bind_fields(&mut self) { - let Self::InProgress(parts) = self else { + let Self::InProgress(parts, _) = self else { return; }; @@ -106,21 +113,12 @@ impl SqlBuilder { } } - pub(crate) fn append(&mut self, suffix: &'static str) { - let Self::InProgress(parts) = self else { - return; - }; - - parts.push(Part::Str(suffix)); - } - pub(crate) fn finish(mut self) -> Result { let mut sql = String::new(); - if let Self::InProgress(parts) = &self { + if let Self::InProgress(parts, _) = &self { for part in parts { match part { - Part::Str(text) => sql.push_str(text), Part::Text(text) => sql.push_str(text), Part::Arg => { self.error("unbound query argument"); @@ -135,7 +133,12 @@ impl SqlBuilder { } match self { - Self::InProgress(_) => Ok(sql), + Self::InProgress(_, output_format_opt) => { + if let Some(output_format) = output_format_opt { + sql.push_str(&format!(" FORMAT {output_format}")) + } + Ok(sql) + } Self::Failed(err) => Err(Error::InvalidParams(err.into())), } } diff --git a/src/ticks.rs b/src/ticks.rs index 6a643a0..b7c0021 100644 --- a/src/ticks.rs +++ b/src/ticks.rs @@ -46,7 +46,7 @@ impl Ticks { } pub(crate) fn reached(&self) -> bool { - self.next_at.map_or(false, |n| Instant::now() >= n) + self.next_at.is_some_and(|n| Instant::now() >= n) } pub(crate) fn reschedule(&mut self) { diff --git a/tests/it/fetch_bytes.rs b/tests/it/fetch_bytes.rs new file mode 100644 index 0000000..42b6012 --- /dev/null +++ b/tests/it/fetch_bytes.rs @@ -0,0 +1,100 @@ +#![cfg(feature = "futures03")] + +use futures::{AsyncBufReadExt, StreamExt}; +use std::str::from_utf8; + +#[tokio::test] +async fn fetch_bytes_single_chunk() { + let client = prepare_database!(); + + let mut raw_cursor = client + .query("SELECT number FROM system.numbers LIMIT 3") + .fetch_bytes("CSV") + .unwrap(); + + let mut total_chunks = 0; + let mut buffer = Vec::::new(); + while let Some(chunk) = raw_cursor.next().await { + buffer.extend(chunk.unwrap()); + total_chunks += 1; + } + + assert_eq!(from_utf8(&buffer).unwrap(), "0\n1\n2\n",); + assert_eq!(total_chunks, 1); +} + +#[tokio::test] +async fn fetch_bytes_multiple_chunks() { + let client = prepare_database!(); + + let mut bytes_cursor = client + .query("SELECT number FROM system.numbers LIMIT 3") + // each number will go into a separate chunk + .with_option("max_block_size", "1") + .fetch_bytes("CSV") + .unwrap(); + + let mut total_chunks = 0; + let mut buffer = Vec::::new(); + while let Some(data) = bytes_cursor.next().await { + buffer.extend(data.unwrap()); + total_chunks += 1; + } + + assert_eq!(from_utf8(&buffer).unwrap(), "0\n1\n2\n",); + assert_eq!(total_chunks, 3); +} + +#[tokio::test] +async fn fetch_bytes_with_error() { + let client = prepare_database!(); + + let mut bytes_cursor = client + .query("SELECT sleepEachRow(0.05) AS s FROM system.numbers LIMIT 30") + .with_option("max_block_size", "1") + .with_option("max_execution_time", "0.01") + .fetch_bytes("JSONEachRow") + .unwrap(); + + let err = bytes_cursor.next().await.unwrap(); + println!("{:?}", err); + assert!(matches!(err, Err(clickhouse::error::Error::BadResponse(_)))); +} + +#[tokio::test] +async fn fetch_bytes_lines_single_chunk() { + let client = prepare_database!(); + + let mut lines_cursor = client + .query("SELECT number FROM system.numbers LIMIT 3") + .fetch_bytes("CSV") + .unwrap() + .lines(); + + let mut buffer = Vec::::new(); + while let Some(data) = lines_cursor.next().await { + buffer.push(data.unwrap()); + } + + assert_eq!(buffer, vec!["0", "1", "2"],); +} + +#[tokio::test] +async fn fetch_bytes_lines_multiple_chunks() { + let client = prepare_database!(); + + let mut lines_cursor = client + .query("SELECT number FROM system.numbers LIMIT 3") + // each number will go into a separate chunk + .with_option("max_block_size", "1") + .fetch_bytes("CSV") + .unwrap() + .lines(); + + let mut buffer = Vec::::new(); + while let Some(data) = lines_cursor.next().await { + buffer.push(data.unwrap()); + } + + assert_eq!(buffer, vec!["0", "1", "2"],); +} diff --git a/tests/it/main.rs b/tests/it/main.rs index c35a6cc..5e0385d 100644 --- a/tests/it/main.rs +++ b/tests/it/main.rs @@ -58,6 +58,7 @@ mod chrono; mod compression; mod cursor_error; mod cursor_stats; +mod fetch_bytes; mod insert; mod inserter; mod ip;