diff --git a/CHANGELOG.md b/CHANGELOG.md
index cb1f42e..8511848 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -7,6 +7,10 @@ and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0
## [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
diff --git a/Cargo.toml b/Cargo.toml
index 51d71b2..b1d23fd 100644
--- a/Cargo.toml
+++ b/Cargo.toml
@@ -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
diff --git a/README.md b/README.md
index 3f2bf71..d6f58e3 100644
--- a/README.md
+++ b/README.md
@@ -440,13 +440,36 @@ How to choose between all these features? Here are some considerations:
}
```
+* `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)`:
+
+ Example
+
+ ```rust,ignore
+ #[derive(Serialize, Deserialize)]
+ enum MyRowVariant {
+ Array(Vec),
+ 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,
+ }
+ ```
+
* [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.
diff --git a/examples/README.md b/examples/README.md
index d1b128a..f3724fc 100644
--- a/examples/README.md
+++ b/examples/README.md
@@ -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
diff --git a/examples/data_types_variant.rs b/examples/data_types_variant.rs
new file mode 100644
index 0000000..e575464
--- /dev/null
+++ b/examples/data_types_variant.rs
@@ -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::()
+ .await?;
+
+ println!("{rows:#?}");
+ Ok(())
+}
+
+fn get_rows() -> Vec {
+ 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),
+ 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,
+}
diff --git a/src/error.rs b/src/error.rs
index 118b32b..0ab1d61 100644
--- a/src/error.rs
+++ b/src/error.rs
@@ -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);
diff --git a/src/rowbinary/de.rs b/src/rowbinary/de.rs
index b0d56c8..d911ded 100644
--- a/src/rowbinary/de.rs
+++ b/src/rowbinary/de.rs
@@ -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
@@ -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>(self, visitor: V) -> Result {
+ self.deserialize_u8(visitor)
+ }
+
#[inline]
fn deserialize_enum>(
self,
- name: &'static str,
+ _name: &'static str,
_variants: &'static [&'static str],
- _visitor: V,
+ visitor: V,
) -> Result {
- 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(self, seed: T) -> Result
+ where
+ T: DeserializeSeed<'data>,
+ {
+ DeserializeSeed::deserialize(seed, &mut *self.deserializer)
+ }
+
+ fn tuple_variant(self, len: usize, visitor: V) -> Result
+ where
+ V: Visitor<'data>,
+ {
+ self.deserializer.deserialize_tuple(len, visitor)
+ }
+
+ fn struct_variant(
+ self,
+ fields: &'static [&'static str],
+ visitor: V,
+ ) -> Result
+ 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(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]
@@ -222,11 +281,6 @@ impl<'cursor, 'data> Deserializer<'data> for &mut RowBinaryDeserializer<'cursor,
self.deserialize_tuple(fields.len(), visitor)
}
- #[inline]
- fn deserialize_identifier>(self, _visitor: V) -> Result {
- panic!("identifiers are unsupported");
- }
-
#[inline]
fn deserialize_newtype_struct>(
self,
diff --git a/src/rowbinary/ser.rs b/src/rowbinary/ser.rs
index 1191140..4fc604d 100644
--- a/src/rowbinary/ser.rs
+++ b/src/rowbinary/ser.rs
@@ -134,12 +134,26 @@ impl<'a, B: BufMut> Serializer for &'a mut RowBinarySerializer {
#[inline]
fn serialize_newtype_variant(
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]
diff --git a/tests/it/main.rs b/tests/it/main.rs
index 3efe24f..87995b4 100644
--- a/tests/it/main.rs
+++ b/tests/it/main.rs
@@ -66,6 +66,7 @@ mod query;
mod time;
mod user_agent;
mod uuid;
+mod variant;
mod watch;
const HOST: &str = "localhost:8123";
diff --git a/tests/it/variant.rs b/tests/it/variant.rs
new file mode 100644
index 0000000..14e8190
--- /dev/null
+++ b/tests/it/variant.rs
@@ -0,0 +1,108 @@
+#![cfg(feature = "time")]
+
+use serde::{Deserialize, Serialize};
+use time::Month::January;
+
+use clickhouse::Row;
+
+// See also: https://clickhouse.com/docs/en/sql-reference/data-types/variant
+
+#[tokio::test]
+async fn variant_data_type() {
+ let client = prepare_database!();
+
+ // NB: Inner Variant types are _always_ sorted alphabetically,
+ // and should be defined in _exactly_ the same order in the enum.
+ #[derive(Debug, PartialEq, Serialize, Deserialize)]
+ enum MyRowVariant {
+ Array(Vec),
+ Boolean(bool),
+ // attributes should work in this case, too
+ #[serde(with = "clickhouse::serde::time::date")]
+ Date(time::Date),
+ 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 {
+ var: MyRowVariant,
+ }
+
+ // No matter the order of the definition on the Variant types, it 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 test_var
+ (
+ `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 ()",
+ )
+ .with_option("allow_experimental_variant_type", "1")
+ .with_option("allow_suspicious_variant_types", "1")
+ .execute()
+ .await
+ .unwrap();
+
+ let vars = [
+ MyRowVariant::Array(vec![1, 2]),
+ MyRowVariant::Boolean(true),
+ MyRowVariant::Date(time::Date::from_calendar_date(2021, January, 1).unwrap()),
+ MyRowVariant::FixedString(*b"foobar"),
+ MyRowVariant::Float32(100.5),
+ MyRowVariant::Float64(200.1),
+ MyRowVariant::Int8(2),
+ MyRowVariant::Int16(3),
+ MyRowVariant::Int32(4),
+ MyRowVariant::Int64(5),
+ MyRowVariant::Int128(6),
+ MyRowVariant::String("my_string".to_string()),
+ MyRowVariant::UInt8(7),
+ MyRowVariant::UInt16(8),
+ MyRowVariant::UInt32(9),
+ MyRowVariant::UInt64(10),
+ MyRowVariant::UInt128(11),
+ ];
+
+ let rows = vars.map(|var| MyRow { var });
+
+ // Write to the table.
+ let mut insert = client.insert("test_var").unwrap();
+ for row in &rows {
+ insert.write(row).await.unwrap();
+ }
+ insert.end().await.unwrap();
+
+ // Read from the table.
+ let result_rows = client
+ .query("SELECT ?fields FROM test_var")
+ .fetch_all::()
+ .await
+ .unwrap();
+
+ assert_eq!(result_rows, rows)
+}