From b618eb3df25219e61a8f6a8fcbf715d9991a0819 Mon Sep 17 00:00:00 2001 From: Chengxu Bian Date: Wed, 6 Mar 2024 21:12:12 -0500 Subject: [PATCH 01/16] temp runtime --- crates/iceberg/src/lib.rs | 1 + crates/iceberg/src/runtime/mod.rs | 1 + crates/iceberg/src/runtime/spawn.rs | 11 +++ crates/iceberg/src/scan.rs | 130 ++++++++++++++-------------- 4 files changed, 79 insertions(+), 64 deletions(-) create mode 100644 crates/iceberg/src/runtime/mod.rs create mode 100644 crates/iceberg/src/runtime/spawn.rs diff --git a/crates/iceberg/src/lib.rs b/crates/iceberg/src/lib.rs index 9ceadcac8..911482c53 100644 --- a/crates/iceberg/src/lib.rs +++ b/crates/iceberg/src/lib.rs @@ -53,3 +53,4 @@ pub mod transaction; pub mod transform; pub mod writer; +mod runtime; diff --git a/crates/iceberg/src/runtime/mod.rs b/crates/iceberg/src/runtime/mod.rs new file mode 100644 index 000000000..f5384ed6e --- /dev/null +++ b/crates/iceberg/src/runtime/mod.rs @@ -0,0 +1 @@ +mod spawn; \ No newline at end of file diff --git a/crates/iceberg/src/runtime/spawn.rs b/crates/iceberg/src/runtime/spawn.rs new file mode 100644 index 000000000..1383af3bc --- /dev/null +++ b/crates/iceberg/src/runtime/spawn.rs @@ -0,0 +1,11 @@ +struct TaskHandle {} + +impl TaskHandle { + pub fn new() -> Self { + Self {} + } +} + +pub fn spawn() -> TaskHandle { + TaskHandle::new() +} \ No newline at end of file diff --git a/crates/iceberg/src/scan.rs b/crates/iceberg/src/scan.rs index 0a3b9a915..b42a18341 100644 --- a/crates/iceberg/src/scan.rs +++ b/crates/iceberg/src/scan.rs @@ -18,12 +18,14 @@ //! Table scan api. use crate::io::FileIO; -use crate::spec::{DataContentType, ManifestEntryRef, SchemaRef, SnapshotRef, TableMetadataRef}; +use crate::spec::{DataContentType, Manifest, ManifestEntryRef, SchemaRef, SnapshotRef, TableMetadataRef}; use crate::table::Table; use crate::{Error, ErrorKind}; use arrow_array::RecordBatch; +use futures::future::try_join_all; use futures::stream::{iter, BoxStream}; use futures::StreamExt; +use itertools::Itertools; /// Builder to create table scan. pub struct TableScanBuilder<'a> { @@ -49,7 +51,7 @@ impl<'a> TableScanBuilder<'a> { } /// Select some columns of the table. - pub fn select(mut self, column_names: impl IntoIterator) -> Self { + pub fn select(mut self, column_names: impl IntoIterator) -> Self { self.column_names = column_names .into_iter() .map(|item| item.to_string()) @@ -138,10 +140,11 @@ impl TableScan { // Generate data file stream let mut file_scan_tasks = Vec::with_capacity(manifest_list.entries().len()); - for manifest_list_entry in manifest_list.entries().iter() { - // Data file - let manifest = manifest_list_entry.load_manifest(&self.file_io).await?; + let tasks = manifest_list.entries().iter().map(|m| m.load_manifest(&self.file_io)).collect_vec(); + let manifests: Vec = try_join_all(tasks).await?; + + for manifest in manifests { for manifest_entry in manifest.entries().iter().filter(|e| e.is_alive()) { match manifest_entry.content_type() { DataContentType::EqualityDeletes | DataContentType::PositionDeletes => { @@ -160,7 +163,6 @@ impl TableScan { } } } - Ok(iter(file_scan_tasks).boxed()) } } @@ -223,7 +225,7 @@ mod tests { "{}/testdata/example_table_metadata_v2.json", env!("CARGO_MANIFEST_DIR") )) - .unwrap(); + .unwrap(); let mut context = Context::new(); context.insert("table_location", &table_location); context.insert("manifest_list_1_location", &manifest_list1_location); @@ -339,64 +341,64 @@ mod tests { current_snapshot.snapshot_id(), vec![], ) - .write(Manifest::new( - ManifestMetadata::builder() - .schema((*current_schema).clone()) - .content(ManifestContentType::Data) - .format_version(FormatVersion::V2) - .partition_spec((**current_partition_spec).clone()) - .schema_id(current_schema.schema_id()) - .build(), - vec![ - ManifestEntry::builder() - .status(ManifestStatus::Added) - .data_file( - DataFile::builder() - .content(DataContentType::Data) - .file_path(format!("{}/1.parquet", &fixture.table_location)) - .file_format(DataFileFormat::Parquet) - .file_size_in_bytes(100) - .record_count(1) - .partition(Struct::from_iter([Some(Literal::long(100))])) - .build(), - ) - .build(), - ManifestEntry::builder() - .status(ManifestStatus::Deleted) - .snapshot_id(parent_snapshot.snapshot_id()) - .sequence_number(parent_snapshot.sequence_number()) - .file_sequence_number(parent_snapshot.sequence_number()) - .data_file( - DataFile::builder() - .content(DataContentType::Data) - .file_path(format!("{}/2.parquet", &fixture.table_location)) - .file_format(DataFileFormat::Parquet) - .file_size_in_bytes(100) - .record_count(1) - .partition(Struct::from_iter([Some(Literal::long(200))])) - .build(), - ) - .build(), - ManifestEntry::builder() - .status(ManifestStatus::Existing) - .snapshot_id(parent_snapshot.snapshot_id()) - .sequence_number(parent_snapshot.sequence_number()) - .file_sequence_number(parent_snapshot.sequence_number()) - .data_file( - DataFile::builder() - .content(DataContentType::Data) - .file_path(format!("{}/3.parquet", &fixture.table_location)) - .file_format(DataFileFormat::Parquet) - .file_size_in_bytes(100) - .record_count(1) - .partition(Struct::from_iter([Some(Literal::long(300))])) - .build(), - ) + .write(Manifest::new( + ManifestMetadata::builder() + .schema((*current_schema).clone()) + .content(ManifestContentType::Data) + .format_version(FormatVersion::V2) + .partition_spec((**current_partition_spec).clone()) + .schema_id(current_schema.schema_id()) .build(), - ], - )) - .await - .unwrap(); + vec![ + ManifestEntry::builder() + .status(ManifestStatus::Added) + .data_file( + DataFile::builder() + .content(DataContentType::Data) + .file_path(format!("{}/1.parquet", &fixture.table_location)) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(100) + .record_count(1) + .partition(Struct::from_iter([Some(Literal::long(100))])) + .build(), + ) + .build(), + ManifestEntry::builder() + .status(ManifestStatus::Deleted) + .snapshot_id(parent_snapshot.snapshot_id()) + .sequence_number(parent_snapshot.sequence_number()) + .file_sequence_number(parent_snapshot.sequence_number()) + .data_file( + DataFile::builder() + .content(DataContentType::Data) + .file_path(format!("{}/2.parquet", &fixture.table_location)) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(100) + .record_count(1) + .partition(Struct::from_iter([Some(Literal::long(200))])) + .build(), + ) + .build(), + ManifestEntry::builder() + .status(ManifestStatus::Existing) + .snapshot_id(parent_snapshot.snapshot_id()) + .sequence_number(parent_snapshot.sequence_number()) + .file_sequence_number(parent_snapshot.sequence_number()) + .data_file( + DataFile::builder() + .content(DataContentType::Data) + .file_path(format!("{}/3.parquet", &fixture.table_location)) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(100) + .record_count(1) + .partition(Struct::from_iter([Some(Literal::long(300))])) + .build(), + ) + .build(), + ], + )) + .await + .unwrap(); // Write to manifest list let mut manifest_list_write = ManifestListWriter::v2( From 410b8bb8968af5ff71c9779aa70395fecdcc9995 Mon Sep 17 00:00:00 2001 From: Chengxu Bian Date: Wed, 6 Mar 2024 23:09:10 -0500 Subject: [PATCH 02/16] POC --- crates/iceberg/Cargo.toml | 7 + crates/iceberg/src/lib.rs | 3 +- .../iceberg/src/runtime/async_std_backend.rs | 12 ++ crates/iceberg/src/runtime/mod.rs | 36 ++++- crates/iceberg/src/runtime/spawn.rs | 11 -- crates/iceberg/src/runtime/tokio_backend.rs | 12 ++ crates/iceberg/src/scan.rs | 141 ++++++++++-------- crates/iceberg/src/spec/manifest_list.rs | 3 +- 8 files changed, 147 insertions(+), 78 deletions(-) create mode 100644 crates/iceberg/src/runtime/async_std_backend.rs delete mode 100644 crates/iceberg/src/runtime/spawn.rs create mode 100644 crates/iceberg/src/runtime/tokio_backend.rs diff --git a/crates/iceberg/Cargo.toml b/crates/iceberg/Cargo.toml index 32288ee81..0f0b715dd 100644 --- a/crates/iceberg/Cargo.toml +++ b/crates/iceberg/Cargo.toml @@ -28,12 +28,18 @@ repository = { workspace = true } license = { workspace = true } keywords = ["iceberg"] +[features] +async-std = ["dep:async-std"] +tokio = ["dep:tokio"] +default = ["tokio"] + [dependencies] anyhow = { workspace = true } apache-avro = { workspace = true } arrow-arith = { workspace = true } arrow-array = { workspace = true } arrow-schema = { workspace = true } +async-std = { version = "1.12.0", optional = true } async-trait = { workspace = true } bimap = { workspace = true } bitvec = { workspace = true } @@ -56,6 +62,7 @@ serde_derive = { workspace = true } serde_json = { workspace = true } serde_repr = { workspace = true } serde_with = { workspace = true } +tokio = { version = "1.36.0", optional = true } typed-builder = { workspace = true } url = { workspace = true } urlencoding = { workspace = true } diff --git a/crates/iceberg/src/lib.rs b/crates/iceberg/src/lib.rs index 911482c53..bbbc25431 100644 --- a/crates/iceberg/src/lib.rs +++ b/crates/iceberg/src/lib.rs @@ -52,5 +52,6 @@ pub mod expr; pub mod transaction; pub mod transform; -pub mod writer; mod runtime; + +pub mod writer; diff --git a/crates/iceberg/src/runtime/async_std_backend.rs b/crates/iceberg/src/runtime/async_std_backend.rs new file mode 100644 index 000000000..18e6c09c1 --- /dev/null +++ b/crates/iceberg/src/runtime/async_std_backend.rs @@ -0,0 +1,12 @@ +use crate::runtime::JoinHandler; +use std::future::Future; + +pub fn spawn(future: F) -> JoinHandler +where + F: Future + Send + 'static, + F::Output: Send + 'static, +{ + JoinHandler { + inner: async_std::task::spawn(future), + } +} diff --git a/crates/iceberg/src/runtime/mod.rs b/crates/iceberg/src/runtime/mod.rs index f5384ed6e..93c1ee22c 100644 --- a/crates/iceberg/src/runtime/mod.rs +++ b/crates/iceberg/src/runtime/mod.rs @@ -1 +1,35 @@ -mod spawn; \ No newline at end of file +use std::future::Future; +use std::pin::Pin; +use std::task::{Context, Poll}; + +#[cfg(feature = "tokio")] +pub mod tokio_backend; +#[cfg(feature = "tokio")] +pub use tokio_backend::*; + +#[cfg(all(feature = "async-std", not(feature = "tokio"),))] +pub mod async_std_backend; +#[cfg(all(feature = "async-std", not(feature = "tokio"),))] +pub use async_std_backend::*; + +pub struct JoinHandler { + #[cfg(feature = "tokio")] + inner: tokio::task::JoinHandle, + #[cfg(all(feature = "async-std", not(feature = "tokio"),))] + inner: async_std::task::JoinHandle, +} + +impl Future for JoinHandler { + type Output = T; + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + #[cfg(all(feature = "async-std", not(feature = "tokio")))] + { + Pin::new(&mut self.inner).poll(cx) + } + // Tokio returns a Poll> + #[cfg(feature = "tokio")] + Pin::new(&mut self.inner) + .poll(cx) + .map(|res| res.expect("tokio spawned task crashed")) + } +} diff --git a/crates/iceberg/src/runtime/spawn.rs b/crates/iceberg/src/runtime/spawn.rs deleted file mode 100644 index 1383af3bc..000000000 --- a/crates/iceberg/src/runtime/spawn.rs +++ /dev/null @@ -1,11 +0,0 @@ -struct TaskHandle {} - -impl TaskHandle { - pub fn new() -> Self { - Self {} - } -} - -pub fn spawn() -> TaskHandle { - TaskHandle::new() -} \ No newline at end of file diff --git a/crates/iceberg/src/runtime/tokio_backend.rs b/crates/iceberg/src/runtime/tokio_backend.rs new file mode 100644 index 000000000..2847940f6 --- /dev/null +++ b/crates/iceberg/src/runtime/tokio_backend.rs @@ -0,0 +1,12 @@ +use crate::runtime::JoinHandler; +use std::future::Future; + +pub fn spawn(future: F) -> JoinHandler +where + F: Future + Send + 'static, + F::Output: Send + 'static, +{ + JoinHandler { + inner: tokio::spawn(future), + } +} diff --git a/crates/iceberg/src/scan.rs b/crates/iceberg/src/scan.rs index b42a18341..8f9c44687 100644 --- a/crates/iceberg/src/scan.rs +++ b/crates/iceberg/src/scan.rs @@ -18,14 +18,17 @@ //! Table scan api. use crate::io::FileIO; -use crate::spec::{DataContentType, Manifest, ManifestEntryRef, SchemaRef, SnapshotRef, TableMetadataRef}; +use crate::spec::{ + DataContentType, Manifest, ManifestEntryRef, SchemaRef, SnapshotRef, TableMetadataRef, +}; use crate::table::Table; -use crate::{Error, ErrorKind}; +use crate::{runtime, Error, ErrorKind}; use arrow_array::RecordBatch; use futures::future::try_join_all; use futures::stream::{iter, BoxStream}; use futures::StreamExt; use itertools::Itertools; +use std::sync::Arc; /// Builder to create table scan. pub struct TableScanBuilder<'a> { @@ -51,7 +54,7 @@ impl<'a> TableScanBuilder<'a> { } /// Select some columns of the table. - pub fn select(mut self, column_names: impl IntoIterator) -> Self { + pub fn select(mut self, column_names: impl IntoIterator) -> Self { self.column_names = column_names .into_iter() .map(|item| item.to_string()) @@ -108,7 +111,7 @@ impl<'a> TableScanBuilder<'a> { Ok(TableScan { snapshot, - file_io: self.table.file_io().clone(), + file_io: Arc::new(self.table.file_io().clone()), table_metadata: self.table.metadata_ref(), column_names: self.column_names, schema, @@ -122,7 +125,7 @@ impl<'a> TableScanBuilder<'a> { pub struct TableScan { snapshot: SnapshotRef, table_metadata: TableMetadataRef, - file_io: FileIO, + file_io: Arc, column_names: Vec, schema: SchemaRef, } @@ -141,7 +144,17 @@ impl TableScan { // Generate data file stream let mut file_scan_tasks = Vec::with_capacity(manifest_list.entries().len()); - let tasks = manifest_list.entries().iter().map(|m| m.load_manifest(&self.file_io)).collect_vec(); + let tasks = manifest_list + .entries() + .iter() + .map(|manifest| { + let cloned_manifest = Arc::new(manifest.clone()); + let file_io = self.file_io.clone(); + async move { cloned_manifest.load_manifest(file_io).await } + }) + .map(runtime::spawn) + .collect_vec(); + let manifests: Vec = try_join_all(tasks).await?; for manifest in manifests { @@ -225,7 +238,7 @@ mod tests { "{}/testdata/example_table_metadata_v2.json", env!("CARGO_MANIFEST_DIR") )) - .unwrap(); + .unwrap(); let mut context = Context::new(); context.insert("table_location", &table_location); context.insert("manifest_list_1_location", &manifest_list1_location); @@ -341,64 +354,64 @@ mod tests { current_snapshot.snapshot_id(), vec![], ) - .write(Manifest::new( - ManifestMetadata::builder() - .schema((*current_schema).clone()) - .content(ManifestContentType::Data) - .format_version(FormatVersion::V2) - .partition_spec((**current_partition_spec).clone()) - .schema_id(current_schema.schema_id()) + .write(Manifest::new( + ManifestMetadata::builder() + .schema((*current_schema).clone()) + .content(ManifestContentType::Data) + .format_version(FormatVersion::V2) + .partition_spec((**current_partition_spec).clone()) + .schema_id(current_schema.schema_id()) + .build(), + vec![ + ManifestEntry::builder() + .status(ManifestStatus::Added) + .data_file( + DataFile::builder() + .content(DataContentType::Data) + .file_path(format!("{}/1.parquet", &fixture.table_location)) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(100) + .record_count(1) + .partition(Struct::from_iter([Some(Literal::long(100))])) + .build(), + ) .build(), - vec![ - ManifestEntry::builder() - .status(ManifestStatus::Added) - .data_file( - DataFile::builder() - .content(DataContentType::Data) - .file_path(format!("{}/1.parquet", &fixture.table_location)) - .file_format(DataFileFormat::Parquet) - .file_size_in_bytes(100) - .record_count(1) - .partition(Struct::from_iter([Some(Literal::long(100))])) - .build(), - ) - .build(), - ManifestEntry::builder() - .status(ManifestStatus::Deleted) - .snapshot_id(parent_snapshot.snapshot_id()) - .sequence_number(parent_snapshot.sequence_number()) - .file_sequence_number(parent_snapshot.sequence_number()) - .data_file( - DataFile::builder() - .content(DataContentType::Data) - .file_path(format!("{}/2.parquet", &fixture.table_location)) - .file_format(DataFileFormat::Parquet) - .file_size_in_bytes(100) - .record_count(1) - .partition(Struct::from_iter([Some(Literal::long(200))])) - .build(), - ) - .build(), - ManifestEntry::builder() - .status(ManifestStatus::Existing) - .snapshot_id(parent_snapshot.snapshot_id()) - .sequence_number(parent_snapshot.sequence_number()) - .file_sequence_number(parent_snapshot.sequence_number()) - .data_file( - DataFile::builder() - .content(DataContentType::Data) - .file_path(format!("{}/3.parquet", &fixture.table_location)) - .file_format(DataFileFormat::Parquet) - .file_size_in_bytes(100) - .record_count(1) - .partition(Struct::from_iter([Some(Literal::long(300))])) - .build(), - ) - .build(), - ], - )) - .await - .unwrap(); + ManifestEntry::builder() + .status(ManifestStatus::Deleted) + .snapshot_id(parent_snapshot.snapshot_id()) + .sequence_number(parent_snapshot.sequence_number()) + .file_sequence_number(parent_snapshot.sequence_number()) + .data_file( + DataFile::builder() + .content(DataContentType::Data) + .file_path(format!("{}/2.parquet", &fixture.table_location)) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(100) + .record_count(1) + .partition(Struct::from_iter([Some(Literal::long(200))])) + .build(), + ) + .build(), + ManifestEntry::builder() + .status(ManifestStatus::Existing) + .snapshot_id(parent_snapshot.snapshot_id()) + .sequence_number(parent_snapshot.sequence_number()) + .file_sequence_number(parent_snapshot.sequence_number()) + .data_file( + DataFile::builder() + .content(DataContentType::Data) + .file_path(format!("{}/3.parquet", &fixture.table_location)) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(100) + .record_count(1) + .partition(Struct::from_iter([Some(Literal::long(300))])) + .build(), + ) + .build(), + ], + )) + .await + .unwrap(); // Write to manifest list let mut manifest_list_write = ManifestListWriter::v2( diff --git a/crates/iceberg/src/spec/manifest_list.rs b/crates/iceberg/src/spec/manifest_list.rs index 8ce40dadc..e67ea3493 100644 --- a/crates/iceberg/src/spec/manifest_list.rs +++ b/crates/iceberg/src/spec/manifest_list.rs @@ -17,6 +17,7 @@ //! ManifestList for Iceberg. +use std::sync::Arc; use std::{collections::HashMap, str::FromStr}; use crate::io::FileIO; @@ -634,7 +635,7 @@ impl ManifestFile { /// Load [`Manifest`]. /// /// This method will also initialize inherited values of [`ManifestEntry`], such as `sequence_number`. - pub async fn load_manifest(&self, file_io: &FileIO) -> Result { + pub async fn load_manifest(&self, file_io: Arc) -> Result { let mut avro = Vec::new(); file_io .new_input(&self.manifest_path)? From 01d7105466d1f80aebcd96aebbab24aa8eaeb46f Mon Sep 17 00:00:00 2001 From: Chengxu Bian Date: Fri, 15 Mar 2024 19:17:17 -0400 Subject: [PATCH 03/16] fix chrono --- crates/iceberg/src/spec/values.rs | 101 +++++++++++++++++------------- 1 file changed, 58 insertions(+), 43 deletions(-) diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index 43f44745c..027f8fc18 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -106,7 +106,7 @@ impl Display for Datum { (_, PrimitiveLiteral::TimestampTZ(val)) => { write!(f, "{}", microseconds_to_datetimetz(*val)) } - (_, PrimitiveLiteral::String(val)) => write!(f, "{}", val), + (_, PrimitiveLiteral::String(val)) => write!(f, r#""{}""#, val), (_, PrimitiveLiteral::UUID(val)) => write!(f, "{}", val), (_, PrimitiveLiteral::Fixed(val)) => display_bytes(val, f), (_, PrimitiveLiteral::Binary(val)) => display_bytes(val, f), @@ -401,10 +401,10 @@ impl Datum { /// ``` pub fn time_from_hms_micro(hour: u32, min: u32, sec: u32, micro: u32) -> Result { let t = NaiveTime::from_hms_micro_opt(hour, min, sec, micro) - .ok_or_else(|| Error::new( - ErrorKind::DataInvalid, - format!("Can't create time from hour: {hour}, min: {min}, second: {sec}, microsecond: {micro}"), - ))?; + .ok_or_else(|| Error::new( + ErrorKind::DataInvalid, + format!("Can't create time from hour: {hour}, min: {min}, second: {sec}, microsecond: {micro}"), + ))?; Ok(Self::time_from_naive_time(t)) } @@ -529,7 +529,7 @@ impl Datum { /// use iceberg::spec::Datum; /// let t = Datum::string("ss"); /// - /// assert_eq!(&format!("{t}"), "ss"); + /// assert_eq!(&format!("{t}"), r#""ss""#); /// ``` pub fn string(s: S) -> Self { Self { @@ -658,6 +658,21 @@ impl Datum { unreachable!("Decimal type must be primitive.") } } + + /// Convert the datum to `target_type`. + pub fn to(self, target_type: &Type) -> Result { + // TODO: We should allow more type conversions + match target_type { + Type::Primitive(typ) if typ == &self.r#type => Ok(self), + _ => Err(Error::new( + ErrorKind::DataInvalid, + format!( + "Can't convert datum from {} type to {} type.", + self.r#type, target_type + ), + )), + } + } } /// Values present in iceberg type @@ -872,10 +887,10 @@ impl Literal { /// ``` pub fn time_from_hms_micro(hour: u32, min: u32, sec: u32, micro: u32) -> Result { let t = NaiveTime::from_hms_micro_opt(hour, min, sec, micro) - .ok_or_else(|| Error::new( - ErrorKind::DataInvalid, - format!("Can't create time from hour: {hour}, min: {min}, second: {sec}, microsecond: {micro}"), - ))?; + .ok_or_else(|| Error::new( + ErrorKind::DataInvalid, + format!("Can't create time from hour: {hour}, min: {min}, second: {sec}, microsecond: {micro}"), + ))?; Ok(Self::time_from_naive_time(t)) } @@ -1530,7 +1545,7 @@ impl Literal { } mod date { - use chrono::{DateTime, Duration, NaiveDate, TimeZone, Utc}; + use chrono::{DateTime, NaiveDate, TimeDelta, TimeZone, Utc}; pub(crate) fn date_to_days(date: &NaiveDate) -> i32 { date.signed_duration_since( @@ -1964,32 +1979,32 @@ mod _serde { }) } else { let list = map.into_iter().map(|(k,v)| { - let raw_k = - RawLiteralEnum::try_from(k, &map_ty.key_field.field_type)?; - let raw_v = v - .map(|v| { - RawLiteralEnum::try_from(v, &map_ty.value_field.field_type) - }) - .transpose()?; - if map_ty.value_field.required { - Ok(Some(RawLiteralEnum::Record(Record { - required: vec![ - (MAP_KEY_FIELD_NAME.to_string(), raw_k), - (MAP_VALUE_FIELD_NAME.to_string(), raw_v.ok_or_else(||Error::new(ErrorKind::DataInvalid, "Map value is required, value cannot be null"))?), - ], - optional: vec![], - }))) - } else { - Ok(Some(RawLiteralEnum::Record(Record { - required: vec![ - (MAP_KEY_FIELD_NAME.to_string(), raw_k), - ], - optional: vec![ - (MAP_VALUE_FIELD_NAME.to_string(), raw_v) - ], - }))) - } - }).collect::>()?; + let raw_k = + RawLiteralEnum::try_from(k, &map_ty.key_field.field_type)?; + let raw_v = v + .map(|v| { + RawLiteralEnum::try_from(v, &map_ty.value_field.field_type) + }) + .transpose()?; + if map_ty.value_field.required { + Ok(Some(RawLiteralEnum::Record(Record { + required: vec![ + (MAP_KEY_FIELD_NAME.to_string(), raw_k), + (MAP_VALUE_FIELD_NAME.to_string(), raw_v.ok_or_else(||Error::new(ErrorKind::DataInvalid, "Map value is required, value cannot be null"))?), + ], + optional: vec![], + }))) + } else { + Ok(Some(RawLiteralEnum::Record(Record { + required: vec![ + (MAP_KEY_FIELD_NAME.to_string(), raw_k), + ], + optional: vec![ + (MAP_VALUE_FIELD_NAME.to_string(), raw_v) + ], + }))) + } + }).collect::>()?; RawLiteralEnum::List(List { list, required: true, @@ -2009,12 +2024,12 @@ mod _serde { pub fn try_into(self, ty: &Type) -> Result, Error> { let invalid_err = |v: &str| { Error::new( - ErrorKind::DataInvalid, - format!( - "Unable to convert raw literal ({}) fail convert to type {} for: type mismatch", - v, ty - ), - ) + ErrorKind::DataInvalid, + format!( + "Unable to convert raw literal ({}) fail convert to type {} for: type mismatch", + v, ty + ), + ) }; let invalid_err_with_reason = |v: &str, reason: &str| { Error::new( From 2916fb58c998ece79961d91dcfbbbface7c46a3f Mon Sep 17 00:00:00 2001 From: Chengxu Bian Date: Fri, 15 Mar 2024 21:06:04 -0400 Subject: [PATCH 04/16] fix dep --- Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Cargo.toml b/Cargo.toml index b8c9195ad..68a7d23bc 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -45,7 +45,7 @@ async-std = "1.12.0" bimap = "0.6" bitvec = "1.0.1" bytes = "1.5" -chrono = "0.4" +chrono = "0.4.34" derive_builder = "0.20.0" either = "1" env_logger = "0.11.0" From aea489d9c4dba6286fab863abe2b250ef9a35099 Mon Sep 17 00:00:00 2001 From: Chengxu Bian Date: Fri, 15 Mar 2024 21:44:55 -0400 Subject: [PATCH 05/16] refine module --- crates/iceberg/Cargo.toml | 2 +- .../iceberg/src/runtime/async_std_backend.rs | 20 +++++++++--- crates/iceberg/src/runtime/mod.rs | 31 ++++++++++--------- crates/iceberg/src/runtime/tokio_backend.rs | 22 ++++++++++--- 4 files changed, 51 insertions(+), 24 deletions(-) diff --git a/crates/iceberg/Cargo.toml b/crates/iceberg/Cargo.toml index e8cf586d9..b625fb055 100644 --- a/crates/iceberg/Cargo.toml +++ b/crates/iceberg/Cargo.toml @@ -39,9 +39,9 @@ apache-avro = { workspace = true } arrow-arith = { workspace = true } arrow-array = { workspace = true } arrow-schema = { workspace = true } +async-std = { workspace = true, optional = true } async-stream = { workspace = true } async-trait = { workspace = true } -async-std = { workspace = true, optional = true } bimap = { workspace = true } bitvec = { workspace = true } bytes = { workspace = true } diff --git a/crates/iceberg/src/runtime/async_std_backend.rs b/crates/iceberg/src/runtime/async_std_backend.rs index 18e6c09c1..54640110e 100644 --- a/crates/iceberg/src/runtime/async_std_backend.rs +++ b/crates/iceberg/src/runtime/async_std_backend.rs @@ -1,12 +1,24 @@ -use crate::runtime::JoinHandler; -use std::future::Future; +use crate::runtime::{JoinHandle, JoinHandleExt}; +use std::{ + future::Future, + pin::Pin, + task::{Context, Poll}, +}; -pub fn spawn(future: F) -> JoinHandler +pub fn spawn(future: F) -> JoinHandle> where F: Future + Send + 'static, F::Output: Send + 'static, { - JoinHandler { + JoinHandle { inner: async_std::task::spawn(future), + _marker: Default::default(), + } +} + +impl JoinHandleExt for async_std::task::JoinHandle { + type Output = T; + fn poll_join(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + self.as_mut().poll(cx) } } diff --git a/crates/iceberg/src/runtime/mod.rs b/crates/iceberg/src/runtime/mod.rs index 93c1ee22c..59bdc0c91 100644 --- a/crates/iceberg/src/runtime/mod.rs +++ b/crates/iceberg/src/runtime/mod.rs @@ -1,4 +1,5 @@ use std::future::Future; +use std::marker::{PhantomData, Unpin}; use std::pin::Pin; use std::task::{Context, Poll}; @@ -12,24 +13,24 @@ pub mod async_std_backend; #[cfg(all(feature = "async-std", not(feature = "tokio"),))] pub use async_std_backend::*; -pub struct JoinHandler { - #[cfg(feature = "tokio")] - inner: tokio::task::JoinHandle, - #[cfg(all(feature = "async-std", not(feature = "tokio"),))] - inner: async_std::task::JoinHandle, +pub trait JoinHandleExt { + type Output; + fn poll_join(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll; } -impl Future for JoinHandler { +pub struct JoinHandle { + inner: J, + _marker: PhantomData, +} + +impl Future for JoinHandle +where + T: Send + 'static + Unpin, + J: JoinHandleExt + Unpin, +{ type Output = T; + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { - #[cfg(all(feature = "async-std", not(feature = "tokio")))] - { - Pin::new(&mut self.inner).poll(cx) - } - // Tokio returns a Poll> - #[cfg(feature = "tokio")] - Pin::new(&mut self.inner) - .poll(cx) - .map(|res| res.expect("tokio spawned task crashed")) + Pin::new(&mut self.inner).poll_join(cx) } } diff --git a/crates/iceberg/src/runtime/tokio_backend.rs b/crates/iceberg/src/runtime/tokio_backend.rs index 2847940f6..7d4a00212 100644 --- a/crates/iceberg/src/runtime/tokio_backend.rs +++ b/crates/iceberg/src/runtime/tokio_backend.rs @@ -1,12 +1,26 @@ -use crate::runtime::JoinHandler; -use std::future::Future; +use crate::runtime::{JoinHandle, JoinHandleExt}; +use std::{ + future::Future, + pin::Pin, + task::{Context, Poll}, +}; -pub fn spawn(future: F) -> JoinHandler +pub fn spawn(future: F) -> JoinHandle> where F: Future + Send + 'static, F::Output: Send + 'static, { - JoinHandler { + JoinHandle { inner: tokio::spawn(future), + _marker: Default::default(), + } +} + +impl JoinHandleExt for tokio::task::JoinHandle { + type Output = T; + fn poll_join(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + self.as_mut() + .poll(cx) + .map(|res| res.expect("tokio spawned task crashed")) } } From dbcdef31b3667ebc6c3dad6d186fb108819f8680 Mon Sep 17 00:00:00 2001 From: Chengxu Bian Date: Fri, 21 Jun 2024 20:58:11 -0400 Subject: [PATCH 06/16] refactor to use a deadly simple way --- crates/iceberg/Cargo.toml | 6 +- .../iceberg/src/runtime/async_std_backend.rs | 24 ----- crates/iceberg/src/runtime/mod.rs | 88 ++++++++++++++----- crates/iceberg/src/runtime/tokio_backend.rs | 26 ------ crates/iceberg/src/spec/manifest_list.rs | 3 +- 5 files changed, 71 insertions(+), 76 deletions(-) delete mode 100644 crates/iceberg/src/runtime/async_std_backend.rs delete mode 100644 crates/iceberg/src/runtime/tokio_backend.rs diff --git a/crates/iceberg/Cargo.toml b/crates/iceberg/Cargo.toml index eb5c05c50..a2db99b87 100644 --- a/crates/iceberg/Cargo.toml +++ b/crates/iceberg/Cargo.toml @@ -30,7 +30,7 @@ keywords = ["iceberg"] [features] async-std = ["dep:async-std"] -tokio = ["dep:tokio"] +tokio = [] # tokio is not an optional dependency now default = ["tokio"] [dependencies] @@ -40,7 +40,7 @@ array-init = { workspace = true } arrow-arith = { workspace = true } arrow-array = { workspace = true } arrow-schema = { workspace = true } -async-std = { workspace = true, optional = true } +async-std = { workspace = true, optional = true, features = ["attributes"] } arrow-select = { workspace = true } async-stream = { workspace = true } async-trait = { workspace = true } @@ -68,7 +68,7 @@ serde_derive = { workspace = true } serde_json = { workspace = true } serde_repr = { workspace = true } serde_with = { workspace = true } -tokio = { workspace = true, optional = true } +tokio = { workspace = true } typed-builder = { workspace = true } url = { workspace = true } urlencoding = { workspace = true } diff --git a/crates/iceberg/src/runtime/async_std_backend.rs b/crates/iceberg/src/runtime/async_std_backend.rs deleted file mode 100644 index 54640110e..000000000 --- a/crates/iceberg/src/runtime/async_std_backend.rs +++ /dev/null @@ -1,24 +0,0 @@ -use crate::runtime::{JoinHandle, JoinHandleExt}; -use std::{ - future::Future, - pin::Pin, - task::{Context, Poll}, -}; - -pub fn spawn(future: F) -> JoinHandle> -where - F: Future + Send + 'static, - F::Output: Send + 'static, -{ - JoinHandle { - inner: async_std::task::spawn(future), - _marker: Default::default(), - } -} - -impl JoinHandleExt for async_std::task::JoinHandle { - type Output = T; - fn poll_join(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { - self.as_mut().poll(cx) - } -} diff --git a/crates/iceberg/src/runtime/mod.rs b/crates/iceberg/src/runtime/mod.rs index 59bdc0c91..37c4dd4bf 100644 --- a/crates/iceberg/src/runtime/mod.rs +++ b/crates/iceberg/src/runtime/mod.rs @@ -1,36 +1,82 @@ use std::future::Future; -use std::marker::{PhantomData, Unpin}; use std::pin::Pin; use std::task::{Context, Poll}; -#[cfg(feature = "tokio")] -pub mod tokio_backend; -#[cfg(feature = "tokio")] -pub use tokio_backend::*; +pub enum JoinHandle { + #[cfg(feature = "tokio")] + Tokio(tokio::task::JoinHandle), + #[cfg(feature = "async-std")] + AsyncStd(async_std::task::JoinHandle), +} -#[cfg(all(feature = "async-std", not(feature = "tokio"),))] -pub mod async_std_backend; -#[cfg(all(feature = "async-std", not(feature = "tokio"),))] -pub use async_std_backend::*; +impl Future for JoinHandle { + type Output = T; -pub trait JoinHandleExt { - type Output; - fn poll_join(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll; + fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + match self.get_mut() { + #[cfg(feature = "tokio")] + JoinHandle::Tokio(handle) => Pin::new(handle) + .poll(cx) + .map(|h| h.expect("tokio spawned task failed")), + #[cfg(feature = "async-std")] + JoinHandle::AsyncStd(handle) => Pin::new(handle).poll(cx), + } + } } -pub struct JoinHandle { - inner: J, - _marker: PhantomData, +pub fn spawn(f: F) -> JoinHandle +where + F: Future + Send + 'static, + F::Output: Send + 'static, +{ + #[cfg(feature = "tokio")] + return JoinHandle::Tokio(tokio::task::spawn(f)); + + #[cfg(feature = "async-std")] + return JoinHandle::AsyncStd(async_std::task::spawn(f)); } -impl Future for JoinHandle +pub fn spawn_blocking(f: F) -> JoinHandle where - T: Send + 'static + Unpin, - J: JoinHandleExt + Unpin, + F: FnOnce() -> T + Send + 'static, + T: Send + 'static, { - type Output = T; + #[cfg(feature = "tokio")] + return JoinHandle::Tokio(tokio::task::spawn_blocking(f)); + + #[cfg(feature = "async-std")] + return JoinHandle::AsyncStd(async_std::task::spawn_blocking(f)); +} + +#[cfg(test)] +mod tests { + use super::*; + + #[cfg(feature = "tokio")] + #[tokio::test] + async fn test_tokio_spawn() { + let handle = spawn(async { 1 + 1 }); + assert_eq!(handle.await, 2); + } + + #[cfg(feature = "tokio")] + #[tokio::test] + async fn test_tokio_spawn_blocking() { + let handle = spawn_blocking(|| 1 + 1); + assert_eq!(handle.await, 2); + } + + #[cfg(feature = "async-std")] + #[async_std::test] + async fn test_async_std_spawn() { + let handle = spawn(async { 1 + 1 }); + assert_eq!(handle.await, 2); + } - fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { - Pin::new(&mut self.inner).poll_join(cx) + #[cfg(feature = "async-std")] + #[async_std::test] + async fn test_async_std_spawn_blocking() { + let handle = spawn_blocking(|| 1 + 1); + assert_eq!(handle.await, 2); } } diff --git a/crates/iceberg/src/runtime/tokio_backend.rs b/crates/iceberg/src/runtime/tokio_backend.rs deleted file mode 100644 index 7d4a00212..000000000 --- a/crates/iceberg/src/runtime/tokio_backend.rs +++ /dev/null @@ -1,26 +0,0 @@ -use crate::runtime::{JoinHandle, JoinHandleExt}; -use std::{ - future::Future, - pin::Pin, - task::{Context, Poll}, -}; - -pub fn spawn(future: F) -> JoinHandle> -where - F: Future + Send + 'static, - F::Output: Send + 'static, -{ - JoinHandle { - inner: tokio::spawn(future), - _marker: Default::default(), - } -} - -impl JoinHandleExt for tokio::task::JoinHandle { - type Output = T; - fn poll_join(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { - self.as_mut() - .poll(cx) - .map(|res| res.expect("tokio spawned task crashed")) - } -} diff --git a/crates/iceberg/src/spec/manifest_list.rs b/crates/iceberg/src/spec/manifest_list.rs index 1145ae9cb..da63815d6 100644 --- a/crates/iceberg/src/spec/manifest_list.rs +++ b/crates/iceberg/src/spec/manifest_list.rs @@ -17,7 +17,6 @@ //! ManifestList for Iceberg. -use std::sync::Arc; use std::{collections::HashMap, str::FromStr}; use crate::io::FileIO; @@ -632,7 +631,7 @@ impl ManifestFile { /// Load [`Manifest`]. /// /// This method will also initialize inherited values of [`ManifestEntry`], such as `sequence_number`. - pub async fn load_manifest(&self, file_io: Arc) -> Result { + pub async fn load_manifest(&self, file_io: &FileIO) -> Result { let mut avro = Vec::new(); file_io .new_input(&self.manifest_path)? From f4bd2ccd5c25ac5300f84d5ec0296869a0563bf0 Mon Sep 17 00:00:00 2001 From: Chengxu Bian Date: Fri, 21 Jun 2024 21:14:38 -0400 Subject: [PATCH 07/16] allow dead_code --- crates/iceberg/src/runtime/mod.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/crates/iceberg/src/runtime/mod.rs b/crates/iceberg/src/runtime/mod.rs index 37c4dd4bf..40ee8da4a 100644 --- a/crates/iceberg/src/runtime/mod.rs +++ b/crates/iceberg/src/runtime/mod.rs @@ -24,6 +24,7 @@ impl Future for JoinHandle { } } +#[allow(dead_code)] pub fn spawn(f: F) -> JoinHandle where F: Future + Send + 'static, @@ -36,6 +37,7 @@ where return JoinHandle::AsyncStd(async_std::task::spawn(f)); } +#[allow(dead_code)] pub fn spawn_blocking(f: F) -> JoinHandle where F: FnOnce() -> T + Send + 'static, From 972bb5c296df92c193525e5d73a20c2ce703deda Mon Sep 17 00:00:00 2001 From: Chengxu Bian Date: Fri, 21 Jun 2024 21:37:31 -0400 Subject: [PATCH 08/16] add license --- crates/iceberg/src/runtime/mod.rs | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/crates/iceberg/src/runtime/mod.rs b/crates/iceberg/src/runtime/mod.rs index 40ee8da4a..f822e5384 100644 --- a/crates/iceberg/src/runtime/mod.rs +++ b/crates/iceberg/src/runtime/mod.rs @@ -1,3 +1,22 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +// This module contains the async runtime abstraction for iceberg. + use std::future::Future; use std::pin::Pin; use std::task::{Context, Poll}; From 15bca8a922e2eadc5dfde1ca567a918f7f027b89 Mon Sep 17 00:00:00 2001 From: Chengxu Bian Date: Fri, 21 Jun 2024 22:37:32 -0400 Subject: [PATCH 09/16] fix clippy and tests --- crates/iceberg/src/runtime/mod.rs | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/crates/iceberg/src/runtime/mod.rs b/crates/iceberg/src/runtime/mod.rs index f822e5384..453b1564a 100644 --- a/crates/iceberg/src/runtime/mod.rs +++ b/crates/iceberg/src/runtime/mod.rs @@ -24,7 +24,7 @@ use std::task::{Context, Poll}; pub enum JoinHandle { #[cfg(feature = "tokio")] Tokio(tokio::task::JoinHandle), - #[cfg(feature = "async-std")] + #[cfg(all(feature = "async-std", not(feature = "tokio")))] AsyncStd(async_std::task::JoinHandle), } @@ -37,7 +37,7 @@ impl Future for JoinHandle { JoinHandle::Tokio(handle) => Pin::new(handle) .poll(cx) .map(|h| h.expect("tokio spawned task failed")), - #[cfg(feature = "async-std")] + #[cfg(all(feature = "async-std", not(feature = "tokio")))] JoinHandle::AsyncStd(handle) => Pin::new(handle).poll(cx), } } @@ -52,7 +52,7 @@ where #[cfg(feature = "tokio")] return JoinHandle::Tokio(tokio::task::spawn(f)); - #[cfg(feature = "async-std")] + #[cfg(all(feature = "async-std", not(feature = "tokio")))] return JoinHandle::AsyncStd(async_std::task::spawn(f)); } @@ -65,7 +65,7 @@ where #[cfg(feature = "tokio")] return JoinHandle::Tokio(tokio::task::spawn_blocking(f)); - #[cfg(feature = "async-std")] + #[cfg(all(feature = "async-std", not(feature = "tokio")))] return JoinHandle::AsyncStd(async_std::task::spawn_blocking(f)); } @@ -87,14 +87,14 @@ mod tests { assert_eq!(handle.await, 2); } - #[cfg(feature = "async-std")] + #[cfg(all(feature = "async-std", not(feature = "tokio")))] #[async_std::test] async fn test_async_std_spawn() { let handle = spawn(async { 1 + 1 }); assert_eq!(handle.await, 2); } - #[cfg(feature = "async-std")] + #[cfg(all(feature = "async-std", not(feature = "tokio")))] #[async_std::test] async fn test_async_std_spawn_blocking() { let handle = spawn_blocking(|| 1 + 1); From 320c8c73af04d14ac28eaa9896b32d6042d06585 Mon Sep 17 00:00:00 2001 From: Chengxu Bian Date: Mon, 24 Jun 2024 16:55:29 -0400 Subject: [PATCH 10/16] clean code --- Cargo.toml | 11 +- crates/iceberg/Cargo.toml | 5 +- crates/iceberg/src/spec/values.rs | 6824 ++++++++++++++--------------- 3 files changed, 3421 insertions(+), 3419 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index ddd029b8c..7adf9992c 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -16,12 +16,13 @@ # under the License. [workspace] +resolver = "2" members = [ - "crates/catalog/*", - "crates/examples", - "crates/iceberg", - "crates/integrations/*", - "crates/test_utils", + "crates/catalog/*", + "crates/examples", + "crates/iceberg", + "crates/integrations/*", + "crates/test_utils", ] [workspace.package] diff --git a/crates/iceberg/Cargo.toml b/crates/iceberg/Cargo.toml index c20c9fbba..0ca2bfb13 100644 --- a/crates/iceberg/Cargo.toml +++ b/crates/iceberg/Cargo.toml @@ -36,7 +36,7 @@ storage-fs = ["opendal/services-fs"] storage-s3 = ["opendal/services-s3"] async-std = ["dep:async-std"] -tokio = [] +tokio = ["dep:tokio"] [dependencies] anyhow = { workspace = true } @@ -75,7 +75,7 @@ serde_derive = { workspace = true } serde_json = { workspace = true } serde_repr = { workspace = true } serde_with = { workspace = true } -tokio = { workspace = true } +tokio = { workspace = true, optional = true } typed-builder = { workspace = true } url = { workspace = true } urlencoding = { workspace = true } @@ -86,3 +86,4 @@ iceberg_test_utils = { path = "../test_utils", features = ["tests"] } pretty_assertions = { workspace = true } tempfile = { workspace = true } tera = { workspace = true } +tokio = { workspace = true } diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index c3b35fb7b..7d56e763c 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -19,2353 +19,2353 @@ * Value in iceberg */ -use std::any::Any; -use std::collections::HashMap; -use std::fmt::{Display, Formatter}; -use std::hash::Hash; -use std::ops::Index; -use std::str::FromStr; - -use bitvec::vec::BitVec; -use chrono::{DateTime, NaiveDate, NaiveDateTime, NaiveTime, TimeZone, Utc}; -use ordered_float::OrderedFloat; -use rust_decimal::Decimal; -use serde::de::{self, MapAccess}; -use serde::ser::SerializeStruct; -use serde::{Deserialize, Serialize}; -use serde_bytes::ByteBuf; -use serde_json::{Map as JsonMap, Number, Value as JsonValue}; -use uuid::Uuid; - -pub use _serde::RawLiteral; - -use crate::error::Result; -use crate::spec::values::date::{date_from_naive_date, days_to_date, unix_epoch}; -use crate::spec::values::time::microseconds_to_time; -use crate::spec::values::timestamp::microseconds_to_datetime; -use crate::spec::values::timestamptz::microseconds_to_datetimetz; -use crate::spec::MAX_DECIMAL_PRECISION; -use crate::{ensure_data_valid, Error, ErrorKind}; - -use super::datatypes::{PrimitiveType, Type}; - -/// Maximum value for [`PrimitiveType::Time`] type in microseconds, e.g. 23 hours 59 minutes 59 seconds 999999 microseconds. -const MAX_TIME_VALUE: i64 = 24 * 60 * 60 * 1_000_000i64 - 1; - -/// Values present in iceberg type -#[derive(Clone, Debug, PartialEq, Hash, Eq)] -pub enum PrimitiveLiteral { - /// 0x00 for false, non-zero byte for true - Boolean(bool), - /// Stored as 4-byte little-endian - Int(i32), - /// Stored as 8-byte little-endian - Long(i64), - /// Stored as 4-byte little-endian - Float(OrderedFloat), - /// Stored as 8-byte little-endian - Double(OrderedFloat), - /// Stores days from the 1970-01-01 in an 4-byte little-endian int - Date(i32), - /// Stores microseconds from midnight in an 8-byte little-endian long - Time(i64), - /// Timestamp without timezone - Timestamp(i64), - /// Timestamp with timezone - Timestamptz(i64), - /// UTF-8 bytes (without length) - String(String), - /// 16-byte big-endian value - UUID(Uuid), - /// Binary value - Fixed(Vec), - /// Binary value (without length) - Binary(Vec), - /// Stores unscaled value as big int. According to iceberg spec, the precision must less than 38(`MAX_DECIMAL_PRECISION`) , so i128 is suit here. - Decimal(i128), -} - -impl PrimitiveLiteral { - /// Returns true if the Literal represents a primitive type - /// that can be a NaN, and that it's value is NaN - pub fn is_nan(&self) -> bool { - match self { - PrimitiveLiteral::Double(val) => val.is_nan(), - PrimitiveLiteral::Float(val) => val.is_nan(), - _ => false, - } - } -} - -/// Literal associated with its type. The value and type pair is checked when construction, so the type and value is -/// guaranteed to be correct when used. -/// -/// By default, we decouple the type and value of a literal, so we can use avoid the cost of storing extra type info -/// for each literal. But associate type with literal can be useful in some cases, for example, in unbound expression. -#[derive(Clone, Debug, PartialEq, Hash, Eq)] -pub struct Datum { - r#type: PrimitiveType, - literal: PrimitiveLiteral, -} - -impl Serialize for Datum { - fn serialize( - &self, - serializer: S, - ) -> std::result::Result { - let mut struct_ser = serializer - .serialize_struct("Datum", 2) - .map_err(serde::ser::Error::custom)?; - struct_ser - .serialize_field("type", &self.r#type) - .map_err(serde::ser::Error::custom)?; - struct_ser - .serialize_field( - "literal", - &RawLiteral::try_from( - Literal::Primitive(self.literal.clone()), - &Type::Primitive(self.r#type.clone()), - ) - .map_err(serde::ser::Error::custom)?, - ) - .map_err(serde::ser::Error::custom)?; - struct_ser.end() - } -} - -impl<'de> Deserialize<'de> for Datum { - fn deserialize>( - deserializer: D, - ) -> std::result::Result { - #[derive(Deserialize)] - #[serde(field_identifier, rename_all = "lowercase")] - enum Field { - Type, - Literal, - } - - struct DatumVisitor; - - impl<'de> serde::de::Visitor<'de> for DatumVisitor { - type Value = Datum; - - fn expecting(&self, formatter: &mut std::fmt::Formatter) -> std::fmt::Result { - formatter.write_str("struct Datum") - } - - fn visit_seq(self, mut seq: A) -> std::result::Result - where - A: serde::de::SeqAccess<'de>, - { - let r#type = seq - .next_element::()? - .ok_or_else(|| serde::de::Error::invalid_length(0, &self))?; - let value = seq - .next_element::()? - .ok_or_else(|| serde::de::Error::invalid_length(1, &self))?; - let Literal::Primitive(primitive) = value - .try_into(&Type::Primitive(r#type.clone())) - .map_err(serde::de::Error::custom)? - .ok_or_else(|| serde::de::Error::custom("None value"))? - else { - return Err(serde::de::Error::custom("Invalid value")); - }; - - Ok(Datum::new(r#type, primitive)) - } - - fn visit_map(self, mut map: V) -> std::result::Result - where - V: MapAccess<'de>, - { - let mut raw_primitive: Option = None; - let mut r#type: Option = None; - while let Some(key) = map.next_key()? { - match key { - Field::Type => { - if r#type.is_some() { - return Err(de::Error::duplicate_field("type")); - } - r#type = Some(map.next_value()?); - } - Field::Literal => { - if raw_primitive.is_some() { - return Err(de::Error::duplicate_field("literal")); - } - raw_primitive = Some(map.next_value()?); - } - } - } - let Some(r#type) = r#type else { - return Err(serde::de::Error::missing_field("type")); - }; - let Some(raw_primitive) = raw_primitive else { - return Err(serde::de::Error::missing_field("literal")); - }; - let Literal::Primitive(primitive) = raw_primitive - .try_into(&Type::Primitive(r#type.clone())) - .map_err(serde::de::Error::custom)? - .ok_or_else(|| serde::de::Error::custom("None value"))? - else { - return Err(serde::de::Error::custom("Invalid value")); - }; - Ok(Datum::new(r#type, primitive)) - } - } - const FIELDS: &[&str] = &["type", "literal"]; - deserializer.deserialize_struct("Datum", FIELDS, DatumVisitor) - } -} - -impl PartialOrd for Datum { - fn partial_cmp(&self, other: &Self) -> Option { - match (&self.literal, &other.literal, &self.r#type, &other.r#type) { - // generate the arm with same type and same literal - ( - PrimitiveLiteral::Boolean(val), - PrimitiveLiteral::Boolean(other_val), - PrimitiveType::Boolean, - PrimitiveType::Boolean, - ) => val.partial_cmp(other_val), - ( - PrimitiveLiteral::Int(val), - PrimitiveLiteral::Int(other_val), - PrimitiveType::Int, - PrimitiveType::Int, - ) => val.partial_cmp(other_val), - ( - PrimitiveLiteral::Long(val), - PrimitiveLiteral::Long(other_val), - PrimitiveType::Long, - PrimitiveType::Long, - ) => val.partial_cmp(other_val), - ( - PrimitiveLiteral::Float(val), - PrimitiveLiteral::Float(other_val), - PrimitiveType::Float, - PrimitiveType::Float, - ) => val.partial_cmp(other_val), - ( - PrimitiveLiteral::Double(val), - PrimitiveLiteral::Double(other_val), - PrimitiveType::Double, - PrimitiveType::Double, - ) => val.partial_cmp(other_val), - ( - PrimitiveLiteral::Date(val), - PrimitiveLiteral::Date(other_val), - PrimitiveType::Date, - PrimitiveType::Date, - ) => val.partial_cmp(other_val), - ( - PrimitiveLiteral::Time(val), - PrimitiveLiteral::Time(other_val), - PrimitiveType::Time, - PrimitiveType::Time, - ) => val.partial_cmp(other_val), - ( - PrimitiveLiteral::Timestamp(val), - PrimitiveLiteral::Timestamp(other_val), - PrimitiveType::Timestamp, - PrimitiveType::Timestamp, - ) => val.partial_cmp(other_val), - ( - PrimitiveLiteral::Timestamptz(val), - PrimitiveLiteral::Timestamptz(other_val), - PrimitiveType::Timestamptz, - PrimitiveType::Timestamptz, - ) => val.partial_cmp(other_val), - ( - PrimitiveLiteral::String(val), - PrimitiveLiteral::String(other_val), - PrimitiveType::String, - PrimitiveType::String, - ) => val.partial_cmp(other_val), - ( - PrimitiveLiteral::UUID(val), - PrimitiveLiteral::UUID(other_val), - PrimitiveType::Uuid, - PrimitiveType::Uuid, - ) => val.partial_cmp(other_val), - ( - PrimitiveLiteral::Fixed(val), - PrimitiveLiteral::Fixed(other_val), - PrimitiveType::Fixed(_), - PrimitiveType::Fixed(_), - ) => val.partial_cmp(other_val), - ( - PrimitiveLiteral::Binary(val), - PrimitiveLiteral::Binary(other_val), - PrimitiveType::Binary, - PrimitiveType::Binary, - ) => val.partial_cmp(other_val), - ( - PrimitiveLiteral::Decimal(val), - PrimitiveLiteral::Decimal(other_val), - PrimitiveType::Decimal { - precision: _, - scale, - }, - PrimitiveType::Decimal { - precision: _, - scale: other_scale, - }, - ) => { - let val = Decimal::from_i128_with_scale(*val, *scale); - let other_val = Decimal::from_i128_with_scale(*other_val, *other_scale); - val.partial_cmp(&other_val) - } - _ => None, - } - } -} - -impl Display for Datum { - fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { - match (&self.r#type, &self.literal) { - (_, PrimitiveLiteral::Boolean(val)) => write!(f, "{}", val), - (_, PrimitiveLiteral::Int(val)) => write!(f, "{}", val), - (_, PrimitiveLiteral::Long(val)) => write!(f, "{}", val), - (_, PrimitiveLiteral::Float(val)) => write!(f, "{}", val), - (_, PrimitiveLiteral::Double(val)) => write!(f, "{}", val), - (_, PrimitiveLiteral::Date(val)) => write!(f, "{}", days_to_date(*val)), - (_, PrimitiveLiteral::Time(val)) => write!(f, "{}", microseconds_to_time(*val)), - (_, PrimitiveLiteral::Timestamp(val)) => { - write!(f, "{}", microseconds_to_datetime(*val)) - } - (_, PrimitiveLiteral::Timestamptz(val)) => { - write!(f, "{}", microseconds_to_datetimetz(*val)) - } - (_, PrimitiveLiteral::String(val)) => write!(f, r#""{}""#, val), - (_, PrimitiveLiteral::UUID(val)) => write!(f, "{}", val), - (_, PrimitiveLiteral::Fixed(val)) => display_bytes(val, f), - (_, PrimitiveLiteral::Binary(val)) => display_bytes(val, f), - ( - PrimitiveType::Decimal { - precision: _, - scale, - }, - PrimitiveLiteral::Decimal(val), - ) => { - write!(f, "{}", Decimal::from_i128_with_scale(*val, *scale)) - } - (_, _) => { - unreachable!() - } - } - } -} - -fn display_bytes(bytes: &[u8], f: &mut Formatter<'_>) -> std::fmt::Result { - let mut s = String::with_capacity(bytes.len() * 2); - for b in bytes { - s.push_str(&format!("{:02X}", b)); - } - f.write_str(&s) -} - -impl From for Literal { - fn from(value: Datum) -> Self { - Literal::Primitive(value.literal) - } -} - -impl From for PrimitiveLiteral { - fn from(value: Datum) -> Self { - value.literal - } -} - -impl Datum { - /// Creates a `Datum` from a `PrimitiveType` and a `PrimitiveLiteral` - pub(crate) fn new(r#type: PrimitiveType, literal: PrimitiveLiteral) -> Self { - Datum { r#type, literal } - } - - /// Create iceberg value from bytes - pub fn try_from_bytes(bytes: &[u8], data_type: PrimitiveType) -> Result { - let literal = match data_type { - PrimitiveType::Boolean => { - if bytes.len() == 1 && bytes[0] == 0u8 { - PrimitiveLiteral::Boolean(false) - } else { - PrimitiveLiteral::Boolean(true) - } - } - PrimitiveType::Int => PrimitiveLiteral::Int(i32::from_le_bytes(bytes.try_into()?)), - PrimitiveType::Long => PrimitiveLiteral::Long(i64::from_le_bytes(bytes.try_into()?)), - PrimitiveType::Float => { - PrimitiveLiteral::Float(OrderedFloat(f32::from_le_bytes(bytes.try_into()?))) - } - PrimitiveType::Double => { - PrimitiveLiteral::Double(OrderedFloat(f64::from_le_bytes(bytes.try_into()?))) - } - PrimitiveType::Date => PrimitiveLiteral::Date(i32::from_le_bytes(bytes.try_into()?)), - PrimitiveType::Time => PrimitiveLiteral::Time(i64::from_le_bytes(bytes.try_into()?)), - PrimitiveType::Timestamp => { - PrimitiveLiteral::Timestamp(i64::from_le_bytes(bytes.try_into()?)) - } - PrimitiveType::Timestamptz => { - PrimitiveLiteral::Timestamptz(i64::from_le_bytes(bytes.try_into()?)) - } - PrimitiveType::String => { - PrimitiveLiteral::String(std::str::from_utf8(bytes)?.to_string()) - } - PrimitiveType::Uuid => { - PrimitiveLiteral::UUID(Uuid::from_u128(u128::from_be_bytes(bytes.try_into()?))) - } - PrimitiveType::Fixed(_) => PrimitiveLiteral::Fixed(Vec::from(bytes)), - PrimitiveType::Binary => PrimitiveLiteral::Binary(Vec::from(bytes)), - PrimitiveType::Decimal { - precision: _, - scale: _, - } => todo!(), - }; - Ok(Datum::new(data_type, literal)) - } - - /// Creates a boolean value. - /// - /// Example: - /// ```rust - /// use iceberg::spec::{Literal, PrimitiveLiteral, Datum}; - /// let t = Datum::bool(true); - /// - /// assert_eq!(format!("{}", t), "true".to_string()); - /// assert_eq!(Literal::from(t), Literal::Primitive(PrimitiveLiteral::Boolean(true))); - /// ``` - pub fn bool>(t: T) -> Self { - Self { - r#type: PrimitiveType::Boolean, - literal: PrimitiveLiteral::Boolean(t.into()), - } - } - - /// Creates a boolean value from string. - /// See [Parse bool from str](https://doc.rust-lang.org/stable/std/primitive.bool.html#impl-FromStr-for-bool) for reference. - /// - /// Example: - /// ```rust - /// use iceberg::spec::{Literal, PrimitiveLiteral, Datum}; - /// let t = Datum::bool_from_str("false").unwrap(); - /// - /// assert_eq!(&format!("{}", t), "false"); - /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Boolean(false)), t.into()); - /// ``` - pub fn bool_from_str>(s: S) -> Result { - let v = s.as_ref().parse::().map_err(|e| { - Error::new(ErrorKind::DataInvalid, "Can't parse string to bool.").with_source(e) - })?; - Ok(Self::bool(v)) - } - - /// Creates an 32bit integer. - /// - /// Example: - /// ```rust - /// use iceberg::spec::{Literal, PrimitiveLiteral, Datum}; - /// let t = Datum::int(23i8); - /// - /// assert_eq!(&format!("{}", t), "23"); - /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Int(23)), t.into()); - /// ``` - pub fn int>(t: T) -> Self { - Self { - r#type: PrimitiveType::Int, - literal: PrimitiveLiteral::Int(t.into()), - } - } - - /// Creates an 64bit integer. - /// - /// Example: - /// ```rust - /// use iceberg::spec::{Literal, PrimitiveLiteral, Datum}; - /// let t = Datum::long(24i8); - /// - /// assert_eq!(&format!("{t}"), "24"); - /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Long(24)), t.into()); - /// ``` - pub fn long>(t: T) -> Self { - Self { - r#type: PrimitiveType::Long, - literal: PrimitiveLiteral::Long(t.into()), - } - } - - /// Creates an 32bit floating point number. - /// - /// Example: - /// ```rust - /// use ordered_float::OrderedFloat; - /// use iceberg::spec::{Literal, PrimitiveLiteral, Datum}; - /// let t = Datum::float( 32.1f32 ); - /// - /// assert_eq!(&format!("{t}"), "32.1"); - /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Float(OrderedFloat(32.1))), t.into()); - /// ``` - pub fn float>(t: T) -> Self { - Self { - r#type: PrimitiveType::Float, - literal: PrimitiveLiteral::Float(OrderedFloat(t.into())), - } - } - - /// Creates an 64bit floating point number. - /// - /// Example: - /// ```rust - /// use ordered_float::OrderedFloat; - /// use iceberg::spec::{Literal, PrimitiveLiteral, Datum}; - /// let t = Datum::double( 32.1f64 ); - /// - /// assert_eq!(&format!("{t}"), "32.1"); - /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Double(OrderedFloat(32.1))), t.into()); - /// ``` - pub fn double>(t: T) -> Self { - Self { - r#type: PrimitiveType::Double, - literal: PrimitiveLiteral::Double(OrderedFloat(t.into())), - } - } - - /// Creates date literal from number of days from unix epoch directly. - /// - /// Example: - /// ```rust - /// - /// use iceberg::spec::{Literal, PrimitiveLiteral, Datum}; - /// // 2 days after 1970-01-01 - /// let t = Datum::date(2); - /// - /// assert_eq!(&format!("{t}"), "1970-01-03"); - /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Date(2)), t.into()); - /// ``` - pub fn date(days: i32) -> Self { - Self { - r#type: PrimitiveType::Date, - literal: PrimitiveLiteral::Date(days), - } - } - - /// Creates date literal in `%Y-%m-%d` format, assume in utc timezone. - /// - /// See [`NaiveDate::from_str`]. - /// - /// Example - /// ```rust - /// use iceberg::spec::{Literal, Datum}; - /// let t = Datum::date_from_str("1970-01-05").unwrap(); - /// - /// assert_eq!(&format!("{t}"), "1970-01-05"); - /// assert_eq!(Literal::date(4), t.into()); - /// ``` - pub fn date_from_str>(s: S) -> Result { - let t = s.as_ref().parse::().map_err(|e| { - Error::new( - ErrorKind::DataInvalid, - format!("Can't parse date from string: {}", s.as_ref()), - ) - .with_source(e) - })?; - - Ok(Self::date(date_from_naive_date(t))) - } - - /// Create date literal from calendar date (year, month and day). - /// - /// See [`NaiveDate::from_ymd_opt`]. - /// - /// Example: - /// - ///```rust - /// use iceberg::spec::{Literal, Datum}; - /// let t = Datum::date_from_ymd(1970, 1, 5).unwrap(); - /// - /// assert_eq!(&format!("{t}"), "1970-01-05"); - /// assert_eq!(Literal::date(4), t.into()); - /// ``` - pub fn date_from_ymd(year: i32, month: u32, day: u32) -> Result { - let t = NaiveDate::from_ymd_opt(year, month, day).ok_or_else(|| { - Error::new( - ErrorKind::DataInvalid, - format!("Can't create date from year: {year}, month: {month}, day: {day}"), - ) - })?; - - Ok(Self::date(date_from_naive_date(t))) - } - - /// Creates time literal in microseconds directly. - /// - /// It will return error when it's negative or too large to fit in 24 hours. - /// - /// Example: - /// - /// ```rust - /// use iceberg::spec::{Literal, Datum}; - /// let micro_secs = { - /// 1 * 3600 * 1_000_000 + // 1 hour - /// 2 * 60 * 1_000_000 + // 2 minutes - /// 1 * 1_000_000 + // 1 second - /// 888999 // microseconds - /// }; - /// - /// let t = Datum::time_micros(micro_secs).unwrap(); - /// - /// assert_eq!(&format!("{t}"), "01:02:01.888999"); - /// assert_eq!(Literal::time(micro_secs), t.into()); - /// - /// let negative_value = -100; - /// assert!(Datum::time_micros(negative_value).is_err()); - /// - /// let too_large_value = 36 * 60 * 60 * 1_000_000; // Too large to fit in 24 hours. - /// assert!(Datum::time_micros(too_large_value).is_err()); - /// ``` - pub fn time_micros(value: i64) -> Result { - ensure_data_valid!( - (0..=MAX_TIME_VALUE).contains(&value), - "Invalid value for Time type: {}", - value - ); - - Ok(Self { - r#type: PrimitiveType::Time, - literal: PrimitiveLiteral::Time(value), - }) - } - - /// Creates time literal from [`chrono::NaiveTime`]. - fn time_from_naive_time(t: NaiveTime) -> Self { - let duration = t - unix_epoch().time(); - // It's safe to unwrap here since less than 24 hours will never overflow. - let micro_secs = duration.num_microseconds().unwrap(); - - Self { - r#type: PrimitiveType::Time, - literal: PrimitiveLiteral::Time(micro_secs), - } - } - - /// Creates time literal in microseconds in `%H:%M:%S:.f` format. - /// - /// See [`NaiveTime::from_str`] for details. - /// - /// Example: - /// ```rust - /// use iceberg::spec::{Literal, Datum}; - /// let t = Datum::time_from_str("01:02:01.888999777").unwrap(); - /// - /// assert_eq!(&format!("{t}"), "01:02:01.888999"); - /// ``` - pub fn time_from_str>(s: S) -> Result { - let t = s.as_ref().parse::().map_err(|e| { - Error::new( - ErrorKind::DataInvalid, - format!("Can't parse time from string: {}", s.as_ref()), - ) - .with_source(e) - })?; - - Ok(Self::time_from_naive_time(t)) - } - - /// Creates time literal from hour, minute, second, and microseconds. - /// - /// See [`NaiveTime::from_hms_micro_opt`]. - /// - /// Example: - /// ```rust - /// - /// use iceberg::spec::{Literal, Datum}; - /// let t = Datum::time_from_hms_micro(22, 15, 33, 111).unwrap(); - /// - /// assert_eq!(&format!("{t}"), "22:15:33.000111"); - /// ``` - pub fn time_from_hms_micro(hour: u32, min: u32, sec: u32, micro: u32) -> Result { - let t = NaiveTime::from_hms_micro_opt(hour, min, sec, micro) + use std::any::Any; + use std::collections::HashMap; + use std::fmt::{Display, Formatter}; + use std::hash::Hash; + use std::ops::Index; + use std::str::FromStr; + + use bitvec::vec::BitVec; + use chrono::{DateTime, NaiveDate, NaiveDateTime, NaiveTime, TimeZone, Utc}; + use ordered_float::OrderedFloat; + use rust_decimal::Decimal; + use serde::de::{self, MapAccess}; + use serde::ser::SerializeStruct; + use serde::{Deserialize, Serialize}; + use serde_bytes::ByteBuf; + use serde_json::{Map as JsonMap, Number, Value as JsonValue}; + use uuid::Uuid; + + pub use _serde::RawLiteral; + + use crate::error::Result; + use crate::spec::values::date::{date_from_naive_date, days_to_date, unix_epoch}; + use crate::spec::values::time::microseconds_to_time; + use crate::spec::values::timestamp::microseconds_to_datetime; + use crate::spec::values::timestamptz::microseconds_to_datetimetz; + use crate::spec::MAX_DECIMAL_PRECISION; + use crate::{ensure_data_valid, Error, ErrorKind}; + + use super::datatypes::{PrimitiveType, Type}; + + /// Maximum value for [`PrimitiveType::Time`] type in microseconds, e.g. 23 hours 59 minutes 59 seconds 999999 microseconds. + const MAX_TIME_VALUE: i64 = 24 * 60 * 60 * 1_000_000i64 - 1; + + /// Values present in iceberg type + #[derive(Clone, Debug, PartialEq, Hash, Eq)] + pub enum PrimitiveLiteral { + /// 0x00 for false, non-zero byte for true + Boolean(bool), + /// Stored as 4-byte little-endian + Int(i32), + /// Stored as 8-byte little-endian + Long(i64), + /// Stored as 4-byte little-endian + Float(OrderedFloat), + /// Stored as 8-byte little-endian + Double(OrderedFloat), + /// Stores days from the 1970-01-01 in an 4-byte little-endian int + Date(i32), + /// Stores microseconds from midnight in an 8-byte little-endian long + Time(i64), + /// Timestamp without timezone + Timestamp(i64), + /// Timestamp with timezone + Timestamptz(i64), + /// UTF-8 bytes (without length) + String(String), + /// 16-byte big-endian value + UUID(Uuid), + /// Binary value + Fixed(Vec), + /// Binary value (without length) + Binary(Vec), + /// Stores unscaled value as big int. According to iceberg spec, the precision must less than 38(`MAX_DECIMAL_PRECISION`) , so i128 is suit here. + Decimal(i128), + } + + impl PrimitiveLiteral { + /// Returns true if the Literal represents a primitive type + /// that can be a NaN, and that it's value is NaN + pub fn is_nan(&self) -> bool { + match self { + PrimitiveLiteral::Double(val) => val.is_nan(), + PrimitiveLiteral::Float(val) => val.is_nan(), + _ => false, + } + } + } + + /// Literal associated with its type. The value and type pair is checked when construction, so the type and value is + /// guaranteed to be correct when used. + /// + /// By default, we decouple the type and value of a literal, so we can use avoid the cost of storing extra type info + /// for each literal. But associate type with literal can be useful in some cases, for example, in unbound expression. + #[derive(Clone, Debug, PartialEq, Hash, Eq)] + pub struct Datum { + r#type: PrimitiveType, + literal: PrimitiveLiteral, + } + + impl Serialize for Datum { + fn serialize( + &self, + serializer: S, + ) -> std::result::Result { + let mut struct_ser = serializer + .serialize_struct("Datum", 2) + .map_err(serde::ser::Error::custom)?; + struct_ser + .serialize_field("type", &self.r#type) + .map_err(serde::ser::Error::custom)?; + struct_ser + .serialize_field( + "literal", + &RawLiteral::try_from( + Literal::Primitive(self.literal.clone()), + &Type::Primitive(self.r#type.clone()), + ) + .map_err(serde::ser::Error::custom)?, + ) + .map_err(serde::ser::Error::custom)?; + struct_ser.end() + } + } + + impl<'de> Deserialize<'de> for Datum { + fn deserialize>( + deserializer: D, + ) -> std::result::Result { + #[derive(Deserialize)] + #[serde(field_identifier, rename_all = "lowercase")] + enum Field { + Type, + Literal, + } + + struct DatumVisitor; + + impl<'de> serde::de::Visitor<'de> for DatumVisitor { + type Value = Datum; + + fn expecting(&self, formatter: &mut std::fmt::Formatter) -> std::fmt::Result { + formatter.write_str("struct Datum") + } + + fn visit_seq(self, mut seq: A) -> std::result::Result + where + A: serde::de::SeqAccess<'de>, + { + let r#type = seq + .next_element::()? + .ok_or_else(|| serde::de::Error::invalid_length(0, &self))?; + let value = seq + .next_element::()? + .ok_or_else(|| serde::de::Error::invalid_length(1, &self))?; + let Literal::Primitive(primitive) = value + .try_into(&Type::Primitive(r#type.clone())) + .map_err(serde::de::Error::custom)? + .ok_or_else(|| serde::de::Error::custom("None value"))? + else { + return Err(serde::de::Error::custom("Invalid value")); + }; + + Ok(Datum::new(r#type, primitive)) + } + + fn visit_map(self, mut map: V) -> std::result::Result + where + V: MapAccess<'de>, + { + let mut raw_primitive: Option = None; + let mut r#type: Option = None; + while let Some(key) = map.next_key()? { + match key { + Field::Type => { + if r#type.is_some() { + return Err(de::Error::duplicate_field("type")); + } + r#type = Some(map.next_value()?); + } + Field::Literal => { + if raw_primitive.is_some() { + return Err(de::Error::duplicate_field("literal")); + } + raw_primitive = Some(map.next_value()?); + } + } + } + let Some(r#type) = r#type else { + return Err(serde::de::Error::missing_field("type")); + }; + let Some(raw_primitive) = raw_primitive else { + return Err(serde::de::Error::missing_field("literal")); + }; + let Literal::Primitive(primitive) = raw_primitive + .try_into(&Type::Primitive(r#type.clone())) + .map_err(serde::de::Error::custom)? + .ok_or_else(|| serde::de::Error::custom("None value"))? + else { + return Err(serde::de::Error::custom("Invalid value")); + }; + Ok(Datum::new(r#type, primitive)) + } + } + const FIELDS: &[&str] = &["type", "literal"]; + deserializer.deserialize_struct("Datum", FIELDS, DatumVisitor) + } + } + + impl PartialOrd for Datum { + fn partial_cmp(&self, other: &Self) -> Option { + match (&self.literal, &other.literal, &self.r#type, &other.r#type) { + // generate the arm with same type and same literal + ( + PrimitiveLiteral::Boolean(val), + PrimitiveLiteral::Boolean(other_val), + PrimitiveType::Boolean, + PrimitiveType::Boolean, + ) => val.partial_cmp(other_val), + ( + PrimitiveLiteral::Int(val), + PrimitiveLiteral::Int(other_val), + PrimitiveType::Int, + PrimitiveType::Int, + ) => val.partial_cmp(other_val), + ( + PrimitiveLiteral::Long(val), + PrimitiveLiteral::Long(other_val), + PrimitiveType::Long, + PrimitiveType::Long, + ) => val.partial_cmp(other_val), + ( + PrimitiveLiteral::Float(val), + PrimitiveLiteral::Float(other_val), + PrimitiveType::Float, + PrimitiveType::Float, + ) => val.partial_cmp(other_val), + ( + PrimitiveLiteral::Double(val), + PrimitiveLiteral::Double(other_val), + PrimitiveType::Double, + PrimitiveType::Double, + ) => val.partial_cmp(other_val), + ( + PrimitiveLiteral::Date(val), + PrimitiveLiteral::Date(other_val), + PrimitiveType::Date, + PrimitiveType::Date, + ) => val.partial_cmp(other_val), + ( + PrimitiveLiteral::Time(val), + PrimitiveLiteral::Time(other_val), + PrimitiveType::Time, + PrimitiveType::Time, + ) => val.partial_cmp(other_val), + ( + PrimitiveLiteral::Timestamp(val), + PrimitiveLiteral::Timestamp(other_val), + PrimitiveType::Timestamp, + PrimitiveType::Timestamp, + ) => val.partial_cmp(other_val), + ( + PrimitiveLiteral::Timestamptz(val), + PrimitiveLiteral::Timestamptz(other_val), + PrimitiveType::Timestamptz, + PrimitiveType::Timestamptz, + ) => val.partial_cmp(other_val), + ( + PrimitiveLiteral::String(val), + PrimitiveLiteral::String(other_val), + PrimitiveType::String, + PrimitiveType::String, + ) => val.partial_cmp(other_val), + ( + PrimitiveLiteral::UUID(val), + PrimitiveLiteral::UUID(other_val), + PrimitiveType::Uuid, + PrimitiveType::Uuid, + ) => val.partial_cmp(other_val), + ( + PrimitiveLiteral::Fixed(val), + PrimitiveLiteral::Fixed(other_val), + PrimitiveType::Fixed(_), + PrimitiveType::Fixed(_), + ) => val.partial_cmp(other_val), + ( + PrimitiveLiteral::Binary(val), + PrimitiveLiteral::Binary(other_val), + PrimitiveType::Binary, + PrimitiveType::Binary, + ) => val.partial_cmp(other_val), + ( + PrimitiveLiteral::Decimal(val), + PrimitiveLiteral::Decimal(other_val), + PrimitiveType::Decimal { + precision: _, + scale, + }, + PrimitiveType::Decimal { + precision: _, + scale: other_scale, + }, + ) => { + let val = Decimal::from_i128_with_scale(*val, *scale); + let other_val = Decimal::from_i128_with_scale(*other_val, *other_scale); + val.partial_cmp(&other_val) + } + _ => None, + } + } + } + + impl Display for Datum { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + match (&self.r#type, &self.literal) { + (_, PrimitiveLiteral::Boolean(val)) => write!(f, "{}", val), + (_, PrimitiveLiteral::Int(val)) => write!(f, "{}", val), + (_, PrimitiveLiteral::Long(val)) => write!(f, "{}", val), + (_, PrimitiveLiteral::Float(val)) => write!(f, "{}", val), + (_, PrimitiveLiteral::Double(val)) => write!(f, "{}", val), + (_, PrimitiveLiteral::Date(val)) => write!(f, "{}", days_to_date(*val)), + (_, PrimitiveLiteral::Time(val)) => write!(f, "{}", microseconds_to_time(*val)), + (_, PrimitiveLiteral::Timestamp(val)) => { + write!(f, "{}", microseconds_to_datetime(*val)) + } + (_, PrimitiveLiteral::Timestamptz(val)) => { + write!(f, "{}", microseconds_to_datetimetz(*val)) + } + (_, PrimitiveLiteral::String(val)) => write!(f, r#""{}""#, val), + (_, PrimitiveLiteral::UUID(val)) => write!(f, "{}", val), + (_, PrimitiveLiteral::Fixed(val)) => display_bytes(val, f), + (_, PrimitiveLiteral::Binary(val)) => display_bytes(val, f), + ( + PrimitiveType::Decimal { + precision: _, + scale, + }, + PrimitiveLiteral::Decimal(val), + ) => { + write!(f, "{}", Decimal::from_i128_with_scale(*val, *scale)) + } + (_, _) => { + unreachable!() + } + } + } + } + + fn display_bytes(bytes: &[u8], f: &mut Formatter<'_>) -> std::fmt::Result { + let mut s = String::with_capacity(bytes.len() * 2); + for b in bytes { + s.push_str(&format!("{:02X}", b)); + } + f.write_str(&s) + } + + impl From for Literal { + fn from(value: Datum) -> Self { + Literal::Primitive(value.literal) + } + } + + impl From for PrimitiveLiteral { + fn from(value: Datum) -> Self { + value.literal + } + } + + impl Datum { + /// Creates a `Datum` from a `PrimitiveType` and a `PrimitiveLiteral` + pub(crate) fn new(r#type: PrimitiveType, literal: PrimitiveLiteral) -> Self { + Datum { r#type, literal } + } + + /// Create iceberg value from bytes + pub fn try_from_bytes(bytes: &[u8], data_type: PrimitiveType) -> Result { + let literal = match data_type { + PrimitiveType::Boolean => { + if bytes.len() == 1 && bytes[0] == 0u8 { + PrimitiveLiteral::Boolean(false) + } else { + PrimitiveLiteral::Boolean(true) + } + } + PrimitiveType::Int => PrimitiveLiteral::Int(i32::from_le_bytes(bytes.try_into()?)), + PrimitiveType::Long => PrimitiveLiteral::Long(i64::from_le_bytes(bytes.try_into()?)), + PrimitiveType::Float => { + PrimitiveLiteral::Float(OrderedFloat(f32::from_le_bytes(bytes.try_into()?))) + } + PrimitiveType::Double => { + PrimitiveLiteral::Double(OrderedFloat(f64::from_le_bytes(bytes.try_into()?))) + } + PrimitiveType::Date => PrimitiveLiteral::Date(i32::from_le_bytes(bytes.try_into()?)), + PrimitiveType::Time => PrimitiveLiteral::Time(i64::from_le_bytes(bytes.try_into()?)), + PrimitiveType::Timestamp => { + PrimitiveLiteral::Timestamp(i64::from_le_bytes(bytes.try_into()?)) + } + PrimitiveType::Timestamptz => { + PrimitiveLiteral::Timestamptz(i64::from_le_bytes(bytes.try_into()?)) + } + PrimitiveType::String => { + PrimitiveLiteral::String(std::str::from_utf8(bytes)?.to_string()) + } + PrimitiveType::Uuid => { + PrimitiveLiteral::UUID(Uuid::from_u128(u128::from_be_bytes(bytes.try_into()?))) + } + PrimitiveType::Fixed(_) => PrimitiveLiteral::Fixed(Vec::from(bytes)), + PrimitiveType::Binary => PrimitiveLiteral::Binary(Vec::from(bytes)), + PrimitiveType::Decimal { + precision: _, + scale: _, + } => todo!(), + }; + Ok(Datum::new(data_type, literal)) + } + + /// Creates a boolean value. + /// + /// Example: + /// ```rust + /// use iceberg::spec::{Literal, PrimitiveLiteral, Datum}; + /// let t = Datum::bool(true); + /// + /// assert_eq!(format!("{}", t), "true".to_string()); + /// assert_eq!(Literal::from(t), Literal::Primitive(PrimitiveLiteral::Boolean(true))); + /// ``` + pub fn bool>(t: T) -> Self { + Self { + r#type: PrimitiveType::Boolean, + literal: PrimitiveLiteral::Boolean(t.into()), + } + } + + /// Creates a boolean value from string. + /// See [Parse bool from str](https://doc.rust-lang.org/stable/std/primitive.bool.html#impl-FromStr-for-bool) for reference. + /// + /// Example: + /// ```rust + /// use iceberg::spec::{Literal, PrimitiveLiteral, Datum}; + /// let t = Datum::bool_from_str("false").unwrap(); + /// + /// assert_eq!(&format!("{}", t), "false"); + /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Boolean(false)), t.into()); + /// ``` + pub fn bool_from_str>(s: S) -> Result { + let v = s.as_ref().parse::().map_err(|e| { + Error::new(ErrorKind::DataInvalid, "Can't parse string to bool.").with_source(e) + })?; + Ok(Self::bool(v)) + } + + /// Creates an 32bit integer. + /// + /// Example: + /// ```rust + /// use iceberg::spec::{Literal, PrimitiveLiteral, Datum}; + /// let t = Datum::int(23i8); + /// + /// assert_eq!(&format!("{}", t), "23"); + /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Int(23)), t.into()); + /// ``` + pub fn int>(t: T) -> Self { + Self { + r#type: PrimitiveType::Int, + literal: PrimitiveLiteral::Int(t.into()), + } + } + + /// Creates an 64bit integer. + /// + /// Example: + /// ```rust + /// use iceberg::spec::{Literal, PrimitiveLiteral, Datum}; + /// let t = Datum::long(24i8); + /// + /// assert_eq!(&format!("{t}"), "24"); + /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Long(24)), t.into()); + /// ``` + pub fn long>(t: T) -> Self { + Self { + r#type: PrimitiveType::Long, + literal: PrimitiveLiteral::Long(t.into()), + } + } + + /// Creates an 32bit floating point number. + /// + /// Example: + /// ```rust + /// use ordered_float::OrderedFloat; + /// use iceberg::spec::{Literal, PrimitiveLiteral, Datum}; + /// let t = Datum::float( 32.1f32 ); + /// + /// assert_eq!(&format!("{t}"), "32.1"); + /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Float(OrderedFloat(32.1))), t.into()); + /// ``` + pub fn float>(t: T) -> Self { + Self { + r#type: PrimitiveType::Float, + literal: PrimitiveLiteral::Float(OrderedFloat(t.into())), + } + } + + /// Creates an 64bit floating point number. + /// + /// Example: + /// ```rust + /// use ordered_float::OrderedFloat; + /// use iceberg::spec::{Literal, PrimitiveLiteral, Datum}; + /// let t = Datum::double( 32.1f64 ); + /// + /// assert_eq!(&format!("{t}"), "32.1"); + /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Double(OrderedFloat(32.1))), t.into()); + /// ``` + pub fn double>(t: T) -> Self { + Self { + r#type: PrimitiveType::Double, + literal: PrimitiveLiteral::Double(OrderedFloat(t.into())), + } + } + + /// Creates date literal from number of days from unix epoch directly. + /// + /// Example: + /// ```rust + /// + /// use iceberg::spec::{Literal, PrimitiveLiteral, Datum}; + /// // 2 days after 1970-01-01 + /// let t = Datum::date(2); + /// + /// assert_eq!(&format!("{t}"), "1970-01-03"); + /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Date(2)), t.into()); + /// ``` + pub fn date(days: i32) -> Self { + Self { + r#type: PrimitiveType::Date, + literal: PrimitiveLiteral::Date(days), + } + } + + /// Creates date literal in `%Y-%m-%d` format, assume in utc timezone. + /// + /// See [`NaiveDate::from_str`]. + /// + /// Example + /// ```rust + /// use iceberg::spec::{Literal, Datum}; + /// let t = Datum::date_from_str("1970-01-05").unwrap(); + /// + /// assert_eq!(&format!("{t}"), "1970-01-05"); + /// assert_eq!(Literal::date(4), t.into()); + /// ``` + pub fn date_from_str>(s: S) -> Result { + let t = s.as_ref().parse::().map_err(|e| { + Error::new( + ErrorKind::DataInvalid, + format!("Can't parse date from string: {}", s.as_ref()), + ) + .with_source(e) + })?; + + Ok(Self::date(date_from_naive_date(t))) + } + + /// Create date literal from calendar date (year, month and day). + /// + /// See [`NaiveDate::from_ymd_opt`]. + /// + /// Example: + /// + ///```rust + /// use iceberg::spec::{Literal, Datum}; + /// let t = Datum::date_from_ymd(1970, 1, 5).unwrap(); + /// + /// assert_eq!(&format!("{t}"), "1970-01-05"); + /// assert_eq!(Literal::date(4), t.into()); + /// ``` + pub fn date_from_ymd(year: i32, month: u32, day: u32) -> Result { + let t = NaiveDate::from_ymd_opt(year, month, day).ok_or_else(|| { + Error::new( + ErrorKind::DataInvalid, + format!("Can't create date from year: {year}, month: {month}, day: {day}"), + ) + })?; + + Ok(Self::date(date_from_naive_date(t))) + } + + /// Creates time literal in microseconds directly. + /// + /// It will return error when it's negative or too large to fit in 24 hours. + /// + /// Example: + /// + /// ```rust + /// use iceberg::spec::{Literal, Datum}; + /// let micro_secs = { + /// 1 * 3600 * 1_000_000 + // 1 hour + /// 2 * 60 * 1_000_000 + // 2 minutes + /// 1 * 1_000_000 + // 1 second + /// 888999 // microseconds + /// }; + /// + /// let t = Datum::time_micros(micro_secs).unwrap(); + /// + /// assert_eq!(&format!("{t}"), "01:02:01.888999"); + /// assert_eq!(Literal::time(micro_secs), t.into()); + /// + /// let negative_value = -100; + /// assert!(Datum::time_micros(negative_value).is_err()); + /// + /// let too_large_value = 36 * 60 * 60 * 1_000_000; // Too large to fit in 24 hours. + /// assert!(Datum::time_micros(too_large_value).is_err()); + /// ``` + pub fn time_micros(value: i64) -> Result { + ensure_data_valid!( + (0..=MAX_TIME_VALUE).contains(&value), + "Invalid value for Time type: {}", + value + ); + + Ok(Self { + r#type: PrimitiveType::Time, + literal: PrimitiveLiteral::Time(value), + }) + } + + /// Creates time literal from [`chrono::NaiveTime`]. + fn time_from_naive_time(t: NaiveTime) -> Self { + let duration = t - unix_epoch().time(); + // It's safe to unwrap here since less than 24 hours will never overflow. + let micro_secs = duration.num_microseconds().unwrap(); + + Self { + r#type: PrimitiveType::Time, + literal: PrimitiveLiteral::Time(micro_secs), + } + } + + /// Creates time literal in microseconds in `%H:%M:%S:.f` format. + /// + /// See [`NaiveTime::from_str`] for details. + /// + /// Example: + /// ```rust + /// use iceberg::spec::{Literal, Datum}; + /// let t = Datum::time_from_str("01:02:01.888999777").unwrap(); + /// + /// assert_eq!(&format!("{t}"), "01:02:01.888999"); + /// ``` + pub fn time_from_str>(s: S) -> Result { + let t = s.as_ref().parse::().map_err(|e| { + Error::new( + ErrorKind::DataInvalid, + format!("Can't parse time from string: {}", s.as_ref()), + ) + .with_source(e) + })?; + + Ok(Self::time_from_naive_time(t)) + } + + /// Creates time literal from hour, minute, second, and microseconds. + /// + /// See [`NaiveTime::from_hms_micro_opt`]. + /// + /// Example: + /// ```rust + /// + /// use iceberg::spec::{Literal, Datum}; + /// let t = Datum::time_from_hms_micro(22, 15, 33, 111).unwrap(); + /// + /// assert_eq!(&format!("{t}"), "22:15:33.000111"); + /// ``` + pub fn time_from_hms_micro(hour: u32, min: u32, sec: u32, micro: u32) -> Result { + let t = NaiveTime::from_hms_micro_opt(hour, min, sec, micro) .ok_or_else(|| Error::new( ErrorKind::DataInvalid, format!("Can't create time from hour: {hour}, min: {min}, second: {sec}, microsecond: {micro}"), ))?; - Ok(Self::time_from_naive_time(t)) - } - - /// Creates a timestamp from unix epoch in microseconds. - /// - /// Example: - /// - /// ```rust - /// - /// use iceberg::spec::Datum; - /// let t = Datum::timestamp_micros(1000); - /// - /// assert_eq!(&format!("{t}"), "1970-01-01 00:00:00.001"); - /// ``` - pub fn timestamp_micros(value: i64) -> Self { - Self { - r#type: PrimitiveType::Timestamp, - literal: PrimitiveLiteral::Timestamp(value), - } - } - - /// Creates a timestamp from [`DateTime`]. - /// - /// Example: - /// - /// ```rust - /// - /// use chrono::{NaiveDate, NaiveDateTime, TimeZone, Utc}; - /// use iceberg::spec::Datum; - /// let t = Datum::timestamp_from_datetime( - /// NaiveDate::from_ymd_opt(1992, 3, 1) - /// .unwrap() - /// .and_hms_micro_opt(1, 2, 3, 88) - /// .unwrap()); - /// - /// assert_eq!(&format!("{t}"), "1992-03-01 01:02:03.000088"); - /// ``` - pub fn timestamp_from_datetime(dt: NaiveDateTime) -> Self { - Self::timestamp_micros(dt.and_utc().timestamp_micros()) - } - - /// Parse a timestamp in [`%Y-%m-%dT%H:%M:%S%.f`] format. - /// - /// See [`NaiveDateTime::from_str`]. - /// - /// Example: - /// - /// ```rust - /// use chrono::{DateTime, FixedOffset, NaiveDate, NaiveDateTime, NaiveTime}; - /// use iceberg::spec::{Literal, Datum}; - /// let t = Datum::timestamp_from_str("1992-03-01T01:02:03.000088").unwrap(); - /// - /// assert_eq!(&format!("{t}"), "1992-03-01 01:02:03.000088"); - /// ``` - pub fn timestamp_from_str>(s: S) -> Result { - let dt = s.as_ref().parse::().map_err(|e| { - Error::new(ErrorKind::DataInvalid, "Can't parse timestamp.").with_source(e) - })?; - - Ok(Self::timestamp_from_datetime(dt)) - } - - /// Creates a timestamp with timezone from unix epoch in microseconds. - /// - /// Example: - /// - /// ```rust - /// - /// use iceberg::spec::Datum; - /// let t = Datum::timestamptz_micros(1000); - /// - /// assert_eq!(&format!("{t}"), "1970-01-01 00:00:00.001 UTC"); - /// ``` - pub fn timestamptz_micros(value: i64) -> Self { - Self { - r#type: PrimitiveType::Timestamptz, - literal: PrimitiveLiteral::Timestamptz(value), - } - } - - /// Creates a timestamp with timezone from [`DateTime`]. - /// Example: - /// - /// ```rust - /// - /// use chrono::{TimeZone, Utc}; - /// use iceberg::spec::Datum; - /// let t = Datum::timestamptz_from_datetime(Utc.timestamp_opt(1000, 0).unwrap()); - /// - /// assert_eq!(&format!("{t}"), "1970-01-01 00:16:40 UTC"); - /// ``` - pub fn timestamptz_from_datetime(dt: DateTime) -> Self { - Self::timestamptz_micros(dt.with_timezone(&Utc).timestamp_micros()) - } - - /// Parse timestamp with timezone in RFC3339 format. - /// - /// See [`DateTime::from_str`]. - /// - /// Example: - /// - /// ```rust - /// use chrono::{DateTime, FixedOffset, NaiveDate, NaiveDateTime, NaiveTime}; - /// use iceberg::spec::{Literal, Datum}; - /// let t = Datum::timestamptz_from_str("1992-03-01T01:02:03.000088+08:00").unwrap(); - /// - /// assert_eq!(&format!("{t}"), "1992-02-29 17:02:03.000088 UTC"); - /// ``` - pub fn timestamptz_from_str>(s: S) -> Result { - let dt = DateTime::::from_str(s.as_ref()).map_err(|e| { - Error::new(ErrorKind::DataInvalid, "Can't parse datetime.").with_source(e) - })?; - - Ok(Self::timestamptz_from_datetime(dt)) - } - - /// Creates a string literal. - /// - /// Example: - /// - /// ```rust - /// use iceberg::spec::Datum; - /// let t = Datum::string("ss"); - /// - /// assert_eq!(&format!("{t}"), r#""ss""#); - /// ``` - pub fn string(s: S) -> Self { - Self { - r#type: PrimitiveType::String, - literal: PrimitiveLiteral::String(s.to_string()), - } - } - - /// Creates uuid literal. - /// - /// Example: - /// - /// ```rust - /// use uuid::uuid; - /// use iceberg::spec::Datum; - /// let t = Datum::uuid(uuid!("a1a2a3a4-b1b2-c1c2-d1d2-d3d4d5d6d7d8")); - /// - /// assert_eq!(&format!("{t}"), "a1a2a3a4-b1b2-c1c2-d1d2-d3d4d5d6d7d8"); - /// ``` - pub fn uuid(uuid: Uuid) -> Self { - Self { - r#type: PrimitiveType::Uuid, - literal: PrimitiveLiteral::UUID(uuid), - } - } - - /// Creates uuid from str. See [`Uuid::parse_str`]. - /// - /// Example: - /// - /// ```rust - /// use iceberg::spec::{Datum}; - /// let t = Datum::uuid_from_str("a1a2a3a4-b1b2-c1c2-d1d2-d3d4d5d6d7d8").unwrap(); - /// - /// assert_eq!(&format!("{t}"), "a1a2a3a4-b1b2-c1c2-d1d2-d3d4d5d6d7d8"); - /// ``` - pub fn uuid_from_str>(s: S) -> Result { - let uuid = Uuid::parse_str(s.as_ref()).map_err(|e| { - Error::new( - ErrorKind::DataInvalid, - format!("Can't parse uuid from string: {}", s.as_ref()), - ) - .with_source(e) - })?; - Ok(Self::uuid(uuid)) - } - - /// Creates a fixed literal from bytes. - /// - /// Example: - /// - /// ```rust - /// use iceberg::spec::{Literal, PrimitiveLiteral, Datum}; - /// let t = Datum::fixed(vec![1u8, 2u8]); - /// - /// assert_eq!(&format!("{t}"), "0102"); - /// ``` - pub fn fixed>(input: I) -> Self { - let value: Vec = input.into_iter().collect(); - Self { - r#type: PrimitiveType::Fixed(value.len() as u64), - literal: PrimitiveLiteral::Fixed(value), - } - } - - /// Creates a binary literal from bytes. - /// - /// Example: - /// - /// ```rust - /// use iceberg::spec::Datum; - /// let t = Datum::binary(vec![1u8, 100u8]); - /// - /// assert_eq!(&format!("{t}"), "0164"); - /// ``` - pub fn binary>(input: I) -> Self { - Self { - r#type: PrimitiveType::Binary, - literal: PrimitiveLiteral::Binary(input.into_iter().collect()), - } - } - - /// Creates decimal literal from string. See [`Decimal::from_str_exact`]. - /// - /// Example: - /// - /// ```rust - /// use itertools::assert_equal; - /// use rust_decimal::Decimal; - /// use iceberg::spec::Datum; - /// let t = Datum::decimal_from_str("123.45").unwrap(); - /// - /// assert_eq!(&format!("{t}"), "123.45"); - /// ``` - pub fn decimal_from_str>(s: S) -> Result { - let decimal = Decimal::from_str_exact(s.as_ref()).map_err(|e| { - Error::new(ErrorKind::DataInvalid, "Can't parse decimal.").with_source(e) - })?; - - Self::decimal(decimal) - } - - /// Try to create a decimal literal from [`Decimal`]. - /// - /// Example: - /// - /// ```rust - /// use rust_decimal::Decimal; - /// use iceberg::spec::Datum; - /// - /// let t = Datum::decimal(Decimal::new(123, 2)).unwrap(); - /// - /// assert_eq!(&format!("{t}"), "1.23"); - /// ``` - pub fn decimal(value: impl Into) -> Result { - let decimal = value.into(); - let scale = decimal.scale(); - - let r#type = Type::decimal(MAX_DECIMAL_PRECISION, scale)?; - if let Type::Primitive(p) = r#type { - Ok(Self { - r#type: p, - literal: PrimitiveLiteral::Decimal(decimal.mantissa()), - }) - } else { - unreachable!("Decimal type must be primitive.") - } - } - - /// Convert the datum to `target_type`. - pub fn to(self, target_type: &Type) -> Result { - // TODO: We should allow more type conversions - match target_type { - Type::Primitive(typ) if typ == &self.r#type => Ok(self), - _ => Err(Error::new( - ErrorKind::DataInvalid, - format!( - "Can't convert datum from {} type to {} type.", - self.r#type, target_type - ), - )), - } - } - - /// Get the primitive literal from datum. - pub fn literal(&self) -> &PrimitiveLiteral { - &self.literal - } - - /// Get the primitive type from datum. - pub fn data_type(&self) -> &PrimitiveType { - &self.r#type - } - - /// Returns true if the Literal represents a primitive type - /// that can be a NaN, and that it's value is NaN - pub fn is_nan(&self) -> bool { - match self.literal { - PrimitiveLiteral::Double(val) => val.is_nan(), - PrimitiveLiteral::Float(val) => val.is_nan(), - _ => false, - } - } -} - -/// Map is a collection of key-value pairs with a key type and a value type. -/// It used in Literal::Map, to make it hashable, the order of key-value pairs is stored in a separate vector -/// so that we can hash the map in a deterministic way. But it also means that the order of key-value pairs is matter -/// for the hash value. -#[derive(Clone, Debug, PartialEq, Eq)] -pub struct Map { - index: HashMap, - pair: Vec<(Literal, Option)>, -} - -impl Map { - /// Creates a new empty map. - pub fn new() -> Self { - Self { - index: HashMap::new(), - pair: Vec::new(), - } - } - - /// Return the number of key-value pairs in the map. - pub fn len(&self) -> usize { - self.pair.len() - } - - /// Returns true if the map contains no elements. - pub fn is_empty(&self) -> bool { - self.pair.is_empty() - } - - /// Inserts a key-value pair into the map. - /// If the map did not have this key present, None is returned. - /// If the map did have this key present, the value is updated, and the old value is returned. - pub fn insert(&mut self, key: Literal, value: Option) -> Option> { - if let Some(index) = self.index.get(&key) { - let old_value = std::mem::replace(&mut self.pair[*index].1, value); - Some(old_value) - } else { - self.pair.push((key.clone(), value)); - self.index.insert(key, self.pair.len() - 1); - None - } - } - - /// Returns a reference to the value corresponding to the key. - /// If the key is not present in the map, None is returned. - pub fn get(&self, key: &Literal) -> Option<&Option> { - self.index.get(key).map(|index| &self.pair[*index].1) - } - - /// The order of map is matter, so this method used to compare two maps has same key-value pairs without considering the order. - pub fn has_same_content(&self, other: &Map) -> bool { - if self.len() != other.len() { - return false; - } - - for (key, value) in &self.pair { - match other.get(key) { - Some(other_value) if value == other_value => (), - _ => return false, - } - } - - true - } -} - -impl Default for Map { - fn default() -> Self { - Self::new() - } -} - -impl Hash for Map { - fn hash(&self, state: &mut H) { - for (key, value) in &self.pair { - key.hash(state); - value.hash(state); - } - } -} - -impl FromIterator<(Literal, Option)> for Map { - fn from_iter)>>(iter: T) -> Self { - let mut map = Map::new(); - for (key, value) in iter { - map.insert(key, value); - } - map - } -} - -impl IntoIterator for Map { - type Item = (Literal, Option); - type IntoIter = std::vec::IntoIter; - - fn into_iter(self) -> Self::IntoIter { - self.pair.into_iter() - } -} - -impl From<[(Literal, Option); N]> for Map { - fn from(value: [(Literal, Option); N]) -> Self { - value.iter().cloned().collect() - } -} - -/// Values present in iceberg type -#[derive(Clone, Debug, PartialEq, Eq, Hash)] -pub enum Literal { - /// A primitive value - Primitive(PrimitiveLiteral), - /// A struct is a tuple of typed values. Each field in the tuple is named and has an integer id that is unique in the table schema. - /// Each field can be either optional or required, meaning that values can (or cannot) be null. Fields may be any type. - /// Fields may have an optional comment or doc string. Fields can have default values. - Struct(Struct), - /// A list is a collection of values with some element type. - /// The element field has an integer id that is unique in the table schema. - /// Elements can be either optional or required. Element types may be any type. - List(Vec>), - /// A map is a collection of key-value pairs with a key type and a value type. - /// Both the key field and value field each have an integer id that is unique in the table schema. - /// Map keys are required and map values can be either optional or required. Both map keys and map values may be any type, including nested types. - Map(Map), -} - -impl Literal { - /// Creates a boolean value. - /// - /// Example: - /// ```rust - /// use iceberg::spec::{Literal, PrimitiveLiteral}; - /// let t = Literal::bool(true); - /// - /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Boolean(true)), t); - /// ``` - pub fn bool>(t: T) -> Self { - Self::Primitive(PrimitiveLiteral::Boolean(t.into())) - } - - /// Creates a boolean value from string. - /// See [Parse bool from str](https://doc.rust-lang.org/stable/std/primitive.bool.html#impl-FromStr-for-bool) for reference. - /// - /// Example: - /// ```rust - /// use iceberg::spec::{Literal, PrimitiveLiteral}; - /// let t = Literal::bool_from_str("false").unwrap(); - /// - /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Boolean(false)), t); - /// ``` - pub fn bool_from_str>(s: S) -> Result { - let v = s.as_ref().parse::().map_err(|e| { - Error::new(ErrorKind::DataInvalid, "Can't parse string to bool.").with_source(e) - })?; - Ok(Self::Primitive(PrimitiveLiteral::Boolean(v))) - } - - /// Creates an 32bit integer. - /// - /// Example: - /// ```rust - /// use iceberg::spec::{Literal, PrimitiveLiteral}; - /// let t = Literal::int(23i8); - /// - /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Int(23)), t); - /// ``` - pub fn int>(t: T) -> Self { - Self::Primitive(PrimitiveLiteral::Int(t.into())) - } - - /// Creates an 64bit integer. - /// - /// Example: - /// ```rust - /// use iceberg::spec::{Literal, PrimitiveLiteral}; - /// let t = Literal::long(24i8); - /// - /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Long(24)), t); - /// ``` - pub fn long>(t: T) -> Self { - Self::Primitive(PrimitiveLiteral::Long(t.into())) - } - - /// Creates an 32bit floating point number. - /// - /// Example: - /// ```rust - /// use ordered_float::OrderedFloat; - /// use iceberg::spec::{Literal, PrimitiveLiteral}; - /// let t = Literal::float( 32.1f32 ); - /// - /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Float(OrderedFloat(32.1))), t); - /// ``` - pub fn float>(t: T) -> Self { - Self::Primitive(PrimitiveLiteral::Float(OrderedFloat(t.into()))) - } - - /// Creates an 32bit floating point number. - /// - /// Example: - /// ```rust - /// use ordered_float::OrderedFloat; - /// use iceberg::spec::{Literal, PrimitiveLiteral}; - /// let t = Literal::double( 32.1f64 ); - /// - /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Double(OrderedFloat(32.1))), t); - /// ``` - pub fn double>(t: T) -> Self { - Self::Primitive(PrimitiveLiteral::Double(OrderedFloat(t.into()))) - } - - /// Creates date literal from number of days from unix epoch directly. - pub fn date(days: i32) -> Self { - Self::Primitive(PrimitiveLiteral::Date(days)) - } - - /// Creates a date in `%Y-%m-%d` format, assume in utc timezone. - /// - /// See [`NaiveDate::from_str`]. - /// - /// Example - /// ```rust - /// use iceberg::spec::Literal; - /// let t = Literal::date_from_str("1970-01-03").unwrap(); - /// - /// assert_eq!(Literal::date(2), t); - /// ``` - pub fn date_from_str>(s: S) -> Result { - let t = s.as_ref().parse::().map_err(|e| { - Error::new( - ErrorKind::DataInvalid, - format!("Can't parse date from string: {}", s.as_ref()), - ) - .with_source(e) - })?; - - Ok(Self::date(date_from_naive_date(t))) - } - - /// Create a date from calendar date (year, month and day). - /// - /// See [`NaiveDate::from_ymd_opt`]. - /// - /// Example: - /// - ///```rust - /// use iceberg::spec::Literal; - /// let t = Literal::date_from_ymd(1970, 1, 5).unwrap(); - /// - /// assert_eq!(Literal::date(4), t); - /// ``` - pub fn date_from_ymd(year: i32, month: u32, day: u32) -> Result { - let t = NaiveDate::from_ymd_opt(year, month, day).ok_or_else(|| { - Error::new( - ErrorKind::DataInvalid, - format!("Can't create date from year: {year}, month: {month}, day: {day}"), - ) - })?; - - Ok(Self::date(date_from_naive_date(t))) - } - - /// Creates time in microseconds directly - pub fn time(value: i64) -> Self { - Self::Primitive(PrimitiveLiteral::Time(value)) - } - - /// Creates time literal from [`chrono::NaiveTime`]. - fn time_from_naive_time(t: NaiveTime) -> Self { - let duration = t - unix_epoch().time(); - // It's safe to unwrap here since less than 24 hours will never overflow. - let micro_secs = duration.num_microseconds().unwrap(); - - Literal::time(micro_secs) - } - - /// Creates time in microseconds in `%H:%M:%S:.f` format. - /// - /// See [`NaiveTime::from_str`] for details. - /// - /// Example: - /// ```rust - /// use iceberg::spec::Literal; - /// let t = Literal::time_from_str("01:02:01.888999777").unwrap(); - /// - /// let micro_secs = { - /// 1 * 3600 * 1_000_000 + // 1 hour - /// 2 * 60 * 1_000_000 + // 2 minutes - /// 1 * 1_000_000 + // 1 second - /// 888999 // microseconds - /// }; - /// assert_eq!(Literal::time(micro_secs), t); - /// ``` - pub fn time_from_str>(s: S) -> Result { - let t = s.as_ref().parse::().map_err(|e| { - Error::new( - ErrorKind::DataInvalid, - format!("Can't parse time from string: {}", s.as_ref()), - ) - .with_source(e) - })?; - - Ok(Self::time_from_naive_time(t)) - } - - /// Creates time literal from hour, minute, second, and microseconds. - /// - /// See [`NaiveTime::from_hms_micro_opt`]. - /// - /// Example: - /// ```rust - /// - /// use iceberg::spec::Literal; - /// let t = Literal::time_from_hms_micro(22, 15, 33, 111).unwrap(); - /// - /// assert_eq!(Literal::time_from_str("22:15:33.000111").unwrap(), t); - /// ``` - pub fn time_from_hms_micro(hour: u32, min: u32, sec: u32, micro: u32) -> Result { - let t = NaiveTime::from_hms_micro_opt(hour, min, sec, micro) + Ok(Self::time_from_naive_time(t)) + } + + /// Creates a timestamp from unix epoch in microseconds. + /// + /// Example: + /// + /// ```rust + /// + /// use iceberg::spec::Datum; + /// let t = Datum::timestamp_micros(1000); + /// + /// assert_eq!(&format!("{t}"), "1970-01-01 00:00:00.001"); + /// ``` + pub fn timestamp_micros(value: i64) -> Self { + Self { + r#type: PrimitiveType::Timestamp, + literal: PrimitiveLiteral::Timestamp(value), + } + } + + /// Creates a timestamp from [`DateTime`]. + /// + /// Example: + /// + /// ```rust + /// + /// use chrono::{NaiveDate, NaiveDateTime, TimeZone, Utc}; + /// use iceberg::spec::Datum; + /// let t = Datum::timestamp_from_datetime( + /// NaiveDate::from_ymd_opt(1992, 3, 1) + /// .unwrap() + /// .and_hms_micro_opt(1, 2, 3, 88) + /// .unwrap()); + /// + /// assert_eq!(&format!("{t}"), "1992-03-01 01:02:03.000088"); + /// ``` + pub fn timestamp_from_datetime(dt: NaiveDateTime) -> Self { + Self::timestamp_micros(dt.and_utc().timestamp_micros()) + } + + /// Parse a timestamp in [`%Y-%m-%dT%H:%M:%S%.f`] format. + /// + /// See [`NaiveDateTime::from_str`]. + /// + /// Example: + /// + /// ```rust + /// use chrono::{DateTime, FixedOffset, NaiveDate, NaiveDateTime, NaiveTime}; + /// use iceberg::spec::{Literal, Datum}; + /// let t = Datum::timestamp_from_str("1992-03-01T01:02:03.000088").unwrap(); + /// + /// assert_eq!(&format!("{t}"), "1992-03-01 01:02:03.000088"); + /// ``` + pub fn timestamp_from_str>(s: S) -> Result { + let dt = s.as_ref().parse::().map_err(|e| { + Error::new(ErrorKind::DataInvalid, "Can't parse timestamp.").with_source(e) + })?; + + Ok(Self::timestamp_from_datetime(dt)) + } + + /// Creates a timestamp with timezone from unix epoch in microseconds. + /// + /// Example: + /// + /// ```rust + /// + /// use iceberg::spec::Datum; + /// let t = Datum::timestamptz_micros(1000); + /// + /// assert_eq!(&format!("{t}"), "1970-01-01 00:00:00.001 UTC"); + /// ``` + pub fn timestamptz_micros(value: i64) -> Self { + Self { + r#type: PrimitiveType::Timestamptz, + literal: PrimitiveLiteral::Timestamptz(value), + } + } + + /// Creates a timestamp with timezone from [`DateTime`]. + /// Example: + /// + /// ```rust + /// + /// use chrono::{TimeZone, Utc}; + /// use iceberg::spec::Datum; + /// let t = Datum::timestamptz_from_datetime(Utc.timestamp_opt(1000, 0).unwrap()); + /// + /// assert_eq!(&format!("{t}"), "1970-01-01 00:16:40 UTC"); + /// ``` + pub fn timestamptz_from_datetime(dt: DateTime) -> Self { + Self::timestamptz_micros(dt.with_timezone(&Utc).timestamp_micros()) + } + + /// Parse timestamp with timezone in RFC3339 format. + /// + /// See [`DateTime::from_str`]. + /// + /// Example: + /// + /// ```rust + /// use chrono::{DateTime, FixedOffset, NaiveDate, NaiveDateTime, NaiveTime}; + /// use iceberg::spec::{Literal, Datum}; + /// let t = Datum::timestamptz_from_str("1992-03-01T01:02:03.000088+08:00").unwrap(); + /// + /// assert_eq!(&format!("{t}"), "1992-02-29 17:02:03.000088 UTC"); + /// ``` + pub fn timestamptz_from_str>(s: S) -> Result { + let dt = DateTime::::from_str(s.as_ref()).map_err(|e| { + Error::new(ErrorKind::DataInvalid, "Can't parse datetime.").with_source(e) + })?; + + Ok(Self::timestamptz_from_datetime(dt)) + } + + /// Creates a string literal. + /// + /// Example: + /// + /// ```rust + /// use iceberg::spec::Datum; + /// let t = Datum::string("ss"); + /// + /// assert_eq!(&format!("{t}"), r#""ss""#); + /// ``` + pub fn string(s: S) -> Self { + Self { + r#type: PrimitiveType::String, + literal: PrimitiveLiteral::String(s.to_string()), + } + } + + /// Creates uuid literal. + /// + /// Example: + /// + /// ```rust + /// use uuid::uuid; + /// use iceberg::spec::Datum; + /// let t = Datum::uuid(uuid!("a1a2a3a4-b1b2-c1c2-d1d2-d3d4d5d6d7d8")); + /// + /// assert_eq!(&format!("{t}"), "a1a2a3a4-b1b2-c1c2-d1d2-d3d4d5d6d7d8"); + /// ``` + pub fn uuid(uuid: Uuid) -> Self { + Self { + r#type: PrimitiveType::Uuid, + literal: PrimitiveLiteral::UUID(uuid), + } + } + + /// Creates uuid from str. See [`Uuid::parse_str`]. + /// + /// Example: + /// + /// ```rust + /// use iceberg::spec::{Datum}; + /// let t = Datum::uuid_from_str("a1a2a3a4-b1b2-c1c2-d1d2-d3d4d5d6d7d8").unwrap(); + /// + /// assert_eq!(&format!("{t}"), "a1a2a3a4-b1b2-c1c2-d1d2-d3d4d5d6d7d8"); + /// ``` + pub fn uuid_from_str>(s: S) -> Result { + let uuid = Uuid::parse_str(s.as_ref()).map_err(|e| { + Error::new( + ErrorKind::DataInvalid, + format!("Can't parse uuid from string: {}", s.as_ref()), + ) + .with_source(e) + })?; + Ok(Self::uuid(uuid)) + } + + /// Creates a fixed literal from bytes. + /// + /// Example: + /// + /// ```rust + /// use iceberg::spec::{Literal, PrimitiveLiteral, Datum}; + /// let t = Datum::fixed(vec![1u8, 2u8]); + /// + /// assert_eq!(&format!("{t}"), "0102"); + /// ``` + pub fn fixed>(input: I) -> Self { + let value: Vec = input.into_iter().collect(); + Self { + r#type: PrimitiveType::Fixed(value.len() as u64), + literal: PrimitiveLiteral::Fixed(value), + } + } + + /// Creates a binary literal from bytes. + /// + /// Example: + /// + /// ```rust + /// use iceberg::spec::Datum; + /// let t = Datum::binary(vec![1u8, 100u8]); + /// + /// assert_eq!(&format!("{t}"), "0164"); + /// ``` + pub fn binary>(input: I) -> Self { + Self { + r#type: PrimitiveType::Binary, + literal: PrimitiveLiteral::Binary(input.into_iter().collect()), + } + } + + /// Creates decimal literal from string. See [`Decimal::from_str_exact`]. + /// + /// Example: + /// + /// ```rust + /// use itertools::assert_equal; + /// use rust_decimal::Decimal; + /// use iceberg::spec::Datum; + /// let t = Datum::decimal_from_str("123.45").unwrap(); + /// + /// assert_eq!(&format!("{t}"), "123.45"); + /// ``` + pub fn decimal_from_str>(s: S) -> Result { + let decimal = Decimal::from_str_exact(s.as_ref()).map_err(|e| { + Error::new(ErrorKind::DataInvalid, "Can't parse decimal.").with_source(e) + })?; + + Self::decimal(decimal) + } + + /// Try to create a decimal literal from [`Decimal`]. + /// + /// Example: + /// + /// ```rust + /// use rust_decimal::Decimal; + /// use iceberg::spec::Datum; + /// + /// let t = Datum::decimal(Decimal::new(123, 2)).unwrap(); + /// + /// assert_eq!(&format!("{t}"), "1.23"); + /// ``` + pub fn decimal(value: impl Into) -> Result { + let decimal = value.into(); + let scale = decimal.scale(); + + let r#type = Type::decimal(MAX_DECIMAL_PRECISION, scale)?; + if let Type::Primitive(p) = r#type { + Ok(Self { + r#type: p, + literal: PrimitiveLiteral::Decimal(decimal.mantissa()), + }) + } else { + unreachable!("Decimal type must be primitive.") + } + } + + /// Convert the datum to `target_type`. + pub fn to(self, target_type: &Type) -> Result { + // TODO: We should allow more type conversions + match target_type { + Type::Primitive(typ) if typ == &self.r#type => Ok(self), + _ => Err(Error::new( + ErrorKind::DataInvalid, + format!( + "Can't convert datum from {} type to {} type.", + self.r#type, target_type + ), + )), + } + } + + /// Get the primitive literal from datum. + pub fn literal(&self) -> &PrimitiveLiteral { + &self.literal + } + + /// Get the primitive type from datum. + pub fn data_type(&self) -> &PrimitiveType { + &self.r#type + } + + /// Returns true if the Literal represents a primitive type + /// that can be a NaN, and that it's value is NaN + pub fn is_nan(&self) -> bool { + match self.literal { + PrimitiveLiteral::Double(val) => val.is_nan(), + PrimitiveLiteral::Float(val) => val.is_nan(), + _ => false, + } + } + } + + /// Map is a collection of key-value pairs with a key type and a value type. + /// It used in Literal::Map, to make it hashable, the order of key-value pairs is stored in a separate vector + /// so that we can hash the map in a deterministic way. But it also means that the order of key-value pairs is matter + /// for the hash value. + #[derive(Clone, Debug, PartialEq, Eq)] + pub struct Map { + index: HashMap, + pair: Vec<(Literal, Option)>, + } + + impl Map { + /// Creates a new empty map. + pub fn new() -> Self { + Self { + index: HashMap::new(), + pair: Vec::new(), + } + } + + /// Return the number of key-value pairs in the map. + pub fn len(&self) -> usize { + self.pair.len() + } + + /// Returns true if the map contains no elements. + pub fn is_empty(&self) -> bool { + self.pair.is_empty() + } + + /// Inserts a key-value pair into the map. + /// If the map did not have this key present, None is returned. + /// If the map did have this key present, the value is updated, and the old value is returned. + pub fn insert(&mut self, key: Literal, value: Option) -> Option> { + if let Some(index) = self.index.get(&key) { + let old_value = std::mem::replace(&mut self.pair[*index].1, value); + Some(old_value) + } else { + self.pair.push((key.clone(), value)); + self.index.insert(key, self.pair.len() - 1); + None + } + } + + /// Returns a reference to the value corresponding to the key. + /// If the key is not present in the map, None is returned. + pub fn get(&self, key: &Literal) -> Option<&Option> { + self.index.get(key).map(|index| &self.pair[*index].1) + } + + /// The order of map is matter, so this method used to compare two maps has same key-value pairs without considering the order. + pub fn has_same_content(&self, other: &Map) -> bool { + if self.len() != other.len() { + return false; + } + + for (key, value) in &self.pair { + match other.get(key) { + Some(other_value) if value == other_value => (), + _ => return false, + } + } + + true + } + } + + impl Default for Map { + fn default() -> Self { + Self::new() + } + } + + impl Hash for Map { + fn hash(&self, state: &mut H) { + for (key, value) in &self.pair { + key.hash(state); + value.hash(state); + } + } + } + + impl FromIterator<(Literal, Option)> for Map { + fn from_iter)>>(iter: T) -> Self { + let mut map = Map::new(); + for (key, value) in iter { + map.insert(key, value); + } + map + } + } + + impl IntoIterator for Map { + type Item = (Literal, Option); + type IntoIter = std::vec::IntoIter; + + fn into_iter(self) -> Self::IntoIter { + self.pair.into_iter() + } + } + + impl From<[(Literal, Option); N]> for Map { + fn from(value: [(Literal, Option); N]) -> Self { + value.iter().cloned().collect() + } + } + + /// Values present in iceberg type + #[derive(Clone, Debug, PartialEq, Eq, Hash)] + pub enum Literal { + /// A primitive value + Primitive(PrimitiveLiteral), + /// A struct is a tuple of typed values. Each field in the tuple is named and has an integer id that is unique in the table schema. + /// Each field can be either optional or required, meaning that values can (or cannot) be null. Fields may be any type. + /// Fields may have an optional comment or doc string. Fields can have default values. + Struct(Struct), + /// A list is a collection of values with some element type. + /// The element field has an integer id that is unique in the table schema. + /// Elements can be either optional or required. Element types may be any type. + List(Vec>), + /// A map is a collection of key-value pairs with a key type and a value type. + /// Both the key field and value field each have an integer id that is unique in the table schema. + /// Map keys are required and map values can be either optional or required. Both map keys and map values may be any type, including nested types. + Map(Map), + } + + impl Literal { + /// Creates a boolean value. + /// + /// Example: + /// ```rust + /// use iceberg::spec::{Literal, PrimitiveLiteral}; + /// let t = Literal::bool(true); + /// + /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Boolean(true)), t); + /// ``` + pub fn bool>(t: T) -> Self { + Self::Primitive(PrimitiveLiteral::Boolean(t.into())) + } + + /// Creates a boolean value from string. + /// See [Parse bool from str](https://doc.rust-lang.org/stable/std/primitive.bool.html#impl-FromStr-for-bool) for reference. + /// + /// Example: + /// ```rust + /// use iceberg::spec::{Literal, PrimitiveLiteral}; + /// let t = Literal::bool_from_str("false").unwrap(); + /// + /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Boolean(false)), t); + /// ``` + pub fn bool_from_str>(s: S) -> Result { + let v = s.as_ref().parse::().map_err(|e| { + Error::new(ErrorKind::DataInvalid, "Can't parse string to bool.").with_source(e) + })?; + Ok(Self::Primitive(PrimitiveLiteral::Boolean(v))) + } + + /// Creates an 32bit integer. + /// + /// Example: + /// ```rust + /// use iceberg::spec::{Literal, PrimitiveLiteral}; + /// let t = Literal::int(23i8); + /// + /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Int(23)), t); + /// ``` + pub fn int>(t: T) -> Self { + Self::Primitive(PrimitiveLiteral::Int(t.into())) + } + + /// Creates an 64bit integer. + /// + /// Example: + /// ```rust + /// use iceberg::spec::{Literal, PrimitiveLiteral}; + /// let t = Literal::long(24i8); + /// + /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Long(24)), t); + /// ``` + pub fn long>(t: T) -> Self { + Self::Primitive(PrimitiveLiteral::Long(t.into())) + } + + /// Creates an 32bit floating point number. + /// + /// Example: + /// ```rust + /// use ordered_float::OrderedFloat; + /// use iceberg::spec::{Literal, PrimitiveLiteral}; + /// let t = Literal::float( 32.1f32 ); + /// + /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Float(OrderedFloat(32.1))), t); + /// ``` + pub fn float>(t: T) -> Self { + Self::Primitive(PrimitiveLiteral::Float(OrderedFloat(t.into()))) + } + + /// Creates an 32bit floating point number. + /// + /// Example: + /// ```rust + /// use ordered_float::OrderedFloat; + /// use iceberg::spec::{Literal, PrimitiveLiteral}; + /// let t = Literal::double( 32.1f64 ); + /// + /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Double(OrderedFloat(32.1))), t); + /// ``` + pub fn double>(t: T) -> Self { + Self::Primitive(PrimitiveLiteral::Double(OrderedFloat(t.into()))) + } + + /// Creates date literal from number of days from unix epoch directly. + pub fn date(days: i32) -> Self { + Self::Primitive(PrimitiveLiteral::Date(days)) + } + + /// Creates a date in `%Y-%m-%d` format, assume in utc timezone. + /// + /// See [`NaiveDate::from_str`]. + /// + /// Example + /// ```rust + /// use iceberg::spec::Literal; + /// let t = Literal::date_from_str("1970-01-03").unwrap(); + /// + /// assert_eq!(Literal::date(2), t); + /// ``` + pub fn date_from_str>(s: S) -> Result { + let t = s.as_ref().parse::().map_err(|e| { + Error::new( + ErrorKind::DataInvalid, + format!("Can't parse date from string: {}", s.as_ref()), + ) + .with_source(e) + })?; + + Ok(Self::date(date_from_naive_date(t))) + } + + /// Create a date from calendar date (year, month and day). + /// + /// See [`NaiveDate::from_ymd_opt`]. + /// + /// Example: + /// + ///```rust + /// use iceberg::spec::Literal; + /// let t = Literal::date_from_ymd(1970, 1, 5).unwrap(); + /// + /// assert_eq!(Literal::date(4), t); + /// ``` + pub fn date_from_ymd(year: i32, month: u32, day: u32) -> Result { + let t = NaiveDate::from_ymd_opt(year, month, day).ok_or_else(|| { + Error::new( + ErrorKind::DataInvalid, + format!("Can't create date from year: {year}, month: {month}, day: {day}"), + ) + })?; + + Ok(Self::date(date_from_naive_date(t))) + } + + /// Creates time in microseconds directly + pub fn time(value: i64) -> Self { + Self::Primitive(PrimitiveLiteral::Time(value)) + } + + /// Creates time literal from [`chrono::NaiveTime`]. + fn time_from_naive_time(t: NaiveTime) -> Self { + let duration = t - unix_epoch().time(); + // It's safe to unwrap here since less than 24 hours will never overflow. + let micro_secs = duration.num_microseconds().unwrap(); + + Literal::time(micro_secs) + } + + /// Creates time in microseconds in `%H:%M:%S:.f` format. + /// + /// See [`NaiveTime::from_str`] for details. + /// + /// Example: + /// ```rust + /// use iceberg::spec::Literal; + /// let t = Literal::time_from_str("01:02:01.888999777").unwrap(); + /// + /// let micro_secs = { + /// 1 * 3600 * 1_000_000 + // 1 hour + /// 2 * 60 * 1_000_000 + // 2 minutes + /// 1 * 1_000_000 + // 1 second + /// 888999 // microseconds + /// }; + /// assert_eq!(Literal::time(micro_secs), t); + /// ``` + pub fn time_from_str>(s: S) -> Result { + let t = s.as_ref().parse::().map_err(|e| { + Error::new( + ErrorKind::DataInvalid, + format!("Can't parse time from string: {}", s.as_ref()), + ) + .with_source(e) + })?; + + Ok(Self::time_from_naive_time(t)) + } + + /// Creates time literal from hour, minute, second, and microseconds. + /// + /// See [`NaiveTime::from_hms_micro_opt`]. + /// + /// Example: + /// ```rust + /// + /// use iceberg::spec::Literal; + /// let t = Literal::time_from_hms_micro(22, 15, 33, 111).unwrap(); + /// + /// assert_eq!(Literal::time_from_str("22:15:33.000111").unwrap(), t); + /// ``` + pub fn time_from_hms_micro(hour: u32, min: u32, sec: u32, micro: u32) -> Result { + let t = NaiveTime::from_hms_micro_opt(hour, min, sec, micro) .ok_or_else(|| Error::new( ErrorKind::DataInvalid, format!("Can't create time from hour: {hour}, min: {min}, second: {sec}, microsecond: {micro}"), ))?; - Ok(Self::time_from_naive_time(t)) - } - - /// Creates a timestamp from unix epoch in microseconds. - pub fn timestamp(value: i64) -> Self { - Self::Primitive(PrimitiveLiteral::Timestamp(value)) - } - - /// Creates a timestamp with timezone from unix epoch in microseconds. - pub fn timestamptz(value: i64) -> Self { - Self::Primitive(PrimitiveLiteral::Timestamptz(value)) - } - - /// Creates a timestamp from [`DateTime`]. - pub fn timestamp_from_datetime(dt: DateTime) -> Self { - Self::timestamp(dt.with_timezone(&Utc).timestamp_micros()) - } - - /// Creates a timestamp with timezone from [`DateTime`]. - pub fn timestamptz_from_datetime(dt: DateTime) -> Self { - Self::timestamptz(dt.with_timezone(&Utc).timestamp_micros()) - } - - /// Parse a timestamp in RFC3339 format. - /// - /// See [`DateTime::from_str`]. - /// - /// Example: - /// - /// ```rust - /// use chrono::{DateTime, FixedOffset, NaiveDate, NaiveDateTime, NaiveTime}; - /// use iceberg::spec::Literal; - /// let t = Literal::timestamp_from_str("2012-12-12 12:12:12.8899-04:00").unwrap(); - /// - /// let t2 = { - /// let date = NaiveDate::from_ymd_opt(2012, 12, 12).unwrap(); - /// let time = NaiveTime::from_hms_micro_opt(12, 12, 12, 889900).unwrap(); - /// let dt = NaiveDateTime::new(date, time); - /// Literal::timestamp_from_datetime(DateTime::::from_local(dt, FixedOffset::west_opt(4 * 3600).unwrap())) - /// }; - /// - /// assert_eq!(t, t2); - /// ``` - pub fn timestamp_from_str>(s: S) -> Result { - let dt = DateTime::::from_str(s.as_ref()).map_err(|e| { - Error::new(ErrorKind::DataInvalid, "Can't parse datetime.").with_source(e) - })?; - - Ok(Self::timestamp_from_datetime(dt)) - } - - /// Similar to [`Literal::timestamp_from_str`], but return timestamp with timezone literal. - pub fn timestamptz_from_str>(s: S) -> Result { - let dt = DateTime::::from_str(s.as_ref()).map_err(|e| { - Error::new(ErrorKind::DataInvalid, "Can't parse datetime.").with_source(e) - })?; - - Ok(Self::timestamptz_from_datetime(dt)) - } - - /// Creates a string literal. - pub fn string(s: S) -> Self { - Self::Primitive(PrimitiveLiteral::String(s.to_string())) - } - - /// Creates uuid literal. - pub fn uuid(uuid: Uuid) -> Self { - Self::Primitive(PrimitiveLiteral::UUID(uuid)) - } - - /// Creates uuid from str. See [`Uuid::parse_str`]. - /// - /// Example: - /// - /// ```rust - /// use uuid::Uuid; - /// use iceberg::spec::Literal; - /// let t1 = Literal::uuid_from_str("a1a2a3a4-b1b2-c1c2-d1d2-d3d4d5d6d7d8").unwrap(); - /// let t2 = Literal::uuid(Uuid::from_u128_le(0xd8d7d6d5d4d3d2d1c2c1b2b1a4a3a2a1)); - /// - /// assert_eq!(t1, t2); - /// ``` - pub fn uuid_from_str>(s: S) -> Result { - let uuid = Uuid::parse_str(s.as_ref()).map_err(|e| { - Error::new( - ErrorKind::DataInvalid, - format!("Can't parse uuid from string: {}", s.as_ref()), - ) - .with_source(e) - })?; - Ok(Self::uuid(uuid)) - } - - /// Creates a fixed literal from bytes. - /// - /// Example: - /// - /// ```rust - /// use iceberg::spec::{Literal, PrimitiveLiteral}; - /// let t1 = Literal::fixed(vec![1u8, 2u8]); - /// let t2 = Literal::Primitive(PrimitiveLiteral::Fixed(vec![1u8, 2u8])); - /// - /// assert_eq!(t1, t2); - /// ``` - pub fn fixed>(input: I) -> Self { - Literal::Primitive(PrimitiveLiteral::Fixed(input.into_iter().collect())) - } - - /// Creates a binary literal from bytes. - /// - /// Example: - /// - /// ```rust - /// use iceberg::spec::{Literal, PrimitiveLiteral}; - /// let t1 = Literal::binary(vec![1u8, 2u8]); - /// let t2 = Literal::Primitive(PrimitiveLiteral::Binary(vec![1u8, 2u8])); - /// - /// assert_eq!(t1, t2); - /// ``` - pub fn binary>(input: I) -> Self { - Literal::Primitive(PrimitiveLiteral::Binary(input.into_iter().collect())) - } - - /// Creates a decimal literal. - pub fn decimal(decimal: i128) -> Self { - Self::Primitive(PrimitiveLiteral::Decimal(decimal)) - } - - /// Creates decimal literal from string. See [`Decimal::from_str_exact`]. - /// - /// Example: - /// - /// ```rust - /// use rust_decimal::Decimal; - /// use iceberg::spec::Literal; - /// let t1 = Literal::decimal(12345); - /// let t2 = Literal::decimal_from_str("123.45").unwrap(); - /// - /// assert_eq!(t1, t2); - /// ``` - pub fn decimal_from_str>(s: S) -> Result { - let decimal = Decimal::from_str_exact(s.as_ref()).map_err(|e| { - Error::new(ErrorKind::DataInvalid, "Can't parse decimal.").with_source(e) - })?; - Ok(Self::decimal(decimal.mantissa())) - } -} - -impl From for ByteBuf { - fn from(value: PrimitiveLiteral) -> Self { - match value { - PrimitiveLiteral::Boolean(val) => { - if val { - ByteBuf::from([1u8]) - } else { - ByteBuf::from([0u8]) - } - } - PrimitiveLiteral::Int(val) => ByteBuf::from(val.to_le_bytes()), - PrimitiveLiteral::Long(val) => ByteBuf::from(val.to_le_bytes()), - PrimitiveLiteral::Float(val) => ByteBuf::from(val.to_le_bytes()), - PrimitiveLiteral::Double(val) => ByteBuf::from(val.to_le_bytes()), - PrimitiveLiteral::Date(val) => ByteBuf::from(val.to_le_bytes()), - PrimitiveLiteral::Time(val) => ByteBuf::from(val.to_le_bytes()), - PrimitiveLiteral::Timestamp(val) => ByteBuf::from(val.to_le_bytes()), - PrimitiveLiteral::Timestamptz(val) => ByteBuf::from(val.to_le_bytes()), - PrimitiveLiteral::String(val) => ByteBuf::from(val.as_bytes()), - PrimitiveLiteral::UUID(val) => ByteBuf::from(val.as_u128().to_be_bytes()), - PrimitiveLiteral::Fixed(val) => ByteBuf::from(val), - PrimitiveLiteral::Binary(val) => ByteBuf::from(val), - PrimitiveLiteral::Decimal(_) => todo!(), - } - } -} - -impl From for ByteBuf { - fn from(value: Literal) -> Self { - match value { - Literal::Primitive(val) => val.into(), - _ => unimplemented!(), - } - } -} - -impl From for Vec { - fn from(value: PrimitiveLiteral) -> Self { - match value { - PrimitiveLiteral::Boolean(val) => { - if val { - Vec::from([1u8]) - } else { - Vec::from([0u8]) - } - } - PrimitiveLiteral::Int(val) => Vec::from(val.to_le_bytes()), - PrimitiveLiteral::Long(val) => Vec::from(val.to_le_bytes()), - PrimitiveLiteral::Float(val) => Vec::from(val.to_le_bytes()), - PrimitiveLiteral::Double(val) => Vec::from(val.to_le_bytes()), - PrimitiveLiteral::Date(val) => Vec::from(val.to_le_bytes()), - PrimitiveLiteral::Time(val) => Vec::from(val.to_le_bytes()), - PrimitiveLiteral::Timestamp(val) => Vec::from(val.to_le_bytes()), - PrimitiveLiteral::Timestamptz(val) => Vec::from(val.to_le_bytes()), - PrimitiveLiteral::String(val) => Vec::from(val.as_bytes()), - PrimitiveLiteral::UUID(val) => Vec::from(val.as_u128().to_be_bytes()), - PrimitiveLiteral::Fixed(val) => val, - PrimitiveLiteral::Binary(val) => val, - PrimitiveLiteral::Decimal(_) => todo!(), - } - } -} - -impl From for Vec { - fn from(value: Literal) -> Self { - match value { - Literal::Primitive(val) => val.into(), - _ => unimplemented!(), - } - } -} - -/// The partition struct stores the tuple of partition values for each file. -/// Its type is derived from the partition fields of the partition spec used to write the manifest file. -/// In v2, the partition struct’s field ids must match the ids from the partition spec. -#[derive(Debug, Clone, PartialEq, Eq, Hash)] -pub struct Struct { - /// Vector to store the field values - fields: Vec, - /// Null bitmap - null_bitmap: BitVec, -} - -impl Struct { - /// Create a empty struct. - pub fn empty() -> Self { - Self { - fields: Vec::new(), - null_bitmap: BitVec::new(), - } - } - - /// Create a iterator to read the field in order of field_value. - pub fn iter(&self) -> impl Iterator> { - self.null_bitmap.iter().zip(self.fields.iter()).map( - |(null, value)| { - if *null { - None - } else { - Some(value) - } - }, - ) - } - - /// returns true if the field at position `index` is null - pub fn is_null_at_index(&self, index: usize) -> bool { - self.null_bitmap[index] - } -} - -impl Index for Struct { - type Output = Literal; - - fn index(&self, idx: usize) -> &Self::Output { - &self.fields[idx] - } -} - -/// An iterator that moves out of a struct. -pub struct StructValueIntoIter { - null_bitmap: bitvec::boxed::IntoIter, - fields: std::vec::IntoIter, -} - -impl Iterator for StructValueIntoIter { - type Item = Option; - - fn next(&mut self) -> Option { - match (self.null_bitmap.next(), self.fields.next()) { - (Some(null), Some(value)) => Some(if null { None } else { Some(value) }), - _ => None, - } - } -} - -impl IntoIterator for Struct { - type Item = Option; - - type IntoIter = StructValueIntoIter; - - fn into_iter(self) -> Self::IntoIter { - StructValueIntoIter { - null_bitmap: self.null_bitmap.into_iter(), - fields: self.fields.into_iter(), - } - } -} - -impl FromIterator> for Struct { - fn from_iter>>(iter: I) -> Self { - let mut fields = Vec::new(); - let mut null_bitmap = BitVec::new(); - - for value in iter.into_iter() { - match value { - Some(value) => { - fields.push(value); - null_bitmap.push(false) - } - None => { - fields.push(Literal::Primitive(PrimitiveLiteral::Boolean(false))); - null_bitmap.push(true) - } - } - } - Struct { - fields, - null_bitmap, - } - } -} - -impl Literal { - /// Create iceberg value from bytes - pub fn try_from_bytes(bytes: &[u8], data_type: &Type) -> Result { - match data_type { - Type::Primitive(primitive_type) => { - let datum = Datum::try_from_bytes(bytes, primitive_type.clone())?; - Ok(Literal::Primitive(datum.literal)) - } - _ => Err(Error::new( - crate::ErrorKind::DataInvalid, - "Converting bytes to non-primitive types is not supported.", - )), - } - } - - /// Create iceberg value from a json value - pub fn try_from_json(value: JsonValue, data_type: &Type) -> Result> { - match data_type { - Type::Primitive(primitive) => match (primitive, value) { - (PrimitiveType::Boolean, JsonValue::Bool(bool)) => { - Ok(Some(Literal::Primitive(PrimitiveLiteral::Boolean(bool)))) - } - (PrimitiveType::Int, JsonValue::Number(number)) => { - Ok(Some(Literal::Primitive(PrimitiveLiteral::Int( - number - .as_i64() - .ok_or(Error::new( - crate::ErrorKind::DataInvalid, - "Failed to convert json number to int", - ))? - .try_into()?, - )))) - } - (PrimitiveType::Long, JsonValue::Number(number)) => Ok(Some(Literal::Primitive( - PrimitiveLiteral::Long(number.as_i64().ok_or(Error::new( - crate::ErrorKind::DataInvalid, - "Failed to convert json number to long", - ))?), - ))), - (PrimitiveType::Float, JsonValue::Number(number)) => Ok(Some(Literal::Primitive( - PrimitiveLiteral::Float(OrderedFloat(number.as_f64().ok_or(Error::new( - crate::ErrorKind::DataInvalid, - "Failed to convert json number to float", - ))? as f32)), - ))), - (PrimitiveType::Double, JsonValue::Number(number)) => Ok(Some(Literal::Primitive( - PrimitiveLiteral::Double(OrderedFloat(number.as_f64().ok_or(Error::new( - crate::ErrorKind::DataInvalid, - "Failed to convert json number to double", - ))?)), - ))), - (PrimitiveType::Date, JsonValue::String(s)) => { - Ok(Some(Literal::Primitive(PrimitiveLiteral::Date( - date::date_to_days(&NaiveDate::parse_from_str(&s, "%Y-%m-%d")?), - )))) - } - (PrimitiveType::Time, JsonValue::String(s)) => { - Ok(Some(Literal::Primitive(PrimitiveLiteral::Time( - time::time_to_microseconds(&NaiveTime::parse_from_str(&s, "%H:%M:%S%.f")?), - )))) - } - (PrimitiveType::Timestamp, JsonValue::String(s)) => Ok(Some(Literal::Primitive( - PrimitiveLiteral::Timestamp(timestamp::datetime_to_microseconds( - &NaiveDateTime::parse_from_str(&s, "%Y-%m-%dT%H:%M:%S%.f")?, - )), - ))), - (PrimitiveType::Timestamptz, JsonValue::String(s)) => { - Ok(Some(Literal::Primitive(PrimitiveLiteral::Timestamptz( - timestamptz::datetimetz_to_microseconds(&Utc.from_utc_datetime( - &NaiveDateTime::parse_from_str(&s, "%Y-%m-%dT%H:%M:%S%.f+00:00")?, - )), - )))) - } - (PrimitiveType::String, JsonValue::String(s)) => { - Ok(Some(Literal::Primitive(PrimitiveLiteral::String(s)))) - } - (PrimitiveType::Uuid, JsonValue::String(s)) => Ok(Some(Literal::Primitive( - PrimitiveLiteral::UUID(Uuid::parse_str(&s)?), - ))), - (PrimitiveType::Fixed(_), JsonValue::String(_)) => todo!(), - (PrimitiveType::Binary, JsonValue::String(_)) => todo!(), - ( - PrimitiveType::Decimal { - precision: _, - scale, - }, - JsonValue::String(s), - ) => { - let mut decimal = Decimal::from_str_exact(&s)?; - decimal.rescale(*scale); - Ok(Some(Literal::Primitive(PrimitiveLiteral::Decimal( - decimal.mantissa(), - )))) - } - (_, JsonValue::Null) => Ok(None), - (i, j) => Err(Error::new( - crate::ErrorKind::DataInvalid, - format!( - "The json value {} doesn't fit to the iceberg type {}.", - j, i - ), - )), - }, - Type::Struct(schema) => { - if let JsonValue::Object(mut object) = value { - Ok(Some(Literal::Struct(Struct::from_iter( - schema.fields().iter().map(|field| { - object.remove(&field.id.to_string()).and_then(|value| { - Literal::try_from_json(value, &field.field_type) - .and_then(|value| { - value.ok_or(Error::new( - ErrorKind::DataInvalid, - "Key of map cannot be null", - )) - }) - .ok() - }) - }), - )))) - } else { - Err(Error::new( - crate::ErrorKind::DataInvalid, - "The json value for a struct type must be an object.", - )) - } - } - Type::List(list) => { - if let JsonValue::Array(array) = value { - Ok(Some(Literal::List( - array - .into_iter() - .map(|value| { - Literal::try_from_json(value, &list.element_field.field_type) - }) - .collect::>>()?, - ))) - } else { - Err(Error::new( - crate::ErrorKind::DataInvalid, - "The json value for a list type must be an array.", - )) - } - } - Type::Map(map) => { - if let JsonValue::Object(mut object) = value { - if let (Some(JsonValue::Array(keys)), Some(JsonValue::Array(values))) = - (object.remove("keys"), object.remove("values")) - { - Ok(Some(Literal::Map(Map::from_iter( - keys.into_iter() - .zip(values.into_iter()) - .map(|(key, value)| { - Ok(( - Literal::try_from_json(key, &map.key_field.field_type) - .and_then(|value| { - value.ok_or(Error::new( - ErrorKind::DataInvalid, - "Key of map cannot be null", - )) - })?, - Literal::try_from_json(value, &map.value_field.field_type)?, - )) - }) - .collect::>>()?, - )))) - } else { - Err(Error::new( - crate::ErrorKind::DataInvalid, - "The json value for a list type must be an array.", - )) - } - } else { - Err(Error::new( - crate::ErrorKind::DataInvalid, - "The json value for a list type must be an array.", - )) - } - } - } - } - - /// Converting iceberg value to json value. - /// - /// See [this spec](https://iceberg.apache.org/spec/#json-single-value-serialization) for reference. - pub fn try_into_json(self, r#type: &Type) -> Result { - match (self, r#type) { - (Literal::Primitive(prim), _) => match prim { - PrimitiveLiteral::Boolean(val) => Ok(JsonValue::Bool(val)), - PrimitiveLiteral::Int(val) => Ok(JsonValue::Number((val).into())), - PrimitiveLiteral::Long(val) => Ok(JsonValue::Number((val).into())), - PrimitiveLiteral::Float(val) => match Number::from_f64(val.0 as f64) { - Some(number) => Ok(JsonValue::Number(number)), - None => Ok(JsonValue::Null), - }, - PrimitiveLiteral::Double(val) => match Number::from_f64(val.0) { - Some(number) => Ok(JsonValue::Number(number)), - None => Ok(JsonValue::Null), - }, - PrimitiveLiteral::Date(val) => { - Ok(JsonValue::String(date::days_to_date(val).to_string())) - } - PrimitiveLiteral::Time(val) => Ok(JsonValue::String( - time::microseconds_to_time(val).to_string(), - )), - PrimitiveLiteral::Timestamp(val) => Ok(JsonValue::String( - timestamp::microseconds_to_datetime(val) - .format("%Y-%m-%dT%H:%M:%S%.f") - .to_string(), - )), - PrimitiveLiteral::Timestamptz(val) => Ok(JsonValue::String( - timestamptz::microseconds_to_datetimetz(val) - .format("%Y-%m-%dT%H:%M:%S%.f+00:00") - .to_string(), - )), - PrimitiveLiteral::String(val) => Ok(JsonValue::String(val.clone())), - PrimitiveLiteral::UUID(val) => Ok(JsonValue::String(val.to_string())), - PrimitiveLiteral::Fixed(val) => Ok(JsonValue::String(val.iter().fold( - String::new(), - |mut acc, x| { - acc.push_str(&format!("{:x}", x)); - acc - }, - ))), - PrimitiveLiteral::Binary(val) => Ok(JsonValue::String(val.iter().fold( - String::new(), - |mut acc, x| { - acc.push_str(&format!("{:x}", x)); - acc - }, - ))), - PrimitiveLiteral::Decimal(val) => match r#type { - Type::Primitive(PrimitiveType::Decimal { - precision: _precision, - scale, - }) => { - let decimal = Decimal::try_from_i128_with_scale(val, *scale)?; - Ok(JsonValue::String(decimal.to_string())) - } - _ => Err(Error::new( - ErrorKind::DataInvalid, - "The iceberg type for decimal literal must be decimal.", - ))?, - }, - }, - (Literal::Struct(s), Type::Struct(struct_type)) => { - let mut id_and_value = Vec::with_capacity(struct_type.fields().len()); - for (value, field) in s.into_iter().zip(struct_type.fields()) { - let json = match value { - Some(val) => val.try_into_json(&field.field_type)?, - None => JsonValue::Null, - }; - id_and_value.push((field.id.to_string(), json)); - } - Ok(JsonValue::Object(JsonMap::from_iter(id_and_value))) - } - (Literal::List(list), Type::List(list_type)) => Ok(JsonValue::Array( - list.into_iter() - .map(|opt| match opt { - Some(literal) => literal.try_into_json(&list_type.element_field.field_type), - None => Ok(JsonValue::Null), - }) - .collect::>>()?, - )), - (Literal::Map(map), Type::Map(map_type)) => { - let mut object = JsonMap::with_capacity(2); - let mut json_keys = Vec::with_capacity(map.len()); - let mut json_values = Vec::with_capacity(map.len()); - for (key, value) in map.into_iter() { - json_keys.push(key.try_into_json(&map_type.key_field.field_type)?); - json_values.push(match value { - Some(literal) => literal.try_into_json(&map_type.value_field.field_type)?, - None => JsonValue::Null, - }); - } - object.insert("keys".to_string(), JsonValue::Array(json_keys)); - object.insert("values".to_string(), JsonValue::Array(json_values)); - Ok(JsonValue::Object(object)) - } - (value, r#type) => Err(Error::new( - ErrorKind::DataInvalid, - format!( - "The iceberg value {:?} doesn't fit to the iceberg type {}.", - value, r#type - ), - )), - } - } - - /// Convert Value to the any type - pub fn into_any(self) -> Box { - match self { - Literal::Primitive(prim) => match prim { - PrimitiveLiteral::Boolean(any) => Box::new(any), - PrimitiveLiteral::Int(any) => Box::new(any), - PrimitiveLiteral::Long(any) => Box::new(any), - PrimitiveLiteral::Float(any) => Box::new(any), - PrimitiveLiteral::Double(any) => Box::new(any), - PrimitiveLiteral::Date(any) => Box::new(any), - PrimitiveLiteral::Time(any) => Box::new(any), - PrimitiveLiteral::Timestamp(any) => Box::new(any), - PrimitiveLiteral::Timestamptz(any) => Box::new(any), - PrimitiveLiteral::Fixed(any) => Box::new(any), - PrimitiveLiteral::Binary(any) => Box::new(any), - PrimitiveLiteral::String(any) => Box::new(any), - PrimitiveLiteral::UUID(any) => Box::new(any), - PrimitiveLiteral::Decimal(any) => Box::new(any), - }, - _ => unimplemented!(), - } - } -} - -mod date { - use chrono::{DateTime, NaiveDate, TimeDelta, TimeZone, Utc}; - - pub(crate) fn date_to_days(date: &NaiveDate) -> i32 { - date.signed_duration_since( - // This is always the same and shouldn't fail - NaiveDate::from_ymd_opt(1970, 1, 1).unwrap(), - ) - .num_days() as i32 - } - - pub(crate) fn days_to_date(days: i32) -> NaiveDate { - // This shouldn't fail until the year 262000 - (chrono::DateTime::UNIX_EPOCH + TimeDelta::try_days(days as i64).unwrap()) - .naive_utc() - .date() - } - - /// Returns unix epoch. - pub(crate) fn unix_epoch() -> DateTime { - Utc.timestamp_nanos(0) - } - - /// Creates date literal from `NaiveDate`, assuming it's utc timezone. - pub(crate) fn date_from_naive_date(date: NaiveDate) -> i32 { - (date - unix_epoch().date_naive()).num_days() as i32 - } -} - -mod time { - use chrono::NaiveTime; - - pub(crate) fn time_to_microseconds(time: &NaiveTime) -> i64 { - time.signed_duration_since( - // This is always the same and shouldn't fail - NaiveTime::from_num_seconds_from_midnight_opt(0, 0).unwrap(), - ) - .num_microseconds() - .unwrap() - } - - pub(crate) fn microseconds_to_time(micros: i64) -> NaiveTime { - let (secs, rem) = (micros / 1_000_000, micros % 1_000_000); - - NaiveTime::from_num_seconds_from_midnight_opt(secs as u32, rem as u32 * 1_000).unwrap() - } -} - -mod timestamp { - use chrono::{DateTime, NaiveDateTime}; - - pub(crate) fn datetime_to_microseconds(time: &NaiveDateTime) -> i64 { - time.and_utc().timestamp_micros() - } - - pub(crate) fn microseconds_to_datetime(micros: i64) -> NaiveDateTime { - // This shouldn't fail until the year 262000 - DateTime::from_timestamp_micros(micros).unwrap().naive_utc() - } -} - -mod timestamptz { - use chrono::DateTime; - use chrono::Utc; - - pub(crate) fn datetimetz_to_microseconds(time: &DateTime) -> i64 { - time.timestamp_micros() - } - - pub(crate) fn microseconds_to_datetimetz(micros: i64) -> DateTime { - let (secs, rem) = (micros / 1_000_000, micros % 1_000_000); - - DateTime::from_timestamp(secs, rem as u32 * 1_000).unwrap() - } -} - -mod _serde { - use serde::{ - de::Visitor, - ser::{SerializeMap, SerializeSeq, SerializeStruct}, - Deserialize, Serialize, - }; - use serde_bytes::ByteBuf; - use serde_derive::Deserialize as DeserializeDerive; - use serde_derive::Serialize as SerializeDerive; - - use crate::{ - spec::{PrimitiveType, Type, MAP_KEY_FIELD_NAME, MAP_VALUE_FIELD_NAME}, - Error, ErrorKind, - }; - - use super::{Literal, Map, PrimitiveLiteral}; - - #[derive(SerializeDerive, DeserializeDerive, Debug)] - #[serde(transparent)] - /// Raw literal representation used for serde. The serialize way is used for Avro serializer. - pub struct RawLiteral(RawLiteralEnum); - - impl RawLiteral { - /// Covert literal to raw literal. - pub fn try_from(literal: Literal, ty: &Type) -> Result { - Ok(Self(RawLiteralEnum::try_from(literal, ty)?)) - } - - /// Convert raw literal to literal. - pub fn try_into(self, ty: &Type) -> Result, Error> { - self.0.try_into(ty) - } - } - - #[derive(SerializeDerive, Clone, Debug)] - #[serde(untagged)] - enum RawLiteralEnum { - Null, - Boolean(bool), - Int(i32), - Long(i64), - Float(f32), - Double(f64), - String(String), - Bytes(ByteBuf), - List(List), - StringMap(StringMap), - Record(Record), - } - - #[derive(Clone, Debug)] - struct Record { - required: Vec<(String, RawLiteralEnum)>, - optional: Vec<(String, Option)>, - } - - impl Serialize for Record { - fn serialize(&self, serializer: S) -> Result - where - S: serde::Serializer, - { - let len = self.required.len() + self.optional.len(); - let mut record = serializer.serialize_struct("", len)?; - for (k, v) in &self.required { - record.serialize_field(Box::leak(k.clone().into_boxed_str()), &v)?; - } - for (k, v) in &self.optional { - record.serialize_field(Box::leak(k.clone().into_boxed_str()), &v)?; - } - record.end() - } - } - - #[derive(Clone, Debug)] - struct List { - list: Vec>, - required: bool, - } - - impl Serialize for List { - fn serialize(&self, serializer: S) -> Result - where - S: serde::Serializer, - { - let mut seq = serializer.serialize_seq(Some(self.list.len()))?; - for value in &self.list { - if self.required { - seq.serialize_element(value.as_ref().ok_or_else(|| { - serde::ser::Error::custom( - "List element is required, element cannot be null", - ) - })?)?; - } else { - seq.serialize_element(&value)?; - } - } - seq.end() - } - } - - #[derive(Clone, Debug)] - struct StringMap { - raw: Vec<(String, Option)>, - required: bool, - } - - impl Serialize for StringMap { - fn serialize(&self, serializer: S) -> Result - where - S: serde::Serializer, - { - let mut map = serializer.serialize_map(Some(self.raw.len()))?; - for (k, v) in &self.raw { - if self.required { - map.serialize_entry( - k, - v.as_ref().ok_or_else(|| { - serde::ser::Error::custom( - "Map element is required, element cannot be null", - ) - })?, - )?; - } else { - map.serialize_entry(k, v)?; - } - } - map.end() - } - } - - impl<'de> Deserialize<'de> for RawLiteralEnum { - fn deserialize(deserializer: D) -> Result - where - D: serde::Deserializer<'de>, - { - struct RawLiteralVisitor; - impl<'de> Visitor<'de> for RawLiteralVisitor { - type Value = RawLiteralEnum; - - fn expecting(&self, formatter: &mut std::fmt::Formatter) -> std::fmt::Result { - formatter.write_str("expect") - } - - fn visit_bool(self, v: bool) -> Result - where - E: serde::de::Error, - { - Ok(RawLiteralEnum::Boolean(v)) - } - - fn visit_i32(self, v: i32) -> Result - where - E: serde::de::Error, - { - Ok(RawLiteralEnum::Int(v)) - } - - fn visit_i64(self, v: i64) -> Result - where - E: serde::de::Error, - { - Ok(RawLiteralEnum::Long(v)) - } - - /// Used in json - fn visit_u64(self, v: u64) -> Result - where - E: serde::de::Error, - { - Ok(RawLiteralEnum::Long(v as i64)) - } - - fn visit_f32(self, v: f32) -> Result - where - E: serde::de::Error, - { - Ok(RawLiteralEnum::Float(v)) - } - - fn visit_f64(self, v: f64) -> Result - where - E: serde::de::Error, - { - Ok(RawLiteralEnum::Double(v)) - } - - fn visit_str(self, v: &str) -> Result - where - E: serde::de::Error, - { - Ok(RawLiteralEnum::String(v.to_string())) - } - - fn visit_bytes(self, v: &[u8]) -> Result - where - E: serde::de::Error, - { - Ok(RawLiteralEnum::Bytes(ByteBuf::from(v))) - } - - fn visit_borrowed_str(self, v: &'de str) -> Result - where - E: serde::de::Error, - { - Ok(RawLiteralEnum::String(v.to_string())) - } - - fn visit_unit(self) -> Result - where - E: serde::de::Error, - { - Ok(RawLiteralEnum::Null) - } - - fn visit_map(self, mut map: A) -> Result - where - A: serde::de::MapAccess<'de>, - { - let mut required = Vec::new(); - while let Some(key) = map.next_key::()? { - let value = map.next_value::()?; - required.push((key, value)); - } - Ok(RawLiteralEnum::Record(Record { - required, - optional: Vec::new(), - })) - } - - fn visit_seq(self, mut seq: A) -> Result - where - A: serde::de::SeqAccess<'de>, - { - let mut list = Vec::new(); - while let Some(value) = seq.next_element::()? { - list.push(Some(value)); - } - Ok(RawLiteralEnum::List(List { - list, - // `required` only used in serialize, just set default in deserialize. - required: false, - })) - } - } - deserializer.deserialize_any(RawLiteralVisitor) - } - } - - impl RawLiteralEnum { - pub fn try_from(literal: Literal, ty: &Type) -> Result { - let raw = match literal { - Literal::Primitive(prim) => match prim { - super::PrimitiveLiteral::Boolean(v) => RawLiteralEnum::Boolean(v), - super::PrimitiveLiteral::Int(v) => RawLiteralEnum::Int(v), - super::PrimitiveLiteral::Long(v) => RawLiteralEnum::Long(v), - super::PrimitiveLiteral::Float(v) => RawLiteralEnum::Float(v.0), - super::PrimitiveLiteral::Double(v) => RawLiteralEnum::Double(v.0), - super::PrimitiveLiteral::Date(v) => RawLiteralEnum::Int(v), - super::PrimitiveLiteral::Time(v) => RawLiteralEnum::Long(v), - super::PrimitiveLiteral::Timestamp(v) => RawLiteralEnum::Long(v), - super::PrimitiveLiteral::Timestamptz(v) => RawLiteralEnum::Long(v), - super::PrimitiveLiteral::String(v) => RawLiteralEnum::String(v), - super::PrimitiveLiteral::UUID(v) => { - RawLiteralEnum::Bytes(ByteBuf::from(v.as_u128().to_be_bytes())) - } - super::PrimitiveLiteral::Fixed(v) => RawLiteralEnum::Bytes(ByteBuf::from(v)), - super::PrimitiveLiteral::Binary(v) => RawLiteralEnum::Bytes(ByteBuf::from(v)), - super::PrimitiveLiteral::Decimal(v) => { - RawLiteralEnum::Bytes(ByteBuf::from(v.to_be_bytes())) - } - }, - Literal::Struct(r#struct) => { - let mut required = Vec::new(); - let mut optional = Vec::new(); - if let Type::Struct(struct_ty) = ty { - for (value, field) in r#struct.into_iter().zip(struct_ty.fields()) { - if field.required { - if let Some(value) = value { - required.push(( - field.name.clone(), - RawLiteralEnum::try_from(value, &field.field_type)?, - )); - } else { - return Err(Error::new( - ErrorKind::DataInvalid, - "Can't convert null to required field", - )); - } - } else if let Some(value) = value { - optional.push(( - field.name.clone(), - Some(RawLiteralEnum::try_from(value, &field.field_type)?), - )); - } else { - optional.push((field.name.clone(), None)); - } - } - } else { - return Err(Error::new( - ErrorKind::DataInvalid, - format!("Type {} should be a struct", ty), - )); - } - RawLiteralEnum::Record(Record { required, optional }) - } - Literal::List(list) => { - if let Type::List(list_ty) = ty { - let list = list - .into_iter() - .map(|v| { - v.map(|v| { - RawLiteralEnum::try_from(v, &list_ty.element_field.field_type) - }) - .transpose() - }) - .collect::>()?; - RawLiteralEnum::List(List { - list, - required: list_ty.element_field.required, - }) - } else { - return Err(Error::new( - ErrorKind::DataInvalid, - format!("Type {} should be a list", ty), - )); - } - } - Literal::Map(map) => { - if let Type::Map(map_ty) = ty { - if let Type::Primitive(PrimitiveType::String) = *map_ty.key_field.field_type - { - let mut raw = Vec::with_capacity(map.len()); - for (k, v) in map { - if let Literal::Primitive(PrimitiveLiteral::String(k)) = k { - raw.push(( - k, - v.map(|v| { - RawLiteralEnum::try_from( - v, - &map_ty.value_field.field_type, - ) - }) - .transpose()?, - )); - } else { - return Err(Error::new( - ErrorKind::DataInvalid, - "literal type is inconsistent with type", - )); - } - } - RawLiteralEnum::StringMap(StringMap { - raw, - required: map_ty.value_field.required, - }) - } else { - let list = map.into_iter().map(|(k,v)| { + Ok(Self::time_from_naive_time(t)) + } + + /// Creates a timestamp from unix epoch in microseconds. + pub fn timestamp(value: i64) -> Self { + Self::Primitive(PrimitiveLiteral::Timestamp(value)) + } + + /// Creates a timestamp with timezone from unix epoch in microseconds. + pub fn timestamptz(value: i64) -> Self { + Self::Primitive(PrimitiveLiteral::Timestamptz(value)) + } + + /// Creates a timestamp from [`DateTime`]. + pub fn timestamp_from_datetime(dt: DateTime) -> Self { + Self::timestamp(dt.with_timezone(&Utc).timestamp_micros()) + } + + /// Creates a timestamp with timezone from [`DateTime`]. + pub fn timestamptz_from_datetime(dt: DateTime) -> Self { + Self::timestamptz(dt.with_timezone(&Utc).timestamp_micros()) + } + + /// Parse a timestamp in RFC3339 format. + /// + /// See [`DateTime::from_str`]. + /// + /// Example: + /// + /// ```rust + /// use chrono::{DateTime, FixedOffset, NaiveDate, NaiveDateTime, NaiveTime}; + /// use iceberg::spec::Literal; + /// let t = Literal::timestamp_from_str("2012-12-12 12:12:12.8899-04:00").unwrap(); + /// + /// let t2 = { + /// let date = NaiveDate::from_ymd_opt(2012, 12, 12).unwrap(); + /// let time = NaiveTime::from_hms_micro_opt(12, 12, 12, 889900).unwrap(); + /// let dt = NaiveDateTime::new(date, time); + /// Literal::timestamp_from_datetime(DateTime::::from_local(dt, FixedOffset::west_opt(4 * 3600).unwrap())) + /// }; + /// + /// assert_eq!(t, t2); + /// ``` + pub fn timestamp_from_str>(s: S) -> Result { + let dt = DateTime::::from_str(s.as_ref()).map_err(|e| { + Error::new(ErrorKind::DataInvalid, "Can't parse datetime.").with_source(e) + })?; + + Ok(Self::timestamp_from_datetime(dt)) + } + + /// Similar to [`Literal::timestamp_from_str`], but return timestamp with timezone literal. + pub fn timestamptz_from_str>(s: S) -> Result { + let dt = DateTime::::from_str(s.as_ref()).map_err(|e| { + Error::new(ErrorKind::DataInvalid, "Can't parse datetime.").with_source(e) + })?; + + Ok(Self::timestamptz_from_datetime(dt)) + } + + /// Creates a string literal. + pub fn string(s: S) -> Self { + Self::Primitive(PrimitiveLiteral::String(s.to_string())) + } + + /// Creates uuid literal. + pub fn uuid(uuid: Uuid) -> Self { + Self::Primitive(PrimitiveLiteral::UUID(uuid)) + } + + /// Creates uuid from str. See [`Uuid::parse_str`]. + /// + /// Example: + /// + /// ```rust + /// use uuid::Uuid; + /// use iceberg::spec::Literal; + /// let t1 = Literal::uuid_from_str("a1a2a3a4-b1b2-c1c2-d1d2-d3d4d5d6d7d8").unwrap(); + /// let t2 = Literal::uuid(Uuid::from_u128_le(0xd8d7d6d5d4d3d2d1c2c1b2b1a4a3a2a1)); + /// + /// assert_eq!(t1, t2); + /// ``` + pub fn uuid_from_str>(s: S) -> Result { + let uuid = Uuid::parse_str(s.as_ref()).map_err(|e| { + Error::new( + ErrorKind::DataInvalid, + format!("Can't parse uuid from string: {}", s.as_ref()), + ) + .with_source(e) + })?; + Ok(Self::uuid(uuid)) + } + + /// Creates a fixed literal from bytes. + /// + /// Example: + /// + /// ```rust + /// use iceberg::spec::{Literal, PrimitiveLiteral}; + /// let t1 = Literal::fixed(vec![1u8, 2u8]); + /// let t2 = Literal::Primitive(PrimitiveLiteral::Fixed(vec![1u8, 2u8])); + /// + /// assert_eq!(t1, t2); + /// ``` + pub fn fixed>(input: I) -> Self { + Literal::Primitive(PrimitiveLiteral::Fixed(input.into_iter().collect())) + } + + /// Creates a binary literal from bytes. + /// + /// Example: + /// + /// ```rust + /// use iceberg::spec::{Literal, PrimitiveLiteral}; + /// let t1 = Literal::binary(vec![1u8, 2u8]); + /// let t2 = Literal::Primitive(PrimitiveLiteral::Binary(vec![1u8, 2u8])); + /// + /// assert_eq!(t1, t2); + /// ``` + pub fn binary>(input: I) -> Self { + Literal::Primitive(PrimitiveLiteral::Binary(input.into_iter().collect())) + } + + /// Creates a decimal literal. + pub fn decimal(decimal: i128) -> Self { + Self::Primitive(PrimitiveLiteral::Decimal(decimal)) + } + + /// Creates decimal literal from string. See [`Decimal::from_str_exact`]. + /// + /// Example: + /// + /// ```rust + /// use rust_decimal::Decimal; + /// use iceberg::spec::Literal; + /// let t1 = Literal::decimal(12345); + /// let t2 = Literal::decimal_from_str("123.45").unwrap(); + /// + /// assert_eq!(t1, t2); + /// ``` + pub fn decimal_from_str>(s: S) -> Result { + let decimal = Decimal::from_str_exact(s.as_ref()).map_err(|e| { + Error::new(ErrorKind::DataInvalid, "Can't parse decimal.").with_source(e) + })?; + Ok(Self::decimal(decimal.mantissa())) + } + } + + impl From for ByteBuf { + fn from(value: PrimitiveLiteral) -> Self { + match value { + PrimitiveLiteral::Boolean(val) => { + if val { + ByteBuf::from([1u8]) + } else { + ByteBuf::from([0u8]) + } + } + PrimitiveLiteral::Int(val) => ByteBuf::from(val.to_le_bytes()), + PrimitiveLiteral::Long(val) => ByteBuf::from(val.to_le_bytes()), + PrimitiveLiteral::Float(val) => ByteBuf::from(val.to_le_bytes()), + PrimitiveLiteral::Double(val) => ByteBuf::from(val.to_le_bytes()), + PrimitiveLiteral::Date(val) => ByteBuf::from(val.to_le_bytes()), + PrimitiveLiteral::Time(val) => ByteBuf::from(val.to_le_bytes()), + PrimitiveLiteral::Timestamp(val) => ByteBuf::from(val.to_le_bytes()), + PrimitiveLiteral::Timestamptz(val) => ByteBuf::from(val.to_le_bytes()), + PrimitiveLiteral::String(val) => ByteBuf::from(val.as_bytes()), + PrimitiveLiteral::UUID(val) => ByteBuf::from(val.as_u128().to_be_bytes()), + PrimitiveLiteral::Fixed(val) => ByteBuf::from(val), + PrimitiveLiteral::Binary(val) => ByteBuf::from(val), + PrimitiveLiteral::Decimal(_) => todo!(), + } + } + } + + impl From for ByteBuf { + fn from(value: Literal) -> Self { + match value { + Literal::Primitive(val) => val.into(), + _ => unimplemented!(), + } + } + } + + impl From for Vec { + fn from(value: PrimitiveLiteral) -> Self { + match value { + PrimitiveLiteral::Boolean(val) => { + if val { + Vec::from([1u8]) + } else { + Vec::from([0u8]) + } + } + PrimitiveLiteral::Int(val) => Vec::from(val.to_le_bytes()), + PrimitiveLiteral::Long(val) => Vec::from(val.to_le_bytes()), + PrimitiveLiteral::Float(val) => Vec::from(val.to_le_bytes()), + PrimitiveLiteral::Double(val) => Vec::from(val.to_le_bytes()), + PrimitiveLiteral::Date(val) => Vec::from(val.to_le_bytes()), + PrimitiveLiteral::Time(val) => Vec::from(val.to_le_bytes()), + PrimitiveLiteral::Timestamp(val) => Vec::from(val.to_le_bytes()), + PrimitiveLiteral::Timestamptz(val) => Vec::from(val.to_le_bytes()), + PrimitiveLiteral::String(val) => Vec::from(val.as_bytes()), + PrimitiveLiteral::UUID(val) => Vec::from(val.as_u128().to_be_bytes()), + PrimitiveLiteral::Fixed(val) => val, + PrimitiveLiteral::Binary(val) => val, + PrimitiveLiteral::Decimal(_) => todo!(), + } + } + } + + impl From for Vec { + fn from(value: Literal) -> Self { + match value { + Literal::Primitive(val) => val.into(), + _ => unimplemented!(), + } + } + } + + /// The partition struct stores the tuple of partition values for each file. + /// Its type is derived from the partition fields of the partition spec used to write the manifest file. + /// In v2, the partition struct’s field ids must match the ids from the partition spec. + #[derive(Debug, Clone, PartialEq, Eq, Hash)] + pub struct Struct { + /// Vector to store the field values + fields: Vec, + /// Null bitmap + null_bitmap: BitVec, + } + + impl Struct { + /// Create a empty struct. + pub fn empty() -> Self { + Self { + fields: Vec::new(), + null_bitmap: BitVec::new(), + } + } + + /// Create a iterator to read the field in order of field_value. + pub fn iter(&self) -> impl Iterator> { + self.null_bitmap.iter().zip(self.fields.iter()).map( + |(null, value)| { + if *null { + None + } else { + Some(value) + } + }, + ) + } + + /// returns true if the field at position `index` is null + pub fn is_null_at_index(&self, index: usize) -> bool { + self.null_bitmap[index] + } + } + + impl Index for Struct { + type Output = Literal; + + fn index(&self, idx: usize) -> &Self::Output { + &self.fields[idx] + } + } + + /// An iterator that moves out of a struct. + pub struct StructValueIntoIter { + null_bitmap: bitvec::boxed::IntoIter, + fields: std::vec::IntoIter, + } + + impl Iterator for StructValueIntoIter { + type Item = Option; + + fn next(&mut self) -> Option { + match (self.null_bitmap.next(), self.fields.next()) { + (Some(null), Some(value)) => Some(if null { None } else { Some(value) }), + _ => None, + } + } + } + + impl IntoIterator for Struct { + type Item = Option; + + type IntoIter = StructValueIntoIter; + + fn into_iter(self) -> Self::IntoIter { + StructValueIntoIter { + null_bitmap: self.null_bitmap.into_iter(), + fields: self.fields.into_iter(), + } + } + } + + impl FromIterator> for Struct { + fn from_iter>>(iter: I) -> Self { + let mut fields = Vec::new(); + let mut null_bitmap = BitVec::new(); + + for value in iter.into_iter() { + match value { + Some(value) => { + fields.push(value); + null_bitmap.push(false) + } + None => { + fields.push(Literal::Primitive(PrimitiveLiteral::Boolean(false))); + null_bitmap.push(true) + } + } + } + Struct { + fields, + null_bitmap, + } + } + } + + impl Literal { + /// Create iceberg value from bytes + pub fn try_from_bytes(bytes: &[u8], data_type: &Type) -> Result { + match data_type { + Type::Primitive(primitive_type) => { + let datum = Datum::try_from_bytes(bytes, primitive_type.clone())?; + Ok(Literal::Primitive(datum.literal)) + } + _ => Err(Error::new( + crate::ErrorKind::DataInvalid, + "Converting bytes to non-primitive types is not supported.", + )), + } + } + + /// Create iceberg value from a json value + pub fn try_from_json(value: JsonValue, data_type: &Type) -> Result> { + match data_type { + Type::Primitive(primitive) => match (primitive, value) { + (PrimitiveType::Boolean, JsonValue::Bool(bool)) => { + Ok(Some(Literal::Primitive(PrimitiveLiteral::Boolean(bool)))) + } + (PrimitiveType::Int, JsonValue::Number(number)) => { + Ok(Some(Literal::Primitive(PrimitiveLiteral::Int( + number + .as_i64() + .ok_or(Error::new( + crate::ErrorKind::DataInvalid, + "Failed to convert json number to int", + ))? + .try_into()?, + )))) + } + (PrimitiveType::Long, JsonValue::Number(number)) => Ok(Some(Literal::Primitive( + PrimitiveLiteral::Long(number.as_i64().ok_or(Error::new( + crate::ErrorKind::DataInvalid, + "Failed to convert json number to long", + ))?), + ))), + (PrimitiveType::Float, JsonValue::Number(number)) => Ok(Some(Literal::Primitive( + PrimitiveLiteral::Float(OrderedFloat(number.as_f64().ok_or(Error::new( + crate::ErrorKind::DataInvalid, + "Failed to convert json number to float", + ))? as f32)), + ))), + (PrimitiveType::Double, JsonValue::Number(number)) => Ok(Some(Literal::Primitive( + PrimitiveLiteral::Double(OrderedFloat(number.as_f64().ok_or(Error::new( + crate::ErrorKind::DataInvalid, + "Failed to convert json number to double", + ))?)), + ))), + (PrimitiveType::Date, JsonValue::String(s)) => { + Ok(Some(Literal::Primitive(PrimitiveLiteral::Date( + date::date_to_days(&NaiveDate::parse_from_str(&s, "%Y-%m-%d")?), + )))) + } + (PrimitiveType::Time, JsonValue::String(s)) => { + Ok(Some(Literal::Primitive(PrimitiveLiteral::Time( + time::time_to_microseconds(&NaiveTime::parse_from_str(&s, "%H:%M:%S%.f")?), + )))) + } + (PrimitiveType::Timestamp, JsonValue::String(s)) => Ok(Some(Literal::Primitive( + PrimitiveLiteral::Timestamp(timestamp::datetime_to_microseconds( + &NaiveDateTime::parse_from_str(&s, "%Y-%m-%dT%H:%M:%S%.f")?, + )), + ))), + (PrimitiveType::Timestamptz, JsonValue::String(s)) => { + Ok(Some(Literal::Primitive(PrimitiveLiteral::Timestamptz( + timestamptz::datetimetz_to_microseconds(&Utc.from_utc_datetime( + &NaiveDateTime::parse_from_str(&s, "%Y-%m-%dT%H:%M:%S%.f+00:00")?, + )), + )))) + } + (PrimitiveType::String, JsonValue::String(s)) => { + Ok(Some(Literal::Primitive(PrimitiveLiteral::String(s)))) + } + (PrimitiveType::Uuid, JsonValue::String(s)) => Ok(Some(Literal::Primitive( + PrimitiveLiteral::UUID(Uuid::parse_str(&s)?), + ))), + (PrimitiveType::Fixed(_), JsonValue::String(_)) => todo!(), + (PrimitiveType::Binary, JsonValue::String(_)) => todo!(), + ( + PrimitiveType::Decimal { + precision: _, + scale, + }, + JsonValue::String(s), + ) => { + let mut decimal = Decimal::from_str_exact(&s)?; + decimal.rescale(*scale); + Ok(Some(Literal::Primitive(PrimitiveLiteral::Decimal( + decimal.mantissa(), + )))) + } + (_, JsonValue::Null) => Ok(None), + (i, j) => Err(Error::new( + crate::ErrorKind::DataInvalid, + format!( + "The json value {} doesn't fit to the iceberg type {}.", + j, i + ), + )), + }, + Type::Struct(schema) => { + if let JsonValue::Object(mut object) = value { + Ok(Some(Literal::Struct(Struct::from_iter( + schema.fields().iter().map(|field| { + object.remove(&field.id.to_string()).and_then(|value| { + Literal::try_from_json(value, &field.field_type) + .and_then(|value| { + value.ok_or(Error::new( + ErrorKind::DataInvalid, + "Key of map cannot be null", + )) + }) + .ok() + }) + }), + )))) + } else { + Err(Error::new( + crate::ErrorKind::DataInvalid, + "The json value for a struct type must be an object.", + )) + } + } + Type::List(list) => { + if let JsonValue::Array(array) = value { + Ok(Some(Literal::List( + array + .into_iter() + .map(|value| { + Literal::try_from_json(value, &list.element_field.field_type) + }) + .collect::>>()?, + ))) + } else { + Err(Error::new( + crate::ErrorKind::DataInvalid, + "The json value for a list type must be an array.", + )) + } + } + Type::Map(map) => { + if let JsonValue::Object(mut object) = value { + if let (Some(JsonValue::Array(keys)), Some(JsonValue::Array(values))) = + (object.remove("keys"), object.remove("values")) + { + Ok(Some(Literal::Map(Map::from_iter( + keys.into_iter() + .zip(values.into_iter()) + .map(|(key, value)| { + Ok(( + Literal::try_from_json(key, &map.key_field.field_type) + .and_then(|value| { + value.ok_or(Error::new( + ErrorKind::DataInvalid, + "Key of map cannot be null", + )) + })?, + Literal::try_from_json(value, &map.value_field.field_type)?, + )) + }) + .collect::>>()?, + )))) + } else { + Err(Error::new( + crate::ErrorKind::DataInvalid, + "The json value for a list type must be an array.", + )) + } + } else { + Err(Error::new( + crate::ErrorKind::DataInvalid, + "The json value for a list type must be an array.", + )) + } + } + } + } + + /// Converting iceberg value to json value. + /// + /// See [this spec](https://iceberg.apache.org/spec/#json-single-value-serialization) for reference. + pub fn try_into_json(self, r#type: &Type) -> Result { + match (self, r#type) { + (Literal::Primitive(prim), _) => match prim { + PrimitiveLiteral::Boolean(val) => Ok(JsonValue::Bool(val)), + PrimitiveLiteral::Int(val) => Ok(JsonValue::Number((val).into())), + PrimitiveLiteral::Long(val) => Ok(JsonValue::Number((val).into())), + PrimitiveLiteral::Float(val) => match Number::from_f64(val.0 as f64) { + Some(number) => Ok(JsonValue::Number(number)), + None => Ok(JsonValue::Null), + }, + PrimitiveLiteral::Double(val) => match Number::from_f64(val.0) { + Some(number) => Ok(JsonValue::Number(number)), + None => Ok(JsonValue::Null), + }, + PrimitiveLiteral::Date(val) => { + Ok(JsonValue::String(date::days_to_date(val).to_string())) + } + PrimitiveLiteral::Time(val) => Ok(JsonValue::String( + time::microseconds_to_time(val).to_string(), + )), + PrimitiveLiteral::Timestamp(val) => Ok(JsonValue::String( + timestamp::microseconds_to_datetime(val) + .format("%Y-%m-%dT%H:%M:%S%.f") + .to_string(), + )), + PrimitiveLiteral::Timestamptz(val) => Ok(JsonValue::String( + timestamptz::microseconds_to_datetimetz(val) + .format("%Y-%m-%dT%H:%M:%S%.f+00:00") + .to_string(), + )), + PrimitiveLiteral::String(val) => Ok(JsonValue::String(val.clone())), + PrimitiveLiteral::UUID(val) => Ok(JsonValue::String(val.to_string())), + PrimitiveLiteral::Fixed(val) => Ok(JsonValue::String(val.iter().fold( + String::new(), + |mut acc, x| { + acc.push_str(&format!("{:x}", x)); + acc + }, + ))), + PrimitiveLiteral::Binary(val) => Ok(JsonValue::String(val.iter().fold( + String::new(), + |mut acc, x| { + acc.push_str(&format!("{:x}", x)); + acc + }, + ))), + PrimitiveLiteral::Decimal(val) => match r#type { + Type::Primitive(PrimitiveType::Decimal { + precision: _precision, + scale, + }) => { + let decimal = Decimal::try_from_i128_with_scale(val, *scale)?; + Ok(JsonValue::String(decimal.to_string())) + } + _ => Err(Error::new( + ErrorKind::DataInvalid, + "The iceberg type for decimal literal must be decimal.", + ))?, + }, + }, + (Literal::Struct(s), Type::Struct(struct_type)) => { + let mut id_and_value = Vec::with_capacity(struct_type.fields().len()); + for (value, field) in s.into_iter().zip(struct_type.fields()) { + let json = match value { + Some(val) => val.try_into_json(&field.field_type)?, + None => JsonValue::Null, + }; + id_and_value.push((field.id.to_string(), json)); + } + Ok(JsonValue::Object(JsonMap::from_iter(id_and_value))) + } + (Literal::List(list), Type::List(list_type)) => Ok(JsonValue::Array( + list.into_iter() + .map(|opt| match opt { + Some(literal) => literal.try_into_json(&list_type.element_field.field_type), + None => Ok(JsonValue::Null), + }) + .collect::>>()?, + )), + (Literal::Map(map), Type::Map(map_type)) => { + let mut object = JsonMap::with_capacity(2); + let mut json_keys = Vec::with_capacity(map.len()); + let mut json_values = Vec::with_capacity(map.len()); + for (key, value) in map.into_iter() { + json_keys.push(key.try_into_json(&map_type.key_field.field_type)?); + json_values.push(match value { + Some(literal) => literal.try_into_json(&map_type.value_field.field_type)?, + None => JsonValue::Null, + }); + } + object.insert("keys".to_string(), JsonValue::Array(json_keys)); + object.insert("values".to_string(), JsonValue::Array(json_values)); + Ok(JsonValue::Object(object)) + } + (value, r#type) => Err(Error::new( + ErrorKind::DataInvalid, + format!( + "The iceberg value {:?} doesn't fit to the iceberg type {}.", + value, r#type + ), + )), + } + } + + /// Convert Value to the any type + pub fn into_any(self) -> Box { + match self { + Literal::Primitive(prim) => match prim { + PrimitiveLiteral::Boolean(any) => Box::new(any), + PrimitiveLiteral::Int(any) => Box::new(any), + PrimitiveLiteral::Long(any) => Box::new(any), + PrimitiveLiteral::Float(any) => Box::new(any), + PrimitiveLiteral::Double(any) => Box::new(any), + PrimitiveLiteral::Date(any) => Box::new(any), + PrimitiveLiteral::Time(any) => Box::new(any), + PrimitiveLiteral::Timestamp(any) => Box::new(any), + PrimitiveLiteral::Timestamptz(any) => Box::new(any), + PrimitiveLiteral::Fixed(any) => Box::new(any), + PrimitiveLiteral::Binary(any) => Box::new(any), + PrimitiveLiteral::String(any) => Box::new(any), + PrimitiveLiteral::UUID(any) => Box::new(any), + PrimitiveLiteral::Decimal(any) => Box::new(any), + }, + _ => unimplemented!(), + } + } + } + + mod date { + use chrono::{DateTime, NaiveDate, TimeDelta, TimeZone, Utc}; + + pub(crate) fn date_to_days(date: &NaiveDate) -> i32 { + date.signed_duration_since( + // This is always the same and shouldn't fail + NaiveDate::from_ymd_opt(1970, 1, 1).unwrap(), + ) + .num_days() as i32 + } + + pub(crate) fn days_to_date(days: i32) -> NaiveDate { + // This shouldn't fail until the year 262000 + (chrono::DateTime::UNIX_EPOCH + TimeDelta::try_days(days as i64).unwrap()) + .naive_utc() + .date() + } + + /// Returns unix epoch. + pub(crate) fn unix_epoch() -> DateTime { + Utc.timestamp_nanos(0) + } + + /// Creates date literal from `NaiveDate`, assuming it's utc timezone. + pub(crate) fn date_from_naive_date(date: NaiveDate) -> i32 { + (date - unix_epoch().date_naive()).num_days() as i32 + } + } + + mod time { + use chrono::NaiveTime; + + pub(crate) fn time_to_microseconds(time: &NaiveTime) -> i64 { + time.signed_duration_since( + // This is always the same and shouldn't fail + NaiveTime::from_num_seconds_from_midnight_opt(0, 0).unwrap(), + ) + .num_microseconds() + .unwrap() + } + + pub(crate) fn microseconds_to_time(micros: i64) -> NaiveTime { + let (secs, rem) = (micros / 1_000_000, micros % 1_000_000); + + NaiveTime::from_num_seconds_from_midnight_opt(secs as u32, rem as u32 * 1_000).unwrap() + } + } + + mod timestamp { + use chrono::{DateTime, NaiveDateTime}; + + pub(crate) fn datetime_to_microseconds(time: &NaiveDateTime) -> i64 { + time.and_utc().timestamp_micros() + } + + pub(crate) fn microseconds_to_datetime(micros: i64) -> NaiveDateTime { + // This shouldn't fail until the year 262000 + DateTime::from_timestamp_micros(micros).unwrap().naive_utc() + } + } + + mod timestamptz { + use chrono::DateTime; + use chrono::Utc; + + pub(crate) fn datetimetz_to_microseconds(time: &DateTime) -> i64 { + time.timestamp_micros() + } + + pub(crate) fn microseconds_to_datetimetz(micros: i64) -> DateTime { + let (secs, rem) = (micros / 1_000_000, micros % 1_000_000); + + DateTime::from_timestamp(secs, rem as u32 * 1_000).unwrap() + } + } + + mod _serde { + use serde::{ + de::Visitor, + ser::{SerializeMap, SerializeSeq, SerializeStruct}, + Deserialize, Serialize, + }; + use serde_bytes::ByteBuf; + use serde_derive::Deserialize as DeserializeDerive; + use serde_derive::Serialize as SerializeDerive; + + use crate::{ + spec::{PrimitiveType, Type, MAP_KEY_FIELD_NAME, MAP_VALUE_FIELD_NAME}, + Error, ErrorKind, + }; + + use super::{Literal, Map, PrimitiveLiteral}; + + #[derive(SerializeDerive, DeserializeDerive, Debug)] + #[serde(transparent)] + /// Raw literal representation used for serde. The serialize way is used for Avro serializer. + pub struct RawLiteral(RawLiteralEnum); + + impl RawLiteral { + /// Covert literal to raw literal. + pub fn try_from(literal: Literal, ty: &Type) -> Result { + Ok(Self(RawLiteralEnum::try_from(literal, ty)?)) + } + + /// Convert raw literal to literal. + pub fn try_into(self, ty: &Type) -> Result, Error> { + self.0.try_into(ty) + } + } + + #[derive(SerializeDerive, Clone, Debug)] + #[serde(untagged)] + enum RawLiteralEnum { + Null, + Boolean(bool), + Int(i32), + Long(i64), + Float(f32), + Double(f64), + String(String), + Bytes(ByteBuf), + List(List), + StringMap(StringMap), + Record(Record), + } + + #[derive(Clone, Debug)] + struct Record { + required: Vec<(String, RawLiteralEnum)>, + optional: Vec<(String, Option)>, + } + + impl Serialize for Record { + fn serialize(&self, serializer: S) -> Result + where + S: serde::Serializer, + { + let len = self.required.len() + self.optional.len(); + let mut record = serializer.serialize_struct("", len)?; + for (k, v) in &self.required { + record.serialize_field(Box::leak(k.clone().into_boxed_str()), &v)?; + } + for (k, v) in &self.optional { + record.serialize_field(Box::leak(k.clone().into_boxed_str()), &v)?; + } + record.end() + } + } + + #[derive(Clone, Debug)] + struct List { + list: Vec>, + required: bool, + } + + impl Serialize for List { + fn serialize(&self, serializer: S) -> Result + where + S: serde::Serializer, + { + let mut seq = serializer.serialize_seq(Some(self.list.len()))?; + for value in &self.list { + if self.required { + seq.serialize_element(value.as_ref().ok_or_else(|| { + serde::ser::Error::custom( + "List element is required, element cannot be null", + ) + })?)?; + } else { + seq.serialize_element(&value)?; + } + } + seq.end() + } + } + + #[derive(Clone, Debug)] + struct StringMap { + raw: Vec<(String, Option)>, + required: bool, + } + + impl Serialize for StringMap { + fn serialize(&self, serializer: S) -> Result + where + S: serde::Serializer, + { + let mut map = serializer.serialize_map(Some(self.raw.len()))?; + for (k, v) in &self.raw { + if self.required { + map.serialize_entry( + k, + v.as_ref().ok_or_else(|| { + serde::ser::Error::custom( + "Map element is required, element cannot be null", + ) + })?, + )?; + } else { + map.serialize_entry(k, v)?; + } + } + map.end() + } + } + + impl<'de> Deserialize<'de> for RawLiteralEnum { + fn deserialize(deserializer: D) -> Result + where + D: serde::Deserializer<'de>, + { + struct RawLiteralVisitor; + impl<'de> Visitor<'de> for RawLiteralVisitor { + type Value = RawLiteralEnum; + + fn expecting(&self, formatter: &mut std::fmt::Formatter) -> std::fmt::Result { + formatter.write_str("expect") + } + + fn visit_bool(self, v: bool) -> Result + where + E: serde::de::Error, + { + Ok(RawLiteralEnum::Boolean(v)) + } + + fn visit_i32(self, v: i32) -> Result + where + E: serde::de::Error, + { + Ok(RawLiteralEnum::Int(v)) + } + + fn visit_i64(self, v: i64) -> Result + where + E: serde::de::Error, + { + Ok(RawLiteralEnum::Long(v)) + } + + /// Used in json + fn visit_u64(self, v: u64) -> Result + where + E: serde::de::Error, + { + Ok(RawLiteralEnum::Long(v as i64)) + } + + fn visit_f32(self, v: f32) -> Result + where + E: serde::de::Error, + { + Ok(RawLiteralEnum::Float(v)) + } + + fn visit_f64(self, v: f64) -> Result + where + E: serde::de::Error, + { + Ok(RawLiteralEnum::Double(v)) + } + + fn visit_str(self, v: &str) -> Result + where + E: serde::de::Error, + { + Ok(RawLiteralEnum::String(v.to_string())) + } + + fn visit_bytes(self, v: &[u8]) -> Result + where + E: serde::de::Error, + { + Ok(RawLiteralEnum::Bytes(ByteBuf::from(v))) + } + + fn visit_borrowed_str(self, v: &'de str) -> Result + where + E: serde::de::Error, + { + Ok(RawLiteralEnum::String(v.to_string())) + } + + fn visit_unit(self) -> Result + where + E: serde::de::Error, + { + Ok(RawLiteralEnum::Null) + } + + fn visit_map(self, mut map: A) -> Result + where + A: serde::de::MapAccess<'de>, + { + let mut required = Vec::new(); + while let Some(key) = map.next_key::()? { + let value = map.next_value::()?; + required.push((key, value)); + } + Ok(RawLiteralEnum::Record(Record { + required, + optional: Vec::new(), + })) + } + + fn visit_seq(self, mut seq: A) -> Result + where + A: serde::de::SeqAccess<'de>, + { + let mut list = Vec::new(); + while let Some(value) = seq.next_element::()? { + list.push(Some(value)); + } + Ok(RawLiteralEnum::List(List { + list, + // `required` only used in serialize, just set default in deserialize. + required: false, + })) + } + } + deserializer.deserialize_any(RawLiteralVisitor) + } + } + + impl RawLiteralEnum { + pub fn try_from(literal: Literal, ty: &Type) -> Result { + let raw = match literal { + Literal::Primitive(prim) => match prim { + super::PrimitiveLiteral::Boolean(v) => RawLiteralEnum::Boolean(v), + super::PrimitiveLiteral::Int(v) => RawLiteralEnum::Int(v), + super::PrimitiveLiteral::Long(v) => RawLiteralEnum::Long(v), + super::PrimitiveLiteral::Float(v) => RawLiteralEnum::Float(v.0), + super::PrimitiveLiteral::Double(v) => RawLiteralEnum::Double(v.0), + super::PrimitiveLiteral::Date(v) => RawLiteralEnum::Int(v), + super::PrimitiveLiteral::Time(v) => RawLiteralEnum::Long(v), + super::PrimitiveLiteral::Timestamp(v) => RawLiteralEnum::Long(v), + super::PrimitiveLiteral::Timestamptz(v) => RawLiteralEnum::Long(v), + super::PrimitiveLiteral::String(v) => RawLiteralEnum::String(v), + super::PrimitiveLiteral::UUID(v) => { + RawLiteralEnum::Bytes(ByteBuf::from(v.as_u128().to_be_bytes())) + } + super::PrimitiveLiteral::Fixed(v) => RawLiteralEnum::Bytes(ByteBuf::from(v)), + super::PrimitiveLiteral::Binary(v) => RawLiteralEnum::Bytes(ByteBuf::from(v)), + super::PrimitiveLiteral::Decimal(v) => { + RawLiteralEnum::Bytes(ByteBuf::from(v.to_be_bytes())) + } + }, + Literal::Struct(r#struct) => { + let mut required = Vec::new(); + let mut optional = Vec::new(); + if let Type::Struct(struct_ty) = ty { + for (value, field) in r#struct.into_iter().zip(struct_ty.fields()) { + if field.required { + if let Some(value) = value { + required.push(( + field.name.clone(), + RawLiteralEnum::try_from(value, &field.field_type)?, + )); + } else { + return Err(Error::new( + ErrorKind::DataInvalid, + "Can't convert null to required field", + )); + } + } else if let Some(value) = value { + optional.push(( + field.name.clone(), + Some(RawLiteralEnum::try_from(value, &field.field_type)?), + )); + } else { + optional.push((field.name.clone(), None)); + } + } + } else { + return Err(Error::new( + ErrorKind::DataInvalid, + format!("Type {} should be a struct", ty), + )); + } + RawLiteralEnum::Record(Record { required, optional }) + } + Literal::List(list) => { + if let Type::List(list_ty) = ty { + let list = list + .into_iter() + .map(|v| { + v.map(|v| { + RawLiteralEnum::try_from(v, &list_ty.element_field.field_type) + }) + .transpose() + }) + .collect::>()?; + RawLiteralEnum::List(List { + list, + required: list_ty.element_field.required, + }) + } else { + return Err(Error::new( + ErrorKind::DataInvalid, + format!("Type {} should be a list", ty), + )); + } + } + Literal::Map(map) => { + if let Type::Map(map_ty) = ty { + if let Type::Primitive(PrimitiveType::String) = *map_ty.key_field.field_type + { + let mut raw = Vec::with_capacity(map.len()); + for (k, v) in map { + if let Literal::Primitive(PrimitiveLiteral::String(k)) = k { + raw.push(( + k, + v.map(|v| { + RawLiteralEnum::try_from( + v, + &map_ty.value_field.field_type, + ) + }) + .transpose()?, + )); + } else { + return Err(Error::new( + ErrorKind::DataInvalid, + "literal type is inconsistent with type", + )); + } + } + RawLiteralEnum::StringMap(StringMap { + raw, + required: map_ty.value_field.required, + }) + } else { + let list = map.into_iter().map(|(k,v)| { let raw_k = RawLiteralEnum::try_from(k, &map_ty.key_field.field_type)?; let raw_v = v @@ -2392,1082 +2392,1082 @@ mod _serde { }))) } }).collect::>()?; - RawLiteralEnum::List(List { - list, - required: true, - }) - } - } else { - return Err(Error::new( - ErrorKind::DataInvalid, - format!("Type {} should be a map", ty), - )); - } - } - }; - Ok(raw) - } - - pub fn try_into(self, ty: &Type) -> Result, Error> { - let invalid_err = |v: &str| { - Error::new( + RawLiteralEnum::List(List { + list, + required: true, + }) + } + } else { + return Err(Error::new( + ErrorKind::DataInvalid, + format!("Type {} should be a map", ty), + )); + } + } + }; + Ok(raw) + } + + pub fn try_into(self, ty: &Type) -> Result, Error> { + let invalid_err = |v: &str| { + Error::new( ErrorKind::DataInvalid, format!( "Unable to convert raw literal ({}) fail convert to type {} for: type mismatch", v, ty ), ) - }; - let invalid_err_with_reason = |v: &str, reason: &str| { - Error::new( - ErrorKind::DataInvalid, - format!( - "Unable to convert raw literal ({}) fail convert to type {} for: {}", - v, ty, reason - ), - ) - }; - match self { - RawLiteralEnum::Null => Ok(None), - RawLiteralEnum::Boolean(v) => Ok(Some(Literal::bool(v))), - RawLiteralEnum::Int(v) => match ty { - Type::Primitive(PrimitiveType::Int) => Ok(Some(Literal::int(v))), - Type::Primitive(PrimitiveType::Long) => Ok(Some(Literal::long(i64::from(v)))), - Type::Primitive(PrimitiveType::Date) => Ok(Some(Literal::date(v))), - _ => Err(invalid_err("int")), - }, - RawLiteralEnum::Long(v) => match ty { - Type::Primitive(PrimitiveType::Int) => Ok(Some(Literal::int( - i32::try_from(v).map_err(|_| invalid_err("long"))?, - ))), - Type::Primitive(PrimitiveType::Date) => Ok(Some(Literal::date( - i32::try_from(v).map_err(|_| invalid_err("long"))?, - ))), - Type::Primitive(PrimitiveType::Long) => Ok(Some(Literal::long(v))), - Type::Primitive(PrimitiveType::Time) => Ok(Some(Literal::time(v))), - Type::Primitive(PrimitiveType::Timestamp) => Ok(Some(Literal::timestamp(v))), - Type::Primitive(PrimitiveType::Timestamptz) => { - Ok(Some(Literal::timestamptz(v))) - } - _ => Err(invalid_err("long")), - }, - RawLiteralEnum::Float(v) => match ty { - Type::Primitive(PrimitiveType::Float) => Ok(Some(Literal::float(v))), - Type::Primitive(PrimitiveType::Double) => { - Ok(Some(Literal::double(f64::from(v)))) - } - _ => Err(invalid_err("float")), - }, - RawLiteralEnum::Double(v) => match ty { - Type::Primitive(PrimitiveType::Float) => { - let v_32 = v as f32; - if v_32.is_finite() { - let v_64 = f64::from(v_32); - if (v_64 - v).abs() > f32::EPSILON as f64 { - // there is a precision loss - return Err(invalid_err("double")); - } - } - Ok(Some(Literal::float(v_32))) - } - Type::Primitive(PrimitiveType::Double) => Ok(Some(Literal::double(v))), - _ => Err(invalid_err("double")), - }, - RawLiteralEnum::String(v) => match ty { - Type::Primitive(PrimitiveType::String) => Ok(Some(Literal::string(v))), - _ => Err(invalid_err("string")), - }, - // # TODO:https://github.com/apache/iceberg-rust/issues/86 - // rust avro don't support deserialize any bytes representation now. - RawLiteralEnum::Bytes(_) => Err(invalid_err_with_reason( - "bytes", - "todo: rust avro doesn't support deserialize any bytes representation now", - )), - RawLiteralEnum::List(v) => { - match ty { - Type::List(ty) => Ok(Some(Literal::List( - v.list - .into_iter() - .map(|v| { - if let Some(v) = v { - v.try_into(&ty.element_field.field_type) - } else { - Ok(None) - } - }) - .collect::>()?, - ))), - Type::Map(map_ty) => { - let key_ty = map_ty.key_field.field_type.as_ref(); - let value_ty = map_ty.value_field.field_type.as_ref(); - let mut map = Map::new(); - for k_v in v.list { - let k_v = k_v.ok_or_else(|| invalid_err_with_reason("list","In deserialize, None will be represented as Some(RawLiteral::Null), all element in list must be valid"))?; - if let RawLiteralEnum::Record(Record { - required, - optional: _, - }) = k_v - { - if required.len() != 2 { - return Err(invalid_err_with_reason("list","Record must contains two element(key and value) of array")); - } - let mut key = None; - let mut value = None; - required.into_iter().for_each(|(k, v)| { - if k == MAP_KEY_FIELD_NAME { - key = Some(v); - } else if k == MAP_VALUE_FIELD_NAME { - value = Some(v); - } - }); - match (key, value) { - (Some(k), Some(v)) => { - let key = k.try_into(key_ty)?.ok_or_else(|| { - invalid_err_with_reason( - "list", - "Key element in Map must be valid", - ) - })?; - let value = v.try_into(value_ty)?; - if map_ty.value_field.required && value.is_none() { - return Err(invalid_err_with_reason( - "list", - "Value element is required in this Map", - )); - } - map.insert(key, value); - } - _ => return Err(invalid_err_with_reason( - "list", - "The elements of record in list are not key and value", - )), - } - } else { - return Err(invalid_err_with_reason( - "list", - "Map should represented as record array.", - )); - } - } - Ok(Some(Literal::Map(map))) - } - Type::Primitive(PrimitiveType::Uuid) => { - if v.list.len() != 16 { - return Err(invalid_err_with_reason( - "list", - "The length of list should be 16", - )); - } - let mut bytes = [0u8; 16]; - for (i, v) in v.list.iter().enumerate() { - if let Some(RawLiteralEnum::Long(v)) = v { - bytes[i] = *v as u8; - } else { - return Err(invalid_err_with_reason( - "list", - "The element of list should be int", - )); - } - } - Ok(Some(Literal::uuid(uuid::Uuid::from_bytes(bytes)))) - } - Type::Primitive(PrimitiveType::Decimal { - precision: _, - scale: _, - }) => { - if v.list.len() != 16 { - return Err(invalid_err_with_reason( - "list", - "The length of list should be 16", - )); - } - let mut bytes = [0u8; 16]; - for (i, v) in v.list.iter().enumerate() { - if let Some(RawLiteralEnum::Long(v)) = v { - bytes[i] = *v as u8; - } else { - return Err(invalid_err_with_reason( - "list", - "The element of list should be int", - )); - } - } - Ok(Some(Literal::decimal(i128::from_be_bytes(bytes)))) - } - Type::Primitive(PrimitiveType::Binary) => { - let bytes = v - .list - .into_iter() - .map(|v| { - if let Some(RawLiteralEnum::Long(v)) = v { - Ok(v as u8) - } else { - Err(invalid_err_with_reason( - "list", - "The element of list should be int", - )) - } - }) - .collect::, Error>>()?; - Ok(Some(Literal::binary(bytes))) - } - Type::Primitive(PrimitiveType::Fixed(size)) => { - if v.list.len() != *size as usize { - return Err(invalid_err_with_reason( - "list", - "The length of list should be equal to size", - )); - } - let bytes = v - .list - .into_iter() - .map(|v| { - if let Some(RawLiteralEnum::Long(v)) = v { - Ok(v as u8) - } else { - Err(invalid_err_with_reason( - "list", - "The element of list should be int", - )) - } - }) - .collect::, Error>>()?; - Ok(Some(Literal::fixed(bytes))) - } - _ => Err(invalid_err("list")), - } - } - RawLiteralEnum::Record(Record { - required, - optional: _, - }) => match ty { - Type::Struct(struct_ty) => { - let iters: Vec> = required - .into_iter() - .map(|(field_name, value)| { - let field = struct_ty - .field_by_name(field_name.as_str()) - .ok_or_else(|| { - invalid_err_with_reason( - "record", - &format!("field {} is not exist", &field_name), - ) - })?; - let value = value.try_into(&field.field_type)?; - Ok(value) - }) - .collect::>()?; - Ok(Some(Literal::Struct(super::Struct::from_iter(iters)))) - } - Type::Map(map_ty) => { - if *map_ty.key_field.field_type != Type::Primitive(PrimitiveType::String) { - return Err(invalid_err_with_reason( - "record", - "Map key must be string", - )); - } - let mut map = Map::new(); - for (k, v) in required { - let value = v.try_into(&map_ty.value_field.field_type)?; - if map_ty.value_field.required && value.is_none() { - return Err(invalid_err_with_reason( - "record", - "Value element is required in this Map", - )); - } - map.insert(Literal::string(k), value); - } - Ok(Some(Literal::Map(map))) - } - _ => Err(invalid_err("record")), - }, - RawLiteralEnum::StringMap(_) => Err(invalid_err("string map")), - } - } - } -} - -#[cfg(test)] -mod tests { - use apache_avro::{to_value, types::Value}; - - use crate::{ - avro::schema_to_avro_schema, - spec::{ - datatypes::{ListType, MapType, NestedField, StructType}, - Schema, - }, - }; - - use super::*; - - fn check_json_serde(json: &str, expected_literal: Literal, expected_type: &Type) { - let raw_json_value = serde_json::from_str::(json).unwrap(); - let desered_literal = - Literal::try_from_json(raw_json_value.clone(), expected_type).unwrap(); - assert_eq!(desered_literal, Some(expected_literal.clone())); - - let expected_json_value: JsonValue = expected_literal.try_into_json(expected_type).unwrap(); - let sered_json = serde_json::to_string(&expected_json_value).unwrap(); - let parsed_json_value = serde_json::from_str::(&sered_json).unwrap(); - - assert_eq!(parsed_json_value, raw_json_value); - } - - fn check_avro_bytes_serde(input: Vec, expected_literal: Literal, expected_type: &Type) { - let raw_schema = r#""bytes""#; - let schema = apache_avro::Schema::parse_str(raw_schema).unwrap(); - - let bytes = ByteBuf::from(input); - let literal = Literal::try_from_bytes(&bytes, expected_type).unwrap(); - assert_eq!(literal, expected_literal); - - let mut writer = apache_avro::Writer::new(&schema, Vec::new()); - writer.append_ser(ByteBuf::from(literal)).unwrap(); - let encoded = writer.into_inner().unwrap(); - let reader = apache_avro::Reader::with_schema(&schema, &*encoded).unwrap(); - - for record in reader { - let result = apache_avro::from_value::(&record.unwrap()).unwrap(); - let desered_literal = Literal::try_from_bytes(&result, expected_type).unwrap(); - assert_eq!(desered_literal, expected_literal); - } - } - - fn check_convert_with_avro(expected_literal: Literal, expected_type: &Type) { - let fields = vec![NestedField::required(1, "col", expected_type.clone()).into()]; - let schema = Schema::builder() - .with_fields(fields.clone()) - .build() - .unwrap(); - let avro_schema = schema_to_avro_schema("test", &schema).unwrap(); - let struct_type = Type::Struct(StructType::new(fields)); - let struct_literal = - Literal::Struct(Struct::from_iter(vec![Some(expected_literal.clone())])); - - let mut writer = apache_avro::Writer::new(&avro_schema, Vec::new()); - let raw_literal = RawLiteral::try_from(struct_literal.clone(), &struct_type).unwrap(); - writer.append_ser(raw_literal).unwrap(); - let encoded = writer.into_inner().unwrap(); - - let reader = apache_avro::Reader::new(&*encoded).unwrap(); - for record in reader { - let result = apache_avro::from_value::(&record.unwrap()).unwrap(); - let desered_literal = result.try_into(&struct_type).unwrap().unwrap(); - assert_eq!(desered_literal, struct_literal); - } - } - - fn check_serialize_avro(literal: Literal, ty: &Type, expect_value: Value) { - let expect_value = Value::Record(vec![("col".to_string(), expect_value)]); - - let fields = vec![NestedField::required(1, "col", ty.clone()).into()]; - let schema = Schema::builder() - .with_fields(fields.clone()) - .build() - .unwrap(); - let avro_schema = schema_to_avro_schema("test", &schema).unwrap(); - let struct_type = Type::Struct(StructType::new(fields)); - let struct_literal = Literal::Struct(Struct::from_iter(vec![Some(literal.clone())])); - let mut writer = apache_avro::Writer::new(&avro_schema, Vec::new()); - let raw_literal = RawLiteral::try_from(struct_literal.clone(), &struct_type).unwrap(); - let value = to_value(raw_literal) - .unwrap() - .resolve(&avro_schema) - .unwrap(); - writer.append_value_ref(&value).unwrap(); - let encoded = writer.into_inner().unwrap(); - - let reader = apache_avro::Reader::new(&*encoded).unwrap(); - for record in reader { - assert_eq!(record.unwrap(), expect_value); - } - } - - #[test] - fn json_boolean() { - let record = r#"true"#; - - check_json_serde( - record, - Literal::Primitive(PrimitiveLiteral::Boolean(true)), - &Type::Primitive(PrimitiveType::Boolean), - ); - } - - #[test] - fn json_int() { - let record = r#"32"#; - - check_json_serde( - record, - Literal::Primitive(PrimitiveLiteral::Int(32)), - &Type::Primitive(PrimitiveType::Int), - ); - } - - #[test] - fn json_long() { - let record = r#"32"#; - - check_json_serde( - record, - Literal::Primitive(PrimitiveLiteral::Long(32)), - &Type::Primitive(PrimitiveType::Long), - ); - } - - #[test] - fn json_float() { - let record = r#"1.0"#; - - check_json_serde( - record, - Literal::Primitive(PrimitiveLiteral::Float(OrderedFloat(1.0))), - &Type::Primitive(PrimitiveType::Float), - ); - } - - #[test] - fn json_double() { - let record = r#"1.0"#; - - check_json_serde( - record, - Literal::Primitive(PrimitiveLiteral::Double(OrderedFloat(1.0))), - &Type::Primitive(PrimitiveType::Double), - ); - } - - #[test] - fn json_date() { - let record = r#""2017-11-16""#; - - check_json_serde( - record, - Literal::Primitive(PrimitiveLiteral::Date(17486)), - &Type::Primitive(PrimitiveType::Date), - ); - } - - #[test] - fn json_time() { - let record = r#""22:31:08.123456""#; - - check_json_serde( - record, - Literal::Primitive(PrimitiveLiteral::Time(81068123456)), - &Type::Primitive(PrimitiveType::Time), - ); - } - - #[test] - fn json_timestamp() { - let record = r#""2017-11-16T22:31:08.123456""#; - - check_json_serde( - record, - Literal::Primitive(PrimitiveLiteral::Timestamp(1510871468123456)), - &Type::Primitive(PrimitiveType::Timestamp), - ); - } - - #[test] - fn json_timestamptz() { - let record = r#""2017-11-16T22:31:08.123456+00:00""#; - - check_json_serde( - record, - Literal::Primitive(PrimitiveLiteral::Timestamptz(1510871468123456)), - &Type::Primitive(PrimitiveType::Timestamptz), - ); - } - - #[test] - fn json_string() { - let record = r#""iceberg""#; - - check_json_serde( - record, - Literal::Primitive(PrimitiveLiteral::String("iceberg".to_string())), - &Type::Primitive(PrimitiveType::String), - ); - } - - #[test] - fn json_uuid() { - let record = r#""f79c3e09-677c-4bbd-a479-3f349cb785e7""#; - - check_json_serde( - record, - Literal::Primitive(PrimitiveLiteral::UUID( - Uuid::parse_str("f79c3e09-677c-4bbd-a479-3f349cb785e7").unwrap(), - )), - &Type::Primitive(PrimitiveType::Uuid), - ); - } - - #[test] - fn json_decimal() { - let record = r#""14.20""#; - - check_json_serde( - record, - Literal::Primitive(PrimitiveLiteral::Decimal(1420)), - &Type::decimal(28, 2).unwrap(), - ); - } - - #[test] - fn json_struct() { - let record = r#"{"1": 1, "2": "bar", "3": null}"#; - - check_json_serde( - record, - Literal::Struct(Struct::from_iter(vec![ - Some(Literal::Primitive(PrimitiveLiteral::Int(1))), - Some(Literal::Primitive(PrimitiveLiteral::String( - "bar".to_string(), - ))), - None, - ])), - &Type::Struct(StructType::new(vec![ - NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), - NestedField::optional(2, "name", Type::Primitive(PrimitiveType::String)).into(), - NestedField::optional(3, "address", Type::Primitive(PrimitiveType::String)).into(), - ])), - ); - } - - #[test] - fn json_list() { - let record = r#"[1, 2, 3, null]"#; - - check_json_serde( - record, - Literal::List(vec![ - Some(Literal::Primitive(PrimitiveLiteral::Int(1))), - Some(Literal::Primitive(PrimitiveLiteral::Int(2))), - Some(Literal::Primitive(PrimitiveLiteral::Int(3))), - None, - ]), - &Type::List(ListType { - element_field: NestedField::list_element( - 0, - Type::Primitive(PrimitiveType::Int), - true, - ) - .into(), - }), - ); - } - - #[test] - fn json_map() { - let record = r#"{ "keys": ["a", "b", "c"], "values": [1, 2, null] }"#; - - check_json_serde( - record, - Literal::Map(Map::from([ - ( - Literal::Primitive(PrimitiveLiteral::String("a".to_string())), - Some(Literal::Primitive(PrimitiveLiteral::Int(1))), - ), - ( - Literal::Primitive(PrimitiveLiteral::String("b".to_string())), - Some(Literal::Primitive(PrimitiveLiteral::Int(2))), - ), - ( - Literal::Primitive(PrimitiveLiteral::String("c".to_string())), - None, - ), - ])), - &Type::Map(MapType { - key_field: NestedField::map_key_element(0, Type::Primitive(PrimitiveType::String)) - .into(), - value_field: NestedField::map_value_element( - 1, - Type::Primitive(PrimitiveType::Int), - true, - ) - .into(), - }), - ); - } - - #[test] - fn avro_bytes_boolean() { - let bytes = vec![1u8]; - - check_avro_bytes_serde( - bytes, - Literal::Primitive(PrimitiveLiteral::Boolean(true)), - &Type::Primitive(PrimitiveType::Boolean), - ); - } - - #[test] - fn avro_bytes_int() { - let bytes = vec![32u8, 0u8, 0u8, 0u8]; - - check_avro_bytes_serde( - bytes, - Literal::Primitive(PrimitiveLiteral::Int(32)), - &Type::Primitive(PrimitiveType::Int), - ); - } - - #[test] - fn avro_bytes_long() { - let bytes = vec![32u8, 0u8, 0u8, 0u8, 0u8, 0u8, 0u8, 0u8]; - - check_avro_bytes_serde( - bytes, - Literal::Primitive(PrimitiveLiteral::Long(32)), - &Type::Primitive(PrimitiveType::Long), - ); - } - - #[test] - fn avro_bytes_float() { - let bytes = vec![0u8, 0u8, 128u8, 63u8]; - - check_avro_bytes_serde( - bytes, - Literal::Primitive(PrimitiveLiteral::Float(OrderedFloat(1.0))), - &Type::Primitive(PrimitiveType::Float), - ); - } - - #[test] - fn avro_bytes_double() { - let bytes = vec![0u8, 0u8, 0u8, 0u8, 0u8, 0u8, 240u8, 63u8]; - - check_avro_bytes_serde( - bytes, - Literal::Primitive(PrimitiveLiteral::Double(OrderedFloat(1.0))), - &Type::Primitive(PrimitiveType::Double), - ); - } - - #[test] - fn avro_bytes_string() { - let bytes = vec![105u8, 99u8, 101u8, 98u8, 101u8, 114u8, 103u8]; - - check_avro_bytes_serde( - bytes, - Literal::Primitive(PrimitiveLiteral::String("iceberg".to_string())), - &Type::Primitive(PrimitiveType::String), - ); - } - - #[test] - fn avro_convert_test_int() { - check_convert_with_avro( - Literal::Primitive(PrimitiveLiteral::Int(32)), - &Type::Primitive(PrimitiveType::Int), - ); - } - - #[test] - fn avro_convert_test_long() { - check_convert_with_avro( - Literal::Primitive(PrimitiveLiteral::Long(32)), - &Type::Primitive(PrimitiveType::Long), - ); - } - - #[test] - fn avro_convert_test_float() { - check_convert_with_avro( - Literal::Primitive(PrimitiveLiteral::Float(OrderedFloat(1.0))), - &Type::Primitive(PrimitiveType::Float), - ); - } - - #[test] - fn avro_convert_test_double() { - check_convert_with_avro( - Literal::Primitive(PrimitiveLiteral::Double(OrderedFloat(1.0))), - &Type::Primitive(PrimitiveType::Double), - ); - } - - #[test] - fn avro_convert_test_string() { - check_convert_with_avro( - Literal::Primitive(PrimitiveLiteral::String("iceberg".to_string())), - &Type::Primitive(PrimitiveType::String), - ); - } - - #[test] - fn avro_convert_test_date() { - check_convert_with_avro( - Literal::Primitive(PrimitiveLiteral::Date(17486)), - &Type::Primitive(PrimitiveType::Date), - ); - } - - #[test] - fn avro_convert_test_time() { - check_convert_with_avro( - Literal::Primitive(PrimitiveLiteral::Time(81068123456)), - &Type::Primitive(PrimitiveType::Time), - ); - } - - #[test] - fn avro_convert_test_timestamp() { - check_convert_with_avro( - Literal::Primitive(PrimitiveLiteral::Timestamp(1510871468123456)), - &Type::Primitive(PrimitiveType::Timestamp), - ); - } - - #[test] - fn avro_convert_test_timestamptz() { - check_convert_with_avro( - Literal::Primitive(PrimitiveLiteral::Timestamptz(1510871468123456)), - &Type::Primitive(PrimitiveType::Timestamptz), - ); - } - - #[test] - fn avro_convert_test_list() { - check_convert_with_avro( - Literal::List(vec![ - Some(Literal::Primitive(PrimitiveLiteral::Int(1))), - Some(Literal::Primitive(PrimitiveLiteral::Int(2))), - Some(Literal::Primitive(PrimitiveLiteral::Int(3))), - None, - ]), - &Type::List(ListType { - element_field: NestedField::list_element( - 0, - Type::Primitive(PrimitiveType::Int), - false, - ) - .into(), - }), - ); - - check_convert_with_avro( - Literal::List(vec![ - Some(Literal::Primitive(PrimitiveLiteral::Int(1))), - Some(Literal::Primitive(PrimitiveLiteral::Int(2))), - Some(Literal::Primitive(PrimitiveLiteral::Int(3))), - ]), - &Type::List(ListType { - element_field: NestedField::list_element( - 0, - Type::Primitive(PrimitiveType::Int), - true, - ) - .into(), - }), - ); - } - - fn check_convert_with_avro_map(expected_literal: Literal, expected_type: &Type) { - let fields = vec![NestedField::required(1, "col", expected_type.clone()).into()]; - let schema = Schema::builder() - .with_fields(fields.clone()) - .build() - .unwrap(); - let avro_schema = schema_to_avro_schema("test", &schema).unwrap(); - let struct_type = Type::Struct(StructType::new(fields)); - let struct_literal = - Literal::Struct(Struct::from_iter(vec![Some(expected_literal.clone())])); - - let mut writer = apache_avro::Writer::new(&avro_schema, Vec::new()); - let raw_literal = RawLiteral::try_from(struct_literal.clone(), &struct_type).unwrap(); - writer.append_ser(raw_literal).unwrap(); - let encoded = writer.into_inner().unwrap(); - - let reader = apache_avro::Reader::new(&*encoded).unwrap(); - for record in reader { - let result = apache_avro::from_value::(&record.unwrap()).unwrap(); - let desered_literal = result.try_into(&struct_type).unwrap().unwrap(); - match (&desered_literal, &struct_literal) { - (Literal::Struct(desered), Literal::Struct(expected)) => { - match (&desered.fields[0], &expected.fields[0]) { - (Literal::Map(desered), Literal::Map(expected)) => { - assert!(desered.has_same_content(expected)) - } - _ => { - unreachable!() - } - } - } - _ => { - panic!("unexpected literal type"); - } - } - } - } - - #[test] - fn avro_convert_test_map() { - check_convert_with_avro_map( - Literal::Map(Map::from([ - ( - Literal::Primitive(PrimitiveLiteral::Int(1)), - Some(Literal::Primitive(PrimitiveLiteral::Long(1))), - ), - ( - Literal::Primitive(PrimitiveLiteral::Int(2)), - Some(Literal::Primitive(PrimitiveLiteral::Long(2))), - ), - (Literal::Primitive(PrimitiveLiteral::Int(3)), None), - ])), - &Type::Map(MapType { - key_field: NestedField::map_key_element(0, Type::Primitive(PrimitiveType::Int)) - .into(), - value_field: NestedField::map_value_element( - 1, - Type::Primitive(PrimitiveType::Long), - false, - ) - .into(), - }), - ); - - check_convert_with_avro_map( - Literal::Map(Map::from([ - ( - Literal::Primitive(PrimitiveLiteral::Int(1)), - Some(Literal::Primitive(PrimitiveLiteral::Long(1))), - ), - ( - Literal::Primitive(PrimitiveLiteral::Int(2)), - Some(Literal::Primitive(PrimitiveLiteral::Long(2))), - ), - ( - Literal::Primitive(PrimitiveLiteral::Int(3)), - Some(Literal::Primitive(PrimitiveLiteral::Long(3))), - ), - ])), - &Type::Map(MapType { - key_field: NestedField::map_key_element(0, Type::Primitive(PrimitiveType::Int)) - .into(), - value_field: NestedField::map_value_element( - 1, - Type::Primitive(PrimitiveType::Long), - true, - ) - .into(), - }), - ); - } - - #[test] - fn avro_convert_test_string_map() { - check_convert_with_avro_map( - Literal::Map(Map::from([ - ( - Literal::Primitive(PrimitiveLiteral::String("a".to_string())), - Some(Literal::Primitive(PrimitiveLiteral::Int(1))), - ), - ( - Literal::Primitive(PrimitiveLiteral::String("b".to_string())), - Some(Literal::Primitive(PrimitiveLiteral::Int(2))), - ), - ( - Literal::Primitive(PrimitiveLiteral::String("c".to_string())), - None, - ), - ])), - &Type::Map(MapType { - key_field: NestedField::map_key_element(0, Type::Primitive(PrimitiveType::String)) - .into(), - value_field: NestedField::map_value_element( - 1, - Type::Primitive(PrimitiveType::Int), - false, - ) - .into(), - }), - ); - - check_convert_with_avro_map( - Literal::Map(Map::from([ - ( - Literal::Primitive(PrimitiveLiteral::String("a".to_string())), - Some(Literal::Primitive(PrimitiveLiteral::Int(1))), - ), - ( - Literal::Primitive(PrimitiveLiteral::String("b".to_string())), - Some(Literal::Primitive(PrimitiveLiteral::Int(2))), - ), - ( - Literal::Primitive(PrimitiveLiteral::String("c".to_string())), - Some(Literal::Primitive(PrimitiveLiteral::Int(3))), - ), - ])), - &Type::Map(MapType { - key_field: NestedField::map_key_element(0, Type::Primitive(PrimitiveType::String)) - .into(), - value_field: NestedField::map_value_element( - 1, - Type::Primitive(PrimitiveType::Int), - true, - ) - .into(), - }), - ); - } - - #[test] - fn avro_convert_test_record() { - check_convert_with_avro( - Literal::Struct(Struct::from_iter(vec![ - Some(Literal::Primitive(PrimitiveLiteral::Int(1))), - Some(Literal::Primitive(PrimitiveLiteral::String( - "bar".to_string(), - ))), - None, - ])), - &Type::Struct(StructType::new(vec![ - NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), - NestedField::optional(2, "name", Type::Primitive(PrimitiveType::String)).into(), - NestedField::optional(3, "address", Type::Primitive(PrimitiveType::String)).into(), - ])), - ); - } - - // # TODO:https://github.com/apache/iceberg-rust/issues/86 - // rust avro don't support deserialize any bytes representation now: - // - binary - // - decimal - #[test] - fn avro_convert_test_binary_ser() { - let literal = Literal::Primitive(PrimitiveLiteral::Binary(vec![1, 2, 3, 4, 5])); - let ty = Type::Primitive(PrimitiveType::Binary); - let expect_value = Value::Bytes(vec![1, 2, 3, 4, 5]); - check_serialize_avro(literal, &ty, expect_value); - } - - #[test] - fn avro_convert_test_decimal_ser() { - let literal = Literal::decimal(12345); - let ty = Type::Primitive(PrimitiveType::Decimal { - precision: 9, - scale: 8, - }); - let expect_value = Value::Decimal(apache_avro::Decimal::from(12345_i128.to_be_bytes())); - check_serialize_avro(literal, &ty, expect_value); - } - - // # TODO:https://github.com/apache/iceberg-rust/issues/86 - // rust avro can't support to convert any byte-like type to fixed in avro now. - // - uuid ser/de - // - fixed ser/de - - #[test] - fn test_parse_timestamp() { - let value = Datum::timestamp_from_str("2021-08-01T01:09:00.0899").unwrap(); - assert_eq!(&format!("{value}"), "2021-08-01 01:09:00.089900"); - - let value = Datum::timestamp_from_str("2021-08-01T01:09:00.0899+0800"); - assert!(value.is_err(), "Parse timestamp with timezone should fail!"); - - let value = Datum::timestamp_from_str("dfa"); - assert!( - value.is_err(), - "Parse timestamp with invalid input should fail!" - ); - } - - #[test] - fn test_parse_timestamptz() { - let value = Datum::timestamptz_from_str("2021-08-01T09:09:00.0899+0800").unwrap(); - assert_eq!(&format!("{value}"), "2021-08-01 01:09:00.089900 UTC"); - - let value = Datum::timestamptz_from_str("2021-08-01T01:09:00.0899"); - assert!( - value.is_err(), - "Parse timestamptz without timezone should fail!" - ); - - let value = Datum::timestamptz_from_str("dfa"); - assert!( - value.is_err(), - "Parse timestamptz with invalid input should fail!" - ); - } - - #[test] - fn test_datum_ser_deser() { - let test_fn = |datum: Datum| { - let json = serde_json::to_value(&datum).unwrap(); - let desered_datum: Datum = serde_json::from_value(json).unwrap(); - assert_eq!(datum, desered_datum); - }; - let datum = Datum::int(1); - test_fn(datum); - let datum = Datum::long(1); - test_fn(datum); - - let datum = Datum::float(1.0); - test_fn(datum); - let datum = Datum::float(0_f32); - test_fn(datum); - let datum = Datum::float(-0_f32); - test_fn(datum); - let datum = Datum::float(f32::MAX); - test_fn(datum); - let datum = Datum::float(f32::MIN); - test_fn(datum); - - // serde_json can't serialize f32::INFINITY, f32::NEG_INFINITY, f32::NAN - let datum = Datum::float(f32::INFINITY); - let json = serde_json::to_string(&datum).unwrap(); - assert!(serde_json::from_str::(&json).is_err()); - let datum = Datum::float(f32::NEG_INFINITY); - let json = serde_json::to_string(&datum).unwrap(); - assert!(serde_json::from_str::(&json).is_err()); - let datum = Datum::float(f32::NAN); - let json = serde_json::to_string(&datum).unwrap(); - assert!(serde_json::from_str::(&json).is_err()); - - let datum = Datum::double(1.0); - test_fn(datum); - let datum = Datum::double(f64::MAX); - test_fn(datum); - let datum = Datum::double(f64::MIN); - test_fn(datum); - - // serde_json can't serialize f32::INFINITY, f32::NEG_INFINITY, f32::NAN - let datum = Datum::double(f64::INFINITY); - let json = serde_json::to_string(&datum).unwrap(); - assert!(serde_json::from_str::(&json).is_err()); - let datum = Datum::double(f64::NEG_INFINITY); - let json = serde_json::to_string(&datum).unwrap(); - assert!(serde_json::from_str::(&json).is_err()); - let datum = Datum::double(f64::NAN); - let json = serde_json::to_string(&datum).unwrap(); - assert!(serde_json::from_str::(&json).is_err()); - - let datum = Datum::string("iceberg"); - test_fn(datum); - let datum = Datum::bool(true); - test_fn(datum); - let datum = Datum::date(17486); - test_fn(datum); - let datum = Datum::time_from_hms_micro(22, 15, 33, 111).unwrap(); - test_fn(datum); - let datum = Datum::timestamp_micros(1510871468123456); - test_fn(datum); - let datum = Datum::timestamptz_micros(1510871468123456); - test_fn(datum); - let datum = Datum::uuid(Uuid::parse_str("f79c3e09-677c-4bbd-a479-3f349cb785e7").unwrap()); - test_fn(datum); - let datum = Datum::decimal(1420).unwrap(); - test_fn(datum); - let datum = Datum::binary(vec![1, 2, 3, 4, 5]); - test_fn(datum); - let datum = Datum::fixed(vec![1, 2, 3, 4, 5]); - test_fn(datum); - } -} + }; + let invalid_err_with_reason = |v: &str, reason: &str| { + Error::new( + ErrorKind::DataInvalid, + format!( + "Unable to convert raw literal ({}) fail convert to type {} for: {}", + v, ty, reason + ), + ) + }; + match self { + RawLiteralEnum::Null => Ok(None), + RawLiteralEnum::Boolean(v) => Ok(Some(Literal::bool(v))), + RawLiteralEnum::Int(v) => match ty { + Type::Primitive(PrimitiveType::Int) => Ok(Some(Literal::int(v))), + Type::Primitive(PrimitiveType::Long) => Ok(Some(Literal::long(i64::from(v)))), + Type::Primitive(PrimitiveType::Date) => Ok(Some(Literal::date(v))), + _ => Err(invalid_err("int")), + }, + RawLiteralEnum::Long(v) => match ty { + Type::Primitive(PrimitiveType::Int) => Ok(Some(Literal::int( + i32::try_from(v).map_err(|_| invalid_err("long"))?, + ))), + Type::Primitive(PrimitiveType::Date) => Ok(Some(Literal::date( + i32::try_from(v).map_err(|_| invalid_err("long"))?, + ))), + Type::Primitive(PrimitiveType::Long) => Ok(Some(Literal::long(v))), + Type::Primitive(PrimitiveType::Time) => Ok(Some(Literal::time(v))), + Type::Primitive(PrimitiveType::Timestamp) => Ok(Some(Literal::timestamp(v))), + Type::Primitive(PrimitiveType::Timestamptz) => { + Ok(Some(Literal::timestamptz(v))) + } + _ => Err(invalid_err("long")), + }, + RawLiteralEnum::Float(v) => match ty { + Type::Primitive(PrimitiveType::Float) => Ok(Some(Literal::float(v))), + Type::Primitive(PrimitiveType::Double) => { + Ok(Some(Literal::double(f64::from(v)))) + } + _ => Err(invalid_err("float")), + }, + RawLiteralEnum::Double(v) => match ty { + Type::Primitive(PrimitiveType::Float) => { + let v_32 = v as f32; + if v_32.is_finite() { + let v_64 = f64::from(v_32); + if (v_64 - v).abs() > f32::EPSILON as f64 { + // there is a precision loss + return Err(invalid_err("double")); + } + } + Ok(Some(Literal::float(v_32))) + } + Type::Primitive(PrimitiveType::Double) => Ok(Some(Literal::double(v))), + _ => Err(invalid_err("double")), + }, + RawLiteralEnum::String(v) => match ty { + Type::Primitive(PrimitiveType::String) => Ok(Some(Literal::string(v))), + _ => Err(invalid_err("string")), + }, + // # TODO:https://github.com/apache/iceberg-rust/issues/86 + // rust avro don't support deserialize any bytes representation now. + RawLiteralEnum::Bytes(_) => Err(invalid_err_with_reason( + "bytes", + "todo: rust avro doesn't support deserialize any bytes representation now", + )), + RawLiteralEnum::List(v) => { + match ty { + Type::List(ty) => Ok(Some(Literal::List( + v.list + .into_iter() + .map(|v| { + if let Some(v) = v { + v.try_into(&ty.element_field.field_type) + } else { + Ok(None) + } + }) + .collect::>()?, + ))), + Type::Map(map_ty) => { + let key_ty = map_ty.key_field.field_type.as_ref(); + let value_ty = map_ty.value_field.field_type.as_ref(); + let mut map = Map::new(); + for k_v in v.list { + let k_v = k_v.ok_or_else(|| invalid_err_with_reason("list","In deserialize, None will be represented as Some(RawLiteral::Null), all element in list must be valid"))?; + if let RawLiteralEnum::Record(Record { + required, + optional: _, + }) = k_v + { + if required.len() != 2 { + return Err(invalid_err_with_reason("list","Record must contains two element(key and value) of array")); + } + let mut key = None; + let mut value = None; + required.into_iter().for_each(|(k, v)| { + if k == MAP_KEY_FIELD_NAME { + key = Some(v); + } else if k == MAP_VALUE_FIELD_NAME { + value = Some(v); + } + }); + match (key, value) { + (Some(k), Some(v)) => { + let key = k.try_into(key_ty)?.ok_or_else(|| { + invalid_err_with_reason( + "list", + "Key element in Map must be valid", + ) + })?; + let value = v.try_into(value_ty)?; + if map_ty.value_field.required && value.is_none() { + return Err(invalid_err_with_reason( + "list", + "Value element is required in this Map", + )); + } + map.insert(key, value); + } + _ => return Err(invalid_err_with_reason( + "list", + "The elements of record in list are not key and value", + )), + } + } else { + return Err(invalid_err_with_reason( + "list", + "Map should represented as record array.", + )); + } + } + Ok(Some(Literal::Map(map))) + } + Type::Primitive(PrimitiveType::Uuid) => { + if v.list.len() != 16 { + return Err(invalid_err_with_reason( + "list", + "The length of list should be 16", + )); + } + let mut bytes = [0u8; 16]; + for (i, v) in v.list.iter().enumerate() { + if let Some(RawLiteralEnum::Long(v)) = v { + bytes[i] = *v as u8; + } else { + return Err(invalid_err_with_reason( + "list", + "The element of list should be int", + )); + } + } + Ok(Some(Literal::uuid(uuid::Uuid::from_bytes(bytes)))) + } + Type::Primitive(PrimitiveType::Decimal { + precision: _, + scale: _, + }) => { + if v.list.len() != 16 { + return Err(invalid_err_with_reason( + "list", + "The length of list should be 16", + )); + } + let mut bytes = [0u8; 16]; + for (i, v) in v.list.iter().enumerate() { + if let Some(RawLiteralEnum::Long(v)) = v { + bytes[i] = *v as u8; + } else { + return Err(invalid_err_with_reason( + "list", + "The element of list should be int", + )); + } + } + Ok(Some(Literal::decimal(i128::from_be_bytes(bytes)))) + } + Type::Primitive(PrimitiveType::Binary) => { + let bytes = v + .list + .into_iter() + .map(|v| { + if let Some(RawLiteralEnum::Long(v)) = v { + Ok(v as u8) + } else { + Err(invalid_err_with_reason( + "list", + "The element of list should be int", + )) + } + }) + .collect::, Error>>()?; + Ok(Some(Literal::binary(bytes))) + } + Type::Primitive(PrimitiveType::Fixed(size)) => { + if v.list.len() != *size as usize { + return Err(invalid_err_with_reason( + "list", + "The length of list should be equal to size", + )); + } + let bytes = v + .list + .into_iter() + .map(|v| { + if let Some(RawLiteralEnum::Long(v)) = v { + Ok(v as u8) + } else { + Err(invalid_err_with_reason( + "list", + "The element of list should be int", + )) + } + }) + .collect::, Error>>()?; + Ok(Some(Literal::fixed(bytes))) + } + _ => Err(invalid_err("list")), + } + } + RawLiteralEnum::Record(Record { + required, + optional: _, + }) => match ty { + Type::Struct(struct_ty) => { + let iters: Vec> = required + .into_iter() + .map(|(field_name, value)| { + let field = struct_ty + .field_by_name(field_name.as_str()) + .ok_or_else(|| { + invalid_err_with_reason( + "record", + &format!("field {} is not exist", &field_name), + ) + })?; + let value = value.try_into(&field.field_type)?; + Ok(value) + }) + .collect::>()?; + Ok(Some(Literal::Struct(super::Struct::from_iter(iters)))) + } + Type::Map(map_ty) => { + if *map_ty.key_field.field_type != Type::Primitive(PrimitiveType::String) { + return Err(invalid_err_with_reason( + "record", + "Map key must be string", + )); + } + let mut map = Map::new(); + for (k, v) in required { + let value = v.try_into(&map_ty.value_field.field_type)?; + if map_ty.value_field.required && value.is_none() { + return Err(invalid_err_with_reason( + "record", + "Value element is required in this Map", + )); + } + map.insert(Literal::string(k), value); + } + Ok(Some(Literal::Map(map))) + } + _ => Err(invalid_err("record")), + }, + RawLiteralEnum::StringMap(_) => Err(invalid_err("string map")), + } + } + } + } + + #[cfg(test)] + mod tests { + use apache_avro::{to_value, types::Value}; + + use crate::{ + avro::schema_to_avro_schema, + spec::{ + datatypes::{ListType, MapType, NestedField, StructType}, + Schema, + }, + }; + + use super::*; + + fn check_json_serde(json: &str, expected_literal: Literal, expected_type: &Type) { + let raw_json_value = serde_json::from_str::(json).unwrap(); + let desered_literal = + Literal::try_from_json(raw_json_value.clone(), expected_type).unwrap(); + assert_eq!(desered_literal, Some(expected_literal.clone())); + + let expected_json_value: JsonValue = expected_literal.try_into_json(expected_type).unwrap(); + let sered_json = serde_json::to_string(&expected_json_value).unwrap(); + let parsed_json_value = serde_json::from_str::(&sered_json).unwrap(); + + assert_eq!(parsed_json_value, raw_json_value); + } + + fn check_avro_bytes_serde(input: Vec, expected_literal: Literal, expected_type: &Type) { + let raw_schema = r#""bytes""#; + let schema = apache_avro::Schema::parse_str(raw_schema).unwrap(); + + let bytes = ByteBuf::from(input); + let literal = Literal::try_from_bytes(&bytes, expected_type).unwrap(); + assert_eq!(literal, expected_literal); + + let mut writer = apache_avro::Writer::new(&schema, Vec::new()); + writer.append_ser(ByteBuf::from(literal)).unwrap(); + let encoded = writer.into_inner().unwrap(); + let reader = apache_avro::Reader::with_schema(&schema, &*encoded).unwrap(); + + for record in reader { + let result = apache_avro::from_value::(&record.unwrap()).unwrap(); + let desered_literal = Literal::try_from_bytes(&result, expected_type).unwrap(); + assert_eq!(desered_literal, expected_literal); + } + } + + fn check_convert_with_avro(expected_literal: Literal, expected_type: &Type) { + let fields = vec![NestedField::required(1, "col", expected_type.clone()).into()]; + let schema = Schema::builder() + .with_fields(fields.clone()) + .build() + .unwrap(); + let avro_schema = schema_to_avro_schema("test", &schema).unwrap(); + let struct_type = Type::Struct(StructType::new(fields)); + let struct_literal = + Literal::Struct(Struct::from_iter(vec![Some(expected_literal.clone())])); + + let mut writer = apache_avro::Writer::new(&avro_schema, Vec::new()); + let raw_literal = RawLiteral::try_from(struct_literal.clone(), &struct_type).unwrap(); + writer.append_ser(raw_literal).unwrap(); + let encoded = writer.into_inner().unwrap(); + + let reader = apache_avro::Reader::new(&*encoded).unwrap(); + for record in reader { + let result = apache_avro::from_value::(&record.unwrap()).unwrap(); + let desered_literal = result.try_into(&struct_type).unwrap().unwrap(); + assert_eq!(desered_literal, struct_literal); + } + } + + fn check_serialize_avro(literal: Literal, ty: &Type, expect_value: Value) { + let expect_value = Value::Record(vec![("col".to_string(), expect_value)]); + + let fields = vec![NestedField::required(1, "col", ty.clone()).into()]; + let schema = Schema::builder() + .with_fields(fields.clone()) + .build() + .unwrap(); + let avro_schema = schema_to_avro_schema("test", &schema).unwrap(); + let struct_type = Type::Struct(StructType::new(fields)); + let struct_literal = Literal::Struct(Struct::from_iter(vec![Some(literal.clone())])); + let mut writer = apache_avro::Writer::new(&avro_schema, Vec::new()); + let raw_literal = RawLiteral::try_from(struct_literal.clone(), &struct_type).unwrap(); + let value = to_value(raw_literal) + .unwrap() + .resolve(&avro_schema) + .unwrap(); + writer.append_value_ref(&value).unwrap(); + let encoded = writer.into_inner().unwrap(); + + let reader = apache_avro::Reader::new(&*encoded).unwrap(); + for record in reader { + assert_eq!(record.unwrap(), expect_value); + } + } + + #[test] + fn json_boolean() { + let record = r#"true"#; + + check_json_serde( + record, + Literal::Primitive(PrimitiveLiteral::Boolean(true)), + &Type::Primitive(PrimitiveType::Boolean), + ); + } + + #[test] + fn json_int() { + let record = r#"32"#; + + check_json_serde( + record, + Literal::Primitive(PrimitiveLiteral::Int(32)), + &Type::Primitive(PrimitiveType::Int), + ); + } + + #[test] + fn json_long() { + let record = r#"32"#; + + check_json_serde( + record, + Literal::Primitive(PrimitiveLiteral::Long(32)), + &Type::Primitive(PrimitiveType::Long), + ); + } + + #[test] + fn json_float() { + let record = r#"1.0"#; + + check_json_serde( + record, + Literal::Primitive(PrimitiveLiteral::Float(OrderedFloat(1.0))), + &Type::Primitive(PrimitiveType::Float), + ); + } + + #[test] + fn json_double() { + let record = r#"1.0"#; + + check_json_serde( + record, + Literal::Primitive(PrimitiveLiteral::Double(OrderedFloat(1.0))), + &Type::Primitive(PrimitiveType::Double), + ); + } + + #[test] + fn json_date() { + let record = r#""2017-11-16""#; + + check_json_serde( + record, + Literal::Primitive(PrimitiveLiteral::Date(17486)), + &Type::Primitive(PrimitiveType::Date), + ); + } + + #[test] + fn json_time() { + let record = r#""22:31:08.123456""#; + + check_json_serde( + record, + Literal::Primitive(PrimitiveLiteral::Time(81068123456)), + &Type::Primitive(PrimitiveType::Time), + ); + } + + #[test] + fn json_timestamp() { + let record = r#""2017-11-16T22:31:08.123456""#; + + check_json_serde( + record, + Literal::Primitive(PrimitiveLiteral::Timestamp(1510871468123456)), + &Type::Primitive(PrimitiveType::Timestamp), + ); + } + + #[test] + fn json_timestamptz() { + let record = r#""2017-11-16T22:31:08.123456+00:00""#; + + check_json_serde( + record, + Literal::Primitive(PrimitiveLiteral::Timestamptz(1510871468123456)), + &Type::Primitive(PrimitiveType::Timestamptz), + ); + } + + #[test] + fn json_string() { + let record = r#""iceberg""#; + + check_json_serde( + record, + Literal::Primitive(PrimitiveLiteral::String("iceberg".to_string())), + &Type::Primitive(PrimitiveType::String), + ); + } + + #[test] + fn json_uuid() { + let record = r#""f79c3e09-677c-4bbd-a479-3f349cb785e7""#; + + check_json_serde( + record, + Literal::Primitive(PrimitiveLiteral::UUID( + Uuid::parse_str("f79c3e09-677c-4bbd-a479-3f349cb785e7").unwrap(), + )), + &Type::Primitive(PrimitiveType::Uuid), + ); + } + + #[test] + fn json_decimal() { + let record = r#""14.20""#; + + check_json_serde( + record, + Literal::Primitive(PrimitiveLiteral::Decimal(1420)), + &Type::decimal(28, 2).unwrap(), + ); + } + + #[test] + fn json_struct() { + let record = r#"{"1": 1, "2": "bar", "3": null}"#; + + check_json_serde( + record, + Literal::Struct(Struct::from_iter(vec![ + Some(Literal::Primitive(PrimitiveLiteral::Int(1))), + Some(Literal::Primitive(PrimitiveLiteral::String( + "bar".to_string(), + ))), + None, + ])), + &Type::Struct(StructType::new(vec![ + NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::optional(2, "name", Type::Primitive(PrimitiveType::String)).into(), + NestedField::optional(3, "address", Type::Primitive(PrimitiveType::String)).into(), + ])), + ); + } + + #[test] + fn json_list() { + let record = r#"[1, 2, 3, null]"#; + + check_json_serde( + record, + Literal::List(vec![ + Some(Literal::Primitive(PrimitiveLiteral::Int(1))), + Some(Literal::Primitive(PrimitiveLiteral::Int(2))), + Some(Literal::Primitive(PrimitiveLiteral::Int(3))), + None, + ]), + &Type::List(ListType { + element_field: NestedField::list_element( + 0, + Type::Primitive(PrimitiveType::Int), + true, + ) + .into(), + }), + ); + } + + #[test] + fn json_map() { + let record = r#"{ "keys": ["a", "b", "c"], "values": [1, 2, null] }"#; + + check_json_serde( + record, + Literal::Map(Map::from([ + ( + Literal::Primitive(PrimitiveLiteral::String("a".to_string())), + Some(Literal::Primitive(PrimitiveLiteral::Int(1))), + ), + ( + Literal::Primitive(PrimitiveLiteral::String("b".to_string())), + Some(Literal::Primitive(PrimitiveLiteral::Int(2))), + ), + ( + Literal::Primitive(PrimitiveLiteral::String("c".to_string())), + None, + ), + ])), + &Type::Map(MapType { + key_field: NestedField::map_key_element(0, Type::Primitive(PrimitiveType::String)) + .into(), + value_field: NestedField::map_value_element( + 1, + Type::Primitive(PrimitiveType::Int), + true, + ) + .into(), + }), + ); + } + + #[test] + fn avro_bytes_boolean() { + let bytes = vec![1u8]; + + check_avro_bytes_serde( + bytes, + Literal::Primitive(PrimitiveLiteral::Boolean(true)), + &Type::Primitive(PrimitiveType::Boolean), + ); + } + + #[test] + fn avro_bytes_int() { + let bytes = vec![32u8, 0u8, 0u8, 0u8]; + + check_avro_bytes_serde( + bytes, + Literal::Primitive(PrimitiveLiteral::Int(32)), + &Type::Primitive(PrimitiveType::Int), + ); + } + + #[test] + fn avro_bytes_long() { + let bytes = vec![32u8, 0u8, 0u8, 0u8, 0u8, 0u8, 0u8, 0u8]; + + check_avro_bytes_serde( + bytes, + Literal::Primitive(PrimitiveLiteral::Long(32)), + &Type::Primitive(PrimitiveType::Long), + ); + } + + #[test] + fn avro_bytes_float() { + let bytes = vec![0u8, 0u8, 128u8, 63u8]; + + check_avro_bytes_serde( + bytes, + Literal::Primitive(PrimitiveLiteral::Float(OrderedFloat(1.0))), + &Type::Primitive(PrimitiveType::Float), + ); + } + + #[test] + fn avro_bytes_double() { + let bytes = vec![0u8, 0u8, 0u8, 0u8, 0u8, 0u8, 240u8, 63u8]; + + check_avro_bytes_serde( + bytes, + Literal::Primitive(PrimitiveLiteral::Double(OrderedFloat(1.0))), + &Type::Primitive(PrimitiveType::Double), + ); + } + + #[test] + fn avro_bytes_string() { + let bytes = vec![105u8, 99u8, 101u8, 98u8, 101u8, 114u8, 103u8]; + + check_avro_bytes_serde( + bytes, + Literal::Primitive(PrimitiveLiteral::String("iceberg".to_string())), + &Type::Primitive(PrimitiveType::String), + ); + } + + #[test] + fn avro_convert_test_int() { + check_convert_with_avro( + Literal::Primitive(PrimitiveLiteral::Int(32)), + &Type::Primitive(PrimitiveType::Int), + ); + } + + #[test] + fn avro_convert_test_long() { + check_convert_with_avro( + Literal::Primitive(PrimitiveLiteral::Long(32)), + &Type::Primitive(PrimitiveType::Long), + ); + } + + #[test] + fn avro_convert_test_float() { + check_convert_with_avro( + Literal::Primitive(PrimitiveLiteral::Float(OrderedFloat(1.0))), + &Type::Primitive(PrimitiveType::Float), + ); + } + + #[test] + fn avro_convert_test_double() { + check_convert_with_avro( + Literal::Primitive(PrimitiveLiteral::Double(OrderedFloat(1.0))), + &Type::Primitive(PrimitiveType::Double), + ); + } + + #[test] + fn avro_convert_test_string() { + check_convert_with_avro( + Literal::Primitive(PrimitiveLiteral::String("iceberg".to_string())), + &Type::Primitive(PrimitiveType::String), + ); + } + + #[test] + fn avro_convert_test_date() { + check_convert_with_avro( + Literal::Primitive(PrimitiveLiteral::Date(17486)), + &Type::Primitive(PrimitiveType::Date), + ); + } + + #[test] + fn avro_convert_test_time() { + check_convert_with_avro( + Literal::Primitive(PrimitiveLiteral::Time(81068123456)), + &Type::Primitive(PrimitiveType::Time), + ); + } + + #[test] + fn avro_convert_test_timestamp() { + check_convert_with_avro( + Literal::Primitive(PrimitiveLiteral::Timestamp(1510871468123456)), + &Type::Primitive(PrimitiveType::Timestamp), + ); + } + + #[test] + fn avro_convert_test_timestamptz() { + check_convert_with_avro( + Literal::Primitive(PrimitiveLiteral::Timestamptz(1510871468123456)), + &Type::Primitive(PrimitiveType::Timestamptz), + ); + } + + #[test] + fn avro_convert_test_list() { + check_convert_with_avro( + Literal::List(vec![ + Some(Literal::Primitive(PrimitiveLiteral::Int(1))), + Some(Literal::Primitive(PrimitiveLiteral::Int(2))), + Some(Literal::Primitive(PrimitiveLiteral::Int(3))), + None, + ]), + &Type::List(ListType { + element_field: NestedField::list_element( + 0, + Type::Primitive(PrimitiveType::Int), + false, + ) + .into(), + }), + ); + + check_convert_with_avro( + Literal::List(vec![ + Some(Literal::Primitive(PrimitiveLiteral::Int(1))), + Some(Literal::Primitive(PrimitiveLiteral::Int(2))), + Some(Literal::Primitive(PrimitiveLiteral::Int(3))), + ]), + &Type::List(ListType { + element_field: NestedField::list_element( + 0, + Type::Primitive(PrimitiveType::Int), + true, + ) + .into(), + }), + ); + } + + fn check_convert_with_avro_map(expected_literal: Literal, expected_type: &Type) { + let fields = vec![NestedField::required(1, "col", expected_type.clone()).into()]; + let schema = Schema::builder() + .with_fields(fields.clone()) + .build() + .unwrap(); + let avro_schema = schema_to_avro_schema("test", &schema).unwrap(); + let struct_type = Type::Struct(StructType::new(fields)); + let struct_literal = + Literal::Struct(Struct::from_iter(vec![Some(expected_literal.clone())])); + + let mut writer = apache_avro::Writer::new(&avro_schema, Vec::new()); + let raw_literal = RawLiteral::try_from(struct_literal.clone(), &struct_type).unwrap(); + writer.append_ser(raw_literal).unwrap(); + let encoded = writer.into_inner().unwrap(); + + let reader = apache_avro::Reader::new(&*encoded).unwrap(); + for record in reader { + let result = apache_avro::from_value::(&record.unwrap()).unwrap(); + let desered_literal = result.try_into(&struct_type).unwrap().unwrap(); + match (&desered_literal, &struct_literal) { + (Literal::Struct(desered), Literal::Struct(expected)) => { + match (&desered.fields[0], &expected.fields[0]) { + (Literal::Map(desered), Literal::Map(expected)) => { + assert!(desered.has_same_content(expected)) + } + _ => { + unreachable!() + } + } + } + _ => { + panic!("unexpected literal type"); + } + } + } + } + + #[test] + fn avro_convert_test_map() { + check_convert_with_avro_map( + Literal::Map(Map::from([ + ( + Literal::Primitive(PrimitiveLiteral::Int(1)), + Some(Literal::Primitive(PrimitiveLiteral::Long(1))), + ), + ( + Literal::Primitive(PrimitiveLiteral::Int(2)), + Some(Literal::Primitive(PrimitiveLiteral::Long(2))), + ), + (Literal::Primitive(PrimitiveLiteral::Int(3)), None), + ])), + &Type::Map(MapType { + key_field: NestedField::map_key_element(0, Type::Primitive(PrimitiveType::Int)) + .into(), + value_field: NestedField::map_value_element( + 1, + Type::Primitive(PrimitiveType::Long), + false, + ) + .into(), + }), + ); + + check_convert_with_avro_map( + Literal::Map(Map::from([ + ( + Literal::Primitive(PrimitiveLiteral::Int(1)), + Some(Literal::Primitive(PrimitiveLiteral::Long(1))), + ), + ( + Literal::Primitive(PrimitiveLiteral::Int(2)), + Some(Literal::Primitive(PrimitiveLiteral::Long(2))), + ), + ( + Literal::Primitive(PrimitiveLiteral::Int(3)), + Some(Literal::Primitive(PrimitiveLiteral::Long(3))), + ), + ])), + &Type::Map(MapType { + key_field: NestedField::map_key_element(0, Type::Primitive(PrimitiveType::Int)) + .into(), + value_field: NestedField::map_value_element( + 1, + Type::Primitive(PrimitiveType::Long), + true, + ) + .into(), + }), + ); + } + + #[test] + fn avro_convert_test_string_map() { + check_convert_with_avro_map( + Literal::Map(Map::from([ + ( + Literal::Primitive(PrimitiveLiteral::String("a".to_string())), + Some(Literal::Primitive(PrimitiveLiteral::Int(1))), + ), + ( + Literal::Primitive(PrimitiveLiteral::String("b".to_string())), + Some(Literal::Primitive(PrimitiveLiteral::Int(2))), + ), + ( + Literal::Primitive(PrimitiveLiteral::String("c".to_string())), + None, + ), + ])), + &Type::Map(MapType { + key_field: NestedField::map_key_element(0, Type::Primitive(PrimitiveType::String)) + .into(), + value_field: NestedField::map_value_element( + 1, + Type::Primitive(PrimitiveType::Int), + false, + ) + .into(), + }), + ); + + check_convert_with_avro_map( + Literal::Map(Map::from([ + ( + Literal::Primitive(PrimitiveLiteral::String("a".to_string())), + Some(Literal::Primitive(PrimitiveLiteral::Int(1))), + ), + ( + Literal::Primitive(PrimitiveLiteral::String("b".to_string())), + Some(Literal::Primitive(PrimitiveLiteral::Int(2))), + ), + ( + Literal::Primitive(PrimitiveLiteral::String("c".to_string())), + Some(Literal::Primitive(PrimitiveLiteral::Int(3))), + ), + ])), + &Type::Map(MapType { + key_field: NestedField::map_key_element(0, Type::Primitive(PrimitiveType::String)) + .into(), + value_field: NestedField::map_value_element( + 1, + Type::Primitive(PrimitiveType::Int), + true, + ) + .into(), + }), + ); + } + + #[test] + fn avro_convert_test_record() { + check_convert_with_avro( + Literal::Struct(Struct::from_iter(vec![ + Some(Literal::Primitive(PrimitiveLiteral::Int(1))), + Some(Literal::Primitive(PrimitiveLiteral::String( + "bar".to_string(), + ))), + None, + ])), + &Type::Struct(StructType::new(vec![ + NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::optional(2, "name", Type::Primitive(PrimitiveType::String)).into(), + NestedField::optional(3, "address", Type::Primitive(PrimitiveType::String)).into(), + ])), + ); + } + + // # TODO:https://github.com/apache/iceberg-rust/issues/86 + // rust avro don't support deserialize any bytes representation now: + // - binary + // - decimal + #[test] + fn avro_convert_test_binary_ser() { + let literal = Literal::Primitive(PrimitiveLiteral::Binary(vec![1, 2, 3, 4, 5])); + let ty = Type::Primitive(PrimitiveType::Binary); + let expect_value = Value::Bytes(vec![1, 2, 3, 4, 5]); + check_serialize_avro(literal, &ty, expect_value); + } + + #[test] + fn avro_convert_test_decimal_ser() { + let literal = Literal::decimal(12345); + let ty = Type::Primitive(PrimitiveType::Decimal { + precision: 9, + scale: 8, + }); + let expect_value = Value::Decimal(apache_avro::Decimal::from(12345_i128.to_be_bytes())); + check_serialize_avro(literal, &ty, expect_value); + } + + // # TODO:https://github.com/apache/iceberg-rust/issues/86 + // rust avro can't support to convert any byte-like type to fixed in avro now. + // - uuid ser/de + // - fixed ser/de + + #[test] + fn test_parse_timestamp() { + let value = Datum::timestamp_from_str("2021-08-01T01:09:00.0899").unwrap(); + assert_eq!(&format!("{value}"), "2021-08-01 01:09:00.089900"); + + let value = Datum::timestamp_from_str("2021-08-01T01:09:00.0899+0800"); + assert!(value.is_err(), "Parse timestamp with timezone should fail!"); + + let value = Datum::timestamp_from_str("dfa"); + assert!( + value.is_err(), + "Parse timestamp with invalid input should fail!" + ); + } + + #[test] + fn test_parse_timestamptz() { + let value = Datum::timestamptz_from_str("2021-08-01T09:09:00.0899+0800").unwrap(); + assert_eq!(&format!("{value}"), "2021-08-01 01:09:00.089900 UTC"); + + let value = Datum::timestamptz_from_str("2021-08-01T01:09:00.0899"); + assert!( + value.is_err(), + "Parse timestamptz without timezone should fail!" + ); + + let value = Datum::timestamptz_from_str("dfa"); + assert!( + value.is_err(), + "Parse timestamptz with invalid input should fail!" + ); + } + + #[test] + fn test_datum_ser_deser() { + let test_fn = |datum: Datum| { + let json = serde_json::to_value(&datum).unwrap(); + let desered_datum: Datum = serde_json::from_value(json).unwrap(); + assert_eq!(datum, desered_datum); + }; + let datum = Datum::int(1); + test_fn(datum); + let datum = Datum::long(1); + test_fn(datum); + + let datum = Datum::float(1.0); + test_fn(datum); + let datum = Datum::float(0_f32); + test_fn(datum); + let datum = Datum::float(-0_f32); + test_fn(datum); + let datum = Datum::float(f32::MAX); + test_fn(datum); + let datum = Datum::float(f32::MIN); + test_fn(datum); + + // serde_json can't serialize f32::INFINITY, f32::NEG_INFINITY, f32::NAN + let datum = Datum::float(f32::INFINITY); + let json = serde_json::to_string(&datum).unwrap(); + assert!(serde_json::from_str::(&json).is_err()); + let datum = Datum::float(f32::NEG_INFINITY); + let json = serde_json::to_string(&datum).unwrap(); + assert!(serde_json::from_str::(&json).is_err()); + let datum = Datum::float(f32::NAN); + let json = serde_json::to_string(&datum).unwrap(); + assert!(serde_json::from_str::(&json).is_err()); + + let datum = Datum::double(1.0); + test_fn(datum); + let datum = Datum::double(f64::MAX); + test_fn(datum); + let datum = Datum::double(f64::MIN); + test_fn(datum); + + // serde_json can't serialize f32::INFINITY, f32::NEG_INFINITY, f32::NAN + let datum = Datum::double(f64::INFINITY); + let json = serde_json::to_string(&datum).unwrap(); + assert!(serde_json::from_str::(&json).is_err()); + let datum = Datum::double(f64::NEG_INFINITY); + let json = serde_json::to_string(&datum).unwrap(); + assert!(serde_json::from_str::(&json).is_err()); + let datum = Datum::double(f64::NAN); + let json = serde_json::to_string(&datum).unwrap(); + assert!(serde_json::from_str::(&json).is_err()); + + let datum = Datum::string("iceberg"); + test_fn(datum); + let datum = Datum::bool(true); + test_fn(datum); + let datum = Datum::date(17486); + test_fn(datum); + let datum = Datum::time_from_hms_micro(22, 15, 33, 111).unwrap(); + test_fn(datum); + let datum = Datum::timestamp_micros(1510871468123456); + test_fn(datum); + let datum = Datum::timestamptz_micros(1510871468123456); + test_fn(datum); + let datum = Datum::uuid(Uuid::parse_str("f79c3e09-677c-4bbd-a479-3f349cb785e7").unwrap()); + test_fn(datum); + let datum = Datum::decimal(1420).unwrap(); + test_fn(datum); + let datum = Datum::binary(vec![1, 2, 3, 4, 5]); + test_fn(datum); + let datum = Datum::fixed(vec![1, 2, 3, 4, 5]); + test_fn(datum); + } + } \ No newline at end of file From 055f9bcf981f85c4891049d35bb14c117e252abb Mon Sep 17 00:00:00 2001 From: Chengxu Bian Date: Mon, 24 Jun 2024 16:57:14 -0400 Subject: [PATCH 11/16] undo --- crates/iceberg/src/spec/values.rs | 6904 ++++++++++++++--------------- 1 file changed, 3452 insertions(+), 3452 deletions(-) diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index 7d56e763c..a90590376 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -19,3455 +19,3455 @@ * Value in iceberg */ - use std::any::Any; - use std::collections::HashMap; - use std::fmt::{Display, Formatter}; - use std::hash::Hash; - use std::ops::Index; - use std::str::FromStr; - - use bitvec::vec::BitVec; - use chrono::{DateTime, NaiveDate, NaiveDateTime, NaiveTime, TimeZone, Utc}; - use ordered_float::OrderedFloat; - use rust_decimal::Decimal; - use serde::de::{self, MapAccess}; - use serde::ser::SerializeStruct; - use serde::{Deserialize, Serialize}; - use serde_bytes::ByteBuf; - use serde_json::{Map as JsonMap, Number, Value as JsonValue}; - use uuid::Uuid; - - pub use _serde::RawLiteral; - - use crate::error::Result; - use crate::spec::values::date::{date_from_naive_date, days_to_date, unix_epoch}; - use crate::spec::values::time::microseconds_to_time; - use crate::spec::values::timestamp::microseconds_to_datetime; - use crate::spec::values::timestamptz::microseconds_to_datetimetz; - use crate::spec::MAX_DECIMAL_PRECISION; - use crate::{ensure_data_valid, Error, ErrorKind}; - - use super::datatypes::{PrimitiveType, Type}; - - /// Maximum value for [`PrimitiveType::Time`] type in microseconds, e.g. 23 hours 59 minutes 59 seconds 999999 microseconds. - const MAX_TIME_VALUE: i64 = 24 * 60 * 60 * 1_000_000i64 - 1; - - /// Values present in iceberg type - #[derive(Clone, Debug, PartialEq, Hash, Eq)] - pub enum PrimitiveLiteral { - /// 0x00 for false, non-zero byte for true - Boolean(bool), - /// Stored as 4-byte little-endian - Int(i32), - /// Stored as 8-byte little-endian - Long(i64), - /// Stored as 4-byte little-endian - Float(OrderedFloat), - /// Stored as 8-byte little-endian - Double(OrderedFloat), - /// Stores days from the 1970-01-01 in an 4-byte little-endian int - Date(i32), - /// Stores microseconds from midnight in an 8-byte little-endian long - Time(i64), - /// Timestamp without timezone - Timestamp(i64), - /// Timestamp with timezone - Timestamptz(i64), - /// UTF-8 bytes (without length) - String(String), - /// 16-byte big-endian value - UUID(Uuid), - /// Binary value - Fixed(Vec), - /// Binary value (without length) - Binary(Vec), - /// Stores unscaled value as big int. According to iceberg spec, the precision must less than 38(`MAX_DECIMAL_PRECISION`) , so i128 is suit here. - Decimal(i128), - } - - impl PrimitiveLiteral { - /// Returns true if the Literal represents a primitive type - /// that can be a NaN, and that it's value is NaN - pub fn is_nan(&self) -> bool { - match self { - PrimitiveLiteral::Double(val) => val.is_nan(), - PrimitiveLiteral::Float(val) => val.is_nan(), - _ => false, - } - } - } - - /// Literal associated with its type. The value and type pair is checked when construction, so the type and value is - /// guaranteed to be correct when used. - /// - /// By default, we decouple the type and value of a literal, so we can use avoid the cost of storing extra type info - /// for each literal. But associate type with literal can be useful in some cases, for example, in unbound expression. - #[derive(Clone, Debug, PartialEq, Hash, Eq)] - pub struct Datum { - r#type: PrimitiveType, - literal: PrimitiveLiteral, - } - - impl Serialize for Datum { - fn serialize( - &self, - serializer: S, - ) -> std::result::Result { - let mut struct_ser = serializer - .serialize_struct("Datum", 2) - .map_err(serde::ser::Error::custom)?; - struct_ser - .serialize_field("type", &self.r#type) - .map_err(serde::ser::Error::custom)?; - struct_ser - .serialize_field( - "literal", - &RawLiteral::try_from( - Literal::Primitive(self.literal.clone()), - &Type::Primitive(self.r#type.clone()), - ) - .map_err(serde::ser::Error::custom)?, - ) - .map_err(serde::ser::Error::custom)?; - struct_ser.end() - } - } - - impl<'de> Deserialize<'de> for Datum { - fn deserialize>( - deserializer: D, - ) -> std::result::Result { - #[derive(Deserialize)] - #[serde(field_identifier, rename_all = "lowercase")] - enum Field { - Type, - Literal, - } - - struct DatumVisitor; - - impl<'de> serde::de::Visitor<'de> for DatumVisitor { - type Value = Datum; - - fn expecting(&self, formatter: &mut std::fmt::Formatter) -> std::fmt::Result { - formatter.write_str("struct Datum") - } - - fn visit_seq(self, mut seq: A) -> std::result::Result - where - A: serde::de::SeqAccess<'de>, - { - let r#type = seq - .next_element::()? - .ok_or_else(|| serde::de::Error::invalid_length(0, &self))?; - let value = seq - .next_element::()? - .ok_or_else(|| serde::de::Error::invalid_length(1, &self))?; - let Literal::Primitive(primitive) = value - .try_into(&Type::Primitive(r#type.clone())) - .map_err(serde::de::Error::custom)? - .ok_or_else(|| serde::de::Error::custom("None value"))? - else { - return Err(serde::de::Error::custom("Invalid value")); - }; - - Ok(Datum::new(r#type, primitive)) - } - - fn visit_map(self, mut map: V) -> std::result::Result - where - V: MapAccess<'de>, - { - let mut raw_primitive: Option = None; - let mut r#type: Option = None; - while let Some(key) = map.next_key()? { - match key { - Field::Type => { - if r#type.is_some() { - return Err(de::Error::duplicate_field("type")); - } - r#type = Some(map.next_value()?); - } - Field::Literal => { - if raw_primitive.is_some() { - return Err(de::Error::duplicate_field("literal")); - } - raw_primitive = Some(map.next_value()?); - } - } - } - let Some(r#type) = r#type else { - return Err(serde::de::Error::missing_field("type")); - }; - let Some(raw_primitive) = raw_primitive else { - return Err(serde::de::Error::missing_field("literal")); - }; - let Literal::Primitive(primitive) = raw_primitive - .try_into(&Type::Primitive(r#type.clone())) - .map_err(serde::de::Error::custom)? - .ok_or_else(|| serde::de::Error::custom("None value"))? - else { - return Err(serde::de::Error::custom("Invalid value")); - }; - Ok(Datum::new(r#type, primitive)) - } - } - const FIELDS: &[&str] = &["type", "literal"]; - deserializer.deserialize_struct("Datum", FIELDS, DatumVisitor) - } - } - - impl PartialOrd for Datum { - fn partial_cmp(&self, other: &Self) -> Option { - match (&self.literal, &other.literal, &self.r#type, &other.r#type) { - // generate the arm with same type and same literal - ( - PrimitiveLiteral::Boolean(val), - PrimitiveLiteral::Boolean(other_val), - PrimitiveType::Boolean, - PrimitiveType::Boolean, - ) => val.partial_cmp(other_val), - ( - PrimitiveLiteral::Int(val), - PrimitiveLiteral::Int(other_val), - PrimitiveType::Int, - PrimitiveType::Int, - ) => val.partial_cmp(other_val), - ( - PrimitiveLiteral::Long(val), - PrimitiveLiteral::Long(other_val), - PrimitiveType::Long, - PrimitiveType::Long, - ) => val.partial_cmp(other_val), - ( - PrimitiveLiteral::Float(val), - PrimitiveLiteral::Float(other_val), - PrimitiveType::Float, - PrimitiveType::Float, - ) => val.partial_cmp(other_val), - ( - PrimitiveLiteral::Double(val), - PrimitiveLiteral::Double(other_val), - PrimitiveType::Double, - PrimitiveType::Double, - ) => val.partial_cmp(other_val), - ( - PrimitiveLiteral::Date(val), - PrimitiveLiteral::Date(other_val), - PrimitiveType::Date, - PrimitiveType::Date, - ) => val.partial_cmp(other_val), - ( - PrimitiveLiteral::Time(val), - PrimitiveLiteral::Time(other_val), - PrimitiveType::Time, - PrimitiveType::Time, - ) => val.partial_cmp(other_val), - ( - PrimitiveLiteral::Timestamp(val), - PrimitiveLiteral::Timestamp(other_val), - PrimitiveType::Timestamp, - PrimitiveType::Timestamp, - ) => val.partial_cmp(other_val), - ( - PrimitiveLiteral::Timestamptz(val), - PrimitiveLiteral::Timestamptz(other_val), - PrimitiveType::Timestamptz, - PrimitiveType::Timestamptz, - ) => val.partial_cmp(other_val), - ( - PrimitiveLiteral::String(val), - PrimitiveLiteral::String(other_val), - PrimitiveType::String, - PrimitiveType::String, - ) => val.partial_cmp(other_val), - ( - PrimitiveLiteral::UUID(val), - PrimitiveLiteral::UUID(other_val), - PrimitiveType::Uuid, - PrimitiveType::Uuid, - ) => val.partial_cmp(other_val), - ( - PrimitiveLiteral::Fixed(val), - PrimitiveLiteral::Fixed(other_val), - PrimitiveType::Fixed(_), - PrimitiveType::Fixed(_), - ) => val.partial_cmp(other_val), - ( - PrimitiveLiteral::Binary(val), - PrimitiveLiteral::Binary(other_val), - PrimitiveType::Binary, - PrimitiveType::Binary, - ) => val.partial_cmp(other_val), - ( - PrimitiveLiteral::Decimal(val), - PrimitiveLiteral::Decimal(other_val), - PrimitiveType::Decimal { - precision: _, - scale, - }, - PrimitiveType::Decimal { - precision: _, - scale: other_scale, - }, - ) => { - let val = Decimal::from_i128_with_scale(*val, *scale); - let other_val = Decimal::from_i128_with_scale(*other_val, *other_scale); - val.partial_cmp(&other_val) - } - _ => None, - } - } - } - - impl Display for Datum { - fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { - match (&self.r#type, &self.literal) { - (_, PrimitiveLiteral::Boolean(val)) => write!(f, "{}", val), - (_, PrimitiveLiteral::Int(val)) => write!(f, "{}", val), - (_, PrimitiveLiteral::Long(val)) => write!(f, "{}", val), - (_, PrimitiveLiteral::Float(val)) => write!(f, "{}", val), - (_, PrimitiveLiteral::Double(val)) => write!(f, "{}", val), - (_, PrimitiveLiteral::Date(val)) => write!(f, "{}", days_to_date(*val)), - (_, PrimitiveLiteral::Time(val)) => write!(f, "{}", microseconds_to_time(*val)), - (_, PrimitiveLiteral::Timestamp(val)) => { - write!(f, "{}", microseconds_to_datetime(*val)) - } - (_, PrimitiveLiteral::Timestamptz(val)) => { - write!(f, "{}", microseconds_to_datetimetz(*val)) - } - (_, PrimitiveLiteral::String(val)) => write!(f, r#""{}""#, val), - (_, PrimitiveLiteral::UUID(val)) => write!(f, "{}", val), - (_, PrimitiveLiteral::Fixed(val)) => display_bytes(val, f), - (_, PrimitiveLiteral::Binary(val)) => display_bytes(val, f), - ( - PrimitiveType::Decimal { - precision: _, - scale, - }, - PrimitiveLiteral::Decimal(val), - ) => { - write!(f, "{}", Decimal::from_i128_with_scale(*val, *scale)) - } - (_, _) => { - unreachable!() - } - } - } - } - - fn display_bytes(bytes: &[u8], f: &mut Formatter<'_>) -> std::fmt::Result { - let mut s = String::with_capacity(bytes.len() * 2); - for b in bytes { - s.push_str(&format!("{:02X}", b)); - } - f.write_str(&s) - } - - impl From for Literal { - fn from(value: Datum) -> Self { - Literal::Primitive(value.literal) - } - } - - impl From for PrimitiveLiteral { - fn from(value: Datum) -> Self { - value.literal - } - } - - impl Datum { - /// Creates a `Datum` from a `PrimitiveType` and a `PrimitiveLiteral` - pub(crate) fn new(r#type: PrimitiveType, literal: PrimitiveLiteral) -> Self { - Datum { r#type, literal } - } - - /// Create iceberg value from bytes - pub fn try_from_bytes(bytes: &[u8], data_type: PrimitiveType) -> Result { - let literal = match data_type { - PrimitiveType::Boolean => { - if bytes.len() == 1 && bytes[0] == 0u8 { - PrimitiveLiteral::Boolean(false) - } else { - PrimitiveLiteral::Boolean(true) - } - } - PrimitiveType::Int => PrimitiveLiteral::Int(i32::from_le_bytes(bytes.try_into()?)), - PrimitiveType::Long => PrimitiveLiteral::Long(i64::from_le_bytes(bytes.try_into()?)), - PrimitiveType::Float => { - PrimitiveLiteral::Float(OrderedFloat(f32::from_le_bytes(bytes.try_into()?))) - } - PrimitiveType::Double => { - PrimitiveLiteral::Double(OrderedFloat(f64::from_le_bytes(bytes.try_into()?))) - } - PrimitiveType::Date => PrimitiveLiteral::Date(i32::from_le_bytes(bytes.try_into()?)), - PrimitiveType::Time => PrimitiveLiteral::Time(i64::from_le_bytes(bytes.try_into()?)), - PrimitiveType::Timestamp => { - PrimitiveLiteral::Timestamp(i64::from_le_bytes(bytes.try_into()?)) - } - PrimitiveType::Timestamptz => { - PrimitiveLiteral::Timestamptz(i64::from_le_bytes(bytes.try_into()?)) - } - PrimitiveType::String => { - PrimitiveLiteral::String(std::str::from_utf8(bytes)?.to_string()) - } - PrimitiveType::Uuid => { - PrimitiveLiteral::UUID(Uuid::from_u128(u128::from_be_bytes(bytes.try_into()?))) - } - PrimitiveType::Fixed(_) => PrimitiveLiteral::Fixed(Vec::from(bytes)), - PrimitiveType::Binary => PrimitiveLiteral::Binary(Vec::from(bytes)), - PrimitiveType::Decimal { - precision: _, - scale: _, - } => todo!(), - }; - Ok(Datum::new(data_type, literal)) - } - - /// Creates a boolean value. - /// - /// Example: - /// ```rust - /// use iceberg::spec::{Literal, PrimitiveLiteral, Datum}; - /// let t = Datum::bool(true); - /// - /// assert_eq!(format!("{}", t), "true".to_string()); - /// assert_eq!(Literal::from(t), Literal::Primitive(PrimitiveLiteral::Boolean(true))); - /// ``` - pub fn bool>(t: T) -> Self { - Self { - r#type: PrimitiveType::Boolean, - literal: PrimitiveLiteral::Boolean(t.into()), - } - } - - /// Creates a boolean value from string. - /// See [Parse bool from str](https://doc.rust-lang.org/stable/std/primitive.bool.html#impl-FromStr-for-bool) for reference. - /// - /// Example: - /// ```rust - /// use iceberg::spec::{Literal, PrimitiveLiteral, Datum}; - /// let t = Datum::bool_from_str("false").unwrap(); - /// - /// assert_eq!(&format!("{}", t), "false"); - /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Boolean(false)), t.into()); - /// ``` - pub fn bool_from_str>(s: S) -> Result { - let v = s.as_ref().parse::().map_err(|e| { - Error::new(ErrorKind::DataInvalid, "Can't parse string to bool.").with_source(e) - })?; - Ok(Self::bool(v)) - } - - /// Creates an 32bit integer. - /// - /// Example: - /// ```rust - /// use iceberg::spec::{Literal, PrimitiveLiteral, Datum}; - /// let t = Datum::int(23i8); - /// - /// assert_eq!(&format!("{}", t), "23"); - /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Int(23)), t.into()); - /// ``` - pub fn int>(t: T) -> Self { - Self { - r#type: PrimitiveType::Int, - literal: PrimitiveLiteral::Int(t.into()), - } - } - - /// Creates an 64bit integer. - /// - /// Example: - /// ```rust - /// use iceberg::spec::{Literal, PrimitiveLiteral, Datum}; - /// let t = Datum::long(24i8); - /// - /// assert_eq!(&format!("{t}"), "24"); - /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Long(24)), t.into()); - /// ``` - pub fn long>(t: T) -> Self { - Self { - r#type: PrimitiveType::Long, - literal: PrimitiveLiteral::Long(t.into()), - } - } - - /// Creates an 32bit floating point number. - /// - /// Example: - /// ```rust - /// use ordered_float::OrderedFloat; - /// use iceberg::spec::{Literal, PrimitiveLiteral, Datum}; - /// let t = Datum::float( 32.1f32 ); - /// - /// assert_eq!(&format!("{t}"), "32.1"); - /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Float(OrderedFloat(32.1))), t.into()); - /// ``` - pub fn float>(t: T) -> Self { - Self { - r#type: PrimitiveType::Float, - literal: PrimitiveLiteral::Float(OrderedFloat(t.into())), - } - } - - /// Creates an 64bit floating point number. - /// - /// Example: - /// ```rust - /// use ordered_float::OrderedFloat; - /// use iceberg::spec::{Literal, PrimitiveLiteral, Datum}; - /// let t = Datum::double( 32.1f64 ); - /// - /// assert_eq!(&format!("{t}"), "32.1"); - /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Double(OrderedFloat(32.1))), t.into()); - /// ``` - pub fn double>(t: T) -> Self { - Self { - r#type: PrimitiveType::Double, - literal: PrimitiveLiteral::Double(OrderedFloat(t.into())), - } - } - - /// Creates date literal from number of days from unix epoch directly. - /// - /// Example: - /// ```rust - /// - /// use iceberg::spec::{Literal, PrimitiveLiteral, Datum}; - /// // 2 days after 1970-01-01 - /// let t = Datum::date(2); - /// - /// assert_eq!(&format!("{t}"), "1970-01-03"); - /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Date(2)), t.into()); - /// ``` - pub fn date(days: i32) -> Self { - Self { - r#type: PrimitiveType::Date, - literal: PrimitiveLiteral::Date(days), - } - } - - /// Creates date literal in `%Y-%m-%d` format, assume in utc timezone. - /// - /// See [`NaiveDate::from_str`]. - /// - /// Example - /// ```rust - /// use iceberg::spec::{Literal, Datum}; - /// let t = Datum::date_from_str("1970-01-05").unwrap(); - /// - /// assert_eq!(&format!("{t}"), "1970-01-05"); - /// assert_eq!(Literal::date(4), t.into()); - /// ``` - pub fn date_from_str>(s: S) -> Result { - let t = s.as_ref().parse::().map_err(|e| { - Error::new( - ErrorKind::DataInvalid, - format!("Can't parse date from string: {}", s.as_ref()), - ) - .with_source(e) - })?; - - Ok(Self::date(date_from_naive_date(t))) - } - - /// Create date literal from calendar date (year, month and day). - /// - /// See [`NaiveDate::from_ymd_opt`]. - /// - /// Example: - /// - ///```rust - /// use iceberg::spec::{Literal, Datum}; - /// let t = Datum::date_from_ymd(1970, 1, 5).unwrap(); - /// - /// assert_eq!(&format!("{t}"), "1970-01-05"); - /// assert_eq!(Literal::date(4), t.into()); - /// ``` - pub fn date_from_ymd(year: i32, month: u32, day: u32) -> Result { - let t = NaiveDate::from_ymd_opt(year, month, day).ok_or_else(|| { - Error::new( - ErrorKind::DataInvalid, - format!("Can't create date from year: {year}, month: {month}, day: {day}"), - ) - })?; - - Ok(Self::date(date_from_naive_date(t))) - } - - /// Creates time literal in microseconds directly. - /// - /// It will return error when it's negative or too large to fit in 24 hours. - /// - /// Example: - /// - /// ```rust - /// use iceberg::spec::{Literal, Datum}; - /// let micro_secs = { - /// 1 * 3600 * 1_000_000 + // 1 hour - /// 2 * 60 * 1_000_000 + // 2 minutes - /// 1 * 1_000_000 + // 1 second - /// 888999 // microseconds - /// }; - /// - /// let t = Datum::time_micros(micro_secs).unwrap(); - /// - /// assert_eq!(&format!("{t}"), "01:02:01.888999"); - /// assert_eq!(Literal::time(micro_secs), t.into()); - /// - /// let negative_value = -100; - /// assert!(Datum::time_micros(negative_value).is_err()); - /// - /// let too_large_value = 36 * 60 * 60 * 1_000_000; // Too large to fit in 24 hours. - /// assert!(Datum::time_micros(too_large_value).is_err()); - /// ``` - pub fn time_micros(value: i64) -> Result { - ensure_data_valid!( - (0..=MAX_TIME_VALUE).contains(&value), - "Invalid value for Time type: {}", - value - ); - - Ok(Self { - r#type: PrimitiveType::Time, - literal: PrimitiveLiteral::Time(value), - }) - } - - /// Creates time literal from [`chrono::NaiveTime`]. - fn time_from_naive_time(t: NaiveTime) -> Self { - let duration = t - unix_epoch().time(); - // It's safe to unwrap here since less than 24 hours will never overflow. - let micro_secs = duration.num_microseconds().unwrap(); - - Self { - r#type: PrimitiveType::Time, - literal: PrimitiveLiteral::Time(micro_secs), - } - } - - /// Creates time literal in microseconds in `%H:%M:%S:.f` format. - /// - /// See [`NaiveTime::from_str`] for details. - /// - /// Example: - /// ```rust - /// use iceberg::spec::{Literal, Datum}; - /// let t = Datum::time_from_str("01:02:01.888999777").unwrap(); - /// - /// assert_eq!(&format!("{t}"), "01:02:01.888999"); - /// ``` - pub fn time_from_str>(s: S) -> Result { - let t = s.as_ref().parse::().map_err(|e| { - Error::new( - ErrorKind::DataInvalid, - format!("Can't parse time from string: {}", s.as_ref()), - ) - .with_source(e) - })?; - - Ok(Self::time_from_naive_time(t)) - } - - /// Creates time literal from hour, minute, second, and microseconds. - /// - /// See [`NaiveTime::from_hms_micro_opt`]. - /// - /// Example: - /// ```rust - /// - /// use iceberg::spec::{Literal, Datum}; - /// let t = Datum::time_from_hms_micro(22, 15, 33, 111).unwrap(); - /// - /// assert_eq!(&format!("{t}"), "22:15:33.000111"); - /// ``` - pub fn time_from_hms_micro(hour: u32, min: u32, sec: u32, micro: u32) -> Result { - let t = NaiveTime::from_hms_micro_opt(hour, min, sec, micro) - .ok_or_else(|| Error::new( - ErrorKind::DataInvalid, - format!("Can't create time from hour: {hour}, min: {min}, second: {sec}, microsecond: {micro}"), - ))?; - Ok(Self::time_from_naive_time(t)) - } - - /// Creates a timestamp from unix epoch in microseconds. - /// - /// Example: - /// - /// ```rust - /// - /// use iceberg::spec::Datum; - /// let t = Datum::timestamp_micros(1000); - /// - /// assert_eq!(&format!("{t}"), "1970-01-01 00:00:00.001"); - /// ``` - pub fn timestamp_micros(value: i64) -> Self { - Self { - r#type: PrimitiveType::Timestamp, - literal: PrimitiveLiteral::Timestamp(value), - } - } - - /// Creates a timestamp from [`DateTime`]. - /// - /// Example: - /// - /// ```rust - /// - /// use chrono::{NaiveDate, NaiveDateTime, TimeZone, Utc}; - /// use iceberg::spec::Datum; - /// let t = Datum::timestamp_from_datetime( - /// NaiveDate::from_ymd_opt(1992, 3, 1) - /// .unwrap() - /// .and_hms_micro_opt(1, 2, 3, 88) - /// .unwrap()); - /// - /// assert_eq!(&format!("{t}"), "1992-03-01 01:02:03.000088"); - /// ``` - pub fn timestamp_from_datetime(dt: NaiveDateTime) -> Self { - Self::timestamp_micros(dt.and_utc().timestamp_micros()) - } - - /// Parse a timestamp in [`%Y-%m-%dT%H:%M:%S%.f`] format. - /// - /// See [`NaiveDateTime::from_str`]. - /// - /// Example: - /// - /// ```rust - /// use chrono::{DateTime, FixedOffset, NaiveDate, NaiveDateTime, NaiveTime}; - /// use iceberg::spec::{Literal, Datum}; - /// let t = Datum::timestamp_from_str("1992-03-01T01:02:03.000088").unwrap(); - /// - /// assert_eq!(&format!("{t}"), "1992-03-01 01:02:03.000088"); - /// ``` - pub fn timestamp_from_str>(s: S) -> Result { - let dt = s.as_ref().parse::().map_err(|e| { - Error::new(ErrorKind::DataInvalid, "Can't parse timestamp.").with_source(e) - })?; - - Ok(Self::timestamp_from_datetime(dt)) - } - - /// Creates a timestamp with timezone from unix epoch in microseconds. - /// - /// Example: - /// - /// ```rust - /// - /// use iceberg::spec::Datum; - /// let t = Datum::timestamptz_micros(1000); - /// - /// assert_eq!(&format!("{t}"), "1970-01-01 00:00:00.001 UTC"); - /// ``` - pub fn timestamptz_micros(value: i64) -> Self { - Self { - r#type: PrimitiveType::Timestamptz, - literal: PrimitiveLiteral::Timestamptz(value), - } - } - - /// Creates a timestamp with timezone from [`DateTime`]. - /// Example: - /// - /// ```rust - /// - /// use chrono::{TimeZone, Utc}; - /// use iceberg::spec::Datum; - /// let t = Datum::timestamptz_from_datetime(Utc.timestamp_opt(1000, 0).unwrap()); - /// - /// assert_eq!(&format!("{t}"), "1970-01-01 00:16:40 UTC"); - /// ``` - pub fn timestamptz_from_datetime(dt: DateTime) -> Self { - Self::timestamptz_micros(dt.with_timezone(&Utc).timestamp_micros()) - } - - /// Parse timestamp with timezone in RFC3339 format. - /// - /// See [`DateTime::from_str`]. - /// - /// Example: - /// - /// ```rust - /// use chrono::{DateTime, FixedOffset, NaiveDate, NaiveDateTime, NaiveTime}; - /// use iceberg::spec::{Literal, Datum}; - /// let t = Datum::timestamptz_from_str("1992-03-01T01:02:03.000088+08:00").unwrap(); - /// - /// assert_eq!(&format!("{t}"), "1992-02-29 17:02:03.000088 UTC"); - /// ``` - pub fn timestamptz_from_str>(s: S) -> Result { - let dt = DateTime::::from_str(s.as_ref()).map_err(|e| { - Error::new(ErrorKind::DataInvalid, "Can't parse datetime.").with_source(e) - })?; - - Ok(Self::timestamptz_from_datetime(dt)) - } - - /// Creates a string literal. - /// - /// Example: - /// - /// ```rust - /// use iceberg::spec::Datum; - /// let t = Datum::string("ss"); - /// - /// assert_eq!(&format!("{t}"), r#""ss""#); - /// ``` - pub fn string(s: S) -> Self { - Self { - r#type: PrimitiveType::String, - literal: PrimitiveLiteral::String(s.to_string()), - } - } - - /// Creates uuid literal. - /// - /// Example: - /// - /// ```rust - /// use uuid::uuid; - /// use iceberg::spec::Datum; - /// let t = Datum::uuid(uuid!("a1a2a3a4-b1b2-c1c2-d1d2-d3d4d5d6d7d8")); - /// - /// assert_eq!(&format!("{t}"), "a1a2a3a4-b1b2-c1c2-d1d2-d3d4d5d6d7d8"); - /// ``` - pub fn uuid(uuid: Uuid) -> Self { - Self { - r#type: PrimitiveType::Uuid, - literal: PrimitiveLiteral::UUID(uuid), - } - } - - /// Creates uuid from str. See [`Uuid::parse_str`]. - /// - /// Example: - /// - /// ```rust - /// use iceberg::spec::{Datum}; - /// let t = Datum::uuid_from_str("a1a2a3a4-b1b2-c1c2-d1d2-d3d4d5d6d7d8").unwrap(); - /// - /// assert_eq!(&format!("{t}"), "a1a2a3a4-b1b2-c1c2-d1d2-d3d4d5d6d7d8"); - /// ``` - pub fn uuid_from_str>(s: S) -> Result { - let uuid = Uuid::parse_str(s.as_ref()).map_err(|e| { - Error::new( - ErrorKind::DataInvalid, - format!("Can't parse uuid from string: {}", s.as_ref()), - ) - .with_source(e) - })?; - Ok(Self::uuid(uuid)) - } - - /// Creates a fixed literal from bytes. - /// - /// Example: - /// - /// ```rust - /// use iceberg::spec::{Literal, PrimitiveLiteral, Datum}; - /// let t = Datum::fixed(vec![1u8, 2u8]); - /// - /// assert_eq!(&format!("{t}"), "0102"); - /// ``` - pub fn fixed>(input: I) -> Self { - let value: Vec = input.into_iter().collect(); - Self { - r#type: PrimitiveType::Fixed(value.len() as u64), - literal: PrimitiveLiteral::Fixed(value), - } - } - - /// Creates a binary literal from bytes. - /// - /// Example: - /// - /// ```rust - /// use iceberg::spec::Datum; - /// let t = Datum::binary(vec![1u8, 100u8]); - /// - /// assert_eq!(&format!("{t}"), "0164"); - /// ``` - pub fn binary>(input: I) -> Self { - Self { - r#type: PrimitiveType::Binary, - literal: PrimitiveLiteral::Binary(input.into_iter().collect()), - } - } - - /// Creates decimal literal from string. See [`Decimal::from_str_exact`]. - /// - /// Example: - /// - /// ```rust - /// use itertools::assert_equal; - /// use rust_decimal::Decimal; - /// use iceberg::spec::Datum; - /// let t = Datum::decimal_from_str("123.45").unwrap(); - /// - /// assert_eq!(&format!("{t}"), "123.45"); - /// ``` - pub fn decimal_from_str>(s: S) -> Result { - let decimal = Decimal::from_str_exact(s.as_ref()).map_err(|e| { - Error::new(ErrorKind::DataInvalid, "Can't parse decimal.").with_source(e) - })?; - - Self::decimal(decimal) - } - - /// Try to create a decimal literal from [`Decimal`]. - /// - /// Example: - /// - /// ```rust - /// use rust_decimal::Decimal; - /// use iceberg::spec::Datum; - /// - /// let t = Datum::decimal(Decimal::new(123, 2)).unwrap(); - /// - /// assert_eq!(&format!("{t}"), "1.23"); - /// ``` - pub fn decimal(value: impl Into) -> Result { - let decimal = value.into(); - let scale = decimal.scale(); - - let r#type = Type::decimal(MAX_DECIMAL_PRECISION, scale)?; - if let Type::Primitive(p) = r#type { - Ok(Self { - r#type: p, - literal: PrimitiveLiteral::Decimal(decimal.mantissa()), - }) - } else { - unreachable!("Decimal type must be primitive.") - } - } - - /// Convert the datum to `target_type`. - pub fn to(self, target_type: &Type) -> Result { - // TODO: We should allow more type conversions - match target_type { - Type::Primitive(typ) if typ == &self.r#type => Ok(self), - _ => Err(Error::new( - ErrorKind::DataInvalid, - format!( - "Can't convert datum from {} type to {} type.", - self.r#type, target_type - ), - )), - } - } - - /// Get the primitive literal from datum. - pub fn literal(&self) -> &PrimitiveLiteral { - &self.literal - } - - /// Get the primitive type from datum. - pub fn data_type(&self) -> &PrimitiveType { - &self.r#type - } - - /// Returns true if the Literal represents a primitive type - /// that can be a NaN, and that it's value is NaN - pub fn is_nan(&self) -> bool { - match self.literal { - PrimitiveLiteral::Double(val) => val.is_nan(), - PrimitiveLiteral::Float(val) => val.is_nan(), - _ => false, - } - } - } - - /// Map is a collection of key-value pairs with a key type and a value type. - /// It used in Literal::Map, to make it hashable, the order of key-value pairs is stored in a separate vector - /// so that we can hash the map in a deterministic way. But it also means that the order of key-value pairs is matter - /// for the hash value. - #[derive(Clone, Debug, PartialEq, Eq)] - pub struct Map { - index: HashMap, - pair: Vec<(Literal, Option)>, - } - - impl Map { - /// Creates a new empty map. - pub fn new() -> Self { - Self { - index: HashMap::new(), - pair: Vec::new(), - } - } - - /// Return the number of key-value pairs in the map. - pub fn len(&self) -> usize { - self.pair.len() - } - - /// Returns true if the map contains no elements. - pub fn is_empty(&self) -> bool { - self.pair.is_empty() - } - - /// Inserts a key-value pair into the map. - /// If the map did not have this key present, None is returned. - /// If the map did have this key present, the value is updated, and the old value is returned. - pub fn insert(&mut self, key: Literal, value: Option) -> Option> { - if let Some(index) = self.index.get(&key) { - let old_value = std::mem::replace(&mut self.pair[*index].1, value); - Some(old_value) - } else { - self.pair.push((key.clone(), value)); - self.index.insert(key, self.pair.len() - 1); - None - } - } - - /// Returns a reference to the value corresponding to the key. - /// If the key is not present in the map, None is returned. - pub fn get(&self, key: &Literal) -> Option<&Option> { - self.index.get(key).map(|index| &self.pair[*index].1) - } - - /// The order of map is matter, so this method used to compare two maps has same key-value pairs without considering the order. - pub fn has_same_content(&self, other: &Map) -> bool { - if self.len() != other.len() { - return false; - } - - for (key, value) in &self.pair { - match other.get(key) { - Some(other_value) if value == other_value => (), - _ => return false, - } - } - - true - } - } - - impl Default for Map { - fn default() -> Self { - Self::new() - } - } - - impl Hash for Map { - fn hash(&self, state: &mut H) { - for (key, value) in &self.pair { - key.hash(state); - value.hash(state); - } - } - } - - impl FromIterator<(Literal, Option)> for Map { - fn from_iter)>>(iter: T) -> Self { - let mut map = Map::new(); - for (key, value) in iter { - map.insert(key, value); - } - map - } - } - - impl IntoIterator for Map { - type Item = (Literal, Option); - type IntoIter = std::vec::IntoIter; - - fn into_iter(self) -> Self::IntoIter { - self.pair.into_iter() - } - } - - impl From<[(Literal, Option); N]> for Map { - fn from(value: [(Literal, Option); N]) -> Self { - value.iter().cloned().collect() - } - } - - /// Values present in iceberg type - #[derive(Clone, Debug, PartialEq, Eq, Hash)] - pub enum Literal { - /// A primitive value - Primitive(PrimitiveLiteral), - /// A struct is a tuple of typed values. Each field in the tuple is named and has an integer id that is unique in the table schema. - /// Each field can be either optional or required, meaning that values can (or cannot) be null. Fields may be any type. - /// Fields may have an optional comment or doc string. Fields can have default values. - Struct(Struct), - /// A list is a collection of values with some element type. - /// The element field has an integer id that is unique in the table schema. - /// Elements can be either optional or required. Element types may be any type. - List(Vec>), - /// A map is a collection of key-value pairs with a key type and a value type. - /// Both the key field and value field each have an integer id that is unique in the table schema. - /// Map keys are required and map values can be either optional or required. Both map keys and map values may be any type, including nested types. - Map(Map), - } - - impl Literal { - /// Creates a boolean value. - /// - /// Example: - /// ```rust - /// use iceberg::spec::{Literal, PrimitiveLiteral}; - /// let t = Literal::bool(true); - /// - /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Boolean(true)), t); - /// ``` - pub fn bool>(t: T) -> Self { - Self::Primitive(PrimitiveLiteral::Boolean(t.into())) - } - - /// Creates a boolean value from string. - /// See [Parse bool from str](https://doc.rust-lang.org/stable/std/primitive.bool.html#impl-FromStr-for-bool) for reference. - /// - /// Example: - /// ```rust - /// use iceberg::spec::{Literal, PrimitiveLiteral}; - /// let t = Literal::bool_from_str("false").unwrap(); - /// - /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Boolean(false)), t); - /// ``` - pub fn bool_from_str>(s: S) -> Result { - let v = s.as_ref().parse::().map_err(|e| { - Error::new(ErrorKind::DataInvalid, "Can't parse string to bool.").with_source(e) - })?; - Ok(Self::Primitive(PrimitiveLiteral::Boolean(v))) - } - - /// Creates an 32bit integer. - /// - /// Example: - /// ```rust - /// use iceberg::spec::{Literal, PrimitiveLiteral}; - /// let t = Literal::int(23i8); - /// - /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Int(23)), t); - /// ``` - pub fn int>(t: T) -> Self { - Self::Primitive(PrimitiveLiteral::Int(t.into())) - } - - /// Creates an 64bit integer. - /// - /// Example: - /// ```rust - /// use iceberg::spec::{Literal, PrimitiveLiteral}; - /// let t = Literal::long(24i8); - /// - /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Long(24)), t); - /// ``` - pub fn long>(t: T) -> Self { - Self::Primitive(PrimitiveLiteral::Long(t.into())) - } - - /// Creates an 32bit floating point number. - /// - /// Example: - /// ```rust - /// use ordered_float::OrderedFloat; - /// use iceberg::spec::{Literal, PrimitiveLiteral}; - /// let t = Literal::float( 32.1f32 ); - /// - /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Float(OrderedFloat(32.1))), t); - /// ``` - pub fn float>(t: T) -> Self { - Self::Primitive(PrimitiveLiteral::Float(OrderedFloat(t.into()))) - } - - /// Creates an 32bit floating point number. - /// - /// Example: - /// ```rust - /// use ordered_float::OrderedFloat; - /// use iceberg::spec::{Literal, PrimitiveLiteral}; - /// let t = Literal::double( 32.1f64 ); - /// - /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Double(OrderedFloat(32.1))), t); - /// ``` - pub fn double>(t: T) -> Self { - Self::Primitive(PrimitiveLiteral::Double(OrderedFloat(t.into()))) - } - - /// Creates date literal from number of days from unix epoch directly. - pub fn date(days: i32) -> Self { - Self::Primitive(PrimitiveLiteral::Date(days)) - } - - /// Creates a date in `%Y-%m-%d` format, assume in utc timezone. - /// - /// See [`NaiveDate::from_str`]. - /// - /// Example - /// ```rust - /// use iceberg::spec::Literal; - /// let t = Literal::date_from_str("1970-01-03").unwrap(); - /// - /// assert_eq!(Literal::date(2), t); - /// ``` - pub fn date_from_str>(s: S) -> Result { - let t = s.as_ref().parse::().map_err(|e| { - Error::new( - ErrorKind::DataInvalid, - format!("Can't parse date from string: {}", s.as_ref()), - ) - .with_source(e) - })?; - - Ok(Self::date(date_from_naive_date(t))) - } - - /// Create a date from calendar date (year, month and day). - /// - /// See [`NaiveDate::from_ymd_opt`]. - /// - /// Example: - /// - ///```rust - /// use iceberg::spec::Literal; - /// let t = Literal::date_from_ymd(1970, 1, 5).unwrap(); - /// - /// assert_eq!(Literal::date(4), t); - /// ``` - pub fn date_from_ymd(year: i32, month: u32, day: u32) -> Result { - let t = NaiveDate::from_ymd_opt(year, month, day).ok_or_else(|| { - Error::new( - ErrorKind::DataInvalid, - format!("Can't create date from year: {year}, month: {month}, day: {day}"), - ) - })?; - - Ok(Self::date(date_from_naive_date(t))) - } - - /// Creates time in microseconds directly - pub fn time(value: i64) -> Self { - Self::Primitive(PrimitiveLiteral::Time(value)) - } - - /// Creates time literal from [`chrono::NaiveTime`]. - fn time_from_naive_time(t: NaiveTime) -> Self { - let duration = t - unix_epoch().time(); - // It's safe to unwrap here since less than 24 hours will never overflow. - let micro_secs = duration.num_microseconds().unwrap(); - - Literal::time(micro_secs) - } - - /// Creates time in microseconds in `%H:%M:%S:.f` format. - /// - /// See [`NaiveTime::from_str`] for details. - /// - /// Example: - /// ```rust - /// use iceberg::spec::Literal; - /// let t = Literal::time_from_str("01:02:01.888999777").unwrap(); - /// - /// let micro_secs = { - /// 1 * 3600 * 1_000_000 + // 1 hour - /// 2 * 60 * 1_000_000 + // 2 minutes - /// 1 * 1_000_000 + // 1 second - /// 888999 // microseconds - /// }; - /// assert_eq!(Literal::time(micro_secs), t); - /// ``` - pub fn time_from_str>(s: S) -> Result { - let t = s.as_ref().parse::().map_err(|e| { - Error::new( - ErrorKind::DataInvalid, - format!("Can't parse time from string: {}", s.as_ref()), - ) - .with_source(e) - })?; - - Ok(Self::time_from_naive_time(t)) - } - - /// Creates time literal from hour, minute, second, and microseconds. - /// - /// See [`NaiveTime::from_hms_micro_opt`]. - /// - /// Example: - /// ```rust - /// - /// use iceberg::spec::Literal; - /// let t = Literal::time_from_hms_micro(22, 15, 33, 111).unwrap(); - /// - /// assert_eq!(Literal::time_from_str("22:15:33.000111").unwrap(), t); - /// ``` - pub fn time_from_hms_micro(hour: u32, min: u32, sec: u32, micro: u32) -> Result { - let t = NaiveTime::from_hms_micro_opt(hour, min, sec, micro) - .ok_or_else(|| Error::new( - ErrorKind::DataInvalid, - format!("Can't create time from hour: {hour}, min: {min}, second: {sec}, microsecond: {micro}"), - ))?; - Ok(Self::time_from_naive_time(t)) - } - - /// Creates a timestamp from unix epoch in microseconds. - pub fn timestamp(value: i64) -> Self { - Self::Primitive(PrimitiveLiteral::Timestamp(value)) - } - - /// Creates a timestamp with timezone from unix epoch in microseconds. - pub fn timestamptz(value: i64) -> Self { - Self::Primitive(PrimitiveLiteral::Timestamptz(value)) - } - - /// Creates a timestamp from [`DateTime`]. - pub fn timestamp_from_datetime(dt: DateTime) -> Self { - Self::timestamp(dt.with_timezone(&Utc).timestamp_micros()) - } - - /// Creates a timestamp with timezone from [`DateTime`]. - pub fn timestamptz_from_datetime(dt: DateTime) -> Self { - Self::timestamptz(dt.with_timezone(&Utc).timestamp_micros()) - } - - /// Parse a timestamp in RFC3339 format. - /// - /// See [`DateTime::from_str`]. - /// - /// Example: - /// - /// ```rust - /// use chrono::{DateTime, FixedOffset, NaiveDate, NaiveDateTime, NaiveTime}; - /// use iceberg::spec::Literal; - /// let t = Literal::timestamp_from_str("2012-12-12 12:12:12.8899-04:00").unwrap(); - /// - /// let t2 = { - /// let date = NaiveDate::from_ymd_opt(2012, 12, 12).unwrap(); - /// let time = NaiveTime::from_hms_micro_opt(12, 12, 12, 889900).unwrap(); - /// let dt = NaiveDateTime::new(date, time); - /// Literal::timestamp_from_datetime(DateTime::::from_local(dt, FixedOffset::west_opt(4 * 3600).unwrap())) - /// }; - /// - /// assert_eq!(t, t2); - /// ``` - pub fn timestamp_from_str>(s: S) -> Result { - let dt = DateTime::::from_str(s.as_ref()).map_err(|e| { - Error::new(ErrorKind::DataInvalid, "Can't parse datetime.").with_source(e) - })?; - - Ok(Self::timestamp_from_datetime(dt)) - } - - /// Similar to [`Literal::timestamp_from_str`], but return timestamp with timezone literal. - pub fn timestamptz_from_str>(s: S) -> Result { - let dt = DateTime::::from_str(s.as_ref()).map_err(|e| { - Error::new(ErrorKind::DataInvalid, "Can't parse datetime.").with_source(e) - })?; - - Ok(Self::timestamptz_from_datetime(dt)) - } - - /// Creates a string literal. - pub fn string(s: S) -> Self { - Self::Primitive(PrimitiveLiteral::String(s.to_string())) - } - - /// Creates uuid literal. - pub fn uuid(uuid: Uuid) -> Self { - Self::Primitive(PrimitiveLiteral::UUID(uuid)) - } - - /// Creates uuid from str. See [`Uuid::parse_str`]. - /// - /// Example: - /// - /// ```rust - /// use uuid::Uuid; - /// use iceberg::spec::Literal; - /// let t1 = Literal::uuid_from_str("a1a2a3a4-b1b2-c1c2-d1d2-d3d4d5d6d7d8").unwrap(); - /// let t2 = Literal::uuid(Uuid::from_u128_le(0xd8d7d6d5d4d3d2d1c2c1b2b1a4a3a2a1)); - /// - /// assert_eq!(t1, t2); - /// ``` - pub fn uuid_from_str>(s: S) -> Result { - let uuid = Uuid::parse_str(s.as_ref()).map_err(|e| { - Error::new( - ErrorKind::DataInvalid, - format!("Can't parse uuid from string: {}", s.as_ref()), - ) - .with_source(e) - })?; - Ok(Self::uuid(uuid)) - } - - /// Creates a fixed literal from bytes. - /// - /// Example: - /// - /// ```rust - /// use iceberg::spec::{Literal, PrimitiveLiteral}; - /// let t1 = Literal::fixed(vec![1u8, 2u8]); - /// let t2 = Literal::Primitive(PrimitiveLiteral::Fixed(vec![1u8, 2u8])); - /// - /// assert_eq!(t1, t2); - /// ``` - pub fn fixed>(input: I) -> Self { - Literal::Primitive(PrimitiveLiteral::Fixed(input.into_iter().collect())) - } - - /// Creates a binary literal from bytes. - /// - /// Example: - /// - /// ```rust - /// use iceberg::spec::{Literal, PrimitiveLiteral}; - /// let t1 = Literal::binary(vec![1u8, 2u8]); - /// let t2 = Literal::Primitive(PrimitiveLiteral::Binary(vec![1u8, 2u8])); - /// - /// assert_eq!(t1, t2); - /// ``` - pub fn binary>(input: I) -> Self { - Literal::Primitive(PrimitiveLiteral::Binary(input.into_iter().collect())) - } - - /// Creates a decimal literal. - pub fn decimal(decimal: i128) -> Self { - Self::Primitive(PrimitiveLiteral::Decimal(decimal)) - } - - /// Creates decimal literal from string. See [`Decimal::from_str_exact`]. - /// - /// Example: - /// - /// ```rust - /// use rust_decimal::Decimal; - /// use iceberg::spec::Literal; - /// let t1 = Literal::decimal(12345); - /// let t2 = Literal::decimal_from_str("123.45").unwrap(); - /// - /// assert_eq!(t1, t2); - /// ``` - pub fn decimal_from_str>(s: S) -> Result { - let decimal = Decimal::from_str_exact(s.as_ref()).map_err(|e| { - Error::new(ErrorKind::DataInvalid, "Can't parse decimal.").with_source(e) - })?; - Ok(Self::decimal(decimal.mantissa())) - } - } - - impl From for ByteBuf { - fn from(value: PrimitiveLiteral) -> Self { - match value { - PrimitiveLiteral::Boolean(val) => { - if val { - ByteBuf::from([1u8]) - } else { - ByteBuf::from([0u8]) - } - } - PrimitiveLiteral::Int(val) => ByteBuf::from(val.to_le_bytes()), - PrimitiveLiteral::Long(val) => ByteBuf::from(val.to_le_bytes()), - PrimitiveLiteral::Float(val) => ByteBuf::from(val.to_le_bytes()), - PrimitiveLiteral::Double(val) => ByteBuf::from(val.to_le_bytes()), - PrimitiveLiteral::Date(val) => ByteBuf::from(val.to_le_bytes()), - PrimitiveLiteral::Time(val) => ByteBuf::from(val.to_le_bytes()), - PrimitiveLiteral::Timestamp(val) => ByteBuf::from(val.to_le_bytes()), - PrimitiveLiteral::Timestamptz(val) => ByteBuf::from(val.to_le_bytes()), - PrimitiveLiteral::String(val) => ByteBuf::from(val.as_bytes()), - PrimitiveLiteral::UUID(val) => ByteBuf::from(val.as_u128().to_be_bytes()), - PrimitiveLiteral::Fixed(val) => ByteBuf::from(val), - PrimitiveLiteral::Binary(val) => ByteBuf::from(val), - PrimitiveLiteral::Decimal(_) => todo!(), - } - } - } - - impl From for ByteBuf { - fn from(value: Literal) -> Self { - match value { - Literal::Primitive(val) => val.into(), - _ => unimplemented!(), - } - } - } - - impl From for Vec { - fn from(value: PrimitiveLiteral) -> Self { - match value { - PrimitiveLiteral::Boolean(val) => { - if val { - Vec::from([1u8]) - } else { - Vec::from([0u8]) - } - } - PrimitiveLiteral::Int(val) => Vec::from(val.to_le_bytes()), - PrimitiveLiteral::Long(val) => Vec::from(val.to_le_bytes()), - PrimitiveLiteral::Float(val) => Vec::from(val.to_le_bytes()), - PrimitiveLiteral::Double(val) => Vec::from(val.to_le_bytes()), - PrimitiveLiteral::Date(val) => Vec::from(val.to_le_bytes()), - PrimitiveLiteral::Time(val) => Vec::from(val.to_le_bytes()), - PrimitiveLiteral::Timestamp(val) => Vec::from(val.to_le_bytes()), - PrimitiveLiteral::Timestamptz(val) => Vec::from(val.to_le_bytes()), - PrimitiveLiteral::String(val) => Vec::from(val.as_bytes()), - PrimitiveLiteral::UUID(val) => Vec::from(val.as_u128().to_be_bytes()), - PrimitiveLiteral::Fixed(val) => val, - PrimitiveLiteral::Binary(val) => val, - PrimitiveLiteral::Decimal(_) => todo!(), - } - } - } - - impl From for Vec { - fn from(value: Literal) -> Self { - match value { - Literal::Primitive(val) => val.into(), - _ => unimplemented!(), - } - } - } - - /// The partition struct stores the tuple of partition values for each file. - /// Its type is derived from the partition fields of the partition spec used to write the manifest file. - /// In v2, the partition struct’s field ids must match the ids from the partition spec. - #[derive(Debug, Clone, PartialEq, Eq, Hash)] - pub struct Struct { - /// Vector to store the field values - fields: Vec, - /// Null bitmap - null_bitmap: BitVec, - } - - impl Struct { - /// Create a empty struct. - pub fn empty() -> Self { - Self { - fields: Vec::new(), - null_bitmap: BitVec::new(), - } - } - - /// Create a iterator to read the field in order of field_value. - pub fn iter(&self) -> impl Iterator> { - self.null_bitmap.iter().zip(self.fields.iter()).map( - |(null, value)| { - if *null { - None - } else { - Some(value) - } - }, - ) - } - - /// returns true if the field at position `index` is null - pub fn is_null_at_index(&self, index: usize) -> bool { - self.null_bitmap[index] - } - } - - impl Index for Struct { - type Output = Literal; - - fn index(&self, idx: usize) -> &Self::Output { - &self.fields[idx] - } - } - - /// An iterator that moves out of a struct. - pub struct StructValueIntoIter { - null_bitmap: bitvec::boxed::IntoIter, - fields: std::vec::IntoIter, - } - - impl Iterator for StructValueIntoIter { - type Item = Option; - - fn next(&mut self) -> Option { - match (self.null_bitmap.next(), self.fields.next()) { - (Some(null), Some(value)) => Some(if null { None } else { Some(value) }), - _ => None, - } - } - } - - impl IntoIterator for Struct { - type Item = Option; - - type IntoIter = StructValueIntoIter; - - fn into_iter(self) -> Self::IntoIter { - StructValueIntoIter { - null_bitmap: self.null_bitmap.into_iter(), - fields: self.fields.into_iter(), - } - } - } - - impl FromIterator> for Struct { - fn from_iter>>(iter: I) -> Self { - let mut fields = Vec::new(); - let mut null_bitmap = BitVec::new(); - - for value in iter.into_iter() { - match value { - Some(value) => { - fields.push(value); - null_bitmap.push(false) - } - None => { - fields.push(Literal::Primitive(PrimitiveLiteral::Boolean(false))); - null_bitmap.push(true) - } - } - } - Struct { - fields, - null_bitmap, - } - } - } - - impl Literal { - /// Create iceberg value from bytes - pub fn try_from_bytes(bytes: &[u8], data_type: &Type) -> Result { - match data_type { - Type::Primitive(primitive_type) => { - let datum = Datum::try_from_bytes(bytes, primitive_type.clone())?; - Ok(Literal::Primitive(datum.literal)) - } - _ => Err(Error::new( - crate::ErrorKind::DataInvalid, - "Converting bytes to non-primitive types is not supported.", - )), - } - } - - /// Create iceberg value from a json value - pub fn try_from_json(value: JsonValue, data_type: &Type) -> Result> { - match data_type { - Type::Primitive(primitive) => match (primitive, value) { - (PrimitiveType::Boolean, JsonValue::Bool(bool)) => { - Ok(Some(Literal::Primitive(PrimitiveLiteral::Boolean(bool)))) - } - (PrimitiveType::Int, JsonValue::Number(number)) => { - Ok(Some(Literal::Primitive(PrimitiveLiteral::Int( - number - .as_i64() - .ok_or(Error::new( - crate::ErrorKind::DataInvalid, - "Failed to convert json number to int", - ))? - .try_into()?, - )))) - } - (PrimitiveType::Long, JsonValue::Number(number)) => Ok(Some(Literal::Primitive( - PrimitiveLiteral::Long(number.as_i64().ok_or(Error::new( - crate::ErrorKind::DataInvalid, - "Failed to convert json number to long", - ))?), - ))), - (PrimitiveType::Float, JsonValue::Number(number)) => Ok(Some(Literal::Primitive( - PrimitiveLiteral::Float(OrderedFloat(number.as_f64().ok_or(Error::new( - crate::ErrorKind::DataInvalid, - "Failed to convert json number to float", - ))? as f32)), - ))), - (PrimitiveType::Double, JsonValue::Number(number)) => Ok(Some(Literal::Primitive( - PrimitiveLiteral::Double(OrderedFloat(number.as_f64().ok_or(Error::new( - crate::ErrorKind::DataInvalid, - "Failed to convert json number to double", - ))?)), - ))), - (PrimitiveType::Date, JsonValue::String(s)) => { - Ok(Some(Literal::Primitive(PrimitiveLiteral::Date( - date::date_to_days(&NaiveDate::parse_from_str(&s, "%Y-%m-%d")?), - )))) - } - (PrimitiveType::Time, JsonValue::String(s)) => { - Ok(Some(Literal::Primitive(PrimitiveLiteral::Time( - time::time_to_microseconds(&NaiveTime::parse_from_str(&s, "%H:%M:%S%.f")?), - )))) - } - (PrimitiveType::Timestamp, JsonValue::String(s)) => Ok(Some(Literal::Primitive( - PrimitiveLiteral::Timestamp(timestamp::datetime_to_microseconds( - &NaiveDateTime::parse_from_str(&s, "%Y-%m-%dT%H:%M:%S%.f")?, - )), - ))), - (PrimitiveType::Timestamptz, JsonValue::String(s)) => { - Ok(Some(Literal::Primitive(PrimitiveLiteral::Timestamptz( - timestamptz::datetimetz_to_microseconds(&Utc.from_utc_datetime( - &NaiveDateTime::parse_from_str(&s, "%Y-%m-%dT%H:%M:%S%.f+00:00")?, - )), - )))) - } - (PrimitiveType::String, JsonValue::String(s)) => { - Ok(Some(Literal::Primitive(PrimitiveLiteral::String(s)))) - } - (PrimitiveType::Uuid, JsonValue::String(s)) => Ok(Some(Literal::Primitive( - PrimitiveLiteral::UUID(Uuid::parse_str(&s)?), - ))), - (PrimitiveType::Fixed(_), JsonValue::String(_)) => todo!(), - (PrimitiveType::Binary, JsonValue::String(_)) => todo!(), - ( - PrimitiveType::Decimal { - precision: _, - scale, - }, - JsonValue::String(s), - ) => { - let mut decimal = Decimal::from_str_exact(&s)?; - decimal.rescale(*scale); - Ok(Some(Literal::Primitive(PrimitiveLiteral::Decimal( - decimal.mantissa(), - )))) - } - (_, JsonValue::Null) => Ok(None), - (i, j) => Err(Error::new( - crate::ErrorKind::DataInvalid, - format!( - "The json value {} doesn't fit to the iceberg type {}.", - j, i - ), - )), - }, - Type::Struct(schema) => { - if let JsonValue::Object(mut object) = value { - Ok(Some(Literal::Struct(Struct::from_iter( - schema.fields().iter().map(|field| { - object.remove(&field.id.to_string()).and_then(|value| { - Literal::try_from_json(value, &field.field_type) - .and_then(|value| { - value.ok_or(Error::new( - ErrorKind::DataInvalid, - "Key of map cannot be null", - )) - }) - .ok() - }) - }), - )))) - } else { - Err(Error::new( - crate::ErrorKind::DataInvalid, - "The json value for a struct type must be an object.", - )) - } - } - Type::List(list) => { - if let JsonValue::Array(array) = value { - Ok(Some(Literal::List( - array - .into_iter() - .map(|value| { - Literal::try_from_json(value, &list.element_field.field_type) - }) - .collect::>>()?, - ))) - } else { - Err(Error::new( - crate::ErrorKind::DataInvalid, - "The json value for a list type must be an array.", - )) - } - } - Type::Map(map) => { - if let JsonValue::Object(mut object) = value { - if let (Some(JsonValue::Array(keys)), Some(JsonValue::Array(values))) = - (object.remove("keys"), object.remove("values")) - { - Ok(Some(Literal::Map(Map::from_iter( - keys.into_iter() - .zip(values.into_iter()) - .map(|(key, value)| { - Ok(( - Literal::try_from_json(key, &map.key_field.field_type) - .and_then(|value| { - value.ok_or(Error::new( - ErrorKind::DataInvalid, - "Key of map cannot be null", - )) - })?, - Literal::try_from_json(value, &map.value_field.field_type)?, - )) - }) - .collect::>>()?, - )))) - } else { - Err(Error::new( - crate::ErrorKind::DataInvalid, - "The json value for a list type must be an array.", - )) - } - } else { - Err(Error::new( - crate::ErrorKind::DataInvalid, - "The json value for a list type must be an array.", - )) - } - } - } - } - - /// Converting iceberg value to json value. - /// - /// See [this spec](https://iceberg.apache.org/spec/#json-single-value-serialization) for reference. - pub fn try_into_json(self, r#type: &Type) -> Result { - match (self, r#type) { - (Literal::Primitive(prim), _) => match prim { - PrimitiveLiteral::Boolean(val) => Ok(JsonValue::Bool(val)), - PrimitiveLiteral::Int(val) => Ok(JsonValue::Number((val).into())), - PrimitiveLiteral::Long(val) => Ok(JsonValue::Number((val).into())), - PrimitiveLiteral::Float(val) => match Number::from_f64(val.0 as f64) { - Some(number) => Ok(JsonValue::Number(number)), - None => Ok(JsonValue::Null), - }, - PrimitiveLiteral::Double(val) => match Number::from_f64(val.0) { - Some(number) => Ok(JsonValue::Number(number)), - None => Ok(JsonValue::Null), - }, - PrimitiveLiteral::Date(val) => { - Ok(JsonValue::String(date::days_to_date(val).to_string())) - } - PrimitiveLiteral::Time(val) => Ok(JsonValue::String( - time::microseconds_to_time(val).to_string(), - )), - PrimitiveLiteral::Timestamp(val) => Ok(JsonValue::String( - timestamp::microseconds_to_datetime(val) - .format("%Y-%m-%dT%H:%M:%S%.f") - .to_string(), - )), - PrimitiveLiteral::Timestamptz(val) => Ok(JsonValue::String( - timestamptz::microseconds_to_datetimetz(val) - .format("%Y-%m-%dT%H:%M:%S%.f+00:00") - .to_string(), - )), - PrimitiveLiteral::String(val) => Ok(JsonValue::String(val.clone())), - PrimitiveLiteral::UUID(val) => Ok(JsonValue::String(val.to_string())), - PrimitiveLiteral::Fixed(val) => Ok(JsonValue::String(val.iter().fold( - String::new(), - |mut acc, x| { - acc.push_str(&format!("{:x}", x)); - acc - }, - ))), - PrimitiveLiteral::Binary(val) => Ok(JsonValue::String(val.iter().fold( - String::new(), - |mut acc, x| { - acc.push_str(&format!("{:x}", x)); - acc - }, - ))), - PrimitiveLiteral::Decimal(val) => match r#type { - Type::Primitive(PrimitiveType::Decimal { - precision: _precision, - scale, - }) => { - let decimal = Decimal::try_from_i128_with_scale(val, *scale)?; - Ok(JsonValue::String(decimal.to_string())) - } - _ => Err(Error::new( - ErrorKind::DataInvalid, - "The iceberg type for decimal literal must be decimal.", - ))?, - }, - }, - (Literal::Struct(s), Type::Struct(struct_type)) => { - let mut id_and_value = Vec::with_capacity(struct_type.fields().len()); - for (value, field) in s.into_iter().zip(struct_type.fields()) { - let json = match value { - Some(val) => val.try_into_json(&field.field_type)?, - None => JsonValue::Null, - }; - id_and_value.push((field.id.to_string(), json)); - } - Ok(JsonValue::Object(JsonMap::from_iter(id_and_value))) - } - (Literal::List(list), Type::List(list_type)) => Ok(JsonValue::Array( - list.into_iter() - .map(|opt| match opt { - Some(literal) => literal.try_into_json(&list_type.element_field.field_type), - None => Ok(JsonValue::Null), - }) - .collect::>>()?, - )), - (Literal::Map(map), Type::Map(map_type)) => { - let mut object = JsonMap::with_capacity(2); - let mut json_keys = Vec::with_capacity(map.len()); - let mut json_values = Vec::with_capacity(map.len()); - for (key, value) in map.into_iter() { - json_keys.push(key.try_into_json(&map_type.key_field.field_type)?); - json_values.push(match value { - Some(literal) => literal.try_into_json(&map_type.value_field.field_type)?, - None => JsonValue::Null, - }); - } - object.insert("keys".to_string(), JsonValue::Array(json_keys)); - object.insert("values".to_string(), JsonValue::Array(json_values)); - Ok(JsonValue::Object(object)) - } - (value, r#type) => Err(Error::new( - ErrorKind::DataInvalid, - format!( - "The iceberg value {:?} doesn't fit to the iceberg type {}.", - value, r#type - ), - )), - } - } - - /// Convert Value to the any type - pub fn into_any(self) -> Box { - match self { - Literal::Primitive(prim) => match prim { - PrimitiveLiteral::Boolean(any) => Box::new(any), - PrimitiveLiteral::Int(any) => Box::new(any), - PrimitiveLiteral::Long(any) => Box::new(any), - PrimitiveLiteral::Float(any) => Box::new(any), - PrimitiveLiteral::Double(any) => Box::new(any), - PrimitiveLiteral::Date(any) => Box::new(any), - PrimitiveLiteral::Time(any) => Box::new(any), - PrimitiveLiteral::Timestamp(any) => Box::new(any), - PrimitiveLiteral::Timestamptz(any) => Box::new(any), - PrimitiveLiteral::Fixed(any) => Box::new(any), - PrimitiveLiteral::Binary(any) => Box::new(any), - PrimitiveLiteral::String(any) => Box::new(any), - PrimitiveLiteral::UUID(any) => Box::new(any), - PrimitiveLiteral::Decimal(any) => Box::new(any), - }, - _ => unimplemented!(), - } - } - } - - mod date { - use chrono::{DateTime, NaiveDate, TimeDelta, TimeZone, Utc}; - - pub(crate) fn date_to_days(date: &NaiveDate) -> i32 { - date.signed_duration_since( - // This is always the same and shouldn't fail - NaiveDate::from_ymd_opt(1970, 1, 1).unwrap(), - ) - .num_days() as i32 - } - - pub(crate) fn days_to_date(days: i32) -> NaiveDate { - // This shouldn't fail until the year 262000 - (chrono::DateTime::UNIX_EPOCH + TimeDelta::try_days(days as i64).unwrap()) - .naive_utc() - .date() - } - - /// Returns unix epoch. - pub(crate) fn unix_epoch() -> DateTime { - Utc.timestamp_nanos(0) - } - - /// Creates date literal from `NaiveDate`, assuming it's utc timezone. - pub(crate) fn date_from_naive_date(date: NaiveDate) -> i32 { - (date - unix_epoch().date_naive()).num_days() as i32 - } - } - - mod time { - use chrono::NaiveTime; - - pub(crate) fn time_to_microseconds(time: &NaiveTime) -> i64 { - time.signed_duration_since( - // This is always the same and shouldn't fail - NaiveTime::from_num_seconds_from_midnight_opt(0, 0).unwrap(), - ) - .num_microseconds() - .unwrap() - } - - pub(crate) fn microseconds_to_time(micros: i64) -> NaiveTime { - let (secs, rem) = (micros / 1_000_000, micros % 1_000_000); - - NaiveTime::from_num_seconds_from_midnight_opt(secs as u32, rem as u32 * 1_000).unwrap() - } - } - - mod timestamp { - use chrono::{DateTime, NaiveDateTime}; - - pub(crate) fn datetime_to_microseconds(time: &NaiveDateTime) -> i64 { - time.and_utc().timestamp_micros() - } - - pub(crate) fn microseconds_to_datetime(micros: i64) -> NaiveDateTime { - // This shouldn't fail until the year 262000 - DateTime::from_timestamp_micros(micros).unwrap().naive_utc() - } - } - - mod timestamptz { - use chrono::DateTime; - use chrono::Utc; - - pub(crate) fn datetimetz_to_microseconds(time: &DateTime) -> i64 { - time.timestamp_micros() - } - - pub(crate) fn microseconds_to_datetimetz(micros: i64) -> DateTime { - let (secs, rem) = (micros / 1_000_000, micros % 1_000_000); - - DateTime::from_timestamp(secs, rem as u32 * 1_000).unwrap() - } - } - - mod _serde { - use serde::{ - de::Visitor, - ser::{SerializeMap, SerializeSeq, SerializeStruct}, - Deserialize, Serialize, - }; - use serde_bytes::ByteBuf; - use serde_derive::Deserialize as DeserializeDerive; - use serde_derive::Serialize as SerializeDerive; - - use crate::{ - spec::{PrimitiveType, Type, MAP_KEY_FIELD_NAME, MAP_VALUE_FIELD_NAME}, - Error, ErrorKind, - }; - - use super::{Literal, Map, PrimitiveLiteral}; - - #[derive(SerializeDerive, DeserializeDerive, Debug)] - #[serde(transparent)] - /// Raw literal representation used for serde. The serialize way is used for Avro serializer. - pub struct RawLiteral(RawLiteralEnum); - - impl RawLiteral { - /// Covert literal to raw literal. - pub fn try_from(literal: Literal, ty: &Type) -> Result { - Ok(Self(RawLiteralEnum::try_from(literal, ty)?)) - } - - /// Convert raw literal to literal. - pub fn try_into(self, ty: &Type) -> Result, Error> { - self.0.try_into(ty) - } - } - - #[derive(SerializeDerive, Clone, Debug)] - #[serde(untagged)] - enum RawLiteralEnum { - Null, - Boolean(bool), - Int(i32), - Long(i64), - Float(f32), - Double(f64), - String(String), - Bytes(ByteBuf), - List(List), - StringMap(StringMap), - Record(Record), - } - - #[derive(Clone, Debug)] - struct Record { - required: Vec<(String, RawLiteralEnum)>, - optional: Vec<(String, Option)>, - } - - impl Serialize for Record { - fn serialize(&self, serializer: S) -> Result - where - S: serde::Serializer, - { - let len = self.required.len() + self.optional.len(); - let mut record = serializer.serialize_struct("", len)?; - for (k, v) in &self.required { - record.serialize_field(Box::leak(k.clone().into_boxed_str()), &v)?; - } - for (k, v) in &self.optional { - record.serialize_field(Box::leak(k.clone().into_boxed_str()), &v)?; - } - record.end() - } - } - - #[derive(Clone, Debug)] - struct List { - list: Vec>, - required: bool, - } - - impl Serialize for List { - fn serialize(&self, serializer: S) -> Result - where - S: serde::Serializer, - { - let mut seq = serializer.serialize_seq(Some(self.list.len()))?; - for value in &self.list { - if self.required { - seq.serialize_element(value.as_ref().ok_or_else(|| { - serde::ser::Error::custom( - "List element is required, element cannot be null", - ) - })?)?; - } else { - seq.serialize_element(&value)?; - } - } - seq.end() - } - } - - #[derive(Clone, Debug)] - struct StringMap { - raw: Vec<(String, Option)>, - required: bool, - } - - impl Serialize for StringMap { - fn serialize(&self, serializer: S) -> Result - where - S: serde::Serializer, - { - let mut map = serializer.serialize_map(Some(self.raw.len()))?; - for (k, v) in &self.raw { - if self.required { - map.serialize_entry( - k, - v.as_ref().ok_or_else(|| { - serde::ser::Error::custom( - "Map element is required, element cannot be null", - ) - })?, - )?; - } else { - map.serialize_entry(k, v)?; - } - } - map.end() - } - } - - impl<'de> Deserialize<'de> for RawLiteralEnum { - fn deserialize(deserializer: D) -> Result - where - D: serde::Deserializer<'de>, - { - struct RawLiteralVisitor; - impl<'de> Visitor<'de> for RawLiteralVisitor { - type Value = RawLiteralEnum; - - fn expecting(&self, formatter: &mut std::fmt::Formatter) -> std::fmt::Result { - formatter.write_str("expect") - } - - fn visit_bool(self, v: bool) -> Result - where - E: serde::de::Error, - { - Ok(RawLiteralEnum::Boolean(v)) - } - - fn visit_i32(self, v: i32) -> Result - where - E: serde::de::Error, - { - Ok(RawLiteralEnum::Int(v)) - } - - fn visit_i64(self, v: i64) -> Result - where - E: serde::de::Error, - { - Ok(RawLiteralEnum::Long(v)) - } - - /// Used in json - fn visit_u64(self, v: u64) -> Result - where - E: serde::de::Error, - { - Ok(RawLiteralEnum::Long(v as i64)) - } - - fn visit_f32(self, v: f32) -> Result - where - E: serde::de::Error, - { - Ok(RawLiteralEnum::Float(v)) - } - - fn visit_f64(self, v: f64) -> Result - where - E: serde::de::Error, - { - Ok(RawLiteralEnum::Double(v)) - } - - fn visit_str(self, v: &str) -> Result - where - E: serde::de::Error, - { - Ok(RawLiteralEnum::String(v.to_string())) - } - - fn visit_bytes(self, v: &[u8]) -> Result - where - E: serde::de::Error, - { - Ok(RawLiteralEnum::Bytes(ByteBuf::from(v))) - } - - fn visit_borrowed_str(self, v: &'de str) -> Result - where - E: serde::de::Error, - { - Ok(RawLiteralEnum::String(v.to_string())) - } - - fn visit_unit(self) -> Result - where - E: serde::de::Error, - { - Ok(RawLiteralEnum::Null) - } - - fn visit_map(self, mut map: A) -> Result - where - A: serde::de::MapAccess<'de>, - { - let mut required = Vec::new(); - while let Some(key) = map.next_key::()? { - let value = map.next_value::()?; - required.push((key, value)); - } - Ok(RawLiteralEnum::Record(Record { - required, - optional: Vec::new(), - })) - } - - fn visit_seq(self, mut seq: A) -> Result - where - A: serde::de::SeqAccess<'de>, - { - let mut list = Vec::new(); - while let Some(value) = seq.next_element::()? { - list.push(Some(value)); - } - Ok(RawLiteralEnum::List(List { - list, - // `required` only used in serialize, just set default in deserialize. - required: false, - })) - } - } - deserializer.deserialize_any(RawLiteralVisitor) - } - } - - impl RawLiteralEnum { - pub fn try_from(literal: Literal, ty: &Type) -> Result { - let raw = match literal { - Literal::Primitive(prim) => match prim { - super::PrimitiveLiteral::Boolean(v) => RawLiteralEnum::Boolean(v), - super::PrimitiveLiteral::Int(v) => RawLiteralEnum::Int(v), - super::PrimitiveLiteral::Long(v) => RawLiteralEnum::Long(v), - super::PrimitiveLiteral::Float(v) => RawLiteralEnum::Float(v.0), - super::PrimitiveLiteral::Double(v) => RawLiteralEnum::Double(v.0), - super::PrimitiveLiteral::Date(v) => RawLiteralEnum::Int(v), - super::PrimitiveLiteral::Time(v) => RawLiteralEnum::Long(v), - super::PrimitiveLiteral::Timestamp(v) => RawLiteralEnum::Long(v), - super::PrimitiveLiteral::Timestamptz(v) => RawLiteralEnum::Long(v), - super::PrimitiveLiteral::String(v) => RawLiteralEnum::String(v), - super::PrimitiveLiteral::UUID(v) => { - RawLiteralEnum::Bytes(ByteBuf::from(v.as_u128().to_be_bytes())) - } - super::PrimitiveLiteral::Fixed(v) => RawLiteralEnum::Bytes(ByteBuf::from(v)), - super::PrimitiveLiteral::Binary(v) => RawLiteralEnum::Bytes(ByteBuf::from(v)), - super::PrimitiveLiteral::Decimal(v) => { - RawLiteralEnum::Bytes(ByteBuf::from(v.to_be_bytes())) - } - }, - Literal::Struct(r#struct) => { - let mut required = Vec::new(); - let mut optional = Vec::new(); - if let Type::Struct(struct_ty) = ty { - for (value, field) in r#struct.into_iter().zip(struct_ty.fields()) { - if field.required { - if let Some(value) = value { - required.push(( - field.name.clone(), - RawLiteralEnum::try_from(value, &field.field_type)?, - )); - } else { - return Err(Error::new( - ErrorKind::DataInvalid, - "Can't convert null to required field", - )); - } - } else if let Some(value) = value { - optional.push(( - field.name.clone(), - Some(RawLiteralEnum::try_from(value, &field.field_type)?), - )); - } else { - optional.push((field.name.clone(), None)); - } - } - } else { - return Err(Error::new( - ErrorKind::DataInvalid, - format!("Type {} should be a struct", ty), - )); - } - RawLiteralEnum::Record(Record { required, optional }) - } - Literal::List(list) => { - if let Type::List(list_ty) = ty { - let list = list - .into_iter() - .map(|v| { - v.map(|v| { - RawLiteralEnum::try_from(v, &list_ty.element_field.field_type) - }) - .transpose() - }) - .collect::>()?; - RawLiteralEnum::List(List { - list, - required: list_ty.element_field.required, - }) - } else { - return Err(Error::new( - ErrorKind::DataInvalid, - format!("Type {} should be a list", ty), - )); - } - } - Literal::Map(map) => { - if let Type::Map(map_ty) = ty { - if let Type::Primitive(PrimitiveType::String) = *map_ty.key_field.field_type - { - let mut raw = Vec::with_capacity(map.len()); - for (k, v) in map { - if let Literal::Primitive(PrimitiveLiteral::String(k)) = k { - raw.push(( - k, - v.map(|v| { - RawLiteralEnum::try_from( - v, - &map_ty.value_field.field_type, - ) - }) - .transpose()?, - )); - } else { - return Err(Error::new( - ErrorKind::DataInvalid, - "literal type is inconsistent with type", - )); - } - } - RawLiteralEnum::StringMap(StringMap { - raw, - required: map_ty.value_field.required, - }) - } else { - let list = map.into_iter().map(|(k,v)| { - let raw_k = - RawLiteralEnum::try_from(k, &map_ty.key_field.field_type)?; - let raw_v = v - .map(|v| { - RawLiteralEnum::try_from(v, &map_ty.value_field.field_type) - }) - .transpose()?; - if map_ty.value_field.required { - Ok(Some(RawLiteralEnum::Record(Record { - required: vec![ - (MAP_KEY_FIELD_NAME.to_string(), raw_k), - (MAP_VALUE_FIELD_NAME.to_string(), raw_v.ok_or_else(||Error::new(ErrorKind::DataInvalid, "Map value is required, value cannot be null"))?), - ], - optional: vec![], - }))) - } else { - Ok(Some(RawLiteralEnum::Record(Record { - required: vec![ - (MAP_KEY_FIELD_NAME.to_string(), raw_k), - ], - optional: vec![ - (MAP_VALUE_FIELD_NAME.to_string(), raw_v) - ], - }))) - } - }).collect::>()?; - RawLiteralEnum::List(List { - list, - required: true, - }) - } - } else { - return Err(Error::new( - ErrorKind::DataInvalid, - format!("Type {} should be a map", ty), - )); - } - } - }; - Ok(raw) - } - - pub fn try_into(self, ty: &Type) -> Result, Error> { - let invalid_err = |v: &str| { - Error::new( - ErrorKind::DataInvalid, - format!( - "Unable to convert raw literal ({}) fail convert to type {} for: type mismatch", - v, ty - ), - ) - }; - let invalid_err_with_reason = |v: &str, reason: &str| { - Error::new( - ErrorKind::DataInvalid, - format!( - "Unable to convert raw literal ({}) fail convert to type {} for: {}", - v, ty, reason - ), - ) - }; - match self { - RawLiteralEnum::Null => Ok(None), - RawLiteralEnum::Boolean(v) => Ok(Some(Literal::bool(v))), - RawLiteralEnum::Int(v) => match ty { - Type::Primitive(PrimitiveType::Int) => Ok(Some(Literal::int(v))), - Type::Primitive(PrimitiveType::Long) => Ok(Some(Literal::long(i64::from(v)))), - Type::Primitive(PrimitiveType::Date) => Ok(Some(Literal::date(v))), - _ => Err(invalid_err("int")), - }, - RawLiteralEnum::Long(v) => match ty { - Type::Primitive(PrimitiveType::Int) => Ok(Some(Literal::int( - i32::try_from(v).map_err(|_| invalid_err("long"))?, - ))), - Type::Primitive(PrimitiveType::Date) => Ok(Some(Literal::date( - i32::try_from(v).map_err(|_| invalid_err("long"))?, - ))), - Type::Primitive(PrimitiveType::Long) => Ok(Some(Literal::long(v))), - Type::Primitive(PrimitiveType::Time) => Ok(Some(Literal::time(v))), - Type::Primitive(PrimitiveType::Timestamp) => Ok(Some(Literal::timestamp(v))), - Type::Primitive(PrimitiveType::Timestamptz) => { - Ok(Some(Literal::timestamptz(v))) - } - _ => Err(invalid_err("long")), - }, - RawLiteralEnum::Float(v) => match ty { - Type::Primitive(PrimitiveType::Float) => Ok(Some(Literal::float(v))), - Type::Primitive(PrimitiveType::Double) => { - Ok(Some(Literal::double(f64::from(v)))) - } - _ => Err(invalid_err("float")), - }, - RawLiteralEnum::Double(v) => match ty { - Type::Primitive(PrimitiveType::Float) => { - let v_32 = v as f32; - if v_32.is_finite() { - let v_64 = f64::from(v_32); - if (v_64 - v).abs() > f32::EPSILON as f64 { - // there is a precision loss - return Err(invalid_err("double")); - } - } - Ok(Some(Literal::float(v_32))) - } - Type::Primitive(PrimitiveType::Double) => Ok(Some(Literal::double(v))), - _ => Err(invalid_err("double")), - }, - RawLiteralEnum::String(v) => match ty { - Type::Primitive(PrimitiveType::String) => Ok(Some(Literal::string(v))), - _ => Err(invalid_err("string")), - }, - // # TODO:https://github.com/apache/iceberg-rust/issues/86 - // rust avro don't support deserialize any bytes representation now. - RawLiteralEnum::Bytes(_) => Err(invalid_err_with_reason( - "bytes", - "todo: rust avro doesn't support deserialize any bytes representation now", - )), - RawLiteralEnum::List(v) => { - match ty { - Type::List(ty) => Ok(Some(Literal::List( - v.list - .into_iter() - .map(|v| { - if let Some(v) = v { - v.try_into(&ty.element_field.field_type) - } else { - Ok(None) - } - }) - .collect::>()?, - ))), - Type::Map(map_ty) => { - let key_ty = map_ty.key_field.field_type.as_ref(); - let value_ty = map_ty.value_field.field_type.as_ref(); - let mut map = Map::new(); - for k_v in v.list { - let k_v = k_v.ok_or_else(|| invalid_err_with_reason("list","In deserialize, None will be represented as Some(RawLiteral::Null), all element in list must be valid"))?; - if let RawLiteralEnum::Record(Record { - required, - optional: _, - }) = k_v - { - if required.len() != 2 { - return Err(invalid_err_with_reason("list","Record must contains two element(key and value) of array")); - } - let mut key = None; - let mut value = None; - required.into_iter().for_each(|(k, v)| { - if k == MAP_KEY_FIELD_NAME { - key = Some(v); - } else if k == MAP_VALUE_FIELD_NAME { - value = Some(v); - } - }); - match (key, value) { - (Some(k), Some(v)) => { - let key = k.try_into(key_ty)?.ok_or_else(|| { - invalid_err_with_reason( - "list", - "Key element in Map must be valid", - ) - })?; - let value = v.try_into(value_ty)?; - if map_ty.value_field.required && value.is_none() { - return Err(invalid_err_with_reason( - "list", - "Value element is required in this Map", - )); - } - map.insert(key, value); - } - _ => return Err(invalid_err_with_reason( - "list", - "The elements of record in list are not key and value", - )), - } - } else { - return Err(invalid_err_with_reason( - "list", - "Map should represented as record array.", - )); - } - } - Ok(Some(Literal::Map(map))) - } - Type::Primitive(PrimitiveType::Uuid) => { - if v.list.len() != 16 { - return Err(invalid_err_with_reason( - "list", - "The length of list should be 16", - )); - } - let mut bytes = [0u8; 16]; - for (i, v) in v.list.iter().enumerate() { - if let Some(RawLiteralEnum::Long(v)) = v { - bytes[i] = *v as u8; - } else { - return Err(invalid_err_with_reason( - "list", - "The element of list should be int", - )); - } - } - Ok(Some(Literal::uuid(uuid::Uuid::from_bytes(bytes)))) - } - Type::Primitive(PrimitiveType::Decimal { - precision: _, - scale: _, - }) => { - if v.list.len() != 16 { - return Err(invalid_err_with_reason( - "list", - "The length of list should be 16", - )); - } - let mut bytes = [0u8; 16]; - for (i, v) in v.list.iter().enumerate() { - if let Some(RawLiteralEnum::Long(v)) = v { - bytes[i] = *v as u8; - } else { - return Err(invalid_err_with_reason( - "list", - "The element of list should be int", - )); - } - } - Ok(Some(Literal::decimal(i128::from_be_bytes(bytes)))) - } - Type::Primitive(PrimitiveType::Binary) => { - let bytes = v - .list - .into_iter() - .map(|v| { - if let Some(RawLiteralEnum::Long(v)) = v { - Ok(v as u8) - } else { - Err(invalid_err_with_reason( - "list", - "The element of list should be int", - )) - } - }) - .collect::, Error>>()?; - Ok(Some(Literal::binary(bytes))) - } - Type::Primitive(PrimitiveType::Fixed(size)) => { - if v.list.len() != *size as usize { - return Err(invalid_err_with_reason( - "list", - "The length of list should be equal to size", - )); - } - let bytes = v - .list - .into_iter() - .map(|v| { - if let Some(RawLiteralEnum::Long(v)) = v { - Ok(v as u8) - } else { - Err(invalid_err_with_reason( - "list", - "The element of list should be int", - )) - } - }) - .collect::, Error>>()?; - Ok(Some(Literal::fixed(bytes))) - } - _ => Err(invalid_err("list")), - } - } - RawLiteralEnum::Record(Record { - required, - optional: _, - }) => match ty { - Type::Struct(struct_ty) => { - let iters: Vec> = required - .into_iter() - .map(|(field_name, value)| { - let field = struct_ty - .field_by_name(field_name.as_str()) - .ok_or_else(|| { - invalid_err_with_reason( - "record", - &format!("field {} is not exist", &field_name), - ) - })?; - let value = value.try_into(&field.field_type)?; - Ok(value) - }) - .collect::>()?; - Ok(Some(Literal::Struct(super::Struct::from_iter(iters)))) - } - Type::Map(map_ty) => { - if *map_ty.key_field.field_type != Type::Primitive(PrimitiveType::String) { - return Err(invalid_err_with_reason( - "record", - "Map key must be string", - )); - } - let mut map = Map::new(); - for (k, v) in required { - let value = v.try_into(&map_ty.value_field.field_type)?; - if map_ty.value_field.required && value.is_none() { - return Err(invalid_err_with_reason( - "record", - "Value element is required in this Map", - )); - } - map.insert(Literal::string(k), value); - } - Ok(Some(Literal::Map(map))) - } - _ => Err(invalid_err("record")), - }, - RawLiteralEnum::StringMap(_) => Err(invalid_err("string map")), - } - } - } - } - - #[cfg(test)] - mod tests { - use apache_avro::{to_value, types::Value}; - - use crate::{ - avro::schema_to_avro_schema, - spec::{ - datatypes::{ListType, MapType, NestedField, StructType}, - Schema, - }, - }; - - use super::*; - - fn check_json_serde(json: &str, expected_literal: Literal, expected_type: &Type) { - let raw_json_value = serde_json::from_str::(json).unwrap(); - let desered_literal = - Literal::try_from_json(raw_json_value.clone(), expected_type).unwrap(); - assert_eq!(desered_literal, Some(expected_literal.clone())); - - let expected_json_value: JsonValue = expected_literal.try_into_json(expected_type).unwrap(); - let sered_json = serde_json::to_string(&expected_json_value).unwrap(); - let parsed_json_value = serde_json::from_str::(&sered_json).unwrap(); - - assert_eq!(parsed_json_value, raw_json_value); - } - - fn check_avro_bytes_serde(input: Vec, expected_literal: Literal, expected_type: &Type) { - let raw_schema = r#""bytes""#; - let schema = apache_avro::Schema::parse_str(raw_schema).unwrap(); - - let bytes = ByteBuf::from(input); - let literal = Literal::try_from_bytes(&bytes, expected_type).unwrap(); - assert_eq!(literal, expected_literal); - - let mut writer = apache_avro::Writer::new(&schema, Vec::new()); - writer.append_ser(ByteBuf::from(literal)).unwrap(); - let encoded = writer.into_inner().unwrap(); - let reader = apache_avro::Reader::with_schema(&schema, &*encoded).unwrap(); - - for record in reader { - let result = apache_avro::from_value::(&record.unwrap()).unwrap(); - let desered_literal = Literal::try_from_bytes(&result, expected_type).unwrap(); - assert_eq!(desered_literal, expected_literal); - } - } - - fn check_convert_with_avro(expected_literal: Literal, expected_type: &Type) { - let fields = vec![NestedField::required(1, "col", expected_type.clone()).into()]; - let schema = Schema::builder() - .with_fields(fields.clone()) - .build() - .unwrap(); - let avro_schema = schema_to_avro_schema("test", &schema).unwrap(); - let struct_type = Type::Struct(StructType::new(fields)); - let struct_literal = - Literal::Struct(Struct::from_iter(vec![Some(expected_literal.clone())])); - - let mut writer = apache_avro::Writer::new(&avro_schema, Vec::new()); - let raw_literal = RawLiteral::try_from(struct_literal.clone(), &struct_type).unwrap(); - writer.append_ser(raw_literal).unwrap(); - let encoded = writer.into_inner().unwrap(); - - let reader = apache_avro::Reader::new(&*encoded).unwrap(); - for record in reader { - let result = apache_avro::from_value::(&record.unwrap()).unwrap(); - let desered_literal = result.try_into(&struct_type).unwrap().unwrap(); - assert_eq!(desered_literal, struct_literal); - } - } - - fn check_serialize_avro(literal: Literal, ty: &Type, expect_value: Value) { - let expect_value = Value::Record(vec![("col".to_string(), expect_value)]); - - let fields = vec![NestedField::required(1, "col", ty.clone()).into()]; - let schema = Schema::builder() - .with_fields(fields.clone()) - .build() - .unwrap(); - let avro_schema = schema_to_avro_schema("test", &schema).unwrap(); - let struct_type = Type::Struct(StructType::new(fields)); - let struct_literal = Literal::Struct(Struct::from_iter(vec![Some(literal.clone())])); - let mut writer = apache_avro::Writer::new(&avro_schema, Vec::new()); - let raw_literal = RawLiteral::try_from(struct_literal.clone(), &struct_type).unwrap(); - let value = to_value(raw_literal) - .unwrap() - .resolve(&avro_schema) - .unwrap(); - writer.append_value_ref(&value).unwrap(); - let encoded = writer.into_inner().unwrap(); - - let reader = apache_avro::Reader::new(&*encoded).unwrap(); - for record in reader { - assert_eq!(record.unwrap(), expect_value); - } - } - - #[test] - fn json_boolean() { - let record = r#"true"#; - - check_json_serde( - record, - Literal::Primitive(PrimitiveLiteral::Boolean(true)), - &Type::Primitive(PrimitiveType::Boolean), - ); - } - - #[test] - fn json_int() { - let record = r#"32"#; - - check_json_serde( - record, - Literal::Primitive(PrimitiveLiteral::Int(32)), - &Type::Primitive(PrimitiveType::Int), - ); - } - - #[test] - fn json_long() { - let record = r#"32"#; - - check_json_serde( - record, - Literal::Primitive(PrimitiveLiteral::Long(32)), - &Type::Primitive(PrimitiveType::Long), - ); - } - - #[test] - fn json_float() { - let record = r#"1.0"#; - - check_json_serde( - record, - Literal::Primitive(PrimitiveLiteral::Float(OrderedFloat(1.0))), - &Type::Primitive(PrimitiveType::Float), - ); - } - - #[test] - fn json_double() { - let record = r#"1.0"#; - - check_json_serde( - record, - Literal::Primitive(PrimitiveLiteral::Double(OrderedFloat(1.0))), - &Type::Primitive(PrimitiveType::Double), - ); - } - - #[test] - fn json_date() { - let record = r#""2017-11-16""#; - - check_json_serde( - record, - Literal::Primitive(PrimitiveLiteral::Date(17486)), - &Type::Primitive(PrimitiveType::Date), - ); - } - - #[test] - fn json_time() { - let record = r#""22:31:08.123456""#; - - check_json_serde( - record, - Literal::Primitive(PrimitiveLiteral::Time(81068123456)), - &Type::Primitive(PrimitiveType::Time), - ); - } - - #[test] - fn json_timestamp() { - let record = r#""2017-11-16T22:31:08.123456""#; - - check_json_serde( - record, - Literal::Primitive(PrimitiveLiteral::Timestamp(1510871468123456)), - &Type::Primitive(PrimitiveType::Timestamp), - ); - } - - #[test] - fn json_timestamptz() { - let record = r#""2017-11-16T22:31:08.123456+00:00""#; - - check_json_serde( - record, - Literal::Primitive(PrimitiveLiteral::Timestamptz(1510871468123456)), - &Type::Primitive(PrimitiveType::Timestamptz), - ); - } - - #[test] - fn json_string() { - let record = r#""iceberg""#; - - check_json_serde( - record, - Literal::Primitive(PrimitiveLiteral::String("iceberg".to_string())), - &Type::Primitive(PrimitiveType::String), - ); - } - - #[test] - fn json_uuid() { - let record = r#""f79c3e09-677c-4bbd-a479-3f349cb785e7""#; - - check_json_serde( - record, - Literal::Primitive(PrimitiveLiteral::UUID( - Uuid::parse_str("f79c3e09-677c-4bbd-a479-3f349cb785e7").unwrap(), - )), - &Type::Primitive(PrimitiveType::Uuid), - ); - } - - #[test] - fn json_decimal() { - let record = r#""14.20""#; - - check_json_serde( - record, - Literal::Primitive(PrimitiveLiteral::Decimal(1420)), - &Type::decimal(28, 2).unwrap(), - ); - } - - #[test] - fn json_struct() { - let record = r#"{"1": 1, "2": "bar", "3": null}"#; - - check_json_serde( - record, - Literal::Struct(Struct::from_iter(vec![ - Some(Literal::Primitive(PrimitiveLiteral::Int(1))), - Some(Literal::Primitive(PrimitiveLiteral::String( - "bar".to_string(), - ))), - None, - ])), - &Type::Struct(StructType::new(vec![ - NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), - NestedField::optional(2, "name", Type::Primitive(PrimitiveType::String)).into(), - NestedField::optional(3, "address", Type::Primitive(PrimitiveType::String)).into(), - ])), - ); - } - - #[test] - fn json_list() { - let record = r#"[1, 2, 3, null]"#; - - check_json_serde( - record, - Literal::List(vec![ - Some(Literal::Primitive(PrimitiveLiteral::Int(1))), - Some(Literal::Primitive(PrimitiveLiteral::Int(2))), - Some(Literal::Primitive(PrimitiveLiteral::Int(3))), - None, - ]), - &Type::List(ListType { - element_field: NestedField::list_element( - 0, - Type::Primitive(PrimitiveType::Int), - true, - ) - .into(), - }), - ); - } - - #[test] - fn json_map() { - let record = r#"{ "keys": ["a", "b", "c"], "values": [1, 2, null] }"#; - - check_json_serde( - record, - Literal::Map(Map::from([ - ( - Literal::Primitive(PrimitiveLiteral::String("a".to_string())), - Some(Literal::Primitive(PrimitiveLiteral::Int(1))), - ), - ( - Literal::Primitive(PrimitiveLiteral::String("b".to_string())), - Some(Literal::Primitive(PrimitiveLiteral::Int(2))), - ), - ( - Literal::Primitive(PrimitiveLiteral::String("c".to_string())), - None, - ), - ])), - &Type::Map(MapType { - key_field: NestedField::map_key_element(0, Type::Primitive(PrimitiveType::String)) - .into(), - value_field: NestedField::map_value_element( - 1, - Type::Primitive(PrimitiveType::Int), - true, - ) - .into(), - }), - ); - } - - #[test] - fn avro_bytes_boolean() { - let bytes = vec![1u8]; - - check_avro_bytes_serde( - bytes, - Literal::Primitive(PrimitiveLiteral::Boolean(true)), - &Type::Primitive(PrimitiveType::Boolean), - ); - } - - #[test] - fn avro_bytes_int() { - let bytes = vec![32u8, 0u8, 0u8, 0u8]; - - check_avro_bytes_serde( - bytes, - Literal::Primitive(PrimitiveLiteral::Int(32)), - &Type::Primitive(PrimitiveType::Int), - ); - } - - #[test] - fn avro_bytes_long() { - let bytes = vec![32u8, 0u8, 0u8, 0u8, 0u8, 0u8, 0u8, 0u8]; - - check_avro_bytes_serde( - bytes, - Literal::Primitive(PrimitiveLiteral::Long(32)), - &Type::Primitive(PrimitiveType::Long), - ); - } - - #[test] - fn avro_bytes_float() { - let bytes = vec![0u8, 0u8, 128u8, 63u8]; - - check_avro_bytes_serde( - bytes, - Literal::Primitive(PrimitiveLiteral::Float(OrderedFloat(1.0))), - &Type::Primitive(PrimitiveType::Float), - ); - } - - #[test] - fn avro_bytes_double() { - let bytes = vec![0u8, 0u8, 0u8, 0u8, 0u8, 0u8, 240u8, 63u8]; - - check_avro_bytes_serde( - bytes, - Literal::Primitive(PrimitiveLiteral::Double(OrderedFloat(1.0))), - &Type::Primitive(PrimitiveType::Double), - ); - } - - #[test] - fn avro_bytes_string() { - let bytes = vec![105u8, 99u8, 101u8, 98u8, 101u8, 114u8, 103u8]; - - check_avro_bytes_serde( - bytes, - Literal::Primitive(PrimitiveLiteral::String("iceberg".to_string())), - &Type::Primitive(PrimitiveType::String), - ); - } - - #[test] - fn avro_convert_test_int() { - check_convert_with_avro( - Literal::Primitive(PrimitiveLiteral::Int(32)), - &Type::Primitive(PrimitiveType::Int), - ); - } - - #[test] - fn avro_convert_test_long() { - check_convert_with_avro( - Literal::Primitive(PrimitiveLiteral::Long(32)), - &Type::Primitive(PrimitiveType::Long), - ); - } - - #[test] - fn avro_convert_test_float() { - check_convert_with_avro( - Literal::Primitive(PrimitiveLiteral::Float(OrderedFloat(1.0))), - &Type::Primitive(PrimitiveType::Float), - ); - } - - #[test] - fn avro_convert_test_double() { - check_convert_with_avro( - Literal::Primitive(PrimitiveLiteral::Double(OrderedFloat(1.0))), - &Type::Primitive(PrimitiveType::Double), - ); - } - - #[test] - fn avro_convert_test_string() { - check_convert_with_avro( - Literal::Primitive(PrimitiveLiteral::String("iceberg".to_string())), - &Type::Primitive(PrimitiveType::String), - ); - } - - #[test] - fn avro_convert_test_date() { - check_convert_with_avro( - Literal::Primitive(PrimitiveLiteral::Date(17486)), - &Type::Primitive(PrimitiveType::Date), - ); - } - - #[test] - fn avro_convert_test_time() { - check_convert_with_avro( - Literal::Primitive(PrimitiveLiteral::Time(81068123456)), - &Type::Primitive(PrimitiveType::Time), - ); - } - - #[test] - fn avro_convert_test_timestamp() { - check_convert_with_avro( - Literal::Primitive(PrimitiveLiteral::Timestamp(1510871468123456)), - &Type::Primitive(PrimitiveType::Timestamp), - ); - } - - #[test] - fn avro_convert_test_timestamptz() { - check_convert_with_avro( - Literal::Primitive(PrimitiveLiteral::Timestamptz(1510871468123456)), - &Type::Primitive(PrimitiveType::Timestamptz), - ); - } - - #[test] - fn avro_convert_test_list() { - check_convert_with_avro( - Literal::List(vec![ - Some(Literal::Primitive(PrimitiveLiteral::Int(1))), - Some(Literal::Primitive(PrimitiveLiteral::Int(2))), - Some(Literal::Primitive(PrimitiveLiteral::Int(3))), - None, - ]), - &Type::List(ListType { - element_field: NestedField::list_element( - 0, - Type::Primitive(PrimitiveType::Int), - false, - ) - .into(), - }), - ); - - check_convert_with_avro( - Literal::List(vec![ - Some(Literal::Primitive(PrimitiveLiteral::Int(1))), - Some(Literal::Primitive(PrimitiveLiteral::Int(2))), - Some(Literal::Primitive(PrimitiveLiteral::Int(3))), - ]), - &Type::List(ListType { - element_field: NestedField::list_element( - 0, - Type::Primitive(PrimitiveType::Int), - true, - ) - .into(), - }), - ); - } - - fn check_convert_with_avro_map(expected_literal: Literal, expected_type: &Type) { - let fields = vec![NestedField::required(1, "col", expected_type.clone()).into()]; - let schema = Schema::builder() - .with_fields(fields.clone()) - .build() - .unwrap(); - let avro_schema = schema_to_avro_schema("test", &schema).unwrap(); - let struct_type = Type::Struct(StructType::new(fields)); - let struct_literal = - Literal::Struct(Struct::from_iter(vec![Some(expected_literal.clone())])); - - let mut writer = apache_avro::Writer::new(&avro_schema, Vec::new()); - let raw_literal = RawLiteral::try_from(struct_literal.clone(), &struct_type).unwrap(); - writer.append_ser(raw_literal).unwrap(); - let encoded = writer.into_inner().unwrap(); - - let reader = apache_avro::Reader::new(&*encoded).unwrap(); - for record in reader { - let result = apache_avro::from_value::(&record.unwrap()).unwrap(); - let desered_literal = result.try_into(&struct_type).unwrap().unwrap(); - match (&desered_literal, &struct_literal) { - (Literal::Struct(desered), Literal::Struct(expected)) => { - match (&desered.fields[0], &expected.fields[0]) { - (Literal::Map(desered), Literal::Map(expected)) => { - assert!(desered.has_same_content(expected)) - } - _ => { - unreachable!() - } - } - } - _ => { - panic!("unexpected literal type"); - } - } - } - } - - #[test] - fn avro_convert_test_map() { - check_convert_with_avro_map( - Literal::Map(Map::from([ - ( - Literal::Primitive(PrimitiveLiteral::Int(1)), - Some(Literal::Primitive(PrimitiveLiteral::Long(1))), - ), - ( - Literal::Primitive(PrimitiveLiteral::Int(2)), - Some(Literal::Primitive(PrimitiveLiteral::Long(2))), - ), - (Literal::Primitive(PrimitiveLiteral::Int(3)), None), - ])), - &Type::Map(MapType { - key_field: NestedField::map_key_element(0, Type::Primitive(PrimitiveType::Int)) - .into(), - value_field: NestedField::map_value_element( - 1, - Type::Primitive(PrimitiveType::Long), - false, - ) - .into(), - }), - ); - - check_convert_with_avro_map( - Literal::Map(Map::from([ - ( - Literal::Primitive(PrimitiveLiteral::Int(1)), - Some(Literal::Primitive(PrimitiveLiteral::Long(1))), - ), - ( - Literal::Primitive(PrimitiveLiteral::Int(2)), - Some(Literal::Primitive(PrimitiveLiteral::Long(2))), - ), - ( - Literal::Primitive(PrimitiveLiteral::Int(3)), - Some(Literal::Primitive(PrimitiveLiteral::Long(3))), - ), - ])), - &Type::Map(MapType { - key_field: NestedField::map_key_element(0, Type::Primitive(PrimitiveType::Int)) - .into(), - value_field: NestedField::map_value_element( - 1, - Type::Primitive(PrimitiveType::Long), - true, - ) - .into(), - }), - ); - } - - #[test] - fn avro_convert_test_string_map() { - check_convert_with_avro_map( - Literal::Map(Map::from([ - ( - Literal::Primitive(PrimitiveLiteral::String("a".to_string())), - Some(Literal::Primitive(PrimitiveLiteral::Int(1))), - ), - ( - Literal::Primitive(PrimitiveLiteral::String("b".to_string())), - Some(Literal::Primitive(PrimitiveLiteral::Int(2))), - ), - ( - Literal::Primitive(PrimitiveLiteral::String("c".to_string())), - None, - ), - ])), - &Type::Map(MapType { - key_field: NestedField::map_key_element(0, Type::Primitive(PrimitiveType::String)) - .into(), - value_field: NestedField::map_value_element( - 1, - Type::Primitive(PrimitiveType::Int), - false, - ) - .into(), - }), - ); - - check_convert_with_avro_map( - Literal::Map(Map::from([ - ( - Literal::Primitive(PrimitiveLiteral::String("a".to_string())), - Some(Literal::Primitive(PrimitiveLiteral::Int(1))), - ), - ( - Literal::Primitive(PrimitiveLiteral::String("b".to_string())), - Some(Literal::Primitive(PrimitiveLiteral::Int(2))), - ), - ( - Literal::Primitive(PrimitiveLiteral::String("c".to_string())), - Some(Literal::Primitive(PrimitiveLiteral::Int(3))), - ), - ])), - &Type::Map(MapType { - key_field: NestedField::map_key_element(0, Type::Primitive(PrimitiveType::String)) - .into(), - value_field: NestedField::map_value_element( - 1, - Type::Primitive(PrimitiveType::Int), - true, - ) - .into(), - }), - ); - } - - #[test] - fn avro_convert_test_record() { - check_convert_with_avro( - Literal::Struct(Struct::from_iter(vec![ - Some(Literal::Primitive(PrimitiveLiteral::Int(1))), - Some(Literal::Primitive(PrimitiveLiteral::String( - "bar".to_string(), - ))), - None, - ])), - &Type::Struct(StructType::new(vec![ - NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), - NestedField::optional(2, "name", Type::Primitive(PrimitiveType::String)).into(), - NestedField::optional(3, "address", Type::Primitive(PrimitiveType::String)).into(), - ])), - ); - } - - // # TODO:https://github.com/apache/iceberg-rust/issues/86 - // rust avro don't support deserialize any bytes representation now: - // - binary - // - decimal - #[test] - fn avro_convert_test_binary_ser() { - let literal = Literal::Primitive(PrimitiveLiteral::Binary(vec![1, 2, 3, 4, 5])); - let ty = Type::Primitive(PrimitiveType::Binary); - let expect_value = Value::Bytes(vec![1, 2, 3, 4, 5]); - check_serialize_avro(literal, &ty, expect_value); - } - - #[test] - fn avro_convert_test_decimal_ser() { - let literal = Literal::decimal(12345); - let ty = Type::Primitive(PrimitiveType::Decimal { - precision: 9, - scale: 8, - }); - let expect_value = Value::Decimal(apache_avro::Decimal::from(12345_i128.to_be_bytes())); - check_serialize_avro(literal, &ty, expect_value); - } - - // # TODO:https://github.com/apache/iceberg-rust/issues/86 - // rust avro can't support to convert any byte-like type to fixed in avro now. - // - uuid ser/de - // - fixed ser/de - - #[test] - fn test_parse_timestamp() { - let value = Datum::timestamp_from_str("2021-08-01T01:09:00.0899").unwrap(); - assert_eq!(&format!("{value}"), "2021-08-01 01:09:00.089900"); - - let value = Datum::timestamp_from_str("2021-08-01T01:09:00.0899+0800"); - assert!(value.is_err(), "Parse timestamp with timezone should fail!"); - - let value = Datum::timestamp_from_str("dfa"); - assert!( - value.is_err(), - "Parse timestamp with invalid input should fail!" - ); - } - - #[test] - fn test_parse_timestamptz() { - let value = Datum::timestamptz_from_str("2021-08-01T09:09:00.0899+0800").unwrap(); - assert_eq!(&format!("{value}"), "2021-08-01 01:09:00.089900 UTC"); - - let value = Datum::timestamptz_from_str("2021-08-01T01:09:00.0899"); - assert!( - value.is_err(), - "Parse timestamptz without timezone should fail!" - ); - - let value = Datum::timestamptz_from_str("dfa"); - assert!( - value.is_err(), - "Parse timestamptz with invalid input should fail!" - ); - } - - #[test] - fn test_datum_ser_deser() { - let test_fn = |datum: Datum| { - let json = serde_json::to_value(&datum).unwrap(); - let desered_datum: Datum = serde_json::from_value(json).unwrap(); - assert_eq!(datum, desered_datum); - }; - let datum = Datum::int(1); - test_fn(datum); - let datum = Datum::long(1); - test_fn(datum); - - let datum = Datum::float(1.0); - test_fn(datum); - let datum = Datum::float(0_f32); - test_fn(datum); - let datum = Datum::float(-0_f32); - test_fn(datum); - let datum = Datum::float(f32::MAX); - test_fn(datum); - let datum = Datum::float(f32::MIN); - test_fn(datum); - - // serde_json can't serialize f32::INFINITY, f32::NEG_INFINITY, f32::NAN - let datum = Datum::float(f32::INFINITY); - let json = serde_json::to_string(&datum).unwrap(); - assert!(serde_json::from_str::(&json).is_err()); - let datum = Datum::float(f32::NEG_INFINITY); - let json = serde_json::to_string(&datum).unwrap(); - assert!(serde_json::from_str::(&json).is_err()); - let datum = Datum::float(f32::NAN); - let json = serde_json::to_string(&datum).unwrap(); - assert!(serde_json::from_str::(&json).is_err()); - - let datum = Datum::double(1.0); - test_fn(datum); - let datum = Datum::double(f64::MAX); - test_fn(datum); - let datum = Datum::double(f64::MIN); - test_fn(datum); - - // serde_json can't serialize f32::INFINITY, f32::NEG_INFINITY, f32::NAN - let datum = Datum::double(f64::INFINITY); - let json = serde_json::to_string(&datum).unwrap(); - assert!(serde_json::from_str::(&json).is_err()); - let datum = Datum::double(f64::NEG_INFINITY); - let json = serde_json::to_string(&datum).unwrap(); - assert!(serde_json::from_str::(&json).is_err()); - let datum = Datum::double(f64::NAN); - let json = serde_json::to_string(&datum).unwrap(); - assert!(serde_json::from_str::(&json).is_err()); - - let datum = Datum::string("iceberg"); - test_fn(datum); - let datum = Datum::bool(true); - test_fn(datum); - let datum = Datum::date(17486); - test_fn(datum); - let datum = Datum::time_from_hms_micro(22, 15, 33, 111).unwrap(); - test_fn(datum); - let datum = Datum::timestamp_micros(1510871468123456); - test_fn(datum); - let datum = Datum::timestamptz_micros(1510871468123456); - test_fn(datum); - let datum = Datum::uuid(Uuid::parse_str("f79c3e09-677c-4bbd-a479-3f349cb785e7").unwrap()); - test_fn(datum); - let datum = Datum::decimal(1420).unwrap(); - test_fn(datum); - let datum = Datum::binary(vec![1, 2, 3, 4, 5]); - test_fn(datum); - let datum = Datum::fixed(vec![1, 2, 3, 4, 5]); - test_fn(datum); - } - } \ No newline at end of file +use std::any::Any; +use std::collections::HashMap; +use std::fmt::{Display, Formatter}; +use std::hash::Hash; +use std::ops::Index; +use std::str::FromStr; + +use bitvec::vec::BitVec; +use chrono::{DateTime, NaiveDate, NaiveDateTime, NaiveTime, TimeZone, Utc}; +use ordered_float::OrderedFloat; +use rust_decimal::Decimal; +use serde::de::{self, MapAccess}; +use serde::ser::SerializeStruct; +use serde::{Deserialize, Serialize}; +use serde_bytes::ByteBuf; +use serde_json::{Map as JsonMap, Number, Value as JsonValue}; +use uuid::Uuid; + +pub use _serde::RawLiteral; + +use crate::error::Result; +use crate::spec::values::date::{date_from_naive_date, days_to_date, unix_epoch}; +use crate::spec::values::time::microseconds_to_time; +use crate::spec::values::timestamp::microseconds_to_datetime; +use crate::spec::values::timestamptz::microseconds_to_datetimetz; +use crate::spec::MAX_DECIMAL_PRECISION; +use crate::{ensure_data_valid, Error, ErrorKind}; + +use super::datatypes::{PrimitiveType, Type}; + +/// Maximum value for [`PrimitiveType::Time`] type in microseconds, e.g. 23 hours 59 minutes 59 seconds 999999 microseconds. +const MAX_TIME_VALUE: i64 = 24 * 60 * 60 * 1_000_000i64 - 1; + +/// Values present in iceberg type +#[derive(Clone, Debug, PartialEq, Hash, Eq)] +pub enum PrimitiveLiteral { + /// 0x00 for false, non-zero byte for true + Boolean(bool), + /// Stored as 4-byte little-endian + Int(i32), + /// Stored as 8-byte little-endian + Long(i64), + /// Stored as 4-byte little-endian + Float(OrderedFloat), + /// Stored as 8-byte little-endian + Double(OrderedFloat), + /// Stores days from the 1970-01-01 in an 4-byte little-endian int + Date(i32), + /// Stores microseconds from midnight in an 8-byte little-endian long + Time(i64), + /// Timestamp without timezone + Timestamp(i64), + /// Timestamp with timezone + Timestamptz(i64), + /// UTF-8 bytes (without length) + String(String), + /// 16-byte big-endian value + UUID(Uuid), + /// Binary value + Fixed(Vec), + /// Binary value (without length) + Binary(Vec), + /// Stores unscaled value as big int. According to iceberg spec, the precision must less than 38(`MAX_DECIMAL_PRECISION`) , so i128 is suit here. + Decimal(i128), +} + +impl PrimitiveLiteral { + /// Returns true if the Literal represents a primitive type + /// that can be a NaN, and that it's value is NaN + pub fn is_nan(&self) -> bool { + match self { + PrimitiveLiteral::Double(val) => val.is_nan(), + PrimitiveLiteral::Float(val) => val.is_nan(), + _ => false, + } + } +} + +/// Literal associated with its type. The value and type pair is checked when construction, so the type and value is +/// guaranteed to be correct when used. +/// +/// By default, we decouple the type and value of a literal, so we can use avoid the cost of storing extra type info +/// for each literal. But associate type with literal can be useful in some cases, for example, in unbound expression. +#[derive(Clone, Debug, PartialEq, Hash, Eq)] +pub struct Datum { + r#type: PrimitiveType, + literal: PrimitiveLiteral, +} + +impl Serialize for Datum { + fn serialize( + &self, + serializer: S, + ) -> std::result::Result { + let mut struct_ser = serializer + .serialize_struct("Datum", 2) + .map_err(serde::ser::Error::custom)?; + struct_ser + .serialize_field("type", &self.r#type) + .map_err(serde::ser::Error::custom)?; + struct_ser + .serialize_field( + "literal", + &RawLiteral::try_from( + Literal::Primitive(self.literal.clone()), + &Type::Primitive(self.r#type.clone()), + ) + .map_err(serde::ser::Error::custom)?, + ) + .map_err(serde::ser::Error::custom)?; + struct_ser.end() + } +} + +impl<'de> Deserialize<'de> for Datum { + fn deserialize>( + deserializer: D, + ) -> std::result::Result { + #[derive(Deserialize)] + #[serde(field_identifier, rename_all = "lowercase")] + enum Field { + Type, + Literal, + } + + struct DatumVisitor; + + impl<'de> serde::de::Visitor<'de> for DatumVisitor { + type Value = Datum; + + fn expecting(&self, formatter: &mut std::fmt::Formatter) -> std::fmt::Result { + formatter.write_str("struct Datum") + } + + fn visit_seq(self, mut seq: A) -> std::result::Result + where + A: serde::de::SeqAccess<'de>, + { + let r#type = seq + .next_element::()? + .ok_or_else(|| serde::de::Error::invalid_length(0, &self))?; + let value = seq + .next_element::()? + .ok_or_else(|| serde::de::Error::invalid_length(1, &self))?; + let Literal::Primitive(primitive) = value + .try_into(&Type::Primitive(r#type.clone())) + .map_err(serde::de::Error::custom)? + .ok_or_else(|| serde::de::Error::custom("None value"))? + else { + return Err(serde::de::Error::custom("Invalid value")); + }; + + Ok(Datum::new(r#type, primitive)) + } + + fn visit_map(self, mut map: V) -> std::result::Result + where + V: MapAccess<'de>, + { + let mut raw_primitive: Option = None; + let mut r#type: Option = None; + while let Some(key) = map.next_key()? { + match key { + Field::Type => { + if r#type.is_some() { + return Err(de::Error::duplicate_field("type")); + } + r#type = Some(map.next_value()?); + } + Field::Literal => { + if raw_primitive.is_some() { + return Err(de::Error::duplicate_field("literal")); + } + raw_primitive = Some(map.next_value()?); + } + } + } + let Some(r#type) = r#type else { + return Err(serde::de::Error::missing_field("type")); + }; + let Some(raw_primitive) = raw_primitive else { + return Err(serde::de::Error::missing_field("literal")); + }; + let Literal::Primitive(primitive) = raw_primitive + .try_into(&Type::Primitive(r#type.clone())) + .map_err(serde::de::Error::custom)? + .ok_or_else(|| serde::de::Error::custom("None value"))? + else { + return Err(serde::de::Error::custom("Invalid value")); + }; + Ok(Datum::new(r#type, primitive)) + } + } + const FIELDS: &[&str] = &["type", "literal"]; + deserializer.deserialize_struct("Datum", FIELDS, DatumVisitor) + } +} + +impl PartialOrd for Datum { + fn partial_cmp(&self, other: &Self) -> Option { + match (&self.literal, &other.literal, &self.r#type, &other.r#type) { + // generate the arm with same type and same literal + ( + PrimitiveLiteral::Boolean(val), + PrimitiveLiteral::Boolean(other_val), + PrimitiveType::Boolean, + PrimitiveType::Boolean, + ) => val.partial_cmp(other_val), + ( + PrimitiveLiteral::Int(val), + PrimitiveLiteral::Int(other_val), + PrimitiveType::Int, + PrimitiveType::Int, + ) => val.partial_cmp(other_val), + ( + PrimitiveLiteral::Long(val), + PrimitiveLiteral::Long(other_val), + PrimitiveType::Long, + PrimitiveType::Long, + ) => val.partial_cmp(other_val), + ( + PrimitiveLiteral::Float(val), + PrimitiveLiteral::Float(other_val), + PrimitiveType::Float, + PrimitiveType::Float, + ) => val.partial_cmp(other_val), + ( + PrimitiveLiteral::Double(val), + PrimitiveLiteral::Double(other_val), + PrimitiveType::Double, + PrimitiveType::Double, + ) => val.partial_cmp(other_val), + ( + PrimitiveLiteral::Date(val), + PrimitiveLiteral::Date(other_val), + PrimitiveType::Date, + PrimitiveType::Date, + ) => val.partial_cmp(other_val), + ( + PrimitiveLiteral::Time(val), + PrimitiveLiteral::Time(other_val), + PrimitiveType::Time, + PrimitiveType::Time, + ) => val.partial_cmp(other_val), + ( + PrimitiveLiteral::Timestamp(val), + PrimitiveLiteral::Timestamp(other_val), + PrimitiveType::Timestamp, + PrimitiveType::Timestamp, + ) => val.partial_cmp(other_val), + ( + PrimitiveLiteral::Timestamptz(val), + PrimitiveLiteral::Timestamptz(other_val), + PrimitiveType::Timestamptz, + PrimitiveType::Timestamptz, + ) => val.partial_cmp(other_val), + ( + PrimitiveLiteral::String(val), + PrimitiveLiteral::String(other_val), + PrimitiveType::String, + PrimitiveType::String, + ) => val.partial_cmp(other_val), + ( + PrimitiveLiteral::UUID(val), + PrimitiveLiteral::UUID(other_val), + PrimitiveType::Uuid, + PrimitiveType::Uuid, + ) => val.partial_cmp(other_val), + ( + PrimitiveLiteral::Fixed(val), + PrimitiveLiteral::Fixed(other_val), + PrimitiveType::Fixed(_), + PrimitiveType::Fixed(_), + ) => val.partial_cmp(other_val), + ( + PrimitiveLiteral::Binary(val), + PrimitiveLiteral::Binary(other_val), + PrimitiveType::Binary, + PrimitiveType::Binary, + ) => val.partial_cmp(other_val), + ( + PrimitiveLiteral::Decimal(val), + PrimitiveLiteral::Decimal(other_val), + PrimitiveType::Decimal { + precision: _, + scale, + }, + PrimitiveType::Decimal { + precision: _, + scale: other_scale, + }, + ) => { + let val = Decimal::from_i128_with_scale(*val, *scale); + let other_val = Decimal::from_i128_with_scale(*other_val, *other_scale); + val.partial_cmp(&other_val) + } + _ => None, + } + } +} + +impl Display for Datum { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + match (&self.r#type, &self.literal) { + (_, PrimitiveLiteral::Boolean(val)) => write!(f, "{}", val), + (_, PrimitiveLiteral::Int(val)) => write!(f, "{}", val), + (_, PrimitiveLiteral::Long(val)) => write!(f, "{}", val), + (_, PrimitiveLiteral::Float(val)) => write!(f, "{}", val), + (_, PrimitiveLiteral::Double(val)) => write!(f, "{}", val), + (_, PrimitiveLiteral::Date(val)) => write!(f, "{}", days_to_date(*val)), + (_, PrimitiveLiteral::Time(val)) => write!(f, "{}", microseconds_to_time(*val)), + (_, PrimitiveLiteral::Timestamp(val)) => { + write!(f, "{}", microseconds_to_datetime(*val)) + } + (_, PrimitiveLiteral::Timestamptz(val)) => { + write!(f, "{}", microseconds_to_datetimetz(*val)) + } + (_, PrimitiveLiteral::String(val)) => write!(f, r#""{}""#, val), + (_, PrimitiveLiteral::UUID(val)) => write!(f, "{}", val), + (_, PrimitiveLiteral::Fixed(val)) => display_bytes(val, f), + (_, PrimitiveLiteral::Binary(val)) => display_bytes(val, f), + ( + PrimitiveType::Decimal { + precision: _, + scale, + }, + PrimitiveLiteral::Decimal(val), + ) => { + write!(f, "{}", Decimal::from_i128_with_scale(*val, *scale)) + } + (_, _) => { + unreachable!() + } + } + } +} + +fn display_bytes(bytes: &[u8], f: &mut Formatter<'_>) -> std::fmt::Result { + let mut s = String::with_capacity(bytes.len() * 2); + for b in bytes { + s.push_str(&format!("{:02X}", b)); + } + f.write_str(&s) +} + +impl From for Literal { + fn from(value: Datum) -> Self { + Literal::Primitive(value.literal) + } +} + +impl From for PrimitiveLiteral { + fn from(value: Datum) -> Self { + value.literal + } +} + +impl Datum { + /// Creates a `Datum` from a `PrimitiveType` and a `PrimitiveLiteral` + pub(crate) fn new(r#type: PrimitiveType, literal: PrimitiveLiteral) -> Self { + Datum { r#type, literal } + } + + /// Create iceberg value from bytes + pub fn try_from_bytes(bytes: &[u8], data_type: PrimitiveType) -> Result { + let literal = match data_type { + PrimitiveType::Boolean => { + if bytes.len() == 1 && bytes[0] == 0u8 { + PrimitiveLiteral::Boolean(false) + } else { + PrimitiveLiteral::Boolean(true) + } + } + PrimitiveType::Int => PrimitiveLiteral::Int(i32::from_le_bytes(bytes.try_into()?)), + PrimitiveType::Long => PrimitiveLiteral::Long(i64::from_le_bytes(bytes.try_into()?)), + PrimitiveType::Float => { + PrimitiveLiteral::Float(OrderedFloat(f32::from_le_bytes(bytes.try_into()?))) + } + PrimitiveType::Double => { + PrimitiveLiteral::Double(OrderedFloat(f64::from_le_bytes(bytes.try_into()?))) + } + PrimitiveType::Date => PrimitiveLiteral::Date(i32::from_le_bytes(bytes.try_into()?)), + PrimitiveType::Time => PrimitiveLiteral::Time(i64::from_le_bytes(bytes.try_into()?)), + PrimitiveType::Timestamp => { + PrimitiveLiteral::Timestamp(i64::from_le_bytes(bytes.try_into()?)) + } + PrimitiveType::Timestamptz => { + PrimitiveLiteral::Timestamptz(i64::from_le_bytes(bytes.try_into()?)) + } + PrimitiveType::String => { + PrimitiveLiteral::String(std::str::from_utf8(bytes)?.to_string()) + } + PrimitiveType::Uuid => { + PrimitiveLiteral::UUID(Uuid::from_u128(u128::from_be_bytes(bytes.try_into()?))) + } + PrimitiveType::Fixed(_) => PrimitiveLiteral::Fixed(Vec::from(bytes)), + PrimitiveType::Binary => PrimitiveLiteral::Binary(Vec::from(bytes)), + PrimitiveType::Decimal { + precision: _, + scale: _, + } => todo!(), + }; + Ok(Datum::new(data_type, literal)) + } + + /// Creates a boolean value. + /// + /// Example: + /// ```rust + /// use iceberg::spec::{Literal, PrimitiveLiteral, Datum}; + /// let t = Datum::bool(true); + /// + /// assert_eq!(format!("{}", t), "true".to_string()); + /// assert_eq!(Literal::from(t), Literal::Primitive(PrimitiveLiteral::Boolean(true))); + /// ``` + pub fn bool>(t: T) -> Self { + Self { + r#type: PrimitiveType::Boolean, + literal: PrimitiveLiteral::Boolean(t.into()), + } + } + + /// Creates a boolean value from string. + /// See [Parse bool from str](https://doc.rust-lang.org/stable/std/primitive.bool.html#impl-FromStr-for-bool) for reference. + /// + /// Example: + /// ```rust + /// use iceberg::spec::{Literal, PrimitiveLiteral, Datum}; + /// let t = Datum::bool_from_str("false").unwrap(); + /// + /// assert_eq!(&format!("{}", t), "false"); + /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Boolean(false)), t.into()); + /// ``` + pub fn bool_from_str>(s: S) -> Result { + let v = s.as_ref().parse::().map_err(|e| { + Error::new(ErrorKind::DataInvalid, "Can't parse string to bool.").with_source(e) + })?; + Ok(Self::bool(v)) + } + + /// Creates an 32bit integer. + /// + /// Example: + /// ```rust + /// use iceberg::spec::{Literal, PrimitiveLiteral, Datum}; + /// let t = Datum::int(23i8); + /// + /// assert_eq!(&format!("{}", t), "23"); + /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Int(23)), t.into()); + /// ``` + pub fn int>(t: T) -> Self { + Self { + r#type: PrimitiveType::Int, + literal: PrimitiveLiteral::Int(t.into()), + } + } + + /// Creates an 64bit integer. + /// + /// Example: + /// ```rust + /// use iceberg::spec::{Literal, PrimitiveLiteral, Datum}; + /// let t = Datum::long(24i8); + /// + /// assert_eq!(&format!("{t}"), "24"); + /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Long(24)), t.into()); + /// ``` + pub fn long>(t: T) -> Self { + Self { + r#type: PrimitiveType::Long, + literal: PrimitiveLiteral::Long(t.into()), + } + } + + /// Creates an 32bit floating point number. + /// + /// Example: + /// ```rust + /// use ordered_float::OrderedFloat; + /// use iceberg::spec::{Literal, PrimitiveLiteral, Datum}; + /// let t = Datum::float( 32.1f32 ); + /// + /// assert_eq!(&format!("{t}"), "32.1"); + /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Float(OrderedFloat(32.1))), t.into()); + /// ``` + pub fn float>(t: T) -> Self { + Self { + r#type: PrimitiveType::Float, + literal: PrimitiveLiteral::Float(OrderedFloat(t.into())), + } + } + + /// Creates an 64bit floating point number. + /// + /// Example: + /// ```rust + /// use ordered_float::OrderedFloat; + /// use iceberg::spec::{Literal, PrimitiveLiteral, Datum}; + /// let t = Datum::double( 32.1f64 ); + /// + /// assert_eq!(&format!("{t}"), "32.1"); + /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Double(OrderedFloat(32.1))), t.into()); + /// ``` + pub fn double>(t: T) -> Self { + Self { + r#type: PrimitiveType::Double, + literal: PrimitiveLiteral::Double(OrderedFloat(t.into())), + } + } + + /// Creates date literal from number of days from unix epoch directly. + /// + /// Example: + /// ```rust + /// + /// use iceberg::spec::{Literal, PrimitiveLiteral, Datum}; + /// // 2 days after 1970-01-01 + /// let t = Datum::date(2); + /// + /// assert_eq!(&format!("{t}"), "1970-01-03"); + /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Date(2)), t.into()); + /// ``` + pub fn date(days: i32) -> Self { + Self { + r#type: PrimitiveType::Date, + literal: PrimitiveLiteral::Date(days), + } + } + + /// Creates date literal in `%Y-%m-%d` format, assume in utc timezone. + /// + /// See [`NaiveDate::from_str`]. + /// + /// Example + /// ```rust + /// use iceberg::spec::{Literal, Datum}; + /// let t = Datum::date_from_str("1970-01-05").unwrap(); + /// + /// assert_eq!(&format!("{t}"), "1970-01-05"); + /// assert_eq!(Literal::date(4), t.into()); + /// ``` + pub fn date_from_str>(s: S) -> Result { + let t = s.as_ref().parse::().map_err(|e| { + Error::new( + ErrorKind::DataInvalid, + format!("Can't parse date from string: {}", s.as_ref()), + ) + .with_source(e) + })?; + + Ok(Self::date(date_from_naive_date(t))) + } + + /// Create date literal from calendar date (year, month and day). + /// + /// See [`NaiveDate::from_ymd_opt`]. + /// + /// Example: + /// + ///```rust + /// use iceberg::spec::{Literal, Datum}; + /// let t = Datum::date_from_ymd(1970, 1, 5).unwrap(); + /// + /// assert_eq!(&format!("{t}"), "1970-01-05"); + /// assert_eq!(Literal::date(4), t.into()); + /// ``` + pub fn date_from_ymd(year: i32, month: u32, day: u32) -> Result { + let t = NaiveDate::from_ymd_opt(year, month, day).ok_or_else(|| { + Error::new( + ErrorKind::DataInvalid, + format!("Can't create date from year: {year}, month: {month}, day: {day}"), + ) + })?; + + Ok(Self::date(date_from_naive_date(t))) + } + + /// Creates time literal in microseconds directly. + /// + /// It will return error when it's negative or too large to fit in 24 hours. + /// + /// Example: + /// + /// ```rust + /// use iceberg::spec::{Literal, Datum}; + /// let micro_secs = { + /// 1 * 3600 * 1_000_000 + // 1 hour + /// 2 * 60 * 1_000_000 + // 2 minutes + /// 1 * 1_000_000 + // 1 second + /// 888999 // microseconds + /// }; + /// + /// let t = Datum::time_micros(micro_secs).unwrap(); + /// + /// assert_eq!(&format!("{t}"), "01:02:01.888999"); + /// assert_eq!(Literal::time(micro_secs), t.into()); + /// + /// let negative_value = -100; + /// assert!(Datum::time_micros(negative_value).is_err()); + /// + /// let too_large_value = 36 * 60 * 60 * 1_000_000; // Too large to fit in 24 hours. + /// assert!(Datum::time_micros(too_large_value).is_err()); + /// ``` + pub fn time_micros(value: i64) -> Result { + ensure_data_valid!( + (0..=MAX_TIME_VALUE).contains(&value), + "Invalid value for Time type: {}", + value + ); + + Ok(Self { + r#type: PrimitiveType::Time, + literal: PrimitiveLiteral::Time(value), + }) + } + + /// Creates time literal from [`chrono::NaiveTime`]. + fn time_from_naive_time(t: NaiveTime) -> Self { + let duration = t - unix_epoch().time(); + // It's safe to unwrap here since less than 24 hours will never overflow. + let micro_secs = duration.num_microseconds().unwrap(); + + Self { + r#type: PrimitiveType::Time, + literal: PrimitiveLiteral::Time(micro_secs), + } + } + + /// Creates time literal in microseconds in `%H:%M:%S:.f` format. + /// + /// See [`NaiveTime::from_str`] for details. + /// + /// Example: + /// ```rust + /// use iceberg::spec::{Literal, Datum}; + /// let t = Datum::time_from_str("01:02:01.888999777").unwrap(); + /// + /// assert_eq!(&format!("{t}"), "01:02:01.888999"); + /// ``` + pub fn time_from_str>(s: S) -> Result { + let t = s.as_ref().parse::().map_err(|e| { + Error::new( + ErrorKind::DataInvalid, + format!("Can't parse time from string: {}", s.as_ref()), + ) + .with_source(e) + })?; + + Ok(Self::time_from_naive_time(t)) + } + + /// Creates time literal from hour, minute, second, and microseconds. + /// + /// See [`NaiveTime::from_hms_micro_opt`]. + /// + /// Example: + /// ```rust + /// + /// use iceberg::spec::{Literal, Datum}; + /// let t = Datum::time_from_hms_micro(22, 15, 33, 111).unwrap(); + /// + /// assert_eq!(&format!("{t}"), "22:15:33.000111"); + /// ``` + pub fn time_from_hms_micro(hour: u32, min: u32, sec: u32, micro: u32) -> Result { + let t = NaiveTime::from_hms_micro_opt(hour, min, sec, micro) + .ok_or_else(|| Error::new( + ErrorKind::DataInvalid, + format!("Can't create time from hour: {hour}, min: {min}, second: {sec}, microsecond: {micro}"), + ))?; + Ok(Self::time_from_naive_time(t)) + } + + /// Creates a timestamp from unix epoch in microseconds. + /// + /// Example: + /// + /// ```rust + /// + /// use iceberg::spec::Datum; + /// let t = Datum::timestamp_micros(1000); + /// + /// assert_eq!(&format!("{t}"), "1970-01-01 00:00:00.001"); + /// ``` + pub fn timestamp_micros(value: i64) -> Self { + Self { + r#type: PrimitiveType::Timestamp, + literal: PrimitiveLiteral::Timestamp(value), + } + } + + /// Creates a timestamp from [`DateTime`]. + /// + /// Example: + /// + /// ```rust + /// + /// use chrono::{NaiveDate, NaiveDateTime, TimeZone, Utc}; + /// use iceberg::spec::Datum; + /// let t = Datum::timestamp_from_datetime( + /// NaiveDate::from_ymd_opt(1992, 3, 1) + /// .unwrap() + /// .and_hms_micro_opt(1, 2, 3, 88) + /// .unwrap()); + /// + /// assert_eq!(&format!("{t}"), "1992-03-01 01:02:03.000088"); + /// ``` + pub fn timestamp_from_datetime(dt: NaiveDateTime) -> Self { + Self::timestamp_micros(dt.and_utc().timestamp_micros()) + } + + /// Parse a timestamp in [`%Y-%m-%dT%H:%M:%S%.f`] format. + /// + /// See [`NaiveDateTime::from_str`]. + /// + /// Example: + /// + /// ```rust + /// use chrono::{DateTime, FixedOffset, NaiveDate, NaiveDateTime, NaiveTime}; + /// use iceberg::spec::{Literal, Datum}; + /// let t = Datum::timestamp_from_str("1992-03-01T01:02:03.000088").unwrap(); + /// + /// assert_eq!(&format!("{t}"), "1992-03-01 01:02:03.000088"); + /// ``` + pub fn timestamp_from_str>(s: S) -> Result { + let dt = s.as_ref().parse::().map_err(|e| { + Error::new(ErrorKind::DataInvalid, "Can't parse timestamp.").with_source(e) + })?; + + Ok(Self::timestamp_from_datetime(dt)) + } + + /// Creates a timestamp with timezone from unix epoch in microseconds. + /// + /// Example: + /// + /// ```rust + /// + /// use iceberg::spec::Datum; + /// let t = Datum::timestamptz_micros(1000); + /// + /// assert_eq!(&format!("{t}"), "1970-01-01 00:00:00.001 UTC"); + /// ``` + pub fn timestamptz_micros(value: i64) -> Self { + Self { + r#type: PrimitiveType::Timestamptz, + literal: PrimitiveLiteral::Timestamptz(value), + } + } + + /// Creates a timestamp with timezone from [`DateTime`]. + /// Example: + /// + /// ```rust + /// + /// use chrono::{TimeZone, Utc}; + /// use iceberg::spec::Datum; + /// let t = Datum::timestamptz_from_datetime(Utc.timestamp_opt(1000, 0).unwrap()); + /// + /// assert_eq!(&format!("{t}"), "1970-01-01 00:16:40 UTC"); + /// ``` + pub fn timestamptz_from_datetime(dt: DateTime) -> Self { + Self::timestamptz_micros(dt.with_timezone(&Utc).timestamp_micros()) + } + + /// Parse timestamp with timezone in RFC3339 format. + /// + /// See [`DateTime::from_str`]. + /// + /// Example: + /// + /// ```rust + /// use chrono::{DateTime, FixedOffset, NaiveDate, NaiveDateTime, NaiveTime}; + /// use iceberg::spec::{Literal, Datum}; + /// let t = Datum::timestamptz_from_str("1992-03-01T01:02:03.000088+08:00").unwrap(); + /// + /// assert_eq!(&format!("{t}"), "1992-02-29 17:02:03.000088 UTC"); + /// ``` + pub fn timestamptz_from_str>(s: S) -> Result { + let dt = DateTime::::from_str(s.as_ref()).map_err(|e| { + Error::new(ErrorKind::DataInvalid, "Can't parse datetime.").with_source(e) + })?; + + Ok(Self::timestamptz_from_datetime(dt)) + } + + /// Creates a string literal. + /// + /// Example: + /// + /// ```rust + /// use iceberg::spec::Datum; + /// let t = Datum::string("ss"); + /// + /// assert_eq!(&format!("{t}"), r#""ss""#); + /// ``` + pub fn string(s: S) -> Self { + Self { + r#type: PrimitiveType::String, + literal: PrimitiveLiteral::String(s.to_string()), + } + } + + /// Creates uuid literal. + /// + /// Example: + /// + /// ```rust + /// use uuid::uuid; + /// use iceberg::spec::Datum; + /// let t = Datum::uuid(uuid!("a1a2a3a4-b1b2-c1c2-d1d2-d3d4d5d6d7d8")); + /// + /// assert_eq!(&format!("{t}"), "a1a2a3a4-b1b2-c1c2-d1d2-d3d4d5d6d7d8"); + /// ``` + pub fn uuid(uuid: Uuid) -> Self { + Self { + r#type: PrimitiveType::Uuid, + literal: PrimitiveLiteral::UUID(uuid), + } + } + + /// Creates uuid from str. See [`Uuid::parse_str`]. + /// + /// Example: + /// + /// ```rust + /// use iceberg::spec::{Datum}; + /// let t = Datum::uuid_from_str("a1a2a3a4-b1b2-c1c2-d1d2-d3d4d5d6d7d8").unwrap(); + /// + /// assert_eq!(&format!("{t}"), "a1a2a3a4-b1b2-c1c2-d1d2-d3d4d5d6d7d8"); + /// ``` + pub fn uuid_from_str>(s: S) -> Result { + let uuid = Uuid::parse_str(s.as_ref()).map_err(|e| { + Error::new( + ErrorKind::DataInvalid, + format!("Can't parse uuid from string: {}", s.as_ref()), + ) + .with_source(e) + })?; + Ok(Self::uuid(uuid)) + } + + /// Creates a fixed literal from bytes. + /// + /// Example: + /// + /// ```rust + /// use iceberg::spec::{Literal, PrimitiveLiteral, Datum}; + /// let t = Datum::fixed(vec![1u8, 2u8]); + /// + /// assert_eq!(&format!("{t}"), "0102"); + /// ``` + pub fn fixed>(input: I) -> Self { + let value: Vec = input.into_iter().collect(); + Self { + r#type: PrimitiveType::Fixed(value.len() as u64), + literal: PrimitiveLiteral::Fixed(value), + } + } + + /// Creates a binary literal from bytes. + /// + /// Example: + /// + /// ```rust + /// use iceberg::spec::Datum; + /// let t = Datum::binary(vec![1u8, 100u8]); + /// + /// assert_eq!(&format!("{t}"), "0164"); + /// ``` + pub fn binary>(input: I) -> Self { + Self { + r#type: PrimitiveType::Binary, + literal: PrimitiveLiteral::Binary(input.into_iter().collect()), + } + } + + /// Creates decimal literal from string. See [`Decimal::from_str_exact`]. + /// + /// Example: + /// + /// ```rust + /// use itertools::assert_equal; + /// use rust_decimal::Decimal; + /// use iceberg::spec::Datum; + /// let t = Datum::decimal_from_str("123.45").unwrap(); + /// + /// assert_eq!(&format!("{t}"), "123.45"); + /// ``` + pub fn decimal_from_str>(s: S) -> Result { + let decimal = Decimal::from_str_exact(s.as_ref()).map_err(|e| { + Error::new(ErrorKind::DataInvalid, "Can't parse decimal.").with_source(e) + })?; + + Self::decimal(decimal) + } + + /// Try to create a decimal literal from [`Decimal`]. + /// + /// Example: + /// + /// ```rust + /// use rust_decimal::Decimal; + /// use iceberg::spec::Datum; + /// + /// let t = Datum::decimal(Decimal::new(123, 2)).unwrap(); + /// + /// assert_eq!(&format!("{t}"), "1.23"); + /// ``` + pub fn decimal(value: impl Into) -> Result { + let decimal = value.into(); + let scale = decimal.scale(); + + let r#type = Type::decimal(MAX_DECIMAL_PRECISION, scale)?; + if let Type::Primitive(p) = r#type { + Ok(Self { + r#type: p, + literal: PrimitiveLiteral::Decimal(decimal.mantissa()), + }) + } else { + unreachable!("Decimal type must be primitive.") + } + } + + /// Convert the datum to `target_type`. + pub fn to(self, target_type: &Type) -> Result { + // TODO: We should allow more type conversions + match target_type { + Type::Primitive(typ) if typ == &self.r#type => Ok(self), + _ => Err(Error::new( + ErrorKind::DataInvalid, + format!( + "Can't convert datum from {} type to {} type.", + self.r#type, target_type + ), + )), + } + } + + /// Get the primitive literal from datum. + pub fn literal(&self) -> &PrimitiveLiteral { + &self.literal + } + + /// Get the primitive type from datum. + pub fn data_type(&self) -> &PrimitiveType { + &self.r#type + } + + /// Returns true if the Literal represents a primitive type + /// that can be a NaN, and that it's value is NaN + pub fn is_nan(&self) -> bool { + match self.literal { + PrimitiveLiteral::Double(val) => val.is_nan(), + PrimitiveLiteral::Float(val) => val.is_nan(), + _ => false, + } + } +} + +/// Map is a collection of key-value pairs with a key type and a value type. +/// It used in Literal::Map, to make it hashable, the order of key-value pairs is stored in a separate vector +/// so that we can hash the map in a deterministic way. But it also means that the order of key-value pairs is matter +/// for the hash value. +#[derive(Clone, Debug, PartialEq, Eq)] +pub struct Map { + index: HashMap, + pair: Vec<(Literal, Option)>, +} + +impl Map { + /// Creates a new empty map. + pub fn new() -> Self { + Self { + index: HashMap::new(), + pair: Vec::new(), + } + } + + /// Return the number of key-value pairs in the map. + pub fn len(&self) -> usize { + self.pair.len() + } + + /// Returns true if the map contains no elements. + pub fn is_empty(&self) -> bool { + self.pair.is_empty() + } + + /// Inserts a key-value pair into the map. + /// If the map did not have this key present, None is returned. + /// If the map did have this key present, the value is updated, and the old value is returned. + pub fn insert(&mut self, key: Literal, value: Option) -> Option> { + if let Some(index) = self.index.get(&key) { + let old_value = std::mem::replace(&mut self.pair[*index].1, value); + Some(old_value) + } else { + self.pair.push((key.clone(), value)); + self.index.insert(key, self.pair.len() - 1); + None + } + } + + /// Returns a reference to the value corresponding to the key. + /// If the key is not present in the map, None is returned. + pub fn get(&self, key: &Literal) -> Option<&Option> { + self.index.get(key).map(|index| &self.pair[*index].1) + } + + /// The order of map is matter, so this method used to compare two maps has same key-value pairs without considering the order. + pub fn has_same_content(&self, other: &Map) -> bool { + if self.len() != other.len() { + return false; + } + + for (key, value) in &self.pair { + match other.get(key) { + Some(other_value) if value == other_value => (), + _ => return false, + } + } + + true + } +} + +impl Default for Map { + fn default() -> Self { + Self::new() + } +} + +impl Hash for Map { + fn hash(&self, state: &mut H) { + for (key, value) in &self.pair { + key.hash(state); + value.hash(state); + } + } +} + +impl FromIterator<(Literal, Option)> for Map { + fn from_iter)>>(iter: T) -> Self { + let mut map = Map::new(); + for (key, value) in iter { + map.insert(key, value); + } + map + } +} + +impl IntoIterator for Map { + type Item = (Literal, Option); + type IntoIter = std::vec::IntoIter; + + fn into_iter(self) -> Self::IntoIter { + self.pair.into_iter() + } +} + +impl From<[(Literal, Option); N]> for Map { + fn from(value: [(Literal, Option); N]) -> Self { + value.iter().cloned().collect() + } +} + +/// Values present in iceberg type +#[derive(Clone, Debug, PartialEq, Eq, Hash)] +pub enum Literal { + /// A primitive value + Primitive(PrimitiveLiteral), + /// A struct is a tuple of typed values. Each field in the tuple is named and has an integer id that is unique in the table schema. + /// Each field can be either optional or required, meaning that values can (or cannot) be null. Fields may be any type. + /// Fields may have an optional comment or doc string. Fields can have default values. + Struct(Struct), + /// A list is a collection of values with some element type. + /// The element field has an integer id that is unique in the table schema. + /// Elements can be either optional or required. Element types may be any type. + List(Vec>), + /// A map is a collection of key-value pairs with a key type and a value type. + /// Both the key field and value field each have an integer id that is unique in the table schema. + /// Map keys are required and map values can be either optional or required. Both map keys and map values may be any type, including nested types. + Map(Map), +} + +impl Literal { + /// Creates a boolean value. + /// + /// Example: + /// ```rust + /// use iceberg::spec::{Literal, PrimitiveLiteral}; + /// let t = Literal::bool(true); + /// + /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Boolean(true)), t); + /// ``` + pub fn bool>(t: T) -> Self { + Self::Primitive(PrimitiveLiteral::Boolean(t.into())) + } + + /// Creates a boolean value from string. + /// See [Parse bool from str](https://doc.rust-lang.org/stable/std/primitive.bool.html#impl-FromStr-for-bool) for reference. + /// + /// Example: + /// ```rust + /// use iceberg::spec::{Literal, PrimitiveLiteral}; + /// let t = Literal::bool_from_str("false").unwrap(); + /// + /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Boolean(false)), t); + /// ``` + pub fn bool_from_str>(s: S) -> Result { + let v = s.as_ref().parse::().map_err(|e| { + Error::new(ErrorKind::DataInvalid, "Can't parse string to bool.").with_source(e) + })?; + Ok(Self::Primitive(PrimitiveLiteral::Boolean(v))) + } + + /// Creates an 32bit integer. + /// + /// Example: + /// ```rust + /// use iceberg::spec::{Literal, PrimitiveLiteral}; + /// let t = Literal::int(23i8); + /// + /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Int(23)), t); + /// ``` + pub fn int>(t: T) -> Self { + Self::Primitive(PrimitiveLiteral::Int(t.into())) + } + + /// Creates an 64bit integer. + /// + /// Example: + /// ```rust + /// use iceberg::spec::{Literal, PrimitiveLiteral}; + /// let t = Literal::long(24i8); + /// + /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Long(24)), t); + /// ``` + pub fn long>(t: T) -> Self { + Self::Primitive(PrimitiveLiteral::Long(t.into())) + } + + /// Creates an 32bit floating point number. + /// + /// Example: + /// ```rust + /// use ordered_float::OrderedFloat; + /// use iceberg::spec::{Literal, PrimitiveLiteral}; + /// let t = Literal::float( 32.1f32 ); + /// + /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Float(OrderedFloat(32.1))), t); + /// ``` + pub fn float>(t: T) -> Self { + Self::Primitive(PrimitiveLiteral::Float(OrderedFloat(t.into()))) + } + + /// Creates an 32bit floating point number. + /// + /// Example: + /// ```rust + /// use ordered_float::OrderedFloat; + /// use iceberg::spec::{Literal, PrimitiveLiteral}; + /// let t = Literal::double( 32.1f64 ); + /// + /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Double(OrderedFloat(32.1))), t); + /// ``` + pub fn double>(t: T) -> Self { + Self::Primitive(PrimitiveLiteral::Double(OrderedFloat(t.into()))) + } + + /// Creates date literal from number of days from unix epoch directly. + pub fn date(days: i32) -> Self { + Self::Primitive(PrimitiveLiteral::Date(days)) + } + + /// Creates a date in `%Y-%m-%d` format, assume in utc timezone. + /// + /// See [`NaiveDate::from_str`]. + /// + /// Example + /// ```rust + /// use iceberg::spec::Literal; + /// let t = Literal::date_from_str("1970-01-03").unwrap(); + /// + /// assert_eq!(Literal::date(2), t); + /// ``` + pub fn date_from_str>(s: S) -> Result { + let t = s.as_ref().parse::().map_err(|e| { + Error::new( + ErrorKind::DataInvalid, + format!("Can't parse date from string: {}", s.as_ref()), + ) + .with_source(e) + })?; + + Ok(Self::date(date_from_naive_date(t))) + } + + /// Create a date from calendar date (year, month and day). + /// + /// See [`NaiveDate::from_ymd_opt`]. + /// + /// Example: + /// + ///```rust + /// use iceberg::spec::Literal; + /// let t = Literal::date_from_ymd(1970, 1, 5).unwrap(); + /// + /// assert_eq!(Literal::date(4), t); + /// ``` + pub fn date_from_ymd(year: i32, month: u32, day: u32) -> Result { + let t = NaiveDate::from_ymd_opt(year, month, day).ok_or_else(|| { + Error::new( + ErrorKind::DataInvalid, + format!("Can't create date from year: {year}, month: {month}, day: {day}"), + ) + })?; + + Ok(Self::date(date_from_naive_date(t))) + } + + /// Creates time in microseconds directly + pub fn time(value: i64) -> Self { + Self::Primitive(PrimitiveLiteral::Time(value)) + } + + /// Creates time literal from [`chrono::NaiveTime`]. + fn time_from_naive_time(t: NaiveTime) -> Self { + let duration = t - unix_epoch().time(); + // It's safe to unwrap here since less than 24 hours will never overflow. + let micro_secs = duration.num_microseconds().unwrap(); + + Literal::time(micro_secs) + } + + /// Creates time in microseconds in `%H:%M:%S:.f` format. + /// + /// See [`NaiveTime::from_str`] for details. + /// + /// Example: + /// ```rust + /// use iceberg::spec::Literal; + /// let t = Literal::time_from_str("01:02:01.888999777").unwrap(); + /// + /// let micro_secs = { + /// 1 * 3600 * 1_000_000 + // 1 hour + /// 2 * 60 * 1_000_000 + // 2 minutes + /// 1 * 1_000_000 + // 1 second + /// 888999 // microseconds + /// }; + /// assert_eq!(Literal::time(micro_secs), t); + /// ``` + pub fn time_from_str>(s: S) -> Result { + let t = s.as_ref().parse::().map_err(|e| { + Error::new( + ErrorKind::DataInvalid, + format!("Can't parse time from string: {}", s.as_ref()), + ) + .with_source(e) + })?; + + Ok(Self::time_from_naive_time(t)) + } + + /// Creates time literal from hour, minute, second, and microseconds. + /// + /// See [`NaiveTime::from_hms_micro_opt`]. + /// + /// Example: + /// ```rust + /// + /// use iceberg::spec::Literal; + /// let t = Literal::time_from_hms_micro(22, 15, 33, 111).unwrap(); + /// + /// assert_eq!(Literal::time_from_str("22:15:33.000111").unwrap(), t); + /// ``` + pub fn time_from_hms_micro(hour: u32, min: u32, sec: u32, micro: u32) -> Result { + let t = NaiveTime::from_hms_micro_opt(hour, min, sec, micro) + .ok_or_else(|| Error::new( + ErrorKind::DataInvalid, + format!("Can't create time from hour: {hour}, min: {min}, second: {sec}, microsecond: {micro}"), + ))?; + Ok(Self::time_from_naive_time(t)) + } + + /// Creates a timestamp from unix epoch in microseconds. + pub fn timestamp(value: i64) -> Self { + Self::Primitive(PrimitiveLiteral::Timestamp(value)) + } + + /// Creates a timestamp with timezone from unix epoch in microseconds. + pub fn timestamptz(value: i64) -> Self { + Self::Primitive(PrimitiveLiteral::Timestamptz(value)) + } + + /// Creates a timestamp from [`DateTime`]. + pub fn timestamp_from_datetime(dt: DateTime) -> Self { + Self::timestamp(dt.with_timezone(&Utc).timestamp_micros()) + } + + /// Creates a timestamp with timezone from [`DateTime`]. + pub fn timestamptz_from_datetime(dt: DateTime) -> Self { + Self::timestamptz(dt.with_timezone(&Utc).timestamp_micros()) + } + + /// Parse a timestamp in RFC3339 format. + /// + /// See [`DateTime::from_str`]. + /// + /// Example: + /// + /// ```rust + /// use chrono::{DateTime, FixedOffset, NaiveDate, NaiveDateTime, NaiveTime}; + /// use iceberg::spec::Literal; + /// let t = Literal::timestamp_from_str("2012-12-12 12:12:12.8899-04:00").unwrap(); + /// + /// let t2 = { + /// let date = NaiveDate::from_ymd_opt(2012, 12, 12).unwrap(); + /// let time = NaiveTime::from_hms_micro_opt(12, 12, 12, 889900).unwrap(); + /// let dt = NaiveDateTime::new(date, time); + /// Literal::timestamp_from_datetime(DateTime::::from_local(dt, FixedOffset::west_opt(4 * 3600).unwrap())) + /// }; + /// + /// assert_eq!(t, t2); + /// ``` + pub fn timestamp_from_str>(s: S) -> Result { + let dt = DateTime::::from_str(s.as_ref()).map_err(|e| { + Error::new(ErrorKind::DataInvalid, "Can't parse datetime.").with_source(e) + })?; + + Ok(Self::timestamp_from_datetime(dt)) + } + + /// Similar to [`Literal::timestamp_from_str`], but return timestamp with timezone literal. + pub fn timestamptz_from_str>(s: S) -> Result { + let dt = DateTime::::from_str(s.as_ref()).map_err(|e| { + Error::new(ErrorKind::DataInvalid, "Can't parse datetime.").with_source(e) + })?; + + Ok(Self::timestamptz_from_datetime(dt)) + } + + /// Creates a string literal. + pub fn string(s: S) -> Self { + Self::Primitive(PrimitiveLiteral::String(s.to_string())) + } + + /// Creates uuid literal. + pub fn uuid(uuid: Uuid) -> Self { + Self::Primitive(PrimitiveLiteral::UUID(uuid)) + } + + /// Creates uuid from str. See [`Uuid::parse_str`]. + /// + /// Example: + /// + /// ```rust + /// use uuid::Uuid; + /// use iceberg::spec::Literal; + /// let t1 = Literal::uuid_from_str("a1a2a3a4-b1b2-c1c2-d1d2-d3d4d5d6d7d8").unwrap(); + /// let t2 = Literal::uuid(Uuid::from_u128_le(0xd8d7d6d5d4d3d2d1c2c1b2b1a4a3a2a1)); + /// + /// assert_eq!(t1, t2); + /// ``` + pub fn uuid_from_str>(s: S) -> Result { + let uuid = Uuid::parse_str(s.as_ref()).map_err(|e| { + Error::new( + ErrorKind::DataInvalid, + format!("Can't parse uuid from string: {}", s.as_ref()), + ) + .with_source(e) + })?; + Ok(Self::uuid(uuid)) + } + + /// Creates a fixed literal from bytes. + /// + /// Example: + /// + /// ```rust + /// use iceberg::spec::{Literal, PrimitiveLiteral}; + /// let t1 = Literal::fixed(vec![1u8, 2u8]); + /// let t2 = Literal::Primitive(PrimitiveLiteral::Fixed(vec![1u8, 2u8])); + /// + /// assert_eq!(t1, t2); + /// ``` + pub fn fixed>(input: I) -> Self { + Literal::Primitive(PrimitiveLiteral::Fixed(input.into_iter().collect())) + } + + /// Creates a binary literal from bytes. + /// + /// Example: + /// + /// ```rust + /// use iceberg::spec::{Literal, PrimitiveLiteral}; + /// let t1 = Literal::binary(vec![1u8, 2u8]); + /// let t2 = Literal::Primitive(PrimitiveLiteral::Binary(vec![1u8, 2u8])); + /// + /// assert_eq!(t1, t2); + /// ``` + pub fn binary>(input: I) -> Self { + Literal::Primitive(PrimitiveLiteral::Binary(input.into_iter().collect())) + } + + /// Creates a decimal literal. + pub fn decimal(decimal: i128) -> Self { + Self::Primitive(PrimitiveLiteral::Decimal(decimal)) + } + + /// Creates decimal literal from string. See [`Decimal::from_str_exact`]. + /// + /// Example: + /// + /// ```rust + /// use rust_decimal::Decimal; + /// use iceberg::spec::Literal; + /// let t1 = Literal::decimal(12345); + /// let t2 = Literal::decimal_from_str("123.45").unwrap(); + /// + /// assert_eq!(t1, t2); + /// ``` + pub fn decimal_from_str>(s: S) -> Result { + let decimal = Decimal::from_str_exact(s.as_ref()).map_err(|e| { + Error::new(ErrorKind::DataInvalid, "Can't parse decimal.").with_source(e) + })?; + Ok(Self::decimal(decimal.mantissa())) + } +} + +impl From for ByteBuf { + fn from(value: PrimitiveLiteral) -> Self { + match value { + PrimitiveLiteral::Boolean(val) => { + if val { + ByteBuf::from([1u8]) + } else { + ByteBuf::from([0u8]) + } + } + PrimitiveLiteral::Int(val) => ByteBuf::from(val.to_le_bytes()), + PrimitiveLiteral::Long(val) => ByteBuf::from(val.to_le_bytes()), + PrimitiveLiteral::Float(val) => ByteBuf::from(val.to_le_bytes()), + PrimitiveLiteral::Double(val) => ByteBuf::from(val.to_le_bytes()), + PrimitiveLiteral::Date(val) => ByteBuf::from(val.to_le_bytes()), + PrimitiveLiteral::Time(val) => ByteBuf::from(val.to_le_bytes()), + PrimitiveLiteral::Timestamp(val) => ByteBuf::from(val.to_le_bytes()), + PrimitiveLiteral::Timestamptz(val) => ByteBuf::from(val.to_le_bytes()), + PrimitiveLiteral::String(val) => ByteBuf::from(val.as_bytes()), + PrimitiveLiteral::UUID(val) => ByteBuf::from(val.as_u128().to_be_bytes()), + PrimitiveLiteral::Fixed(val) => ByteBuf::from(val), + PrimitiveLiteral::Binary(val) => ByteBuf::from(val), + PrimitiveLiteral::Decimal(_) => todo!(), + } + } +} + +impl From for ByteBuf { + fn from(value: Literal) -> Self { + match value { + Literal::Primitive(val) => val.into(), + _ => unimplemented!(), + } + } +} + +impl From for Vec { + fn from(value: PrimitiveLiteral) -> Self { + match value { + PrimitiveLiteral::Boolean(val) => { + if val { + Vec::from([1u8]) + } else { + Vec::from([0u8]) + } + } + PrimitiveLiteral::Int(val) => Vec::from(val.to_le_bytes()), + PrimitiveLiteral::Long(val) => Vec::from(val.to_le_bytes()), + PrimitiveLiteral::Float(val) => Vec::from(val.to_le_bytes()), + PrimitiveLiteral::Double(val) => Vec::from(val.to_le_bytes()), + PrimitiveLiteral::Date(val) => Vec::from(val.to_le_bytes()), + PrimitiveLiteral::Time(val) => Vec::from(val.to_le_bytes()), + PrimitiveLiteral::Timestamp(val) => Vec::from(val.to_le_bytes()), + PrimitiveLiteral::Timestamptz(val) => Vec::from(val.to_le_bytes()), + PrimitiveLiteral::String(val) => Vec::from(val.as_bytes()), + PrimitiveLiteral::UUID(val) => Vec::from(val.as_u128().to_be_bytes()), + PrimitiveLiteral::Fixed(val) => val, + PrimitiveLiteral::Binary(val) => val, + PrimitiveLiteral::Decimal(_) => todo!(), + } + } +} + +impl From for Vec { + fn from(value: Literal) -> Self { + match value { + Literal::Primitive(val) => val.into(), + _ => unimplemented!(), + } + } +} + +/// The partition struct stores the tuple of partition values for each file. +/// Its type is derived from the partition fields of the partition spec used to write the manifest file. +/// In v2, the partition struct’s field ids must match the ids from the partition spec. +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct Struct { + /// Vector to store the field values + fields: Vec, + /// Null bitmap + null_bitmap: BitVec, +} + +impl Struct { + /// Create a empty struct. + pub fn empty() -> Self { + Self { + fields: Vec::new(), + null_bitmap: BitVec::new(), + } + } + + /// Create a iterator to read the field in order of field_value. + pub fn iter(&self) -> impl Iterator> { + self.null_bitmap.iter().zip(self.fields.iter()).map( + |(null, value)| { + if *null { + None + } else { + Some(value) + } + }, + ) + } + + /// returns true if the field at position `index` is null + pub fn is_null_at_index(&self, index: usize) -> bool { + self.null_bitmap[index] + } +} + +impl Index for Struct { + type Output = Literal; + + fn index(&self, idx: usize) -> &Self::Output { + &self.fields[idx] + } +} + +/// An iterator that moves out of a struct. +pub struct StructValueIntoIter { + null_bitmap: bitvec::boxed::IntoIter, + fields: std::vec::IntoIter, +} + +impl Iterator for StructValueIntoIter { + type Item = Option; + + fn next(&mut self) -> Option { + match (self.null_bitmap.next(), self.fields.next()) { + (Some(null), Some(value)) => Some(if null { None } else { Some(value) }), + _ => None, + } + } +} + +impl IntoIterator for Struct { + type Item = Option; + + type IntoIter = StructValueIntoIter; + + fn into_iter(self) -> Self::IntoIter { + StructValueIntoIter { + null_bitmap: self.null_bitmap.into_iter(), + fields: self.fields.into_iter(), + } + } +} + +impl FromIterator> for Struct { + fn from_iter>>(iter: I) -> Self { + let mut fields = Vec::new(); + let mut null_bitmap = BitVec::new(); + + for value in iter.into_iter() { + match value { + Some(value) => { + fields.push(value); + null_bitmap.push(false) + } + None => { + fields.push(Literal::Primitive(PrimitiveLiteral::Boolean(false))); + null_bitmap.push(true) + } + } + } + Struct { + fields, + null_bitmap, + } + } +} + +impl Literal { + /// Create iceberg value from bytes + pub fn try_from_bytes(bytes: &[u8], data_type: &Type) -> Result { + match data_type { + Type::Primitive(primitive_type) => { + let datum = Datum::try_from_bytes(bytes, primitive_type.clone())?; + Ok(Literal::Primitive(datum.literal)) + } + _ => Err(Error::new( + crate::ErrorKind::DataInvalid, + "Converting bytes to non-primitive types is not supported.", + )), + } + } + + /// Create iceberg value from a json value + pub fn try_from_json(value: JsonValue, data_type: &Type) -> Result> { + match data_type { + Type::Primitive(primitive) => match (primitive, value) { + (PrimitiveType::Boolean, JsonValue::Bool(bool)) => { + Ok(Some(Literal::Primitive(PrimitiveLiteral::Boolean(bool)))) + } + (PrimitiveType::Int, JsonValue::Number(number)) => { + Ok(Some(Literal::Primitive(PrimitiveLiteral::Int( + number + .as_i64() + .ok_or(Error::new( + crate::ErrorKind::DataInvalid, + "Failed to convert json number to int", + ))? + .try_into()?, + )))) + } + (PrimitiveType::Long, JsonValue::Number(number)) => Ok(Some(Literal::Primitive( + PrimitiveLiteral::Long(number.as_i64().ok_or(Error::new( + crate::ErrorKind::DataInvalid, + "Failed to convert json number to long", + ))?), + ))), + (PrimitiveType::Float, JsonValue::Number(number)) => Ok(Some(Literal::Primitive( + PrimitiveLiteral::Float(OrderedFloat(number.as_f64().ok_or(Error::new( + crate::ErrorKind::DataInvalid, + "Failed to convert json number to float", + ))? as f32)), + ))), + (PrimitiveType::Double, JsonValue::Number(number)) => Ok(Some(Literal::Primitive( + PrimitiveLiteral::Double(OrderedFloat(number.as_f64().ok_or(Error::new( + crate::ErrorKind::DataInvalid, + "Failed to convert json number to double", + ))?)), + ))), + (PrimitiveType::Date, JsonValue::String(s)) => { + Ok(Some(Literal::Primitive(PrimitiveLiteral::Date( + date::date_to_days(&NaiveDate::parse_from_str(&s, "%Y-%m-%d")?), + )))) + } + (PrimitiveType::Time, JsonValue::String(s)) => { + Ok(Some(Literal::Primitive(PrimitiveLiteral::Time( + time::time_to_microseconds(&NaiveTime::parse_from_str(&s, "%H:%M:%S%.f")?), + )))) + } + (PrimitiveType::Timestamp, JsonValue::String(s)) => Ok(Some(Literal::Primitive( + PrimitiveLiteral::Timestamp(timestamp::datetime_to_microseconds( + &NaiveDateTime::parse_from_str(&s, "%Y-%m-%dT%H:%M:%S%.f")?, + )), + ))), + (PrimitiveType::Timestamptz, JsonValue::String(s)) => { + Ok(Some(Literal::Primitive(PrimitiveLiteral::Timestamptz( + timestamptz::datetimetz_to_microseconds(&Utc.from_utc_datetime( + &NaiveDateTime::parse_from_str(&s, "%Y-%m-%dT%H:%M:%S%.f+00:00")?, + )), + )))) + } + (PrimitiveType::String, JsonValue::String(s)) => { + Ok(Some(Literal::Primitive(PrimitiveLiteral::String(s)))) + } + (PrimitiveType::Uuid, JsonValue::String(s)) => Ok(Some(Literal::Primitive( + PrimitiveLiteral::UUID(Uuid::parse_str(&s)?), + ))), + (PrimitiveType::Fixed(_), JsonValue::String(_)) => todo!(), + (PrimitiveType::Binary, JsonValue::String(_)) => todo!(), + ( + PrimitiveType::Decimal { + precision: _, + scale, + }, + JsonValue::String(s), + ) => { + let mut decimal = Decimal::from_str_exact(&s)?; + decimal.rescale(*scale); + Ok(Some(Literal::Primitive(PrimitiveLiteral::Decimal( + decimal.mantissa(), + )))) + } + (_, JsonValue::Null) => Ok(None), + (i, j) => Err(Error::new( + crate::ErrorKind::DataInvalid, + format!( + "The json value {} doesn't fit to the iceberg type {}.", + j, i + ), + )), + }, + Type::Struct(schema) => { + if let JsonValue::Object(mut object) = value { + Ok(Some(Literal::Struct(Struct::from_iter( + schema.fields().iter().map(|field| { + object.remove(&field.id.to_string()).and_then(|value| { + Literal::try_from_json(value, &field.field_type) + .and_then(|value| { + value.ok_or(Error::new( + ErrorKind::DataInvalid, + "Key of map cannot be null", + )) + }) + .ok() + }) + }), + )))) + } else { + Err(Error::new( + crate::ErrorKind::DataInvalid, + "The json value for a struct type must be an object.", + )) + } + } + Type::List(list) => { + if let JsonValue::Array(array) = value { + Ok(Some(Literal::List( + array + .into_iter() + .map(|value| { + Literal::try_from_json(value, &list.element_field.field_type) + }) + .collect::>>()?, + ))) + } else { + Err(Error::new( + crate::ErrorKind::DataInvalid, + "The json value for a list type must be an array.", + )) + } + } + Type::Map(map) => { + if let JsonValue::Object(mut object) = value { + if let (Some(JsonValue::Array(keys)), Some(JsonValue::Array(values))) = + (object.remove("keys"), object.remove("values")) + { + Ok(Some(Literal::Map(Map::from_iter( + keys.into_iter() + .zip(values.into_iter()) + .map(|(key, value)| { + Ok(( + Literal::try_from_json(key, &map.key_field.field_type) + .and_then(|value| { + value.ok_or(Error::new( + ErrorKind::DataInvalid, + "Key of map cannot be null", + )) + })?, + Literal::try_from_json(value, &map.value_field.field_type)?, + )) + }) + .collect::>>()?, + )))) + } else { + Err(Error::new( + crate::ErrorKind::DataInvalid, + "The json value for a list type must be an array.", + )) + } + } else { + Err(Error::new( + crate::ErrorKind::DataInvalid, + "The json value for a list type must be an array.", + )) + } + } + } + } + + /// Converting iceberg value to json value. + /// + /// See [this spec](https://iceberg.apache.org/spec/#json-single-value-serialization) for reference. + pub fn try_into_json(self, r#type: &Type) -> Result { + match (self, r#type) { + (Literal::Primitive(prim), _) => match prim { + PrimitiveLiteral::Boolean(val) => Ok(JsonValue::Bool(val)), + PrimitiveLiteral::Int(val) => Ok(JsonValue::Number((val).into())), + PrimitiveLiteral::Long(val) => Ok(JsonValue::Number((val).into())), + PrimitiveLiteral::Float(val) => match Number::from_f64(val.0 as f64) { + Some(number) => Ok(JsonValue::Number(number)), + None => Ok(JsonValue::Null), + }, + PrimitiveLiteral::Double(val) => match Number::from_f64(val.0) { + Some(number) => Ok(JsonValue::Number(number)), + None => Ok(JsonValue::Null), + }, + PrimitiveLiteral::Date(val) => { + Ok(JsonValue::String(date::days_to_date(val).to_string())) + } + PrimitiveLiteral::Time(val) => Ok(JsonValue::String( + time::microseconds_to_time(val).to_string(), + )), + PrimitiveLiteral::Timestamp(val) => Ok(JsonValue::String( + timestamp::microseconds_to_datetime(val) + .format("%Y-%m-%dT%H:%M:%S%.f") + .to_string(), + )), + PrimitiveLiteral::Timestamptz(val) => Ok(JsonValue::String( + timestamptz::microseconds_to_datetimetz(val) + .format("%Y-%m-%dT%H:%M:%S%.f+00:00") + .to_string(), + )), + PrimitiveLiteral::String(val) => Ok(JsonValue::String(val.clone())), + PrimitiveLiteral::UUID(val) => Ok(JsonValue::String(val.to_string())), + PrimitiveLiteral::Fixed(val) => Ok(JsonValue::String(val.iter().fold( + String::new(), + |mut acc, x| { + acc.push_str(&format!("{:x}", x)); + acc + }, + ))), + PrimitiveLiteral::Binary(val) => Ok(JsonValue::String(val.iter().fold( + String::new(), + |mut acc, x| { + acc.push_str(&format!("{:x}", x)); + acc + }, + ))), + PrimitiveLiteral::Decimal(val) => match r#type { + Type::Primitive(PrimitiveType::Decimal { + precision: _precision, + scale, + }) => { + let decimal = Decimal::try_from_i128_with_scale(val, *scale)?; + Ok(JsonValue::String(decimal.to_string())) + } + _ => Err(Error::new( + ErrorKind::DataInvalid, + "The iceberg type for decimal literal must be decimal.", + ))?, + }, + }, + (Literal::Struct(s), Type::Struct(struct_type)) => { + let mut id_and_value = Vec::with_capacity(struct_type.fields().len()); + for (value, field) in s.into_iter().zip(struct_type.fields()) { + let json = match value { + Some(val) => val.try_into_json(&field.field_type)?, + None => JsonValue::Null, + }; + id_and_value.push((field.id.to_string(), json)); + } + Ok(JsonValue::Object(JsonMap::from_iter(id_and_value))) + } + (Literal::List(list), Type::List(list_type)) => Ok(JsonValue::Array( + list.into_iter() + .map(|opt| match opt { + Some(literal) => literal.try_into_json(&list_type.element_field.field_type), + None => Ok(JsonValue::Null), + }) + .collect::>>()?, + )), + (Literal::Map(map), Type::Map(map_type)) => { + let mut object = JsonMap::with_capacity(2); + let mut json_keys = Vec::with_capacity(map.len()); + let mut json_values = Vec::with_capacity(map.len()); + for (key, value) in map.into_iter() { + json_keys.push(key.try_into_json(&map_type.key_field.field_type)?); + json_values.push(match value { + Some(literal) => literal.try_into_json(&map_type.value_field.field_type)?, + None => JsonValue::Null, + }); + } + object.insert("keys".to_string(), JsonValue::Array(json_keys)); + object.insert("values".to_string(), JsonValue::Array(json_values)); + Ok(JsonValue::Object(object)) + } + (value, r#type) => Err(Error::new( + ErrorKind::DataInvalid, + format!( + "The iceberg value {:?} doesn't fit to the iceberg type {}.", + value, r#type + ), + )), + } + } + + /// Convert Value to the any type + pub fn into_any(self) -> Box { + match self { + Literal::Primitive(prim) => match prim { + PrimitiveLiteral::Boolean(any) => Box::new(any), + PrimitiveLiteral::Int(any) => Box::new(any), + PrimitiveLiteral::Long(any) => Box::new(any), + PrimitiveLiteral::Float(any) => Box::new(any), + PrimitiveLiteral::Double(any) => Box::new(any), + PrimitiveLiteral::Date(any) => Box::new(any), + PrimitiveLiteral::Time(any) => Box::new(any), + PrimitiveLiteral::Timestamp(any) => Box::new(any), + PrimitiveLiteral::Timestamptz(any) => Box::new(any), + PrimitiveLiteral::Fixed(any) => Box::new(any), + PrimitiveLiteral::Binary(any) => Box::new(any), + PrimitiveLiteral::String(any) => Box::new(any), + PrimitiveLiteral::UUID(any) => Box::new(any), + PrimitiveLiteral::Decimal(any) => Box::new(any), + }, + _ => unimplemented!(), + } + } +} + +mod date { + use chrono::{DateTime, NaiveDate, TimeDelta, TimeZone, Utc}; + + pub(crate) fn date_to_days(date: &NaiveDate) -> i32 { + date.signed_duration_since( + // This is always the same and shouldn't fail + NaiveDate::from_ymd_opt(1970, 1, 1).unwrap(), + ) + .num_days() as i32 + } + + pub(crate) fn days_to_date(days: i32) -> NaiveDate { + // This shouldn't fail until the year 262000 + (chrono::DateTime::UNIX_EPOCH + TimeDelta::try_days(days as i64).unwrap()) + .naive_utc() + .date() + } + + /// Returns unix epoch. + pub(crate) fn unix_epoch() -> DateTime { + Utc.timestamp_nanos(0) + } + + /// Creates date literal from `NaiveDate`, assuming it's utc timezone. + pub(crate) fn date_from_naive_date(date: NaiveDate) -> i32 { + (date - unix_epoch().date_naive()).num_days() as i32 + } +} + +mod time { + use chrono::NaiveTime; + + pub(crate) fn time_to_microseconds(time: &NaiveTime) -> i64 { + time.signed_duration_since( + // This is always the same and shouldn't fail + NaiveTime::from_num_seconds_from_midnight_opt(0, 0).unwrap(), + ) + .num_microseconds() + .unwrap() + } + + pub(crate) fn microseconds_to_time(micros: i64) -> NaiveTime { + let (secs, rem) = (micros / 1_000_000, micros % 1_000_000); + + NaiveTime::from_num_seconds_from_midnight_opt(secs as u32, rem as u32 * 1_000).unwrap() + } +} + +mod timestamp { + use chrono::{DateTime, NaiveDateTime}; + + pub(crate) fn datetime_to_microseconds(time: &NaiveDateTime) -> i64 { + time.and_utc().timestamp_micros() + } + + pub(crate) fn microseconds_to_datetime(micros: i64) -> NaiveDateTime { + // This shouldn't fail until the year 262000 + DateTime::from_timestamp_micros(micros).unwrap().naive_utc() + } +} + +mod timestamptz { + use chrono::DateTime; + use chrono::Utc; + + pub(crate) fn datetimetz_to_microseconds(time: &DateTime) -> i64 { + time.timestamp_micros() + } + + pub(crate) fn microseconds_to_datetimetz(micros: i64) -> DateTime { + let (secs, rem) = (micros / 1_000_000, micros % 1_000_000); + + DateTime::from_timestamp(secs, rem as u32 * 1_000).unwrap() + } +} + +mod _serde { + use serde::{ + de::Visitor, + ser::{SerializeMap, SerializeSeq, SerializeStruct}, + Deserialize, Serialize, + }; + use serde_bytes::ByteBuf; + use serde_derive::Deserialize as DeserializeDerive; + use serde_derive::Serialize as SerializeDerive; + + use crate::{ + spec::{PrimitiveType, Type, MAP_KEY_FIELD_NAME, MAP_VALUE_FIELD_NAME}, + Error, ErrorKind, + }; + + use super::{Literal, Map, PrimitiveLiteral}; + + #[derive(SerializeDerive, DeserializeDerive, Debug)] + #[serde(transparent)] + /// Raw literal representation used for serde. The serialize way is used for Avro serializer. + pub struct RawLiteral(RawLiteralEnum); + + impl RawLiteral { + /// Covert literal to raw literal. + pub fn try_from(literal: Literal, ty: &Type) -> Result { + Ok(Self(RawLiteralEnum::try_from(literal, ty)?)) + } + + /// Convert raw literal to literal. + pub fn try_into(self, ty: &Type) -> Result, Error> { + self.0.try_into(ty) + } + } + + #[derive(SerializeDerive, Clone, Debug)] + #[serde(untagged)] + enum RawLiteralEnum { + Null, + Boolean(bool), + Int(i32), + Long(i64), + Float(f32), + Double(f64), + String(String), + Bytes(ByteBuf), + List(List), + StringMap(StringMap), + Record(Record), + } + + #[derive(Clone, Debug)] + struct Record { + required: Vec<(String, RawLiteralEnum)>, + optional: Vec<(String, Option)>, + } + + impl Serialize for Record { + fn serialize(&self, serializer: S) -> Result + where + S: serde::Serializer, + { + let len = self.required.len() + self.optional.len(); + let mut record = serializer.serialize_struct("", len)?; + for (k, v) in &self.required { + record.serialize_field(Box::leak(k.clone().into_boxed_str()), &v)?; + } + for (k, v) in &self.optional { + record.serialize_field(Box::leak(k.clone().into_boxed_str()), &v)?; + } + record.end() + } + } + + #[derive(Clone, Debug)] + struct List { + list: Vec>, + required: bool, + } + + impl Serialize for List { + fn serialize(&self, serializer: S) -> Result + where + S: serde::Serializer, + { + let mut seq = serializer.serialize_seq(Some(self.list.len()))?; + for value in &self.list { + if self.required { + seq.serialize_element(value.as_ref().ok_or_else(|| { + serde::ser::Error::custom( + "List element is required, element cannot be null", + ) + })?)?; + } else { + seq.serialize_element(&value)?; + } + } + seq.end() + } + } + + #[derive(Clone, Debug)] + struct StringMap { + raw: Vec<(String, Option)>, + required: bool, + } + + impl Serialize for StringMap { + fn serialize(&self, serializer: S) -> Result + where + S: serde::Serializer, + { + let mut map = serializer.serialize_map(Some(self.raw.len()))?; + for (k, v) in &self.raw { + if self.required { + map.serialize_entry( + k, + v.as_ref().ok_or_else(|| { + serde::ser::Error::custom( + "Map element is required, element cannot be null", + ) + })?, + )?; + } else { + map.serialize_entry(k, v)?; + } + } + map.end() + } + } + + impl<'de> Deserialize<'de> for RawLiteralEnum { + fn deserialize(deserializer: D) -> Result + where + D: serde::Deserializer<'de>, + { + struct RawLiteralVisitor; + impl<'de> Visitor<'de> for RawLiteralVisitor { + type Value = RawLiteralEnum; + + fn expecting(&self, formatter: &mut std::fmt::Formatter) -> std::fmt::Result { + formatter.write_str("expect") + } + + fn visit_bool(self, v: bool) -> Result + where + E: serde::de::Error, + { + Ok(RawLiteralEnum::Boolean(v)) + } + + fn visit_i32(self, v: i32) -> Result + where + E: serde::de::Error, + { + Ok(RawLiteralEnum::Int(v)) + } + + fn visit_i64(self, v: i64) -> Result + where + E: serde::de::Error, + { + Ok(RawLiteralEnum::Long(v)) + } + + /// Used in json + fn visit_u64(self, v: u64) -> Result + where + E: serde::de::Error, + { + Ok(RawLiteralEnum::Long(v as i64)) + } + + fn visit_f32(self, v: f32) -> Result + where + E: serde::de::Error, + { + Ok(RawLiteralEnum::Float(v)) + } + + fn visit_f64(self, v: f64) -> Result + where + E: serde::de::Error, + { + Ok(RawLiteralEnum::Double(v)) + } + + fn visit_str(self, v: &str) -> Result + where + E: serde::de::Error, + { + Ok(RawLiteralEnum::String(v.to_string())) + } + + fn visit_bytes(self, v: &[u8]) -> Result + where + E: serde::de::Error, + { + Ok(RawLiteralEnum::Bytes(ByteBuf::from(v))) + } + + fn visit_borrowed_str(self, v: &'de str) -> Result + where + E: serde::de::Error, + { + Ok(RawLiteralEnum::String(v.to_string())) + } + + fn visit_unit(self) -> Result + where + E: serde::de::Error, + { + Ok(RawLiteralEnum::Null) + } + + fn visit_map(self, mut map: A) -> Result + where + A: serde::de::MapAccess<'de>, + { + let mut required = Vec::new(); + while let Some(key) = map.next_key::()? { + let value = map.next_value::()?; + required.push((key, value)); + } + Ok(RawLiteralEnum::Record(Record { + required, + optional: Vec::new(), + })) + } + + fn visit_seq(self, mut seq: A) -> Result + where + A: serde::de::SeqAccess<'de>, + { + let mut list = Vec::new(); + while let Some(value) = seq.next_element::()? { + list.push(Some(value)); + } + Ok(RawLiteralEnum::List(List { + list, + // `required` only used in serialize, just set default in deserialize. + required: false, + })) + } + } + deserializer.deserialize_any(RawLiteralVisitor) + } + } + + impl RawLiteralEnum { + pub fn try_from(literal: Literal, ty: &Type) -> Result { + let raw = match literal { + Literal::Primitive(prim) => match prim { + super::PrimitiveLiteral::Boolean(v) => RawLiteralEnum::Boolean(v), + super::PrimitiveLiteral::Int(v) => RawLiteralEnum::Int(v), + super::PrimitiveLiteral::Long(v) => RawLiteralEnum::Long(v), + super::PrimitiveLiteral::Float(v) => RawLiteralEnum::Float(v.0), + super::PrimitiveLiteral::Double(v) => RawLiteralEnum::Double(v.0), + super::PrimitiveLiteral::Date(v) => RawLiteralEnum::Int(v), + super::PrimitiveLiteral::Time(v) => RawLiteralEnum::Long(v), + super::PrimitiveLiteral::Timestamp(v) => RawLiteralEnum::Long(v), + super::PrimitiveLiteral::Timestamptz(v) => RawLiteralEnum::Long(v), + super::PrimitiveLiteral::String(v) => RawLiteralEnum::String(v), + super::PrimitiveLiteral::UUID(v) => { + RawLiteralEnum::Bytes(ByteBuf::from(v.as_u128().to_be_bytes())) + } + super::PrimitiveLiteral::Fixed(v) => RawLiteralEnum::Bytes(ByteBuf::from(v)), + super::PrimitiveLiteral::Binary(v) => RawLiteralEnum::Bytes(ByteBuf::from(v)), + super::PrimitiveLiteral::Decimal(v) => { + RawLiteralEnum::Bytes(ByteBuf::from(v.to_be_bytes())) + } + }, + Literal::Struct(r#struct) => { + let mut required = Vec::new(); + let mut optional = Vec::new(); + if let Type::Struct(struct_ty) = ty { + for (value, field) in r#struct.into_iter().zip(struct_ty.fields()) { + if field.required { + if let Some(value) = value { + required.push(( + field.name.clone(), + RawLiteralEnum::try_from(value, &field.field_type)?, + )); + } else { + return Err(Error::new( + ErrorKind::DataInvalid, + "Can't convert null to required field", + )); + } + } else if let Some(value) = value { + optional.push(( + field.name.clone(), + Some(RawLiteralEnum::try_from(value, &field.field_type)?), + )); + } else { + optional.push((field.name.clone(), None)); + } + } + } else { + return Err(Error::new( + ErrorKind::DataInvalid, + format!("Type {} should be a struct", ty), + )); + } + RawLiteralEnum::Record(Record { required, optional }) + } + Literal::List(list) => { + if let Type::List(list_ty) = ty { + let list = list + .into_iter() + .map(|v| { + v.map(|v| { + RawLiteralEnum::try_from(v, &list_ty.element_field.field_type) + }) + .transpose() + }) + .collect::>()?; + RawLiteralEnum::List(List { + list, + required: list_ty.element_field.required, + }) + } else { + return Err(Error::new( + ErrorKind::DataInvalid, + format!("Type {} should be a list", ty), + )); + } + } + Literal::Map(map) => { + if let Type::Map(map_ty) = ty { + if let Type::Primitive(PrimitiveType::String) = *map_ty.key_field.field_type + { + let mut raw = Vec::with_capacity(map.len()); + for (k, v) in map { + if let Literal::Primitive(PrimitiveLiteral::String(k)) = k { + raw.push(( + k, + v.map(|v| { + RawLiteralEnum::try_from( + v, + &map_ty.value_field.field_type, + ) + }) + .transpose()?, + )); + } else { + return Err(Error::new( + ErrorKind::DataInvalid, + "literal type is inconsistent with type", + )); + } + } + RawLiteralEnum::StringMap(StringMap { + raw, + required: map_ty.value_field.required, + }) + } else { + let list = map.into_iter().map(|(k,v)| { + let raw_k = + RawLiteralEnum::try_from(k, &map_ty.key_field.field_type)?; + let raw_v = v + .map(|v| { + RawLiteralEnum::try_from(v, &map_ty.value_field.field_type) + }) + .transpose()?; + if map_ty.value_field.required { + Ok(Some(RawLiteralEnum::Record(Record { + required: vec![ + (MAP_KEY_FIELD_NAME.to_string(), raw_k), + (MAP_VALUE_FIELD_NAME.to_string(), raw_v.ok_or_else(||Error::new(ErrorKind::DataInvalid, "Map value is required, value cannot be null"))?), + ], + optional: vec![], + }))) + } else { + Ok(Some(RawLiteralEnum::Record(Record { + required: vec![ + (MAP_KEY_FIELD_NAME.to_string(), raw_k), + ], + optional: vec![ + (MAP_VALUE_FIELD_NAME.to_string(), raw_v) + ], + }))) + } + }).collect::>()?; + RawLiteralEnum::List(List { + list, + required: true, + }) + } + } else { + return Err(Error::new( + ErrorKind::DataInvalid, + format!("Type {} should be a map", ty), + )); + } + } + }; + Ok(raw) + } + + pub fn try_into(self, ty: &Type) -> Result, Error> { + let invalid_err = |v: &str| { + Error::new( + ErrorKind::DataInvalid, + format!( + "Unable to convert raw literal ({}) fail convert to type {} for: type mismatch", + v, ty + ), + ) + }; + let invalid_err_with_reason = |v: &str, reason: &str| { + Error::new( + ErrorKind::DataInvalid, + format!( + "Unable to convert raw literal ({}) fail convert to type {} for: {}", + v, ty, reason + ), + ) + }; + match self { + RawLiteralEnum::Null => Ok(None), + RawLiteralEnum::Boolean(v) => Ok(Some(Literal::bool(v))), + RawLiteralEnum::Int(v) => match ty { + Type::Primitive(PrimitiveType::Int) => Ok(Some(Literal::int(v))), + Type::Primitive(PrimitiveType::Long) => Ok(Some(Literal::long(i64::from(v)))), + Type::Primitive(PrimitiveType::Date) => Ok(Some(Literal::date(v))), + _ => Err(invalid_err("int")), + }, + RawLiteralEnum::Long(v) => match ty { + Type::Primitive(PrimitiveType::Int) => Ok(Some(Literal::int( + i32::try_from(v).map_err(|_| invalid_err("long"))?, + ))), + Type::Primitive(PrimitiveType::Date) => Ok(Some(Literal::date( + i32::try_from(v).map_err(|_| invalid_err("long"))?, + ))), + Type::Primitive(PrimitiveType::Long) => Ok(Some(Literal::long(v))), + Type::Primitive(PrimitiveType::Time) => Ok(Some(Literal::time(v))), + Type::Primitive(PrimitiveType::Timestamp) => Ok(Some(Literal::timestamp(v))), + Type::Primitive(PrimitiveType::Timestamptz) => { + Ok(Some(Literal::timestamptz(v))) + } + _ => Err(invalid_err("long")), + }, + RawLiteralEnum::Float(v) => match ty { + Type::Primitive(PrimitiveType::Float) => Ok(Some(Literal::float(v))), + Type::Primitive(PrimitiveType::Double) => { + Ok(Some(Literal::double(f64::from(v)))) + } + _ => Err(invalid_err("float")), + }, + RawLiteralEnum::Double(v) => match ty { + Type::Primitive(PrimitiveType::Float) => { + let v_32 = v as f32; + if v_32.is_finite() { + let v_64 = f64::from(v_32); + if (v_64 - v).abs() > f32::EPSILON as f64 { + // there is a precision loss + return Err(invalid_err("double")); + } + } + Ok(Some(Literal::float(v_32))) + } + Type::Primitive(PrimitiveType::Double) => Ok(Some(Literal::double(v))), + _ => Err(invalid_err("double")), + }, + RawLiteralEnum::String(v) => match ty { + Type::Primitive(PrimitiveType::String) => Ok(Some(Literal::string(v))), + _ => Err(invalid_err("string")), + }, + // # TODO:https://github.com/apache/iceberg-rust/issues/86 + // rust avro don't support deserialize any bytes representation now. + RawLiteralEnum::Bytes(_) => Err(invalid_err_with_reason( + "bytes", + "todo: rust avro doesn't support deserialize any bytes representation now", + )), + RawLiteralEnum::List(v) => { + match ty { + Type::List(ty) => Ok(Some(Literal::List( + v.list + .into_iter() + .map(|v| { + if let Some(v) = v { + v.try_into(&ty.element_field.field_type) + } else { + Ok(None) + } + }) + .collect::>()?, + ))), + Type::Map(map_ty) => { + let key_ty = map_ty.key_field.field_type.as_ref(); + let value_ty = map_ty.value_field.field_type.as_ref(); + let mut map = Map::new(); + for k_v in v.list { + let k_v = k_v.ok_or_else(|| invalid_err_with_reason("list","In deserialize, None will be represented as Some(RawLiteral::Null), all element in list must be valid"))?; + if let RawLiteralEnum::Record(Record { + required, + optional: _, + }) = k_v + { + if required.len() != 2 { + return Err(invalid_err_with_reason("list","Record must contains two element(key and value) of array")); + } + let mut key = None; + let mut value = None; + required.into_iter().for_each(|(k, v)| { + if k == MAP_KEY_FIELD_NAME { + key = Some(v); + } else if k == MAP_VALUE_FIELD_NAME { + value = Some(v); + } + }); + match (key, value) { + (Some(k), Some(v)) => { + let key = k.try_into(key_ty)?.ok_or_else(|| { + invalid_err_with_reason( + "list", + "Key element in Map must be valid", + ) + })?; + let value = v.try_into(value_ty)?; + if map_ty.value_field.required && value.is_none() { + return Err(invalid_err_with_reason( + "list", + "Value element is required in this Map", + )); + } + map.insert(key, value); + } + _ => return Err(invalid_err_with_reason( + "list", + "The elements of record in list are not key and value", + )), + } + } else { + return Err(invalid_err_with_reason( + "list", + "Map should represented as record array.", + )); + } + } + Ok(Some(Literal::Map(map))) + } + Type::Primitive(PrimitiveType::Uuid) => { + if v.list.len() != 16 { + return Err(invalid_err_with_reason( + "list", + "The length of list should be 16", + )); + } + let mut bytes = [0u8; 16]; + for (i, v) in v.list.iter().enumerate() { + if let Some(RawLiteralEnum::Long(v)) = v { + bytes[i] = *v as u8; + } else { + return Err(invalid_err_with_reason( + "list", + "The element of list should be int", + )); + } + } + Ok(Some(Literal::uuid(uuid::Uuid::from_bytes(bytes)))) + } + Type::Primitive(PrimitiveType::Decimal { + precision: _, + scale: _, + }) => { + if v.list.len() != 16 { + return Err(invalid_err_with_reason( + "list", + "The length of list should be 16", + )); + } + let mut bytes = [0u8; 16]; + for (i, v) in v.list.iter().enumerate() { + if let Some(RawLiteralEnum::Long(v)) = v { + bytes[i] = *v as u8; + } else { + return Err(invalid_err_with_reason( + "list", + "The element of list should be int", + )); + } + } + Ok(Some(Literal::decimal(i128::from_be_bytes(bytes)))) + } + Type::Primitive(PrimitiveType::Binary) => { + let bytes = v + .list + .into_iter() + .map(|v| { + if let Some(RawLiteralEnum::Long(v)) = v { + Ok(v as u8) + } else { + Err(invalid_err_with_reason( + "list", + "The element of list should be int", + )) + } + }) + .collect::, Error>>()?; + Ok(Some(Literal::binary(bytes))) + } + Type::Primitive(PrimitiveType::Fixed(size)) => { + if v.list.len() != *size as usize { + return Err(invalid_err_with_reason( + "list", + "The length of list should be equal to size", + )); + } + let bytes = v + .list + .into_iter() + .map(|v| { + if let Some(RawLiteralEnum::Long(v)) = v { + Ok(v as u8) + } else { + Err(invalid_err_with_reason( + "list", + "The element of list should be int", + )) + } + }) + .collect::, Error>>()?; + Ok(Some(Literal::fixed(bytes))) + } + _ => Err(invalid_err("list")), + } + } + RawLiteralEnum::Record(Record { + required, + optional: _, + }) => match ty { + Type::Struct(struct_ty) => { + let iters: Vec> = required + .into_iter() + .map(|(field_name, value)| { + let field = struct_ty + .field_by_name(field_name.as_str()) + .ok_or_else(|| { + invalid_err_with_reason( + "record", + &format!("field {} is not exist", &field_name), + ) + })?; + let value = value.try_into(&field.field_type)?; + Ok(value) + }) + .collect::>()?; + Ok(Some(Literal::Struct(super::Struct::from_iter(iters)))) + } + Type::Map(map_ty) => { + if *map_ty.key_field.field_type != Type::Primitive(PrimitiveType::String) { + return Err(invalid_err_with_reason( + "record", + "Map key must be string", + )); + } + let mut map = Map::new(); + for (k, v) in required { + let value = v.try_into(&map_ty.value_field.field_type)?; + if map_ty.value_field.required && value.is_none() { + return Err(invalid_err_with_reason( + "record", + "Value element is required in this Map", + )); + } + map.insert(Literal::string(k), value); + } + Ok(Some(Literal::Map(map))) + } + _ => Err(invalid_err("record")), + }, + RawLiteralEnum::StringMap(_) => Err(invalid_err("string map")), + } + } + } +} + +#[cfg(test)] +mod tests { + use apache_avro::{to_value, types::Value}; + + use crate::{ + avro::schema_to_avro_schema, + spec::{ + datatypes::{ListType, MapType, NestedField, StructType}, + Schema, + }, + }; + + use super::*; + + fn check_json_serde(json: &str, expected_literal: Literal, expected_type: &Type) { + let raw_json_value = serde_json::from_str::(json).unwrap(); + let desered_literal = + Literal::try_from_json(raw_json_value.clone(), expected_type).unwrap(); + assert_eq!(desered_literal, Some(expected_literal.clone())); + + let expected_json_value: JsonValue = expected_literal.try_into_json(expected_type).unwrap(); + let sered_json = serde_json::to_string(&expected_json_value).unwrap(); + let parsed_json_value = serde_json::from_str::(&sered_json).unwrap(); + + assert_eq!(parsed_json_value, raw_json_value); + } + + fn check_avro_bytes_serde(input: Vec, expected_literal: Literal, expected_type: &Type) { + let raw_schema = r#""bytes""#; + let schema = apache_avro::Schema::parse_str(raw_schema).unwrap(); + + let bytes = ByteBuf::from(input); + let literal = Literal::try_from_bytes(&bytes, expected_type).unwrap(); + assert_eq!(literal, expected_literal); + + let mut writer = apache_avro::Writer::new(&schema, Vec::new()); + writer.append_ser(ByteBuf::from(literal)).unwrap(); + let encoded = writer.into_inner().unwrap(); + let reader = apache_avro::Reader::with_schema(&schema, &*encoded).unwrap(); + + for record in reader { + let result = apache_avro::from_value::(&record.unwrap()).unwrap(); + let desered_literal = Literal::try_from_bytes(&result, expected_type).unwrap(); + assert_eq!(desered_literal, expected_literal); + } + } + + fn check_convert_with_avro(expected_literal: Literal, expected_type: &Type) { + let fields = vec![NestedField::required(1, "col", expected_type.clone()).into()]; + let schema = Schema::builder() + .with_fields(fields.clone()) + .build() + .unwrap(); + let avro_schema = schema_to_avro_schema("test", &schema).unwrap(); + let struct_type = Type::Struct(StructType::new(fields)); + let struct_literal = + Literal::Struct(Struct::from_iter(vec![Some(expected_literal.clone())])); + + let mut writer = apache_avro::Writer::new(&avro_schema, Vec::new()); + let raw_literal = RawLiteral::try_from(struct_literal.clone(), &struct_type).unwrap(); + writer.append_ser(raw_literal).unwrap(); + let encoded = writer.into_inner().unwrap(); + + let reader = apache_avro::Reader::new(&*encoded).unwrap(); + for record in reader { + let result = apache_avro::from_value::(&record.unwrap()).unwrap(); + let desered_literal = result.try_into(&struct_type).unwrap().unwrap(); + assert_eq!(desered_literal, struct_literal); + } + } + + fn check_serialize_avro(literal: Literal, ty: &Type, expect_value: Value) { + let expect_value = Value::Record(vec![("col".to_string(), expect_value)]); + + let fields = vec![NestedField::required(1, "col", ty.clone()).into()]; + let schema = Schema::builder() + .with_fields(fields.clone()) + .build() + .unwrap(); + let avro_schema = schema_to_avro_schema("test", &schema).unwrap(); + let struct_type = Type::Struct(StructType::new(fields)); + let struct_literal = Literal::Struct(Struct::from_iter(vec![Some(literal.clone())])); + let mut writer = apache_avro::Writer::new(&avro_schema, Vec::new()); + let raw_literal = RawLiteral::try_from(struct_literal.clone(), &struct_type).unwrap(); + let value = to_value(raw_literal) + .unwrap() + .resolve(&avro_schema) + .unwrap(); + writer.append_value_ref(&value).unwrap(); + let encoded = writer.into_inner().unwrap(); + + let reader = apache_avro::Reader::new(&*encoded).unwrap(); + for record in reader { + assert_eq!(record.unwrap(), expect_value); + } + } + + #[test] + fn json_boolean() { + let record = r#"true"#; + + check_json_serde( + record, + Literal::Primitive(PrimitiveLiteral::Boolean(true)), + &Type::Primitive(PrimitiveType::Boolean), + ); + } + + #[test] + fn json_int() { + let record = r#"32"#; + + check_json_serde( + record, + Literal::Primitive(PrimitiveLiteral::Int(32)), + &Type::Primitive(PrimitiveType::Int), + ); + } + + #[test] + fn json_long() { + let record = r#"32"#; + + check_json_serde( + record, + Literal::Primitive(PrimitiveLiteral::Long(32)), + &Type::Primitive(PrimitiveType::Long), + ); + } + + #[test] + fn json_float() { + let record = r#"1.0"#; + + check_json_serde( + record, + Literal::Primitive(PrimitiveLiteral::Float(OrderedFloat(1.0))), + &Type::Primitive(PrimitiveType::Float), + ); + } + + #[test] + fn json_double() { + let record = r#"1.0"#; + + check_json_serde( + record, + Literal::Primitive(PrimitiveLiteral::Double(OrderedFloat(1.0))), + &Type::Primitive(PrimitiveType::Double), + ); + } + + #[test] + fn json_date() { + let record = r#""2017-11-16""#; + + check_json_serde( + record, + Literal::Primitive(PrimitiveLiteral::Date(17486)), + &Type::Primitive(PrimitiveType::Date), + ); + } + + #[test] + fn json_time() { + let record = r#""22:31:08.123456""#; + + check_json_serde( + record, + Literal::Primitive(PrimitiveLiteral::Time(81068123456)), + &Type::Primitive(PrimitiveType::Time), + ); + } + + #[test] + fn json_timestamp() { + let record = r#""2017-11-16T22:31:08.123456""#; + + check_json_serde( + record, + Literal::Primitive(PrimitiveLiteral::Timestamp(1510871468123456)), + &Type::Primitive(PrimitiveType::Timestamp), + ); + } + + #[test] + fn json_timestamptz() { + let record = r#""2017-11-16T22:31:08.123456+00:00""#; + + check_json_serde( + record, + Literal::Primitive(PrimitiveLiteral::Timestamptz(1510871468123456)), + &Type::Primitive(PrimitiveType::Timestamptz), + ); + } + + #[test] + fn json_string() { + let record = r#""iceberg""#; + + check_json_serde( + record, + Literal::Primitive(PrimitiveLiteral::String("iceberg".to_string())), + &Type::Primitive(PrimitiveType::String), + ); + } + + #[test] + fn json_uuid() { + let record = r#""f79c3e09-677c-4bbd-a479-3f349cb785e7""#; + + check_json_serde( + record, + Literal::Primitive(PrimitiveLiteral::UUID( + Uuid::parse_str("f79c3e09-677c-4bbd-a479-3f349cb785e7").unwrap(), + )), + &Type::Primitive(PrimitiveType::Uuid), + ); + } + + #[test] + fn json_decimal() { + let record = r#""14.20""#; + + check_json_serde( + record, + Literal::Primitive(PrimitiveLiteral::Decimal(1420)), + &Type::decimal(28, 2).unwrap(), + ); + } + + #[test] + fn json_struct() { + let record = r#"{"1": 1, "2": "bar", "3": null}"#; + + check_json_serde( + record, + Literal::Struct(Struct::from_iter(vec![ + Some(Literal::Primitive(PrimitiveLiteral::Int(1))), + Some(Literal::Primitive(PrimitiveLiteral::String( + "bar".to_string(), + ))), + None, + ])), + &Type::Struct(StructType::new(vec![ + NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::optional(2, "name", Type::Primitive(PrimitiveType::String)).into(), + NestedField::optional(3, "address", Type::Primitive(PrimitiveType::String)).into(), + ])), + ); + } + + #[test] + fn json_list() { + let record = r#"[1, 2, 3, null]"#; + + check_json_serde( + record, + Literal::List(vec![ + Some(Literal::Primitive(PrimitiveLiteral::Int(1))), + Some(Literal::Primitive(PrimitiveLiteral::Int(2))), + Some(Literal::Primitive(PrimitiveLiteral::Int(3))), + None, + ]), + &Type::List(ListType { + element_field: NestedField::list_element( + 0, + Type::Primitive(PrimitiveType::Int), + true, + ) + .into(), + }), + ); + } + + #[test] + fn json_map() { + let record = r#"{ "keys": ["a", "b", "c"], "values": [1, 2, null] }"#; + + check_json_serde( + record, + Literal::Map(Map::from([ + ( + Literal::Primitive(PrimitiveLiteral::String("a".to_string())), + Some(Literal::Primitive(PrimitiveLiteral::Int(1))), + ), + ( + Literal::Primitive(PrimitiveLiteral::String("b".to_string())), + Some(Literal::Primitive(PrimitiveLiteral::Int(2))), + ), + ( + Literal::Primitive(PrimitiveLiteral::String("c".to_string())), + None, + ), + ])), + &Type::Map(MapType { + key_field: NestedField::map_key_element(0, Type::Primitive(PrimitiveType::String)) + .into(), + value_field: NestedField::map_value_element( + 1, + Type::Primitive(PrimitiveType::Int), + true, + ) + .into(), + }), + ); + } + + #[test] + fn avro_bytes_boolean() { + let bytes = vec![1u8]; + + check_avro_bytes_serde( + bytes, + Literal::Primitive(PrimitiveLiteral::Boolean(true)), + &Type::Primitive(PrimitiveType::Boolean), + ); + } + + #[test] + fn avro_bytes_int() { + let bytes = vec![32u8, 0u8, 0u8, 0u8]; + + check_avro_bytes_serde( + bytes, + Literal::Primitive(PrimitiveLiteral::Int(32)), + &Type::Primitive(PrimitiveType::Int), + ); + } + + #[test] + fn avro_bytes_long() { + let bytes = vec![32u8, 0u8, 0u8, 0u8, 0u8, 0u8, 0u8, 0u8]; + + check_avro_bytes_serde( + bytes, + Literal::Primitive(PrimitiveLiteral::Long(32)), + &Type::Primitive(PrimitiveType::Long), + ); + } + + #[test] + fn avro_bytes_float() { + let bytes = vec![0u8, 0u8, 128u8, 63u8]; + + check_avro_bytes_serde( + bytes, + Literal::Primitive(PrimitiveLiteral::Float(OrderedFloat(1.0))), + &Type::Primitive(PrimitiveType::Float), + ); + } + + #[test] + fn avro_bytes_double() { + let bytes = vec![0u8, 0u8, 0u8, 0u8, 0u8, 0u8, 240u8, 63u8]; + + check_avro_bytes_serde( + bytes, + Literal::Primitive(PrimitiveLiteral::Double(OrderedFloat(1.0))), + &Type::Primitive(PrimitiveType::Double), + ); + } + + #[test] + fn avro_bytes_string() { + let bytes = vec![105u8, 99u8, 101u8, 98u8, 101u8, 114u8, 103u8]; + + check_avro_bytes_serde( + bytes, + Literal::Primitive(PrimitiveLiteral::String("iceberg".to_string())), + &Type::Primitive(PrimitiveType::String), + ); + } + + #[test] + fn avro_convert_test_int() { + check_convert_with_avro( + Literal::Primitive(PrimitiveLiteral::Int(32)), + &Type::Primitive(PrimitiveType::Int), + ); + } + + #[test] + fn avro_convert_test_long() { + check_convert_with_avro( + Literal::Primitive(PrimitiveLiteral::Long(32)), + &Type::Primitive(PrimitiveType::Long), + ); + } + + #[test] + fn avro_convert_test_float() { + check_convert_with_avro( + Literal::Primitive(PrimitiveLiteral::Float(OrderedFloat(1.0))), + &Type::Primitive(PrimitiveType::Float), + ); + } + + #[test] + fn avro_convert_test_double() { + check_convert_with_avro( + Literal::Primitive(PrimitiveLiteral::Double(OrderedFloat(1.0))), + &Type::Primitive(PrimitiveType::Double), + ); + } + + #[test] + fn avro_convert_test_string() { + check_convert_with_avro( + Literal::Primitive(PrimitiveLiteral::String("iceberg".to_string())), + &Type::Primitive(PrimitiveType::String), + ); + } + + #[test] + fn avro_convert_test_date() { + check_convert_with_avro( + Literal::Primitive(PrimitiveLiteral::Date(17486)), + &Type::Primitive(PrimitiveType::Date), + ); + } + + #[test] + fn avro_convert_test_time() { + check_convert_with_avro( + Literal::Primitive(PrimitiveLiteral::Time(81068123456)), + &Type::Primitive(PrimitiveType::Time), + ); + } + + #[test] + fn avro_convert_test_timestamp() { + check_convert_with_avro( + Literal::Primitive(PrimitiveLiteral::Timestamp(1510871468123456)), + &Type::Primitive(PrimitiveType::Timestamp), + ); + } + + #[test] + fn avro_convert_test_timestamptz() { + check_convert_with_avro( + Literal::Primitive(PrimitiveLiteral::Timestamptz(1510871468123456)), + &Type::Primitive(PrimitiveType::Timestamptz), + ); + } + + #[test] + fn avro_convert_test_list() { + check_convert_with_avro( + Literal::List(vec![ + Some(Literal::Primitive(PrimitiveLiteral::Int(1))), + Some(Literal::Primitive(PrimitiveLiteral::Int(2))), + Some(Literal::Primitive(PrimitiveLiteral::Int(3))), + None, + ]), + &Type::List(ListType { + element_field: NestedField::list_element( + 0, + Type::Primitive(PrimitiveType::Int), + false, + ) + .into(), + }), + ); + + check_convert_with_avro( + Literal::List(vec![ + Some(Literal::Primitive(PrimitiveLiteral::Int(1))), + Some(Literal::Primitive(PrimitiveLiteral::Int(2))), + Some(Literal::Primitive(PrimitiveLiteral::Int(3))), + ]), + &Type::List(ListType { + element_field: NestedField::list_element( + 0, + Type::Primitive(PrimitiveType::Int), + true, + ) + .into(), + }), + ); + } + + fn check_convert_with_avro_map(expected_literal: Literal, expected_type: &Type) { + let fields = vec![NestedField::required(1, "col", expected_type.clone()).into()]; + let schema = Schema::builder() + .with_fields(fields.clone()) + .build() + .unwrap(); + let avro_schema = schema_to_avro_schema("test", &schema).unwrap(); + let struct_type = Type::Struct(StructType::new(fields)); + let struct_literal = + Literal::Struct(Struct::from_iter(vec![Some(expected_literal.clone())])); + + let mut writer = apache_avro::Writer::new(&avro_schema, Vec::new()); + let raw_literal = RawLiteral::try_from(struct_literal.clone(), &struct_type).unwrap(); + writer.append_ser(raw_literal).unwrap(); + let encoded = writer.into_inner().unwrap(); + + let reader = apache_avro::Reader::new(&*encoded).unwrap(); + for record in reader { + let result = apache_avro::from_value::(&record.unwrap()).unwrap(); + let desered_literal = result.try_into(&struct_type).unwrap().unwrap(); + match (&desered_literal, &struct_literal) { + (Literal::Struct(desered), Literal::Struct(expected)) => { + match (&desered.fields[0], &expected.fields[0]) { + (Literal::Map(desered), Literal::Map(expected)) => { + assert!(desered.has_same_content(expected)) + } + _ => { + unreachable!() + } + } + } + _ => { + panic!("unexpected literal type"); + } + } + } + } + + #[test] + fn avro_convert_test_map() { + check_convert_with_avro_map( + Literal::Map(Map::from([ + ( + Literal::Primitive(PrimitiveLiteral::Int(1)), + Some(Literal::Primitive(PrimitiveLiteral::Long(1))), + ), + ( + Literal::Primitive(PrimitiveLiteral::Int(2)), + Some(Literal::Primitive(PrimitiveLiteral::Long(2))), + ), + (Literal::Primitive(PrimitiveLiteral::Int(3)), None), + ])), + &Type::Map(MapType { + key_field: NestedField::map_key_element(0, Type::Primitive(PrimitiveType::Int)) + .into(), + value_field: NestedField::map_value_element( + 1, + Type::Primitive(PrimitiveType::Long), + false, + ) + .into(), + }), + ); + + check_convert_with_avro_map( + Literal::Map(Map::from([ + ( + Literal::Primitive(PrimitiveLiteral::Int(1)), + Some(Literal::Primitive(PrimitiveLiteral::Long(1))), + ), + ( + Literal::Primitive(PrimitiveLiteral::Int(2)), + Some(Literal::Primitive(PrimitiveLiteral::Long(2))), + ), + ( + Literal::Primitive(PrimitiveLiteral::Int(3)), + Some(Literal::Primitive(PrimitiveLiteral::Long(3))), + ), + ])), + &Type::Map(MapType { + key_field: NestedField::map_key_element(0, Type::Primitive(PrimitiveType::Int)) + .into(), + value_field: NestedField::map_value_element( + 1, + Type::Primitive(PrimitiveType::Long), + true, + ) + .into(), + }), + ); + } + + #[test] + fn avro_convert_test_string_map() { + check_convert_with_avro_map( + Literal::Map(Map::from([ + ( + Literal::Primitive(PrimitiveLiteral::String("a".to_string())), + Some(Literal::Primitive(PrimitiveLiteral::Int(1))), + ), + ( + Literal::Primitive(PrimitiveLiteral::String("b".to_string())), + Some(Literal::Primitive(PrimitiveLiteral::Int(2))), + ), + ( + Literal::Primitive(PrimitiveLiteral::String("c".to_string())), + None, + ), + ])), + &Type::Map(MapType { + key_field: NestedField::map_key_element(0, Type::Primitive(PrimitiveType::String)) + .into(), + value_field: NestedField::map_value_element( + 1, + Type::Primitive(PrimitiveType::Int), + false, + ) + .into(), + }), + ); + + check_convert_with_avro_map( + Literal::Map(Map::from([ + ( + Literal::Primitive(PrimitiveLiteral::String("a".to_string())), + Some(Literal::Primitive(PrimitiveLiteral::Int(1))), + ), + ( + Literal::Primitive(PrimitiveLiteral::String("b".to_string())), + Some(Literal::Primitive(PrimitiveLiteral::Int(2))), + ), + ( + Literal::Primitive(PrimitiveLiteral::String("c".to_string())), + Some(Literal::Primitive(PrimitiveLiteral::Int(3))), + ), + ])), + &Type::Map(MapType { + key_field: NestedField::map_key_element(0, Type::Primitive(PrimitiveType::String)) + .into(), + value_field: NestedField::map_value_element( + 1, + Type::Primitive(PrimitiveType::Int), + true, + ) + .into(), + }), + ); + } + + #[test] + fn avro_convert_test_record() { + check_convert_with_avro( + Literal::Struct(Struct::from_iter(vec![ + Some(Literal::Primitive(PrimitiveLiteral::Int(1))), + Some(Literal::Primitive(PrimitiveLiteral::String( + "bar".to_string(), + ))), + None, + ])), + &Type::Struct(StructType::new(vec![ + NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::optional(2, "name", Type::Primitive(PrimitiveType::String)).into(), + NestedField::optional(3, "address", Type::Primitive(PrimitiveType::String)).into(), + ])), + ); + } + + // # TODO:https://github.com/apache/iceberg-rust/issues/86 + // rust avro don't support deserialize any bytes representation now: + // - binary + // - decimal + #[test] + fn avro_convert_test_binary_ser() { + let literal = Literal::Primitive(PrimitiveLiteral::Binary(vec![1, 2, 3, 4, 5])); + let ty = Type::Primitive(PrimitiveType::Binary); + let expect_value = Value::Bytes(vec![1, 2, 3, 4, 5]); + check_serialize_avro(literal, &ty, expect_value); + } + + #[test] + fn avro_convert_test_decimal_ser() { + let literal = Literal::decimal(12345); + let ty = Type::Primitive(PrimitiveType::Decimal { + precision: 9, + scale: 8, + }); + let expect_value = Value::Decimal(apache_avro::Decimal::from(12345_i128.to_be_bytes())); + check_serialize_avro(literal, &ty, expect_value); + } + + // # TODO:https://github.com/apache/iceberg-rust/issues/86 + // rust avro can't support to convert any byte-like type to fixed in avro now. + // - uuid ser/de + // - fixed ser/de + + #[test] + fn test_parse_timestamp() { + let value = Datum::timestamp_from_str("2021-08-01T01:09:00.0899").unwrap(); + assert_eq!(&format!("{value}"), "2021-08-01 01:09:00.089900"); + + let value = Datum::timestamp_from_str("2021-08-01T01:09:00.0899+0800"); + assert!(value.is_err(), "Parse timestamp with timezone should fail!"); + + let value = Datum::timestamp_from_str("dfa"); + assert!( + value.is_err(), + "Parse timestamp with invalid input should fail!" + ); + } + + #[test] + fn test_parse_timestamptz() { + let value = Datum::timestamptz_from_str("2021-08-01T09:09:00.0899+0800").unwrap(); + assert_eq!(&format!("{value}"), "2021-08-01 01:09:00.089900 UTC"); + + let value = Datum::timestamptz_from_str("2021-08-01T01:09:00.0899"); + assert!( + value.is_err(), + "Parse timestamptz without timezone should fail!" + ); + + let value = Datum::timestamptz_from_str("dfa"); + assert!( + value.is_err(), + "Parse timestamptz with invalid input should fail!" + ); + } + + #[test] + fn test_datum_ser_deser() { + let test_fn = |datum: Datum| { + let json = serde_json::to_value(&datum).unwrap(); + let desered_datum: Datum = serde_json::from_value(json).unwrap(); + assert_eq!(datum, desered_datum); + }; + let datum = Datum::int(1); + test_fn(datum); + let datum = Datum::long(1); + test_fn(datum); + + let datum = Datum::float(1.0); + test_fn(datum); + let datum = Datum::float(0_f32); + test_fn(datum); + let datum = Datum::float(-0_f32); + test_fn(datum); + let datum = Datum::float(f32::MAX); + test_fn(datum); + let datum = Datum::float(f32::MIN); + test_fn(datum); + + // serde_json can't serialize f32::INFINITY, f32::NEG_INFINITY, f32::NAN + let datum = Datum::float(f32::INFINITY); + let json = serde_json::to_string(&datum).unwrap(); + assert!(serde_json::from_str::(&json).is_err()); + let datum = Datum::float(f32::NEG_INFINITY); + let json = serde_json::to_string(&datum).unwrap(); + assert!(serde_json::from_str::(&json).is_err()); + let datum = Datum::float(f32::NAN); + let json = serde_json::to_string(&datum).unwrap(); + assert!(serde_json::from_str::(&json).is_err()); + + let datum = Datum::double(1.0); + test_fn(datum); + let datum = Datum::double(f64::MAX); + test_fn(datum); + let datum = Datum::double(f64::MIN); + test_fn(datum); + + // serde_json can't serialize f32::INFINITY, f32::NEG_INFINITY, f32::NAN + let datum = Datum::double(f64::INFINITY); + let json = serde_json::to_string(&datum).unwrap(); + assert!(serde_json::from_str::(&json).is_err()); + let datum = Datum::double(f64::NEG_INFINITY); + let json = serde_json::to_string(&datum).unwrap(); + assert!(serde_json::from_str::(&json).is_err()); + let datum = Datum::double(f64::NAN); + let json = serde_json::to_string(&datum).unwrap(); + assert!(serde_json::from_str::(&json).is_err()); + + let datum = Datum::string("iceberg"); + test_fn(datum); + let datum = Datum::bool(true); + test_fn(datum); + let datum = Datum::date(17486); + test_fn(datum); + let datum = Datum::time_from_hms_micro(22, 15, 33, 111).unwrap(); + test_fn(datum); + let datum = Datum::timestamp_micros(1510871468123456); + test_fn(datum); + let datum = Datum::timestamptz_micros(1510871468123456); + test_fn(datum); + let datum = Datum::uuid(Uuid::parse_str("f79c3e09-677c-4bbd-a479-3f349cb785e7").unwrap()); + test_fn(datum); + let datum = Datum::decimal(1420).unwrap(); + test_fn(datum); + let datum = Datum::binary(vec![1, 2, 3, 4, 5]); + test_fn(datum); + let datum = Datum::fixed(vec![1, 2, 3, 4, 5]); + test_fn(datum); + } +} From e8b05cb3e7d356333f22405ae461511a12c9897f Mon Sep 17 00:00:00 2001 From: Chengxu Bian Date: Tue, 25 Jun 2024 19:52:13 -0400 Subject: [PATCH 12/16] add async-std ci test --- .github/workflows/ci.yml | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index f0f8f0f5b..5be609c0b 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -71,6 +71,9 @@ jobs: - name: Test run: cargo test --no-fail-fast --all-targets --all-features --workspace - + + - name: Async-std Test + run: cargo test --no-fail-fast --all-targets --no-default-features --features "async-std" --features "storage-fs" --workspace + - name: Doc Test run: cargo test --no-fail-fast --doc --all-features --workspace From df039d047cc8410063e009ce66dbbed988e46879 Mon Sep 17 00:00:00 2001 From: Chengxu Bian Date: Tue, 2 Jul 2024 11:42:27 -0400 Subject: [PATCH 13/16] rm tokio dev-dep --- crates/iceberg/Cargo.toml | 1 - 1 file changed, 1 deletion(-) diff --git a/crates/iceberg/Cargo.toml b/crates/iceberg/Cargo.toml index 0ca2bfb13..c43f54f77 100644 --- a/crates/iceberg/Cargo.toml +++ b/crates/iceberg/Cargo.toml @@ -86,4 +86,3 @@ iceberg_test_utils = { path = "../test_utils", features = ["tests"] } pretty_assertions = { workspace = true } tempfile = { workspace = true } tera = { workspace = true } -tokio = { workspace = true } From 9432b2de7831f6977001c1ff84b493786d83c810 Mon Sep 17 00:00:00 2001 From: Chengxu Bian Date: Tue, 2 Jul 2024 11:45:10 -0400 Subject: [PATCH 14/16] make tokio dev dep --- crates/iceberg/Cargo.toml | 1 + 1 file changed, 1 insertion(+) diff --git a/crates/iceberg/Cargo.toml b/crates/iceberg/Cargo.toml index e0f8408f3..b0a21badf 100644 --- a/crates/iceberg/Cargo.toml +++ b/crates/iceberg/Cargo.toml @@ -76,6 +76,7 @@ serde_json = { workspace = true } serde_repr = { workspace = true } serde_with = { workspace = true } typed-builder = { workspace = true } +tokio = { workspace = true, optional = true } url = { workspace = true } urlencoding = { workspace = true } uuid = { workspace = true } From 34eb38b5c75116ac6270aa6c34a1396477e3573d Mon Sep 17 00:00:00 2001 From: Chengxu Bian Date: Thu, 4 Jul 2024 18:37:21 -0400 Subject: [PATCH 15/16] fix sort --- crates/iceberg/Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/iceberg/Cargo.toml b/crates/iceberg/Cargo.toml index b0a21badf..0ca2bfb13 100644 --- a/crates/iceberg/Cargo.toml +++ b/crates/iceberg/Cargo.toml @@ -75,8 +75,8 @@ serde_derive = { workspace = true } serde_json = { workspace = true } serde_repr = { workspace = true } serde_with = { workspace = true } -typed-builder = { workspace = true } tokio = { workspace = true, optional = true } +typed-builder = { workspace = true } url = { workspace = true } urlencoding = { workspace = true } uuid = { workspace = true } From 12eecb82aa52e4727cb658c70d9d08829f1f6dc8 Mon Sep 17 00:00:00 2001 From: Chengxu Bian Date: Thu, 4 Jul 2024 20:24:25 -0400 Subject: [PATCH 16/16] rm tokio dev --- crates/iceberg/Cargo.toml | 1 - 1 file changed, 1 deletion(-) diff --git a/crates/iceberg/Cargo.toml b/crates/iceberg/Cargo.toml index 0ca2bfb13..c43f54f77 100644 --- a/crates/iceberg/Cargo.toml +++ b/crates/iceberg/Cargo.toml @@ -86,4 +86,3 @@ iceberg_test_utils = { path = "../test_utils", features = ["tests"] } pretty_assertions = { workspace = true } tempfile = { workspace = true } tera = { workspace = true } -tokio = { workspace = true }