Skip to content

Commit

Permalink
feat: variant data type support (#170)
Browse files Browse the repository at this point in the history
  • Loading branch information
slvrtrn authored Nov 21, 2024
1 parent a9cb053 commit 9951588
Show file tree
Hide file tree
Showing 10 changed files with 398 additions and 16 deletions.
4 changes: 4 additions & 0 deletions CHANGELOG.md
Original file line number Diff line number Diff line change
Expand Up @@ -7,6 +7,10 @@ and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0
<!-- next-header -->

## [Unreleased] - ReleaseDate
### Added
- [Variant data type](https://clickhouse.com/docs/en/sql-reference/data-types/variant) support ([#170]).

[#170]: https://github.com/ClickHouse/clickhouse-rs/pull/170

## [0.13.1] - 2024-10-21
### Added
Expand Down
4 changes: 4 additions & 0 deletions Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,10 @@ required-features = ["rustls-tls"]
name = "data_types_derive_simple"
required-features = ["time", "uuid"]

[[example]]
name = "data_types_variant"
required-features = ["time"]

[profile.release]
debug = true

Expand Down
25 changes: 24 additions & 1 deletion README.md
Original file line number Diff line number Diff line change
Expand Up @@ -440,13 +440,36 @@ How to choose between all these features? Here are some considerations:
}
```
</details>
* `Variant` data type is supported as a Rust enum. As the inner Variant types are _always_ sorted alphabetically, Rust enum variants should be defined in the _exactly_ same order as it is in the data type; their names are irrelevant, only the order of the types matters. This following example has a column defined as `Variant(Array(UInt16), Bool, Date, String, UInt32)`:
<details>
<summary>Example</summary>
```rust,ignore
#[derive(Serialize, Deserialize)]
enum MyRowVariant {
Array(Vec<i16>),
Boolean(bool),
#[serde(with = "clickhouse::serde::time::date")]
Date(time::Date),
String(String),
UInt32(u32),
}
#[derive(Row, Serialize, Deserialize)]
struct MyRow {
id: u64,
var: MyRowVariant,
}
```
</details>
* [New `JSON` data type](https://clickhouse.com/docs/en/sql-reference/data-types/newjson) is currently supported as a string when using ClickHouse 24.10+. See [this example](examples/data_types_new_json.rs) for more details.
* `Variant`, `Dynamic` types are not supported for now.
* `Dynamic` data type is not supported for now.
See also the additional examples:
* [Simpler ClickHouse data types](examples/data_types_derive_simple.rs)
* [Container-like ClickHouse data types](examples/data_types_derive_containers.rs)
* [Variant data type](examples/data_types_variant.rs)
## Mocking
The crate provides utils for mocking CH server and testing DDL, `SELECT`, `INSERT` and `WATCH` queries.
Expand Down
1 change: 1 addition & 0 deletions examples/README.md
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ If something is missing, or you found a mistake in one of these examples, please

- [data_types_derive_simple.rs](data_types_derive_simple.rs) - deriving simpler ClickHouse data types in a struct. Required cargo features: `time`, `uuid`.
- [data_types_derive_containers.rs](data_types_derive_containers.rs) - deriving container-like (Array, Tuple, Map, Nested, Geo) ClickHouse data types in a struct.
- [data_types_variant.rs](data_types_variant.rs) - working with the [Variant data type](https://clickhouse.com/docs/en/sql-reference/data-types/variant).
- [data_types_new_json.rs](data_types_new_json.rs) - working with the [new JSON data type](https://clickhouse.com/docs/en/sql-reference/data-types/newjson) as a String.

### Special cases
Expand Down
169 changes: 169 additions & 0 deletions examples/data_types_variant.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,169 @@
use clickhouse_derive::Row;
use serde::{Deserialize, Serialize};

use clickhouse::sql::Identifier;
use clickhouse::{error::Result, Client};

// See also: https://clickhouse.com/docs/en/sql-reference/data-types/variant

#[tokio::main]
async fn main() -> Result<()> {
let table_name = "chrs_data_types_variant";
let client = Client::default().with_url("http://localhost:8123");

// No matter the order of the definition on the Variant types in the DDL, this particular Variant will always be sorted as follows:
// Variant(Array(UInt16), Bool, FixedString(6), Float32, Float64, Int128, Int16, Int32, Int64, Int8, String, UInt128, UInt16, UInt32, UInt64, UInt8)
client
.query(
"
CREATE OR REPLACE TABLE ?
(
`id` UInt64,
`var` Variant(
Array(UInt16),
Bool,
Date,
FixedString(6),
Float32, Float64,
Int128, Int16, Int32, Int64, Int8,
String,
UInt128, UInt16, UInt32, UInt64, UInt8
)
)
ENGINE = MergeTree
ORDER BY id",
)
.bind(Identifier(table_name))
.with_option("allow_experimental_variant_type", "1")
// This is required only if we are mixing similar types in the Variant definition
// In this case, this is various Int/UInt types, Float32/Float64, and String/FixedString
// Omit this option if there are no similar types in the definition
.with_option("allow_suspicious_variant_types", "1")
.execute()
.await?;

let mut insert = client.insert(table_name)?;
let rows_to_insert = get_rows();
for row in rows_to_insert {
insert.write(&row).await?;
}
insert.end().await?;

let rows = client
.query("SELECT ?fields FROM ?")
.bind(Identifier(table_name))
.fetch_all::<MyRow>()
.await?;

println!("{rows:#?}");
Ok(())
}

fn get_rows() -> Vec<MyRow> {
vec![
MyRow {
id: 1,
var: MyRowVariant::Array(vec![1, 2]),
},
MyRow {
id: 2,
var: MyRowVariant::Boolean(true),
},
MyRow {
id: 3,
var: MyRowVariant::Date(
time::Date::from_calendar_date(2021, time::Month::January, 1).unwrap(),
),
},
MyRow {
id: 4,
var: MyRowVariant::FixedString(*b"foobar"),
},
MyRow {
id: 5,
var: MyRowVariant::Float32(100.5),
},
MyRow {
id: 6,
var: MyRowVariant::Float64(200.1),
},
MyRow {
id: 7,
var: MyRowVariant::Int8(2),
},
MyRow {
id: 8,
var: MyRowVariant::Int16(3),
},
MyRow {
id: 9,
var: MyRowVariant::Int32(4),
},
MyRow {
id: 10,
var: MyRowVariant::Int64(5),
},
MyRow {
id: 11,
var: MyRowVariant::Int128(6),
},
MyRow {
id: 12,
var: MyRowVariant::String("my_string".to_string()),
},
MyRow {
id: 13,
var: MyRowVariant::UInt8(7),
},
MyRow {
id: 14,
var: MyRowVariant::UInt16(8),
},
MyRow {
id: 15,
var: MyRowVariant::UInt32(9),
},
MyRow {
id: 16,
var: MyRowVariant::UInt64(10),
},
MyRow {
id: 17,
var: MyRowVariant::UInt128(11),
},
]
}

// As the inner Variant types are _always_ sorted alphabetically,
// Rust enum variants should be defined in the _exactly_ same order as it is in the data type;
// their names are irrelevant, only the order of the types matters.
// This enum represents Variant(Array(UInt16), Bool, Date, FixedString(6), Float32, Float64, Int128, Int16, Int32, Int64, Int8, String, UInt128, UInt16, UInt32, UInt64, UInt8)
#[derive(Debug, PartialEq, Serialize, Deserialize)]
enum MyRowVariant {
Array(Vec<i16>),
Boolean(bool),
// attributes should work in this case, too
#[serde(with = "clickhouse::serde::time::date")]
Date(time::Date),
// NB: by default, fetched as raw bytes
FixedString([u8; 6]),
Float32(f32),
Float64(f64),
Int128(i128),
Int16(i16),
Int32(i32),
Int64(i64),
Int8(i8),
String(String),
UInt128(u128),
UInt16(i16),
UInt32(u32),
UInt64(u64),
UInt8(i8),
}

#[derive(Debug, PartialEq, Row, Serialize, Deserialize)]
struct MyRow {
id: u64,
var: MyRowVariant,
}
4 changes: 4 additions & 0 deletions src/error.rs
Original file line number Diff line number Diff line change
Expand Up @@ -32,12 +32,16 @@ pub enum Error {
InvalidUtf8Encoding(#[from] Utf8Error),
#[error("tag for enum is not valid")]
InvalidTagEncoding(usize),
#[error("max number of types in the Variant data type is 255, got {0}")]
VariantDiscriminatorIsOutOfBound(usize),
#[error("a custom error message from serde: {0}")]
Custom(String),
#[error("bad response: {0}")]
BadResponse(String),
#[error("timeout expired")]
TimedOut,
#[error("unsupported: {0}")]
Unsupported(String),
}

assert_impl_all!(Error: StdError, Send, Sync);
Expand Down
74 changes: 64 additions & 10 deletions src/rowbinary/de.rs
Original file line number Diff line number Diff line change
@@ -1,13 +1,13 @@
use std::{convert::TryFrom, mem, str};

use crate::error::{Error, Result};
use bytes::Buf;
use serde::de::{EnumAccess, VariantAccess};
use serde::{
de::{DeserializeSeed, Deserializer, SeqAccess, Visitor},
Deserialize,
};

use crate::error::{Error, Result};

/// Deserializes a value from `input` with a row encoded in `RowBinary`.
///
/// It accepts _a reference to_ a byte slice because it somehow leads to a more
Expand Down Expand Up @@ -146,14 +146,73 @@ impl<'cursor, 'data> Deserializer<'data> for &mut RowBinaryDeserializer<'cursor,
visitor.visit_byte_buf(self.read_vec(size)?)
}

#[inline]
fn deserialize_identifier<V: Visitor<'data>>(self, visitor: V) -> Result<V::Value> {
self.deserialize_u8(visitor)
}

#[inline]
fn deserialize_enum<V: Visitor<'data>>(
self,
name: &'static str,
_name: &'static str,
_variants: &'static [&'static str],
_visitor: V,
visitor: V,
) -> Result<V::Value> {
panic!("enums are unsupported: `{name}`");
struct Access<'de, 'cursor, 'data> {
deserializer: &'de mut RowBinaryDeserializer<'cursor, 'data>,
}
struct VariantDeserializer<'de, 'cursor, 'data> {
deserializer: &'de mut RowBinaryDeserializer<'cursor, 'data>,
}
impl<'data> VariantAccess<'data> for VariantDeserializer<'_, '_, 'data> {
type Error = Error;

fn unit_variant(self) -> Result<()> {
Err(Error::Unsupported("unit variants".to_string()))
}

fn newtype_variant_seed<T>(self, seed: T) -> Result<T::Value>
where
T: DeserializeSeed<'data>,
{
DeserializeSeed::deserialize(seed, &mut *self.deserializer)
}

fn tuple_variant<V>(self, len: usize, visitor: V) -> Result<V::Value>
where
V: Visitor<'data>,
{
self.deserializer.deserialize_tuple(len, visitor)
}

fn struct_variant<V>(
self,
fields: &'static [&'static str],
visitor: V,
) -> Result<V::Value>
where
V: Visitor<'data>,
{
self.deserializer.deserialize_tuple(fields.len(), visitor)
}
}

impl<'de, 'cursor, 'data> EnumAccess<'data> for Access<'de, 'cursor, 'data> {
type Error = Error;
type Variant = VariantDeserializer<'de, 'cursor, 'data>;

fn variant_seed<T>(self, seed: T) -> Result<(T::Value, Self::Variant), Self::Error>
where
T: DeserializeSeed<'data>,
{
let value = seed.deserialize(&mut *self.deserializer)?;
let deserializer = VariantDeserializer {
deserializer: self.deserializer,
};
Ok((value, deserializer))
}
}
visitor.visit_enum(Access { deserializer: self })
}

#[inline]
Expand Down Expand Up @@ -222,11 +281,6 @@ impl<'cursor, 'data> Deserializer<'data> for &mut RowBinaryDeserializer<'cursor,
self.deserialize_tuple(fields.len(), visitor)
}

#[inline]
fn deserialize_identifier<V: Visitor<'data>>(self, _visitor: V) -> Result<V::Value> {
panic!("identifiers are unsupported");
}

#[inline]
fn deserialize_newtype_struct<V: Visitor<'data>>(
self,
Expand Down
24 changes: 19 additions & 5 deletions src/rowbinary/ser.rs
Original file line number Diff line number Diff line change
Expand Up @@ -134,12 +134,26 @@ impl<'a, B: BufMut> Serializer for &'a mut RowBinarySerializer<B> {
#[inline]
fn serialize_newtype_variant<T: Serialize + ?Sized>(
self,
name: &'static str,
_variant_index: u32,
variant: &'static str,
_value: &T,
_name: &'static str,
variant_index: u32,
_variant: &'static str,
value: &T,
) -> Result<()> {
panic!("newtype variant types are unsupported: `{name}::{variant}`");
// 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 {
return Err(Error::VariantDiscriminatorIsOutOfBound(
variant_index as usize,
));
}
self.buffer.put_u8(variant_index as u8);
value.serialize(self)
}

#[inline]
Expand Down
1 change: 1 addition & 0 deletions tests/it/main.rs
Original file line number Diff line number Diff line change
Expand Up @@ -66,6 +66,7 @@ mod query;
mod time;
mod user_agent;
mod uuid;
mod variant;
mod watch;

const HOST: &str = "localhost:8123";
Expand Down
Loading

0 comments on commit 9951588

Please sign in to comment.