From 1a5236e14b2b9b25c8dafa0e340e713b58d14530 Mon Sep 17 00:00:00 2001 From: Jan Kaul Date: Wed, 2 Aug 2023 20:15:24 +0200 Subject: [PATCH 01/42] implement values --- crates/iceberg/Cargo.toml | 1 + crates/iceberg/src/error.rs | 16 ++ crates/iceberg/src/spec/mod.rs | 1 + crates/iceberg/src/spec/values.rs | 396 ++++++++++++++++++++++++++++++ 4 files changed, 414 insertions(+) create mode 100644 crates/iceberg/src/spec/values.rs diff --git a/crates/iceberg/Cargo.toml b/crates/iceberg/Cargo.toml index 72c10c7cb..5c21f089c 100644 --- a/crates/iceberg/Cargo.toml +++ b/crates/iceberg/Cargo.toml @@ -34,6 +34,7 @@ serde_json = "^1.0" serde_derive = "^1.0" anyhow = "1.0.72" once_cell = "1" +rust_decimal = "1.31.0" [dev-dependencies] pretty_assertions = "1.4.0" diff --git a/crates/iceberg/src/error.rs b/crates/iceberg/src/error.rs index c64d35193..7ab5c916a 100644 --- a/crates/iceberg/src/error.rs +++ b/crates/iceberg/src/error.rs @@ -44,6 +44,14 @@ pub enum ErrorKind { /// /// This error is returned when given iceberg feature is not supported. FeatureUnsupported, + /// Could not cast a byte slice to appropriate array + /// + /// This error occurs when converting Bytes to iceberg values. + ByteArrayConversionFailed, + /// Could not interpret bytes as value + /// + /// This error occurs when the value can't be represented as bytes + ValueByteConversionFailed, } impl ErrorKind { @@ -59,6 +67,8 @@ impl From for &'static str { ErrorKind::Unexpected => "Unexpected", ErrorKind::DataInvalid => "DataInvalid", ErrorKind::FeatureUnsupported => "FeatureUnsupported", + ErrorKind::ByteArrayConversionFailed => "ByteArrayConversionFailed", + ErrorKind::ValueByteConversionFailed => "ValueByteConversionFailed", } } } @@ -259,6 +269,12 @@ define_from_err!( "handling invalid utf-8 characters" ); +define_from_err!( + std::array::TryFromSliceError, + ErrorKind::DataInvalid, + "failed to convert byte slive to array" +); + #[cfg(test)] mod tests { use anyhow::anyhow; diff --git a/crates/iceberg/src/spec/mod.rs b/crates/iceberg/src/spec/mod.rs index 3b998a773..eb66abcdf 100644 --- a/crates/iceberg/src/spec/mod.rs +++ b/crates/iceberg/src/spec/mod.rs @@ -19,3 +19,4 @@ pub mod datatypes; pub mod schema; +pub mod values; diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs new file mode 100644 index 000000000..1a76d42c0 --- /dev/null +++ b/crates/iceberg/src/spec/values.rs @@ -0,0 +1,396 @@ +/*! + * Value in iceberg + */ + +use std::{any::Any, collections::HashMap, fmt, ops::Deref}; + +use rust_decimal::Decimal; +use serde::{ + de::{MapAccess, Visitor}, + ser::SerializeStruct, + Deserialize, Deserializer, Serialize, +}; +use serde_bytes::ByteBuf; + +use crate::Error; + +use super::datatypes::{PrimitiveType, Type}; + +/// Values present in iceberg type +#[derive(Serialize, Deserialize, Clone, Debug, PartialEq)] +#[serde(untagged)] +pub enum Value { + /// 0x00 for false, non-zero byte for true + Boolean(bool), + /// Stored as 4-byte little-endian + Int(i32), + /// Stored as 8-byte little-endian + LongInt(i64), + /// Stored as 4-byte little-endian + Float(f32), + /// Stored as 8-byte little-endian + Double(f64), + /// 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), + /// Stores microseconds from 1970-01-01 00:00:00.000000 in an 8-byte little-endian long + Timestamp(i64), + /// Stores microseconds from 1970-01-01 00:00:00.000000 in an 8-byte little-endian long + TimestampTZ(i64), + /// UTF-8 bytes (without length) + String(String), + /// 16-byte big-endian value + UUID(i128), + /// Binary value + Fixed(usize, Vec), + /// Binary value (without length) + Binary(Vec), + /// Stores unscaled value as two’s-complement big-endian binary, + /// using the minimum number of bytes for the value + Decimal(Decimal), + /// 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(HashMap>), +} + +impl Into for Value { + fn into(self) -> ByteBuf { + match self { + Self::Boolean(val) => { + if val { + ByteBuf::from([0u8]) + } else { + ByteBuf::from([1u8]) + } + } + Self::Int(val) => ByteBuf::from(val.to_le_bytes()), + Self::LongInt(val) => ByteBuf::from(val.to_le_bytes()), + Self::Float(val) => ByteBuf::from(val.to_le_bytes()), + Self::Double(val) => ByteBuf::from(val.to_le_bytes()), + Self::Date(val) => ByteBuf::from(val.to_le_bytes()), + Self::Time(val) => ByteBuf::from(val.to_le_bytes()), + Self::Timestamp(val) => ByteBuf::from(val.to_le_bytes()), + Self::TimestampTZ(val) => ByteBuf::from(val.to_le_bytes()), + Self::String(val) => ByteBuf::from(val.as_bytes()), + Self::UUID(val) => ByteBuf::from(val.to_be_bytes()), + Self::Fixed(_, val) => ByteBuf::from(val), + Self::Binary(val) => ByteBuf::from(val), + _ => todo!(), + } + } +} + +/// 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)] +pub struct Struct { + /// Vector to store the field values + fields: Vec>, + /// A lookup that matches the field name to the entry in the vector + lookup: HashMap, +} + +impl Deref for Struct { + type Target = [Option]; + + fn deref(&self) -> &Self::Target { + &self.fields + } +} + +impl Struct { + /// Get reference to partition value + pub fn get(&self, name: &str) -> Option<&Option> { + self.fields.get(*self.lookup.get(name)?) + } + /// Get mutable reference to partition value + pub fn get_mut(&mut self, name: &str) -> Option<&mut Option> { + self.fields.get_mut(*self.lookup.get(name)?) + } +} + +impl Serialize for Struct { + fn serialize(&self, serializer: S) -> Result + where + S: serde::Serializer, + { + let mut record = serializer.serialize_struct("r102", self.fields.len())?; + for (i, value) in self.fields.iter().enumerate() { + let (key, _) = self.lookup.iter().find(|(_, value)| **value == i).unwrap(); + record.serialize_field(Box::leak(key.clone().into_boxed_str()), value)?; + } + record.end() + } +} + +impl<'de> Deserialize<'de> for Struct { + fn deserialize(deserializer: D) -> Result + where + D: Deserializer<'de>, + { + struct PartitionStructVisitor; + + impl<'de> Visitor<'de> for PartitionStructVisitor { + type Value = Struct; + + fn expecting(&self, formatter: &mut fmt::Formatter) -> fmt::Result { + formatter.write_str("map") + } + + fn visit_map(self, mut map: V) -> Result + where + V: MapAccess<'de>, + { + let mut fields: Vec> = Vec::new(); + let mut lookup: HashMap = HashMap::new(); + let mut index = 0; + while let Some(key) = map.next_key()? { + fields.push(map.next_value()?); + lookup.insert(key, index); + index += 1; + } + Ok(Struct { fields, lookup }) + } + } + deserializer.deserialize_struct( + "r102", + Box::leak(vec![].into_boxed_slice()), + PartitionStructVisitor, + ) + } +} + +impl FromIterator<(String, Option)> for Struct { + fn from_iter)>>(iter: I) -> Self { + let mut fields = Vec::new(); + let mut lookup = HashMap::new(); + + for (i, (key, value)) in iter.into_iter().enumerate() { + fields.push(value); + lookup.insert(key, i); + } + + Struct { fields, lookup } + } +} + +impl Value { + #[inline] + /// Create iceberg value from bytes + pub fn try_from_bytes(bytes: &[u8], data_type: &Type) -> Result { + match data_type { + Type::Primitive(primitive) => match primitive { + PrimitiveType::Boolean => { + if bytes.len() == 1 && bytes[0] == 0u8 { + Ok(Value::Boolean(false)) + } else { + Ok(Value::Boolean(true)) + } + } + PrimitiveType::Int => Ok(Value::Int(i32::from_le_bytes(bytes.try_into()?))), + PrimitiveType::Long => Ok(Value::LongInt(i64::from_le_bytes(bytes.try_into()?))), + PrimitiveType::Float => Ok(Value::Float(f32::from_le_bytes(bytes.try_into()?))), + PrimitiveType::Double => Ok(Value::Double(f64::from_le_bytes(bytes.try_into()?))), + PrimitiveType::Date => Ok(Value::Date(i32::from_le_bytes(bytes.try_into()?))), + PrimitiveType::Time => Ok(Value::Time(i64::from_le_bytes(bytes.try_into()?))), + PrimitiveType::Timestamp => { + Ok(Value::Timestamp(i64::from_le_bytes(bytes.try_into()?))) + } + PrimitiveType::Timestamptz => { + Ok(Value::TimestampTZ(i64::from_le_bytes(bytes.try_into()?))) + } + PrimitiveType::String => Ok(Value::String(std::str::from_utf8(bytes)?.to_string())), + PrimitiveType::Uuid => Ok(Value::UUID(i128::from_be_bytes(bytes.try_into()?))), + PrimitiveType::Fixed(len) => Ok(Value::Fixed(*len as usize, Vec::from(bytes))), + PrimitiveType::Binary => Ok(Value::Binary(Vec::from(bytes))), + _ => Err(Error::new( + crate::ErrorKind::ValueByteConversionFailed, + "Converting bytes to decimal is not supported.", + )), + }, + _ => Err(Error::new( + crate::ErrorKind::ValueByteConversionFailed, + "Converting bytes to non-primitive types is not supported.", + )), + } + } + + /// Get datatype of value + pub fn datatype(&self) -> Type { + match self { + &Value::Boolean(_) => Type::Primitive(PrimitiveType::Boolean), + &Value::Int(_) => Type::Primitive(PrimitiveType::Int), + &Value::LongInt(_) => Type::Primitive(PrimitiveType::Long), + &Value::Float(_) => Type::Primitive(PrimitiveType::Float), + &Value::Double(_) => Type::Primitive(PrimitiveType::Double), + &Value::Date(_) => Type::Primitive(PrimitiveType::Date), + &Value::Time(_) => Type::Primitive(PrimitiveType::Time), + &Value::Timestamp(_) => Type::Primitive(PrimitiveType::Timestamp), + &Value::TimestampTZ(_) => Type::Primitive(PrimitiveType::Timestamptz), + &Value::Fixed(len, _) => Type::Primitive(PrimitiveType::Fixed(len as u64)), + &Value::Binary(_) => Type::Primitive(PrimitiveType::Binary), + &Value::String(_) => Type::Primitive(PrimitiveType::String), + &Value::UUID(_) => Type::Primitive(PrimitiveType::Uuid), + &Value::Decimal(dec) => Type::Primitive(PrimitiveType::Decimal { + precision: 38, + scale: dec.scale(), + }), + _ => unimplemented!(), + } + } + + /// Convert Value to the any type + pub fn into_any(self) -> Box { + match self { + Value::Boolean(any) => Box::new(any), + Value::Int(any) => Box::new(any), + Value::LongInt(any) => Box::new(any), + Value::Float(any) => Box::new(any), + Value::Double(any) => Box::new(any), + Value::Date(any) => Box::new(any), + Value::Time(any) => Box::new(any), + Value::Timestamp(any) => Box::new(any), + Value::TimestampTZ(any) => Box::new(any), + Value::Fixed(_, any) => Box::new(any), + Value::Binary(any) => Box::new(any), + Value::String(any) => Box::new(any), + Value::UUID(any) => Box::new(any), + Value::Decimal(any) => Box::new(any), + _ => unimplemented!(), + } + } +} + +#[cfg(test)] +mod tests { + + use super::*; + + #[test] + pub fn boolean() { + let input = Value::Boolean(true); + + let raw_schema = r#"{"type": "boolean"}"#; + + let schema = apache_avro::Schema::parse_str(&raw_schema).unwrap(); + + let mut writer = apache_avro::Writer::new(&schema, Vec::new()); + + writer.append_ser(input.clone()).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(); + assert_eq!(input, result); + } + } + + #[test] + pub fn int() { + let input = Value::Int(42); + + let raw_schema = r#"{"type": "int"}"#; + + let schema = apache_avro::Schema::parse_str(&raw_schema).unwrap(); + + let mut writer = apache_avro::Writer::new(&schema, Vec::new()); + + writer.append_ser(input.clone()).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(); + assert_eq!(input, result); + } + } + + #[test] + pub fn float() { + let input = Value::Float(42.0); + + let raw_schema = r#"{"type": "float"}"#; + + let schema = apache_avro::Schema::parse_str(&raw_schema).unwrap(); + + let mut writer = apache_avro::Writer::new(&schema, Vec::new()); + + writer.append_ser(input.clone()).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(); + assert_eq!(input, result); + } + } + + #[test] + pub fn string() { + let input = Value::String("test".to_string()); + + let raw_schema = r#"{"type": "string"}"#; + + let schema = apache_avro::Schema::parse_str(&raw_schema).unwrap(); + + let mut writer = apache_avro::Writer::new(&schema, Vec::new()); + + writer.append_ser(input.clone()).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(); + assert_eq!(input, result); + } + } + + #[test] + pub fn struct_value() { + let input = Value::Struct(Struct::from_iter(vec![( + "name".to_string(), + Some(Value::String("Alice".to_string())), + )])); + + let raw_schema = r#"{"type": "record","name": "r102","fields": [{ + "name": "name", + "type": ["null","string"], + "default": null + }]}"#; + + let schema = apache_avro::Schema::parse_str(&raw_schema).unwrap(); + + let mut writer = apache_avro::Writer::new(&schema, Vec::new()); + + writer.append_ser(input.clone()).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(); + assert_eq!(input, result); + } + } +} From 1f9de91e1649f9b42dfc78be2a0b97d16b20dc24 Mon Sep 17 00:00:00 2001 From: Jan Kaul Date: Wed, 2 Aug 2023 20:22:29 +0200 Subject: [PATCH 02/42] improve getters --- crates/iceberg/src/spec/values.rs | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index 1a76d42c0..3f4821ff5 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -111,12 +111,16 @@ impl Deref for Struct { impl Struct { /// Get reference to partition value - pub fn get(&self, name: &str) -> Option<&Option> { - self.fields.get(*self.lookup.get(name)?) + pub fn get(&self, name: &str) -> Option<&Value> { + self.fields + .get(*self.lookup.get(name)?) + .and_then(|x| x.as_ref()) } /// Get mutable reference to partition value - pub fn get_mut(&mut self, name: &str) -> Option<&mut Option> { - self.fields.get_mut(*self.lookup.get(name)?) + pub fn get_mut(&mut self, name: &str) -> Option<&mut Value> { + self.fields + .get_mut(*self.lookup.get(name)?) + .and_then(|x| x.as_mut()) } } From 20f8e2cfb4459f3930920a244df90d6f4476a92a Mon Sep 17 00:00:00 2001 From: Jan Kaul Date: Wed, 2 Aug 2023 20:37:49 +0200 Subject: [PATCH 03/42] fix clippy warnings --- crates/iceberg/src/spec/values.rs | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index 3f4821ff5..5ad55b902 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -287,7 +287,7 @@ mod tests { let raw_schema = r#"{"type": "boolean"}"#; - let schema = apache_avro::Schema::parse_str(&raw_schema).unwrap(); + let schema = apache_avro::Schema::parse_str(raw_schema).unwrap(); let mut writer = apache_avro::Writer::new(&schema, Vec::new()); @@ -309,7 +309,7 @@ mod tests { let raw_schema = r#"{"type": "int"}"#; - let schema = apache_avro::Schema::parse_str(&raw_schema).unwrap(); + let schema = apache_avro::Schema::parse_str(raw_schema).unwrap(); let mut writer = apache_avro::Writer::new(&schema, Vec::new()); @@ -331,7 +331,7 @@ mod tests { let raw_schema = r#"{"type": "float"}"#; - let schema = apache_avro::Schema::parse_str(&raw_schema).unwrap(); + let schema = apache_avro::Schema::parse_str(raw_schema).unwrap(); let mut writer = apache_avro::Writer::new(&schema, Vec::new()); @@ -353,7 +353,7 @@ mod tests { let raw_schema = r#"{"type": "string"}"#; - let schema = apache_avro::Schema::parse_str(&raw_schema).unwrap(); + let schema = apache_avro::Schema::parse_str(raw_schema).unwrap(); let mut writer = apache_avro::Writer::new(&schema, Vec::new()); @@ -382,7 +382,7 @@ mod tests { "default": null }]}"#; - let schema = apache_avro::Schema::parse_str(&raw_schema).unwrap(); + let schema = apache_avro::Schema::parse_str(raw_schema).unwrap(); let mut writer = apache_avro::Writer::new(&schema, Vec::new()); From 5be7a31df12d21ca095b5e148682e40167f00e10 Mon Sep 17 00:00:00 2001 From: Jan Kaul Date: Wed, 2 Aug 2023 20:39:20 +0200 Subject: [PATCH 04/42] fix clippy warnings --- crates/iceberg/src/spec/values.rs | 28 ++++++++++++++-------------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index 5ad55b902..6a14c6fdc 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -233,20 +233,20 @@ impl Value { /// Get datatype of value pub fn datatype(&self) -> Type { match self { - &Value::Boolean(_) => Type::Primitive(PrimitiveType::Boolean), - &Value::Int(_) => Type::Primitive(PrimitiveType::Int), - &Value::LongInt(_) => Type::Primitive(PrimitiveType::Long), - &Value::Float(_) => Type::Primitive(PrimitiveType::Float), - &Value::Double(_) => Type::Primitive(PrimitiveType::Double), - &Value::Date(_) => Type::Primitive(PrimitiveType::Date), - &Value::Time(_) => Type::Primitive(PrimitiveType::Time), - &Value::Timestamp(_) => Type::Primitive(PrimitiveType::Timestamp), - &Value::TimestampTZ(_) => Type::Primitive(PrimitiveType::Timestamptz), - &Value::Fixed(len, _) => Type::Primitive(PrimitiveType::Fixed(len as u64)), - &Value::Binary(_) => Type::Primitive(PrimitiveType::Binary), - &Value::String(_) => Type::Primitive(PrimitiveType::String), - &Value::UUID(_) => Type::Primitive(PrimitiveType::Uuid), - &Value::Decimal(dec) => Type::Primitive(PrimitiveType::Decimal { + Value::Boolean(_) => Type::Primitive(PrimitiveType::Boolean), + Value::Int(_) => Type::Primitive(PrimitiveType::Int), + Value::LongInt(_) => Type::Primitive(PrimitiveType::Long), + Value::Float(_) => Type::Primitive(PrimitiveType::Float), + Value::Double(_) => Type::Primitive(PrimitiveType::Double), + Value::Date(_) => Type::Primitive(PrimitiveType::Date), + Value::Time(_) => Type::Primitive(PrimitiveType::Time), + Value::Timestamp(_) => Type::Primitive(PrimitiveType::Timestamp), + Value::TimestampTZ(_) => Type::Primitive(PrimitiveType::Timestamptz), + Value::Fixed(len, _) => Type::Primitive(PrimitiveType::Fixed(*len as u64)), + Value::Binary(_) => Type::Primitive(PrimitiveType::Binary), + Value::String(_) => Type::Primitive(PrimitiveType::String), + Value::UUID(_) => Type::Primitive(PrimitiveType::Uuid), + Value::Decimal(dec) => Type::Primitive(PrimitiveType::Decimal { precision: 38, scale: dec.scale(), }), From 2ad06e20a74a859a2d68a53e789dfdb5b1a53d4f Mon Sep 17 00:00:00 2001 From: Jan Kaul Date: Wed, 2 Aug 2023 20:42:21 +0200 Subject: [PATCH 05/42] change into bytebuf to from --- crates/iceberg/src/spec/values.rs | 32 +++++++++++++++---------------- 1 file changed, 16 insertions(+), 16 deletions(-) diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index 6a14c6fdc..971ce0d3f 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -63,28 +63,28 @@ pub enum Value { Map(HashMap>), } -impl Into for Value { - fn into(self) -> ByteBuf { - match self { - Self::Boolean(val) => { +impl From for ByteBuf { + fn from(value: Value) -> Self { + match value { + Value::Boolean(val) => { if val { ByteBuf::from([0u8]) } else { ByteBuf::from([1u8]) } } - Self::Int(val) => ByteBuf::from(val.to_le_bytes()), - Self::LongInt(val) => ByteBuf::from(val.to_le_bytes()), - Self::Float(val) => ByteBuf::from(val.to_le_bytes()), - Self::Double(val) => ByteBuf::from(val.to_le_bytes()), - Self::Date(val) => ByteBuf::from(val.to_le_bytes()), - Self::Time(val) => ByteBuf::from(val.to_le_bytes()), - Self::Timestamp(val) => ByteBuf::from(val.to_le_bytes()), - Self::TimestampTZ(val) => ByteBuf::from(val.to_le_bytes()), - Self::String(val) => ByteBuf::from(val.as_bytes()), - Self::UUID(val) => ByteBuf::from(val.to_be_bytes()), - Self::Fixed(_, val) => ByteBuf::from(val), - Self::Binary(val) => ByteBuf::from(val), + Value::Int(val) => ByteBuf::from(val.to_le_bytes()), + Value::LongInt(val) => ByteBuf::from(val.to_le_bytes()), + Value::Float(val) => ByteBuf::from(val.to_le_bytes()), + Value::Double(val) => ByteBuf::from(val.to_le_bytes()), + Value::Date(val) => ByteBuf::from(val.to_le_bytes()), + Value::Time(val) => ByteBuf::from(val.to_le_bytes()), + Value::Timestamp(val) => ByteBuf::from(val.to_le_bytes()), + Value::TimestampTZ(val) => ByteBuf::from(val.to_le_bytes()), + Value::String(val) => ByteBuf::from(val.as_bytes()), + Value::UUID(val) => ByteBuf::from(val.to_be_bytes()), + Value::Fixed(_, val) => ByteBuf::from(val), + Value::Binary(val) => ByteBuf::from(val), _ => todo!(), } } From c3fc2c6804de606caf134c0131dbd1cdc9e45bdb Mon Sep 17 00:00:00 2001 From: Jan Kaul Date: Wed, 2 Aug 2023 20:44:22 +0200 Subject: [PATCH 06/42] add license header --- crates/iceberg/src/spec/values.rs | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index 971ce0d3f..3ddafdf27 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -1,3 +1,20 @@ +// 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. + /*! * Value in iceberg */ From 50e41e02730756156f4336e5249dda4a3f0bfb24 Mon Sep 17 00:00:00 2001 From: JanKaul Date: Thu, 3 Aug 2023 08:04:41 +0200 Subject: [PATCH 07/42] Use Long instead of LongInt Co-authored-by: Renjie Liu --- crates/iceberg/src/spec/values.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index 3ddafdf27..63a5336e3 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -42,7 +42,7 @@ pub enum Value { /// Stored as 4-byte little-endian Int(i32), /// Stored as 8-byte little-endian - LongInt(i64), + Long(i64), /// Stored as 4-byte little-endian Float(f32), /// Stored as 8-byte little-endian From 4775f1610bba05c5dc0500970d131d74561d8b39 Mon Sep 17 00:00:00 2001 From: Jan Kaul Date: Thu, 3 Aug 2023 08:06:43 +0200 Subject: [PATCH 08/42] use more general error kind --- crates/iceberg/src/error.rs | 10 ---------- crates/iceberg/src/spec/values.rs | 12 ++++++------ 2 files changed, 6 insertions(+), 16 deletions(-) diff --git a/crates/iceberg/src/error.rs b/crates/iceberg/src/error.rs index 7ab5c916a..d35f2ed81 100644 --- a/crates/iceberg/src/error.rs +++ b/crates/iceberg/src/error.rs @@ -44,14 +44,6 @@ pub enum ErrorKind { /// /// This error is returned when given iceberg feature is not supported. FeatureUnsupported, - /// Could not cast a byte slice to appropriate array - /// - /// This error occurs when converting Bytes to iceberg values. - ByteArrayConversionFailed, - /// Could not interpret bytes as value - /// - /// This error occurs when the value can't be represented as bytes - ValueByteConversionFailed, } impl ErrorKind { @@ -67,8 +59,6 @@ impl From for &'static str { ErrorKind::Unexpected => "Unexpected", ErrorKind::DataInvalid => "DataInvalid", ErrorKind::FeatureUnsupported => "FeatureUnsupported", - ErrorKind::ByteArrayConversionFailed => "ByteArrayConversionFailed", - ErrorKind::ValueByteConversionFailed => "ValueByteConversionFailed", } } } diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index 63a5336e3..000bf0021 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -91,7 +91,7 @@ impl From for ByteBuf { } } Value::Int(val) => ByteBuf::from(val.to_le_bytes()), - Value::LongInt(val) => ByteBuf::from(val.to_le_bytes()), + Value::Long(val) => ByteBuf::from(val.to_le_bytes()), Value::Float(val) => ByteBuf::from(val.to_le_bytes()), Value::Double(val) => ByteBuf::from(val.to_le_bytes()), Value::Date(val) => ByteBuf::from(val.to_le_bytes()), @@ -220,7 +220,7 @@ impl Value { } } PrimitiveType::Int => Ok(Value::Int(i32::from_le_bytes(bytes.try_into()?))), - PrimitiveType::Long => Ok(Value::LongInt(i64::from_le_bytes(bytes.try_into()?))), + PrimitiveType::Long => Ok(Value::Long(i64::from_le_bytes(bytes.try_into()?))), PrimitiveType::Float => Ok(Value::Float(f32::from_le_bytes(bytes.try_into()?))), PrimitiveType::Double => Ok(Value::Double(f64::from_le_bytes(bytes.try_into()?))), PrimitiveType::Date => Ok(Value::Date(i32::from_le_bytes(bytes.try_into()?))), @@ -236,12 +236,12 @@ impl Value { PrimitiveType::Fixed(len) => Ok(Value::Fixed(*len as usize, Vec::from(bytes))), PrimitiveType::Binary => Ok(Value::Binary(Vec::from(bytes))), _ => Err(Error::new( - crate::ErrorKind::ValueByteConversionFailed, + crate::ErrorKind::DataInvalid, "Converting bytes to decimal is not supported.", )), }, _ => Err(Error::new( - crate::ErrorKind::ValueByteConversionFailed, + crate::ErrorKind::DataInvalid, "Converting bytes to non-primitive types is not supported.", )), } @@ -252,7 +252,7 @@ impl Value { match self { Value::Boolean(_) => Type::Primitive(PrimitiveType::Boolean), Value::Int(_) => Type::Primitive(PrimitiveType::Int), - Value::LongInt(_) => Type::Primitive(PrimitiveType::Long), + Value::Long(_) => Type::Primitive(PrimitiveType::Long), Value::Float(_) => Type::Primitive(PrimitiveType::Float), Value::Double(_) => Type::Primitive(PrimitiveType::Double), Value::Date(_) => Type::Primitive(PrimitiveType::Date), @@ -276,7 +276,7 @@ impl Value { match self { Value::Boolean(any) => Box::new(any), Value::Int(any) => Box::new(any), - Value::LongInt(any) => Box::new(any), + Value::Long(any) => Box::new(any), Value::Float(any) => Box::new(any), Value::Double(any) => Box::new(any), Value::Date(any) => Box::new(any), From bb597039e5da82adaeb521ff882304a5c0e05945 Mon Sep 17 00:00:00 2001 From: Jan Kaul Date: Thu, 3 Aug 2023 10:55:12 +0200 Subject: [PATCH 09/42] use Naivetime --- crates/iceberg/Cargo.toml | 1 + crates/iceberg/src/spec/values.rs | 92 +++++++++++++++++++++++++------ 2 files changed, 75 insertions(+), 18 deletions(-) diff --git a/crates/iceberg/Cargo.toml b/crates/iceberg/Cargo.toml index 5c21f089c..cc1332bdb 100644 --- a/crates/iceberg/Cargo.toml +++ b/crates/iceberg/Cargo.toml @@ -35,6 +35,7 @@ serde_derive = "^1.0" anyhow = "1.0.72" once_cell = "1" rust_decimal = "1.31.0" +chrono = "0.4" [dev-dependencies] pretty_assertions = "1.4.0" diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index 000bf0021..5d75f5211 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -21,6 +21,7 @@ use std::{any::Any, collections::HashMap, fmt, ops::Deref}; +use chrono::NaiveTime; use rust_decimal::Decimal; use serde::{ de::{MapAccess, Visitor}, @@ -50,7 +51,7 @@ pub enum Value { /// 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), + Time(#[serde(with = "time")] NaiveTime), /// Stores microseconds from 1970-01-01 00:00:00.000000 in an 8-byte little-endian long Timestamp(i64), /// Stores microseconds from 1970-01-01 00:00:00.000000 in an 8-byte little-endian long @@ -80,28 +81,31 @@ pub enum Value { Map(HashMap>), } -impl From for ByteBuf { - fn from(value: Value) -> Self { +impl TryFrom for ByteBuf { + type Error = Error; + fn try_from(value: Value) -> Result { match value { Value::Boolean(val) => { if val { - ByteBuf::from([0u8]) + Ok(ByteBuf::from([0u8])) } else { - ByteBuf::from([1u8]) + Ok(ByteBuf::from([1u8])) } } - Value::Int(val) => ByteBuf::from(val.to_le_bytes()), - Value::Long(val) => ByteBuf::from(val.to_le_bytes()), - Value::Float(val) => ByteBuf::from(val.to_le_bytes()), - Value::Double(val) => ByteBuf::from(val.to_le_bytes()), - Value::Date(val) => ByteBuf::from(val.to_le_bytes()), - Value::Time(val) => ByteBuf::from(val.to_le_bytes()), - Value::Timestamp(val) => ByteBuf::from(val.to_le_bytes()), - Value::TimestampTZ(val) => ByteBuf::from(val.to_le_bytes()), - Value::String(val) => ByteBuf::from(val.as_bytes()), - Value::UUID(val) => ByteBuf::from(val.to_be_bytes()), - Value::Fixed(_, val) => ByteBuf::from(val), - Value::Binary(val) => ByteBuf::from(val), + Value::Int(val) => Ok(ByteBuf::from(val.to_le_bytes())), + Value::Long(val) => Ok(ByteBuf::from(val.to_le_bytes())), + Value::Float(val) => Ok(ByteBuf::from(val.to_le_bytes())), + Value::Double(val) => Ok(ByteBuf::from(val.to_le_bytes())), + Value::Date(val) => Ok(ByteBuf::from(val.to_le_bytes())), + Value::Time(val) => Ok(ByteBuf::from( + time::time_to_microseconds(&val)?.to_le_bytes(), + )), + Value::Timestamp(val) => Ok(ByteBuf::from(val.to_le_bytes())), + Value::TimestampTZ(val) => Ok(ByteBuf::from(val.to_le_bytes())), + Value::String(val) => Ok(ByteBuf::from(val.as_bytes())), + Value::UUID(val) => Ok(ByteBuf::from(val.to_be_bytes())), + Value::Fixed(_, val) => Ok(ByteBuf::from(val)), + Value::Binary(val) => Ok(ByteBuf::from(val)), _ => todo!(), } } @@ -224,7 +228,9 @@ impl Value { PrimitiveType::Float => Ok(Value::Float(f32::from_le_bytes(bytes.try_into()?))), PrimitiveType::Double => Ok(Value::Double(f64::from_le_bytes(bytes.try_into()?))), PrimitiveType::Date => Ok(Value::Date(i32::from_le_bytes(bytes.try_into()?))), - PrimitiveType::Time => Ok(Value::Time(i64::from_le_bytes(bytes.try_into()?))), + PrimitiveType::Time => Ok(Value::Time(time::microseconds_to_time( + i64::from_le_bytes(bytes.try_into()?), + )?)), PrimitiveType::Timestamp => { Ok(Value::Timestamp(i64::from_le_bytes(bytes.try_into()?))) } @@ -293,6 +299,56 @@ impl Value { } } +mod time { + use chrono::NaiveTime; + use serde::{de, ser, Deserialize, Deserializer, Serialize, Serializer}; + + use crate::Error; + + pub fn serialize(value: &NaiveTime, serializer: S) -> Result + where + S: Serializer, + { + let micros = + time_to_microseconds(value).map_err(|err| ser::Error::custom(err.to_string()))?; + micros.serialize(serializer) + } + + pub fn deserialize<'de, D>(deserializer: D) -> Result + where + D: Deserializer<'de>, + { + let micros = i64::deserialize(deserializer)?; + + Ok(microseconds_to_time(micros).map_err(|err| de::Error::custom(err.to_string()))?) + } + + pub(crate) fn time_to_microseconds(time: &NaiveTime) -> Result { + time.signed_duration_since(NaiveTime::from_num_seconds_from_midnight_opt(0, 0).ok_or( + Error::new( + crate::ErrorKind::DataInvalid, + "Failed to get time from midnight", + ), + )?) + .num_microseconds() + .ok_or(Error::new( + crate::ErrorKind::DataInvalid, + "Failed to convert time to microseconds", + )) + } + + pub(crate) fn microseconds_to_time(micros: i64) -> Result { + let (secs, rem) = (micros / 1_000_000, micros % 1_000_000); + + NaiveTime::from_num_seconds_from_midnight_opt(secs as u32, rem as u32 * 1000).ok_or( + Error::new( + crate::ErrorKind::DataInvalid, + "Failed to convert microseconds to time", + ), + ) + } +} + #[cfg(test)] mod tests { From 1713492585d18619e5211d7b41fe4657eb2d3aea Mon Sep 17 00:00:00 2001 From: Jan Kaul Date: Thu, 3 Aug 2023 11:11:10 +0200 Subject: [PATCH 10/42] use naivedate --- crates/iceberg/src/spec/values.rs | 50 ++++++++++++++++++++++++++++--- 1 file changed, 46 insertions(+), 4 deletions(-) diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index 5d75f5211..db6b093ef 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -21,7 +21,7 @@ use std::{any::Any, collections::HashMap, fmt, ops::Deref}; -use chrono::NaiveTime; +use chrono::{NaiveDate, NaiveTime}; use rust_decimal::Decimal; use serde::{ de::{MapAccess, Visitor}, @@ -49,7 +49,7 @@ pub enum Value { /// Stored as 8-byte little-endian Double(f64), /// Stores days from the 1970-01-01 in an 4-byte little-endian int - Date(i32), + Date(#[serde(with = "date")] NaiveDate), /// Stores microseconds from midnight in an 8-byte little-endian long Time(#[serde(with = "time")] NaiveTime), /// Stores microseconds from 1970-01-01 00:00:00.000000 in an 8-byte little-endian long @@ -96,7 +96,7 @@ impl TryFrom for ByteBuf { Value::Long(val) => Ok(ByteBuf::from(val.to_le_bytes())), Value::Float(val) => Ok(ByteBuf::from(val.to_le_bytes())), Value::Double(val) => Ok(ByteBuf::from(val.to_le_bytes())), - Value::Date(val) => Ok(ByteBuf::from(val.to_le_bytes())), + Value::Date(val) => Ok(ByteBuf::from(date::date_to_days(&val)?.to_le_bytes())), Value::Time(val) => Ok(ByteBuf::from( time::time_to_microseconds(&val)?.to_le_bytes(), )), @@ -227,7 +227,9 @@ impl Value { PrimitiveType::Long => Ok(Value::Long(i64::from_le_bytes(bytes.try_into()?))), PrimitiveType::Float => Ok(Value::Float(f32::from_le_bytes(bytes.try_into()?))), PrimitiveType::Double => Ok(Value::Double(f64::from_le_bytes(bytes.try_into()?))), - PrimitiveType::Date => Ok(Value::Date(i32::from_le_bytes(bytes.try_into()?))), + PrimitiveType::Date => Ok(Value::Date(date::days_to_date(i32::from_le_bytes( + bytes.try_into()?, + ))?)), PrimitiveType::Time => Ok(Value::Time(time::microseconds_to_time( i64::from_le_bytes(bytes.try_into()?), )?)), @@ -299,6 +301,46 @@ impl Value { } } +mod date { + use chrono::NaiveDate; + use serde::{de, ser, Deserialize, Deserializer, Serialize, Serializer}; + + use crate::Error; + + pub fn serialize(value: &NaiveDate, serializer: S) -> Result + where + S: Serializer, + { + let days = date_to_days(value).map_err(|err| ser::Error::custom(err.to_string()))?; + days.serialize(serializer) + } + + pub fn deserialize<'de, D>(deserializer: D) -> Result + where + D: Deserializer<'de>, + { + let days = i32::deserialize(deserializer)?; + + Ok(days_to_date(days).map_err(|err| de::Error::custom(err.to_string()))?) + } + + pub(crate) fn date_to_days(date: &NaiveDate) -> Result { + Ok(date + .signed_duration_since(NaiveDate::from_ymd_opt(1970, 0, 0).ok_or(Error::new( + crate::ErrorKind::DataInvalid, + "Failed to get time from midnight", + ))?) + .num_days() as i32) + } + + pub(crate) fn days_to_date(days: i32) -> Result { + NaiveDate::from_num_days_from_ce_opt(days).ok_or(Error::new( + crate::ErrorKind::DataInvalid, + "Failed to convert microseconds to time", + )) + } +} + mod time { use chrono::NaiveTime; use serde::{de, ser, Deserialize, Deserializer, Serialize, Serializer}; From 53182e4f74fde5975b2f28265c27f572601fe317 Mon Sep 17 00:00:00 2001 From: Jan Kaul Date: Thu, 3 Aug 2023 11:28:11 +0200 Subject: [PATCH 11/42] use naivedatetime --- crates/iceberg/src/spec/values.rs | 72 ++++++++++++++++++++++++++----- 1 file changed, 61 insertions(+), 11 deletions(-) diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index db6b093ef..91ba84133 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -21,7 +21,7 @@ use std::{any::Any, collections::HashMap, fmt, ops::Deref}; -use chrono::{NaiveDate, NaiveTime}; +use chrono::{NaiveDate, NaiveDateTime, NaiveTime}; use rust_decimal::Decimal; use serde::{ de::{MapAccess, Visitor}, @@ -53,9 +53,9 @@ pub enum Value { /// Stores microseconds from midnight in an 8-byte little-endian long Time(#[serde(with = "time")] NaiveTime), /// Stores microseconds from 1970-01-01 00:00:00.000000 in an 8-byte little-endian long - Timestamp(i64), + Timestamp(#[serde(with = "timestamp")] NaiveDateTime), /// Stores microseconds from 1970-01-01 00:00:00.000000 in an 8-byte little-endian long - TimestampTZ(i64), + TimestampTZ(#[serde(with = "timestamp")] NaiveDateTime), /// UTF-8 bytes (without length) String(String), /// 16-byte big-endian value @@ -100,8 +100,12 @@ impl TryFrom for ByteBuf { Value::Time(val) => Ok(ByteBuf::from( time::time_to_microseconds(&val)?.to_le_bytes(), )), - Value::Timestamp(val) => Ok(ByteBuf::from(val.to_le_bytes())), - Value::TimestampTZ(val) => Ok(ByteBuf::from(val.to_le_bytes())), + Value::Timestamp(val) => Ok(ByteBuf::from( + timestamp::datetime_to_microseconds(&val)?.to_le_bytes(), + )), + Value::TimestampTZ(val) => Ok(ByteBuf::from( + timestamp::datetime_to_microseconds(&val)?.to_le_bytes(), + )), Value::String(val) => Ok(ByteBuf::from(val.as_bytes())), Value::UUID(val) => Ok(ByteBuf::from(val.to_be_bytes())), Value::Fixed(_, val) => Ok(ByteBuf::from(val)), @@ -233,12 +237,12 @@ impl Value { PrimitiveType::Time => Ok(Value::Time(time::microseconds_to_time( i64::from_le_bytes(bytes.try_into()?), )?)), - PrimitiveType::Timestamp => { - Ok(Value::Timestamp(i64::from_le_bytes(bytes.try_into()?))) - } - PrimitiveType::Timestamptz => { - Ok(Value::TimestampTZ(i64::from_le_bytes(bytes.try_into()?))) - } + PrimitiveType::Timestamp => Ok(Value::Timestamp( + timestamp::microseconds_to_datetime(i64::from_le_bytes(bytes.try_into()?))?, + )), + PrimitiveType::Timestamptz => Ok(Value::TimestampTZ( + timestamp::microseconds_to_datetime(i64::from_le_bytes(bytes.try_into()?))?, + )), PrimitiveType::String => Ok(Value::String(std::str::from_utf8(bytes)?.to_string())), PrimitiveType::Uuid => Ok(Value::UUID(i128::from_be_bytes(bytes.try_into()?))), PrimitiveType::Fixed(len) => Ok(Value::Fixed(*len as usize, Vec::from(bytes))), @@ -391,6 +395,52 @@ mod time { } } +mod timestamp { + use chrono::NaiveDateTime; + use serde::{de, ser, Deserialize, Deserializer, Serialize, Serializer}; + + use crate::Error; + + pub fn serialize(value: &NaiveDateTime, serializer: S) -> Result + where + S: Serializer, + { + let micros = + datetime_to_microseconds(value).map_err(|err| ser::Error::custom(err.to_string()))?; + micros.serialize(serializer) + } + + pub fn deserialize<'de, D>(deserializer: D) -> Result + where + D: Deserializer<'de>, + { + let micros = i64::deserialize(deserializer)?; + + Ok(microseconds_to_datetime(micros).map_err(|err| de::Error::custom(err.to_string()))?) + } + + pub(crate) fn datetime_to_microseconds(time: &NaiveDateTime) -> Result { + time.signed_duration_since(NaiveDateTime::from_timestamp_opt(0, 0).ok_or(Error::new( + crate::ErrorKind::DataInvalid, + "Failed to get time from midnight", + ))?) + .num_microseconds() + .ok_or(Error::new( + crate::ErrorKind::DataInvalid, + "Failed to convert time to microseconds", + )) + } + + pub(crate) fn microseconds_to_datetime(micros: i64) -> Result { + let (secs, rem) = (micros / 1_000_000, micros % 1_000_000); + + NaiveDateTime::from_timestamp_opt(secs, rem as u32 * 1000).ok_or(Error::new( + crate::ErrorKind::DataInvalid, + "Failed to convert microseconds to time", + )) + } +} + #[cfg(test)] mod tests { From af46ba1affbbf507b9fa58b933dc0ef51d62b8e2 Mon Sep 17 00:00:00 2001 From: Jan Kaul Date: Thu, 3 Aug 2023 11:28:41 +0200 Subject: [PATCH 12/42] fix clippy warnings --- crates/iceberg/src/spec/values.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index 91ba84133..9b1f17891 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -325,7 +325,7 @@ mod date { { let days = i32::deserialize(deserializer)?; - Ok(days_to_date(days).map_err(|err| de::Error::custom(err.to_string()))?) + days_to_date(days).map_err(|err| de::Error::custom(err.to_string())) } pub(crate) fn date_to_days(date: &NaiveDate) -> Result { @@ -366,7 +366,7 @@ mod time { { let micros = i64::deserialize(deserializer)?; - Ok(microseconds_to_time(micros).map_err(|err| de::Error::custom(err.to_string()))?) + microseconds_to_time(micros).map_err(|err| de::Error::custom(err.to_string())) } pub(crate) fn time_to_microseconds(time: &NaiveTime) -> Result { @@ -416,7 +416,7 @@ mod timestamp { { let micros = i64::deserialize(deserializer)?; - Ok(microseconds_to_datetime(micros).map_err(|err| de::Error::custom(err.to_string()))?) + microseconds_to_datetime(micros).map_err(|err| de::Error::custom(err.to_string())) } pub(crate) fn datetime_to_microseconds(time: &NaiveDateTime) -> Result { From 1982d4c20ac4b7e086418238ea4338bff6ec84e9 Mon Sep 17 00:00:00 2001 From: Jan Kaul Date: Thu, 3 Aug 2023 11:36:00 +0200 Subject: [PATCH 13/42] use uuid --- crates/iceberg/Cargo.toml | 1 + crates/iceberg/src/spec/values.rs | 9 ++++++--- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/crates/iceberg/Cargo.toml b/crates/iceberg/Cargo.toml index cc1332bdb..ce307ec63 100644 --- a/crates/iceberg/Cargo.toml +++ b/crates/iceberg/Cargo.toml @@ -36,6 +36,7 @@ anyhow = "1.0.72" once_cell = "1" rust_decimal = "1.31.0" chrono = "0.4" +uuid = "1.4.1" [dev-dependencies] pretty_assertions = "1.4.0" diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index 9b1f17891..5c68d490c 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -29,6 +29,7 @@ use serde::{ Deserialize, Deserializer, Serialize, }; use serde_bytes::ByteBuf; +use uuid::Uuid; use crate::Error; @@ -59,7 +60,7 @@ pub enum Value { /// UTF-8 bytes (without length) String(String), /// 16-byte big-endian value - UUID(i128), + UUID(Uuid), /// Binary value Fixed(usize, Vec), /// Binary value (without length) @@ -107,7 +108,7 @@ impl TryFrom for ByteBuf { timestamp::datetime_to_microseconds(&val)?.to_le_bytes(), )), Value::String(val) => Ok(ByteBuf::from(val.as_bytes())), - Value::UUID(val) => Ok(ByteBuf::from(val.to_be_bytes())), + Value::UUID(val) => Ok(ByteBuf::from(val.as_u128().to_be_bytes())), Value::Fixed(_, val) => Ok(ByteBuf::from(val)), Value::Binary(val) => Ok(ByteBuf::from(val)), _ => todo!(), @@ -244,7 +245,9 @@ impl Value { timestamp::microseconds_to_datetime(i64::from_le_bytes(bytes.try_into()?))?, )), PrimitiveType::String => Ok(Value::String(std::str::from_utf8(bytes)?.to_string())), - PrimitiveType::Uuid => Ok(Value::UUID(i128::from_be_bytes(bytes.try_into()?))), + PrimitiveType::Uuid => Ok(Value::UUID(Uuid::from_u128(u128::from_be_bytes( + bytes.try_into()?, + )))), PrimitiveType::Fixed(len) => Ok(Value::Fixed(*len as usize, Vec::from(bytes))), PrimitiveType::Binary => Ok(Value::Binary(Vec::from(bytes))), _ => Err(Error::new( From 7cab82b13a8e328bc63bb4fdcff39e1c814d1008 Mon Sep 17 00:00:00 2001 From: Jan Kaul Date: Thu, 3 Aug 2023 11:47:55 +0200 Subject: [PATCH 14/42] use orderedfloat --- crates/iceberg/Cargo.toml | 1 + crates/iceberg/src/spec/values.rs | 53 ++++++++++++++++++++++++++++--- 2 files changed, 49 insertions(+), 5 deletions(-) diff --git a/crates/iceberg/Cargo.toml b/crates/iceberg/Cargo.toml index ce307ec63..f827c12e3 100644 --- a/crates/iceberg/Cargo.toml +++ b/crates/iceberg/Cargo.toml @@ -37,6 +37,7 @@ once_cell = "1" rust_decimal = "1.31.0" chrono = "0.4" uuid = "1.4.1" +ordered-float = "3.7.0" [dev-dependencies] pretty_assertions = "1.4.0" diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index 5c68d490c..b8d450608 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -22,6 +22,7 @@ use std::{any::Any, collections::HashMap, fmt, ops::Deref}; use chrono::{NaiveDate, NaiveDateTime, NaiveTime}; +use ordered_float::OrderedFloat; use rust_decimal::Decimal; use serde::{ de::{MapAccess, Visitor}, @@ -46,9 +47,9 @@ pub enum Value { /// Stored as 8-byte little-endian Long(i64), /// Stored as 4-byte little-endian - Float(f32), + Float(#[serde(with = "float")] OrderedFloat), /// Stored as 8-byte little-endian - Double(f64), + Double(#[serde(with = "double")] OrderedFloat), /// Stores days from the 1970-01-01 in an 4-byte little-endian int Date(#[serde(with = "date")] NaiveDate), /// Stores microseconds from midnight in an 8-byte little-endian long @@ -230,8 +231,12 @@ impl Value { } PrimitiveType::Int => Ok(Value::Int(i32::from_le_bytes(bytes.try_into()?))), PrimitiveType::Long => Ok(Value::Long(i64::from_le_bytes(bytes.try_into()?))), - PrimitiveType::Float => Ok(Value::Float(f32::from_le_bytes(bytes.try_into()?))), - PrimitiveType::Double => Ok(Value::Double(f64::from_le_bytes(bytes.try_into()?))), + PrimitiveType::Float => Ok(Value::Float(OrderedFloat(f32::from_le_bytes( + bytes.try_into()?, + )))), + PrimitiveType::Double => Ok(Value::Double(OrderedFloat(f64::from_le_bytes( + bytes.try_into()?, + )))), PrimitiveType::Date => Ok(Value::Date(date::days_to_date(i32::from_le_bytes( bytes.try_into()?, ))?)), @@ -308,6 +313,44 @@ impl Value { } } +mod float { + use ordered_float::OrderedFloat; + use serde::{Deserialize, Deserializer, Serialize, Serializer}; + + pub fn serialize(value: &OrderedFloat, serializer: S) -> Result + where + S: Serializer, + { + value.serialize(serializer) + } + + pub fn deserialize<'de, D>(deserializer: D) -> Result, D::Error> + where + D: Deserializer<'de>, + { + f32::deserialize(deserializer).map(|x| OrderedFloat(x)) + } +} + +mod double { + use ordered_float::OrderedFloat; + use serde::{Deserialize, Deserializer, Serialize, Serializer}; + + pub fn serialize(value: &OrderedFloat, serializer: S) -> Result + where + S: Serializer, + { + value.serialize(serializer) + } + + pub fn deserialize<'de, D>(deserializer: D) -> Result, D::Error> + where + D: Deserializer<'de>, + { + f64::deserialize(deserializer).map(|x| OrderedFloat(x)) + } +} + mod date { use chrono::NaiveDate; use serde::{de, ser, Deserialize, Deserializer, Serialize, Serializer}; @@ -495,7 +538,7 @@ mod tests { #[test] pub fn float() { - let input = Value::Float(42.0); + let input = Value::Float(OrderedFloat(42.0)); let raw_schema = r#"{"type": "float"}"#; From 831f82185c1f39c31d4c69b8b8eb592a23a9fec6 Mon Sep 17 00:00:00 2001 From: Jan Kaul Date: Thu, 3 Aug 2023 11:48:26 +0200 Subject: [PATCH 15/42] fix clippy warnings --- crates/iceberg/src/spec/values.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index b8d450608..9f08e4e54 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -328,7 +328,7 @@ mod float { where D: Deserializer<'de>, { - f32::deserialize(deserializer).map(|x| OrderedFloat(x)) + f32::deserialize(deserializer).map(OrderedFloat) } } @@ -347,7 +347,7 @@ mod double { where D: Deserializer<'de>, { - f64::deserialize(deserializer).map(|x| OrderedFloat(x)) + f64::deserialize(deserializer).map(OrderedFloat) } } From 522d85f7d5b035c519b34eaf4070b91d2a98f567 Mon Sep 17 00:00:00 2001 From: Jan Kaul Date: Thu, 3 Aug 2023 12:19:05 +0200 Subject: [PATCH 16/42] fix tests --- crates/iceberg/src/spec/values.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index 9f08e4e54..c7516c11f 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -496,7 +496,7 @@ mod tests { pub fn boolean() { let input = Value::Boolean(true); - let raw_schema = r#"{"type": "boolean"}"#; + let raw_schema = r#""boolean""#; let schema = apache_avro::Schema::parse_str(raw_schema).unwrap(); @@ -518,7 +518,7 @@ mod tests { pub fn int() { let input = Value::Int(42); - let raw_schema = r#"{"type": "int"}"#; + let raw_schema = r#""int""#; let schema = apache_avro::Schema::parse_str(raw_schema).unwrap(); @@ -540,7 +540,7 @@ mod tests { pub fn float() { let input = Value::Float(OrderedFloat(42.0)); - let raw_schema = r#"{"type": "float"}"#; + let raw_schema = r#""float""#; let schema = apache_avro::Schema::parse_str(raw_schema).unwrap(); @@ -562,7 +562,7 @@ mod tests { pub fn string() { let input = Value::String("test".to_string()); - let raw_schema = r#"{"type": "string"}"#; + let raw_schema = r#""string""#; let schema = apache_avro::Schema::parse_str(raw_schema).unwrap(); From 8f916954e8d0fb8580832f89f4cea42bdbbb7765 Mon Sep 17 00:00:00 2001 From: Jan Kaul Date: Thu, 3 Aug 2023 15:25:10 +0200 Subject: [PATCH 17/42] use datatime utz --- crates/iceberg/src/spec/values.rs | 60 ++++++++++++++++++++++++++++--- 1 file changed, 56 insertions(+), 4 deletions(-) diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index c7516c11f..aa272df48 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -21,7 +21,7 @@ use std::{any::Any, collections::HashMap, fmt, ops::Deref}; -use chrono::{NaiveDate, NaiveDateTime, NaiveTime}; +use chrono::{DateTime, NaiveDate, NaiveDateTime, NaiveTime, Utc}; use ordered_float::OrderedFloat; use rust_decimal::Decimal; use serde::{ @@ -57,7 +57,7 @@ pub enum Value { /// Stores microseconds from 1970-01-01 00:00:00.000000 in an 8-byte little-endian long Timestamp(#[serde(with = "timestamp")] NaiveDateTime), /// Stores microseconds from 1970-01-01 00:00:00.000000 in an 8-byte little-endian long - TimestampTZ(#[serde(with = "timestamp")] NaiveDateTime), + TimestampTZ(#[serde(with = "timestamptz")] DateTime), /// UTF-8 bytes (without length) String(String), /// 16-byte big-endian value @@ -106,7 +106,7 @@ impl TryFrom for ByteBuf { timestamp::datetime_to_microseconds(&val)?.to_le_bytes(), )), Value::TimestampTZ(val) => Ok(ByteBuf::from( - timestamp::datetime_to_microseconds(&val)?.to_le_bytes(), + timestamptz::datetimetz_to_microseconds(&val)?.to_le_bytes(), )), Value::String(val) => Ok(ByteBuf::from(val.as_bytes())), Value::UUID(val) => Ok(ByteBuf::from(val.as_u128().to_be_bytes())), @@ -247,7 +247,7 @@ impl Value { timestamp::microseconds_to_datetime(i64::from_le_bytes(bytes.try_into()?))?, )), PrimitiveType::Timestamptz => Ok(Value::TimestampTZ( - timestamp::microseconds_to_datetime(i64::from_le_bytes(bytes.try_into()?))?, + timestamptz::microseconds_to_datetimetz(i64::from_le_bytes(bytes.try_into()?))?, )), PrimitiveType::String => Ok(Value::String(std::str::from_utf8(bytes)?.to_string())), PrimitiveType::Uuid => Ok(Value::UUID(Uuid::from_u128(u128::from_be_bytes( @@ -487,6 +487,58 @@ mod timestamp { } } +mod timestamptz { + use chrono::{DateTime, NaiveDateTime, Utc}; + use serde::{de, ser, Deserialize, Deserializer, Serialize, Serializer}; + + use crate::Error; + + pub fn serialize(value: &DateTime, serializer: S) -> Result + where + S: Serializer, + { + let micros = + datetimetz_to_microseconds(value).map_err(|err| ser::Error::custom(err.to_string()))?; + micros.serialize(serializer) + } + + pub fn deserialize<'de, D>(deserializer: D) -> Result, D::Error> + where + D: Deserializer<'de>, + { + let micros = i64::deserialize(deserializer)?; + + microseconds_to_datetimetz(micros).map_err(|err| de::Error::custom(err.to_string())) + } + + pub(crate) fn datetimetz_to_microseconds(time: &DateTime) -> Result { + time.signed_duration_since(DateTime::::from_utc( + NaiveDateTime::from_timestamp_opt(0, 0).ok_or(Error::new( + crate::ErrorKind::DataInvalid, + "Failed to get time from midnight", + ))?, + Utc, + )) + .num_microseconds() + .ok_or(Error::new( + crate::ErrorKind::DataInvalid, + "Failed to convert time to microseconds", + )) + } + + pub(crate) fn microseconds_to_datetimetz(micros: i64) -> Result, Error> { + let (secs, rem) = (micros / 1_000_000, micros % 1_000_000); + + Ok(DateTime::::from_utc( + NaiveDateTime::from_timestamp_opt(secs, rem as u32 * 1000).ok_or(Error::new( + crate::ErrorKind::DataInvalid, + "Failed to convert microseconds to time", + ))?, + Utc, + )) + } +} + #[cfg(test)] mod tests { From 8432e8af9be7fea2346d7b2a4fae8662346e4ceb Mon Sep 17 00:00:00 2001 From: Jan Kaul Date: Thu, 3 Aug 2023 15:26:22 +0200 Subject: [PATCH 18/42] fix docs --- crates/iceberg/src/spec/values.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index aa272df48..1cb804d12 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -54,9 +54,9 @@ pub enum Value { Date(#[serde(with = "date")] NaiveDate), /// Stores microseconds from midnight in an 8-byte little-endian long Time(#[serde(with = "time")] NaiveTime), - /// Stores microseconds from 1970-01-01 00:00:00.000000 in an 8-byte little-endian long + /// Timestamp without timezone Timestamp(#[serde(with = "timestamp")] NaiveDateTime), - /// Stores microseconds from 1970-01-01 00:00:00.000000 in an 8-byte little-endian long + /// Timestamp with timezone TimestampTZ(#[serde(with = "timestamptz")] DateTime), /// UTF-8 bytes (without length) String(String), From abd43271211d9ee145952158958e535028c205c5 Mon Sep 17 00:00:00 2001 From: Jan Kaul Date: Fri, 4 Aug 2023 07:32:00 +0200 Subject: [PATCH 19/42] rename value to literal --- crates/iceberg/src/spec/values.rs | 160 +++++++++++++++--------------- 1 file changed, 81 insertions(+), 79 deletions(-) diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index 1cb804d12..ebcd5cd3d 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -39,7 +39,7 @@ use super::datatypes::{PrimitiveType, Type}; /// Values present in iceberg type #[derive(Serialize, Deserialize, Clone, Debug, PartialEq)] #[serde(untagged)] -pub enum Value { +pub enum Literal { /// 0x00 for false, non-zero byte for true Boolean(bool), /// Stored as 4-byte little-endian @@ -76,42 +76,42 @@ pub enum Value { /// 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>), + 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(HashMap>), + Map(HashMap>), } -impl TryFrom for ByteBuf { +impl TryFrom for ByteBuf { type Error = Error; - fn try_from(value: Value) -> Result { + fn try_from(value: Literal) -> Result { match value { - Value::Boolean(val) => { + Literal::Boolean(val) => { if val { Ok(ByteBuf::from([0u8])) } else { Ok(ByteBuf::from([1u8])) } } - Value::Int(val) => Ok(ByteBuf::from(val.to_le_bytes())), - Value::Long(val) => Ok(ByteBuf::from(val.to_le_bytes())), - Value::Float(val) => Ok(ByteBuf::from(val.to_le_bytes())), - Value::Double(val) => Ok(ByteBuf::from(val.to_le_bytes())), - Value::Date(val) => Ok(ByteBuf::from(date::date_to_days(&val)?.to_le_bytes())), - Value::Time(val) => Ok(ByteBuf::from( + Literal::Int(val) => Ok(ByteBuf::from(val.to_le_bytes())), + Literal::Long(val) => Ok(ByteBuf::from(val.to_le_bytes())), + Literal::Float(val) => Ok(ByteBuf::from(val.to_le_bytes())), + Literal::Double(val) => Ok(ByteBuf::from(val.to_le_bytes())), + Literal::Date(val) => Ok(ByteBuf::from(date::date_to_days(&val)?.to_le_bytes())), + Literal::Time(val) => Ok(ByteBuf::from( time::time_to_microseconds(&val)?.to_le_bytes(), )), - Value::Timestamp(val) => Ok(ByteBuf::from( + Literal::Timestamp(val) => Ok(ByteBuf::from( timestamp::datetime_to_microseconds(&val)?.to_le_bytes(), )), - Value::TimestampTZ(val) => Ok(ByteBuf::from( + Literal::TimestampTZ(val) => Ok(ByteBuf::from( timestamptz::datetimetz_to_microseconds(&val)?.to_le_bytes(), )), - Value::String(val) => Ok(ByteBuf::from(val.as_bytes())), - Value::UUID(val) => Ok(ByteBuf::from(val.as_u128().to_be_bytes())), - Value::Fixed(_, val) => Ok(ByteBuf::from(val)), - Value::Binary(val) => Ok(ByteBuf::from(val)), + Literal::String(val) => Ok(ByteBuf::from(val.as_bytes())), + Literal::UUID(val) => Ok(ByteBuf::from(val.as_u128().to_be_bytes())), + Literal::Fixed(_, val) => Ok(ByteBuf::from(val)), + Literal::Binary(val) => Ok(ByteBuf::from(val)), _ => todo!(), } } @@ -123,13 +123,13 @@ impl TryFrom for ByteBuf { #[derive(Debug, Clone, PartialEq)] pub struct Struct { /// Vector to store the field values - fields: Vec>, + fields: Vec>, /// A lookup that matches the field name to the entry in the vector lookup: HashMap, } impl Deref for Struct { - type Target = [Option]; + type Target = [Option]; fn deref(&self) -> &Self::Target { &self.fields @@ -138,13 +138,13 @@ impl Deref for Struct { impl Struct { /// Get reference to partition value - pub fn get(&self, name: &str) -> Option<&Value> { + pub fn get(&self, name: &str) -> Option<&Literal> { self.fields .get(*self.lookup.get(name)?) .and_then(|x| x.as_ref()) } /// Get mutable reference to partition value - pub fn get_mut(&mut self, name: &str) -> Option<&mut Value> { + pub fn get_mut(&mut self, name: &str) -> Option<&mut Literal> { self.fields .get_mut(*self.lookup.get(name)?) .and_then(|x| x.as_mut()) @@ -183,7 +183,7 @@ impl<'de> Deserialize<'de> for Struct { where V: MapAccess<'de>, { - let mut fields: Vec> = Vec::new(); + let mut fields: Vec> = Vec::new(); let mut lookup: HashMap = HashMap::new(); let mut index = 0; while let Some(key) = map.next_key()? { @@ -202,8 +202,8 @@ impl<'de> Deserialize<'de> for Struct { } } -impl FromIterator<(String, Option)> for Struct { - fn from_iter)>>(iter: I) -> Self { +impl FromIterator<(String, Option)> for Struct { + fn from_iter)>>(iter: I) -> Self { let mut fields = Vec::new(); let mut lookup = HashMap::new(); @@ -216,7 +216,7 @@ impl FromIterator<(String, Option)> for Struct { } } -impl Value { +impl Literal { #[inline] /// Create iceberg value from bytes pub fn try_from_bytes(bytes: &[u8], data_type: &Type) -> Result { @@ -224,37 +224,39 @@ impl Value { Type::Primitive(primitive) => match primitive { PrimitiveType::Boolean => { if bytes.len() == 1 && bytes[0] == 0u8 { - Ok(Value::Boolean(false)) + Ok(Literal::Boolean(false)) } else { - Ok(Value::Boolean(true)) + Ok(Literal::Boolean(true)) } } - PrimitiveType::Int => Ok(Value::Int(i32::from_le_bytes(bytes.try_into()?))), - PrimitiveType::Long => Ok(Value::Long(i64::from_le_bytes(bytes.try_into()?))), - PrimitiveType::Float => Ok(Value::Float(OrderedFloat(f32::from_le_bytes( + PrimitiveType::Int => Ok(Literal::Int(i32::from_le_bytes(bytes.try_into()?))), + PrimitiveType::Long => Ok(Literal::Long(i64::from_le_bytes(bytes.try_into()?))), + PrimitiveType::Float => Ok(Literal::Float(OrderedFloat(f32::from_le_bytes( bytes.try_into()?, )))), - PrimitiveType::Double => Ok(Value::Double(OrderedFloat(f64::from_le_bytes( + PrimitiveType::Double => Ok(Literal::Double(OrderedFloat(f64::from_le_bytes( bytes.try_into()?, )))), - PrimitiveType::Date => Ok(Value::Date(date::days_to_date(i32::from_le_bytes( + PrimitiveType::Date => Ok(Literal::Date(date::days_to_date(i32::from_le_bytes( bytes.try_into()?, ))?)), - PrimitiveType::Time => Ok(Value::Time(time::microseconds_to_time( + PrimitiveType::Time => Ok(Literal::Time(time::microseconds_to_time( i64::from_le_bytes(bytes.try_into()?), )?)), - PrimitiveType::Timestamp => Ok(Value::Timestamp( + PrimitiveType::Timestamp => Ok(Literal::Timestamp( timestamp::microseconds_to_datetime(i64::from_le_bytes(bytes.try_into()?))?, )), - PrimitiveType::Timestamptz => Ok(Value::TimestampTZ( + PrimitiveType::Timestamptz => Ok(Literal::TimestampTZ( timestamptz::microseconds_to_datetimetz(i64::from_le_bytes(bytes.try_into()?))?, )), - PrimitiveType::String => Ok(Value::String(std::str::from_utf8(bytes)?.to_string())), - PrimitiveType::Uuid => Ok(Value::UUID(Uuid::from_u128(u128::from_be_bytes( + PrimitiveType::String => { + Ok(Literal::String(std::str::from_utf8(bytes)?.to_string())) + } + PrimitiveType::Uuid => Ok(Literal::UUID(Uuid::from_u128(u128::from_be_bytes( bytes.try_into()?, )))), - PrimitiveType::Fixed(len) => Ok(Value::Fixed(*len as usize, Vec::from(bytes))), - PrimitiveType::Binary => Ok(Value::Binary(Vec::from(bytes))), + PrimitiveType::Fixed(len) => Ok(Literal::Fixed(*len as usize, Vec::from(bytes))), + PrimitiveType::Binary => Ok(Literal::Binary(Vec::from(bytes))), _ => Err(Error::new( crate::ErrorKind::DataInvalid, "Converting bytes to decimal is not supported.", @@ -270,20 +272,20 @@ impl Value { /// Get datatype of value pub fn datatype(&self) -> Type { match self { - Value::Boolean(_) => Type::Primitive(PrimitiveType::Boolean), - Value::Int(_) => Type::Primitive(PrimitiveType::Int), - Value::Long(_) => Type::Primitive(PrimitiveType::Long), - Value::Float(_) => Type::Primitive(PrimitiveType::Float), - Value::Double(_) => Type::Primitive(PrimitiveType::Double), - Value::Date(_) => Type::Primitive(PrimitiveType::Date), - Value::Time(_) => Type::Primitive(PrimitiveType::Time), - Value::Timestamp(_) => Type::Primitive(PrimitiveType::Timestamp), - Value::TimestampTZ(_) => Type::Primitive(PrimitiveType::Timestamptz), - Value::Fixed(len, _) => Type::Primitive(PrimitiveType::Fixed(*len as u64)), - Value::Binary(_) => Type::Primitive(PrimitiveType::Binary), - Value::String(_) => Type::Primitive(PrimitiveType::String), - Value::UUID(_) => Type::Primitive(PrimitiveType::Uuid), - Value::Decimal(dec) => Type::Primitive(PrimitiveType::Decimal { + Literal::Boolean(_) => Type::Primitive(PrimitiveType::Boolean), + Literal::Int(_) => Type::Primitive(PrimitiveType::Int), + Literal::Long(_) => Type::Primitive(PrimitiveType::Long), + Literal::Float(_) => Type::Primitive(PrimitiveType::Float), + Literal::Double(_) => Type::Primitive(PrimitiveType::Double), + Literal::Date(_) => Type::Primitive(PrimitiveType::Date), + Literal::Time(_) => Type::Primitive(PrimitiveType::Time), + Literal::Timestamp(_) => Type::Primitive(PrimitiveType::Timestamp), + Literal::TimestampTZ(_) => Type::Primitive(PrimitiveType::Timestamptz), + Literal::Fixed(len, _) => Type::Primitive(PrimitiveType::Fixed(*len as u64)), + Literal::Binary(_) => Type::Primitive(PrimitiveType::Binary), + Literal::String(_) => Type::Primitive(PrimitiveType::String), + Literal::UUID(_) => Type::Primitive(PrimitiveType::Uuid), + Literal::Decimal(dec) => Type::Primitive(PrimitiveType::Decimal { precision: 38, scale: dec.scale(), }), @@ -294,20 +296,20 @@ impl Value { /// Convert Value to the any type pub fn into_any(self) -> Box { match self { - Value::Boolean(any) => Box::new(any), - Value::Int(any) => Box::new(any), - Value::Long(any) => Box::new(any), - Value::Float(any) => Box::new(any), - Value::Double(any) => Box::new(any), - Value::Date(any) => Box::new(any), - Value::Time(any) => Box::new(any), - Value::Timestamp(any) => Box::new(any), - Value::TimestampTZ(any) => Box::new(any), - Value::Fixed(_, any) => Box::new(any), - Value::Binary(any) => Box::new(any), - Value::String(any) => Box::new(any), - Value::UUID(any) => Box::new(any), - Value::Decimal(any) => Box::new(any), + Literal::Boolean(any) => Box::new(any), + Literal::Int(any) => Box::new(any), + Literal::Long(any) => Box::new(any), + Literal::Float(any) => Box::new(any), + Literal::Double(any) => Box::new(any), + Literal::Date(any) => Box::new(any), + Literal::Time(any) => Box::new(any), + Literal::Timestamp(any) => Box::new(any), + Literal::TimestampTZ(any) => Box::new(any), + Literal::Fixed(_, any) => Box::new(any), + Literal::Binary(any) => Box::new(any), + Literal::String(any) => Box::new(any), + Literal::UUID(any) => Box::new(any), + Literal::Decimal(any) => Box::new(any), _ => unimplemented!(), } } @@ -546,7 +548,7 @@ mod tests { #[test] pub fn boolean() { - let input = Value::Boolean(true); + let input = Literal::Boolean(true); let raw_schema = r#""boolean""#; @@ -561,14 +563,14 @@ mod tests { let reader = apache_avro::Reader::new(&*encoded).unwrap(); for record in reader { - let result = apache_avro::from_value::(&record.unwrap()).unwrap(); + let result = apache_avro::from_value::(&record.unwrap()).unwrap(); assert_eq!(input, result); } } #[test] pub fn int() { - let input = Value::Int(42); + let input = Literal::Int(42); let raw_schema = r#""int""#; @@ -583,14 +585,14 @@ mod tests { let reader = apache_avro::Reader::new(&*encoded).unwrap(); for record in reader { - let result = apache_avro::from_value::(&record.unwrap()).unwrap(); + let result = apache_avro::from_value::(&record.unwrap()).unwrap(); assert_eq!(input, result); } } #[test] pub fn float() { - let input = Value::Float(OrderedFloat(42.0)); + let input = Literal::Float(OrderedFloat(42.0)); let raw_schema = r#""float""#; @@ -605,14 +607,14 @@ mod tests { let reader = apache_avro::Reader::new(&*encoded).unwrap(); for record in reader { - let result = apache_avro::from_value::(&record.unwrap()).unwrap(); + let result = apache_avro::from_value::(&record.unwrap()).unwrap(); assert_eq!(input, result); } } #[test] pub fn string() { - let input = Value::String("test".to_string()); + let input = Literal::String("test".to_string()); let raw_schema = r#""string""#; @@ -627,16 +629,16 @@ mod tests { let reader = apache_avro::Reader::new(&*encoded).unwrap(); for record in reader { - let result = apache_avro::from_value::(&record.unwrap()).unwrap(); + let result = apache_avro::from_value::(&record.unwrap()).unwrap(); assert_eq!(input, result); } } #[test] pub fn struct_value() { - let input = Value::Struct(Struct::from_iter(vec![( + let input = Literal::Struct(Struct::from_iter(vec![( "name".to_string(), - Some(Value::String("Alice".to_string())), + Some(Literal::String("Alice".to_string())), )])); let raw_schema = r#"{"type": "record","name": "r102","fields": [{ @@ -656,7 +658,7 @@ mod tests { let reader = apache_avro::Reader::new(&*encoded).unwrap(); for record in reader { - let result = apache_avro::from_value::(&record.unwrap()).unwrap(); + let result = apache_avro::from_value::(&record.unwrap()).unwrap(); assert_eq!(input, result); } } From 7dda6131c5a489c01d59930f2511dbdca09c3774 Mon Sep 17 00:00:00 2001 From: Jan Kaul Date: Fri, 4 Aug 2023 07:45:58 +0200 Subject: [PATCH 20/42] introduce primitive literal --- crates/iceberg/src/spec/values.rs | 217 +++++++++++++++++------------- 1 file changed, 125 insertions(+), 92 deletions(-) diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index ebcd5cd3d..7b64746e6 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -39,7 +39,7 @@ use super::datatypes::{PrimitiveType, Type}; /// Values present in iceberg type #[derive(Serialize, Deserialize, Clone, Debug, PartialEq)] #[serde(untagged)] -pub enum Literal { +pub enum PrimitiveLiteral { /// 0x00 for false, non-zero byte for true Boolean(bool), /// Stored as 4-byte little-endian @@ -69,6 +69,14 @@ pub enum Literal { /// Stores unscaled value as two’s-complement big-endian binary, /// using the minimum number of bytes for the value Decimal(Decimal), +} + +/// Values present in iceberg type +#[derive(Serialize, Deserialize, Clone, Debug, PartialEq)] +#[serde(untagged)] +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. @@ -87,32 +95,40 @@ impl TryFrom for ByteBuf { type Error = Error; fn try_from(value: Literal) -> Result { match value { - Literal::Boolean(val) => { - if val { - Ok(ByteBuf::from([0u8])) - } else { - Ok(ByteBuf::from([1u8])) + Literal::Primitive(prim) => match prim { + PrimitiveLiteral::Boolean(val) => { + if val { + Ok(ByteBuf::from([0u8])) + } else { + Ok(ByteBuf::from([1u8])) + } } - } - Literal::Int(val) => Ok(ByteBuf::from(val.to_le_bytes())), - Literal::Long(val) => Ok(ByteBuf::from(val.to_le_bytes())), - Literal::Float(val) => Ok(ByteBuf::from(val.to_le_bytes())), - Literal::Double(val) => Ok(ByteBuf::from(val.to_le_bytes())), - Literal::Date(val) => Ok(ByteBuf::from(date::date_to_days(&val)?.to_le_bytes())), - Literal::Time(val) => Ok(ByteBuf::from( - time::time_to_microseconds(&val)?.to_le_bytes(), - )), - Literal::Timestamp(val) => Ok(ByteBuf::from( - timestamp::datetime_to_microseconds(&val)?.to_le_bytes(), - )), - Literal::TimestampTZ(val) => Ok(ByteBuf::from( - timestamptz::datetimetz_to_microseconds(&val)?.to_le_bytes(), + PrimitiveLiteral::Int(val) => Ok(ByteBuf::from(val.to_le_bytes())), + PrimitiveLiteral::Long(val) => Ok(ByteBuf::from(val.to_le_bytes())), + PrimitiveLiteral::Float(val) => Ok(ByteBuf::from(val.to_le_bytes())), + PrimitiveLiteral::Double(val) => Ok(ByteBuf::from(val.to_le_bytes())), + PrimitiveLiteral::Date(val) => { + Ok(ByteBuf::from(date::date_to_days(&val)?.to_le_bytes())) + } + PrimitiveLiteral::Time(val) => Ok(ByteBuf::from( + time::time_to_microseconds(&val)?.to_le_bytes(), + )), + PrimitiveLiteral::Timestamp(val) => Ok(ByteBuf::from( + timestamp::datetime_to_microseconds(&val)?.to_le_bytes(), + )), + PrimitiveLiteral::TimestampTZ(val) => Ok(ByteBuf::from( + timestamptz::datetimetz_to_microseconds(&val)?.to_le_bytes(), + )), + PrimitiveLiteral::String(val) => Ok(ByteBuf::from(val.as_bytes())), + PrimitiveLiteral::UUID(val) => Ok(ByteBuf::from(val.as_u128().to_be_bytes())), + PrimitiveLiteral::Fixed(_, val) => Ok(ByteBuf::from(val)), + PrimitiveLiteral::Binary(val) => Ok(ByteBuf::from(val)), + PrimitiveLiteral::Decimal(_) => todo!(), + }, + _ => Err(Error::new( + crate::ErrorKind::DataInvalid, + "Complex types can't be converted to bytes", )), - Literal::String(val) => Ok(ByteBuf::from(val.as_bytes())), - Literal::UUID(val) => Ok(ByteBuf::from(val.as_u128().to_be_bytes())), - Literal::Fixed(_, val) => Ok(ByteBuf::from(val)), - Literal::Binary(val) => Ok(ByteBuf::from(val)), - _ => todo!(), } } } @@ -224,43 +240,51 @@ impl Literal { Type::Primitive(primitive) => match primitive { PrimitiveType::Boolean => { if bytes.len() == 1 && bytes[0] == 0u8 { - Ok(Literal::Boolean(false)) + Ok(Literal::Primitive(PrimitiveLiteral::Boolean(false))) } else { - Ok(Literal::Boolean(true)) + Ok(Literal::Primitive(PrimitiveLiteral::Boolean(true))) } } - PrimitiveType::Int => Ok(Literal::Int(i32::from_le_bytes(bytes.try_into()?))), - PrimitiveType::Long => Ok(Literal::Long(i64::from_le_bytes(bytes.try_into()?))), - PrimitiveType::Float => Ok(Literal::Float(OrderedFloat(f32::from_le_bytes( - bytes.try_into()?, - )))), - PrimitiveType::Double => Ok(Literal::Double(OrderedFloat(f64::from_le_bytes( - bytes.try_into()?, - )))), - PrimitiveType::Date => Ok(Literal::Date(date::days_to_date(i32::from_le_bytes( - bytes.try_into()?, - ))?)), - PrimitiveType::Time => Ok(Literal::Time(time::microseconds_to_time( + PrimitiveType::Int => Ok(Literal::Primitive(PrimitiveLiteral::Int( + i32::from_le_bytes(bytes.try_into()?), + ))), + PrimitiveType::Long => Ok(Literal::Primitive(PrimitiveLiteral::Long( i64::from_le_bytes(bytes.try_into()?), - )?)), - PrimitiveType::Timestamp => Ok(Literal::Timestamp( + ))), + PrimitiveType::Float => Ok(Literal::Primitive(PrimitiveLiteral::Float( + OrderedFloat(f32::from_le_bytes(bytes.try_into()?)), + ))), + PrimitiveType::Double => Ok(Literal::Primitive(PrimitiveLiteral::Double( + OrderedFloat(f64::from_le_bytes(bytes.try_into()?)), + ))), + PrimitiveType::Date => Ok(Literal::Primitive(PrimitiveLiteral::Date( + date::days_to_date(i32::from_le_bytes(bytes.try_into()?))?, + ))), + PrimitiveType::Time => Ok(Literal::Primitive(PrimitiveLiteral::Time( + time::microseconds_to_time(i64::from_le_bytes(bytes.try_into()?))?, + ))), + PrimitiveType::Timestamp => Ok(Literal::Primitive(PrimitiveLiteral::Timestamp( timestamp::microseconds_to_datetime(i64::from_le_bytes(bytes.try_into()?))?, + ))), + PrimitiveType::Timestamptz => Ok(Literal::Primitive( + PrimitiveLiteral::TimestampTZ(timestamptz::microseconds_to_datetimetz( + i64::from_le_bytes(bytes.try_into()?), + )?), )), - PrimitiveType::Timestamptz => Ok(Literal::TimestampTZ( - timestamptz::microseconds_to_datetimetz(i64::from_le_bytes(bytes.try_into()?))?, - )), - PrimitiveType::String => { - Ok(Literal::String(std::str::from_utf8(bytes)?.to_string())) - } - PrimitiveType::Uuid => Ok(Literal::UUID(Uuid::from_u128(u128::from_be_bytes( - bytes.try_into()?, - )))), - PrimitiveType::Fixed(len) => Ok(Literal::Fixed(*len as usize, Vec::from(bytes))), - PrimitiveType::Binary => Ok(Literal::Binary(Vec::from(bytes))), - _ => Err(Error::new( - crate::ErrorKind::DataInvalid, - "Converting bytes to decimal is not supported.", - )), + PrimitiveType::String => Ok(Literal::Primitive(PrimitiveLiteral::String( + std::str::from_utf8(bytes)?.to_string(), + ))), + PrimitiveType::Uuid => Ok(Literal::Primitive(PrimitiveLiteral::UUID( + Uuid::from_u128(u128::from_be_bytes(bytes.try_into()?)), + ))), + PrimitiveType::Fixed(len) => Ok(Literal::Primitive(PrimitiveLiteral::Fixed( + *len as usize, + Vec::from(bytes), + ))), + PrimitiveType::Binary => Ok(Literal::Primitive(PrimitiveLiteral::Binary( + Vec::from(bytes), + ))), + _ => todo!(), }, _ => Err(Error::new( crate::ErrorKind::DataInvalid, @@ -272,23 +296,27 @@ impl Literal { /// Get datatype of value pub fn datatype(&self) -> Type { match self { - Literal::Boolean(_) => Type::Primitive(PrimitiveType::Boolean), - Literal::Int(_) => Type::Primitive(PrimitiveType::Int), - Literal::Long(_) => Type::Primitive(PrimitiveType::Long), - Literal::Float(_) => Type::Primitive(PrimitiveType::Float), - Literal::Double(_) => Type::Primitive(PrimitiveType::Double), - Literal::Date(_) => Type::Primitive(PrimitiveType::Date), - Literal::Time(_) => Type::Primitive(PrimitiveType::Time), - Literal::Timestamp(_) => Type::Primitive(PrimitiveType::Timestamp), - Literal::TimestampTZ(_) => Type::Primitive(PrimitiveType::Timestamptz), - Literal::Fixed(len, _) => Type::Primitive(PrimitiveType::Fixed(*len as u64)), - Literal::Binary(_) => Type::Primitive(PrimitiveType::Binary), - Literal::String(_) => Type::Primitive(PrimitiveType::String), - Literal::UUID(_) => Type::Primitive(PrimitiveType::Uuid), - Literal::Decimal(dec) => Type::Primitive(PrimitiveType::Decimal { - precision: 38, - scale: dec.scale(), - }), + Literal::Primitive(prim) => match prim { + PrimitiveLiteral::Boolean(_) => Type::Primitive(PrimitiveType::Boolean), + PrimitiveLiteral::Int(_) => Type::Primitive(PrimitiveType::Int), + PrimitiveLiteral::Long(_) => Type::Primitive(PrimitiveType::Long), + PrimitiveLiteral::Float(_) => Type::Primitive(PrimitiveType::Float), + PrimitiveLiteral::Double(_) => Type::Primitive(PrimitiveType::Double), + PrimitiveLiteral::Date(_) => Type::Primitive(PrimitiveType::Date), + PrimitiveLiteral::Time(_) => Type::Primitive(PrimitiveType::Time), + PrimitiveLiteral::Timestamp(_) => Type::Primitive(PrimitiveType::Timestamp), + PrimitiveLiteral::TimestampTZ(_) => Type::Primitive(PrimitiveType::Timestamptz), + PrimitiveLiteral::Fixed(len, _) => { + Type::Primitive(PrimitiveType::Fixed(*len as u64)) + } + PrimitiveLiteral::Binary(_) => Type::Primitive(PrimitiveType::Binary), + PrimitiveLiteral::String(_) => Type::Primitive(PrimitiveType::String), + PrimitiveLiteral::UUID(_) => Type::Primitive(PrimitiveType::Uuid), + PrimitiveLiteral::Decimal(dec) => Type::Primitive(PrimitiveType::Decimal { + precision: 38, + scale: dec.scale(), + }), + }, _ => unimplemented!(), } } @@ -296,20 +324,23 @@ impl Literal { /// Convert Value to the any type pub fn into_any(self) -> Box { match self { - Literal::Boolean(any) => Box::new(any), - Literal::Int(any) => Box::new(any), - Literal::Long(any) => Box::new(any), - Literal::Float(any) => Box::new(any), - Literal::Double(any) => Box::new(any), - Literal::Date(any) => Box::new(any), - Literal::Time(any) => Box::new(any), - Literal::Timestamp(any) => Box::new(any), - Literal::TimestampTZ(any) => Box::new(any), - Literal::Fixed(_, any) => Box::new(any), - Literal::Binary(any) => Box::new(any), - Literal::String(any) => Box::new(any), - Literal::UUID(any) => Box::new(any), - Literal::Decimal(any) => Box::new(any), + 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!(), } } @@ -548,7 +579,7 @@ mod tests { #[test] pub fn boolean() { - let input = Literal::Boolean(true); + let input = Literal::Primitive(PrimitiveLiteral::Boolean(true)); let raw_schema = r#""boolean""#; @@ -570,7 +601,7 @@ mod tests { #[test] pub fn int() { - let input = Literal::Int(42); + let input = Literal::Primitive(PrimitiveLiteral::Int(42)); let raw_schema = r#""int""#; @@ -592,7 +623,7 @@ mod tests { #[test] pub fn float() { - let input = Literal::Float(OrderedFloat(42.0)); + let input = Literal::Primitive(PrimitiveLiteral::Float(OrderedFloat(42.0))); let raw_schema = r#""float""#; @@ -614,7 +645,7 @@ mod tests { #[test] pub fn string() { - let input = Literal::String("test".to_string()); + let input = Literal::Primitive(PrimitiveLiteral::String("test".to_string())); let raw_schema = r#""string""#; @@ -638,7 +669,9 @@ mod tests { pub fn struct_value() { let input = Literal::Struct(Struct::from_iter(vec![( "name".to_string(), - Some(Literal::String("Alice".to_string())), + Some(Literal::Primitive(PrimitiveLiteral::String( + "Alice".to_string(), + ))), )])); let raw_schema = r#"{"type": "record","name": "r102","fields": [{ From 10d21832a8c8ce79059bdae9057d3c6a47d4b347 Mon Sep 17 00:00:00 2001 From: Jan Kaul Date: Fri, 4 Aug 2023 07:56:36 +0200 Subject: [PATCH 21/42] remove length from fixed --- crates/iceberg/src/spec/values.rs | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index 7b64746e6..efa1dd6c0 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -63,7 +63,7 @@ pub enum PrimitiveLiteral { /// 16-byte big-endian value UUID(Uuid), /// Binary value - Fixed(usize, Vec), + Fixed(Vec), /// Binary value (without length) Binary(Vec), /// Stores unscaled value as two’s-complement big-endian binary, @@ -121,7 +121,7 @@ impl TryFrom for ByteBuf { )), PrimitiveLiteral::String(val) => Ok(ByteBuf::from(val.as_bytes())), PrimitiveLiteral::UUID(val) => Ok(ByteBuf::from(val.as_u128().to_be_bytes())), - PrimitiveLiteral::Fixed(_, val) => Ok(ByteBuf::from(val)), + PrimitiveLiteral::Fixed(val) => Ok(ByteBuf::from(val)), PrimitiveLiteral::Binary(val) => Ok(ByteBuf::from(val)), PrimitiveLiteral::Decimal(_) => todo!(), }, @@ -277,8 +277,7 @@ impl Literal { PrimitiveType::Uuid => Ok(Literal::Primitive(PrimitiveLiteral::UUID( Uuid::from_u128(u128::from_be_bytes(bytes.try_into()?)), ))), - PrimitiveType::Fixed(len) => Ok(Literal::Primitive(PrimitiveLiteral::Fixed( - *len as usize, + PrimitiveType::Fixed(_) => Ok(Literal::Primitive(PrimitiveLiteral::Fixed( Vec::from(bytes), ))), PrimitiveType::Binary => Ok(Literal::Primitive(PrimitiveLiteral::Binary( @@ -306,8 +305,8 @@ impl Literal { PrimitiveLiteral::Time(_) => Type::Primitive(PrimitiveType::Time), PrimitiveLiteral::Timestamp(_) => Type::Primitive(PrimitiveType::Timestamp), PrimitiveLiteral::TimestampTZ(_) => Type::Primitive(PrimitiveType::Timestamptz), - PrimitiveLiteral::Fixed(len, _) => { - Type::Primitive(PrimitiveType::Fixed(*len as u64)) + PrimitiveLiteral::Fixed(vec) => { + Type::Primitive(PrimitiveType::Fixed(vec.len() as u64)) } PrimitiveLiteral::Binary(_) => Type::Primitive(PrimitiveType::Binary), PrimitiveLiteral::String(_) => Type::Primitive(PrimitiveType::String), @@ -334,7 +333,7 @@ impl Literal { 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::Fixed(any) => Box::new(any), PrimitiveLiteral::Binary(any) => Box::new(any), PrimitiveLiteral::String(any) => Box::new(any), PrimitiveLiteral::UUID(any) => Box::new(any), From c37640bdad239f2beebb9494f4f89c4c490f3337 Mon Sep 17 00:00:00 2001 From: Jan Kaul Date: Fri, 4 Aug 2023 10:52:57 +0200 Subject: [PATCH 22/42] serialize json via serde_json value --- crates/iceberg/src/error.rs | 18 ++ crates/iceberg/src/spec/values.rs | 274 ++++++++++++++++++++++++++++++ 2 files changed, 292 insertions(+) diff --git a/crates/iceberg/src/error.rs b/crates/iceberg/src/error.rs index d35f2ed81..6cb41db79 100644 --- a/crates/iceberg/src/error.rs +++ b/crates/iceberg/src/error.rs @@ -265,6 +265,24 @@ define_from_err!( "failed to convert byte slive to array" ); +define_from_err!( + std::num::TryFromIntError, + ErrorKind::DataInvalid, + "failed to convert integer" +); + +define_from_err!( + chrono::ParseError, + ErrorKind::DataInvalid, + "Failed to parse string to date or time" +); + +define_from_err!( + uuid::Error, + ErrorKind::DataInvalid, + "Failed to convert between uuid und iceberg value" +); + #[cfg(test)] mod tests { use anyhow::anyhow; diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index efa1dd6c0..49938349c 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -30,6 +30,7 @@ use serde::{ Deserialize, Deserializer, Serialize, }; use serde_bytes::ByteBuf; +use serde_json::{Number, Value as JsonValue}; use uuid::Uuid; use crate::Error; @@ -133,6 +134,60 @@ impl TryFrom for ByteBuf { } } +impl TryFrom for JsonValue { + type Error = Error; + fn try_from(value: Literal) -> Result { + match value { + 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) => Ok(JsonValue::Number( + Number::from_f64(val.0 as f64).ok_or(Error::new( + crate::ErrorKind::DataInvalid, + "Failed to convert float to json", + ))?, + )), + PrimitiveLiteral::Double(val) => Ok(JsonValue::Number( + Number::from_f64(val.0).ok_or(Error::new( + crate::ErrorKind::DataInvalid, + "Failed to convert float to json", + ))?, + )), + PrimitiveLiteral::Date(val) => Ok(JsonValue::String(val.to_string())), + PrimitiveLiteral::Time(val) => Ok(JsonValue::String(val.to_string())), + PrimitiveLiteral::Timestamp(val) => Ok(JsonValue::String( + val.format("%Y-%m-%dT%H:%M:%S%.f").to_string(), + )), + PrimitiveLiteral::TimestampTZ(val) => Ok(JsonValue::String( + val.format("%Y-%m-%dT%H:%M:%S%.f+00:00").to_string(), + )), + PrimitiveLiteral::String(val) => Ok(JsonValue::String(val)), + PrimitiveLiteral::UUID(val) => Ok(JsonValue::String(val.to_string())), + PrimitiveLiteral::Fixed(val) => Ok(JsonValue::String(val.into_iter().fold( + String::new(), + |mut acc, x| { + acc.push_str(&format!("{:x}", x)); + acc + }, + ))), + PrimitiveLiteral::Binary(val) => Ok(JsonValue::String(val.into_iter().fold( + String::new(), + |mut acc, x| { + acc.push_str(&format!("{:x}", x)); + acc + }, + ))), + PrimitiveLiteral::Decimal(_) => todo!(), + }, + _ => Err(Error::new( + crate::ErrorKind::DataInvalid, + "Complex types can't be converted to bytes", + )), + } + } +} + /// 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. @@ -292,6 +347,76 @@ impl Literal { } } + /// 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(Literal::Primitive(PrimitiveLiteral::Boolean(bool))) + } + (PrimitiveType::Int, JsonValue::Number(number)) => { + Ok(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(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(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(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(Literal::Primitive( + PrimitiveLiteral::Date(NaiveDate::parse_from_str(&s, "%Y-%m-%d")?), + )), + (PrimitiveType::Time, JsonValue::String(s)) => Ok(Literal::Primitive( + PrimitiveLiteral::Time(NaiveTime::parse_from_str(&s, "%H:%M:%S%.f")?), + )), + (PrimitiveType::Timestamp, JsonValue::String(s)) => { + Ok(Literal::Primitive(PrimitiveLiteral::Timestamp( + NaiveDateTime::parse_from_str(&s, "%Y-%m-%dT%H:%M:%S%.f")?, + ))) + } + (PrimitiveType::Timestamptz, JsonValue::String(s)) => Ok(Literal::Primitive( + PrimitiveLiteral::TimestampTZ(DateTime::from_utc( + NaiveDateTime::parse_from_str(&s, "%Y-%m-%dT%H:%M:%S%.f+00:00")?, + Utc, + )), + )), + (PrimitiveType::String, JsonValue::String(s)) => { + Ok(Literal::Primitive(PrimitiveLiteral::String(s))) + } + (PrimitiveType::Uuid, JsonValue::String(s)) => Ok(Literal::Primitive( + PrimitiveLiteral::UUID(Uuid::parse_str(&s)?), + )), + (PrimitiveType::Fixed(_), JsonValue::String(_)) => todo!(), + (PrimitiveType::Binary, JsonValue::String(_)) => todo!(), + _ => todo!(), + }, + _ => Err(Error::new( + crate::ErrorKind::DataInvalid, + "Converting bytes to non-primitive types is not supported.", + )), + } + } + /// Get datatype of value pub fn datatype(&self) -> Type { match self { @@ -576,6 +701,155 @@ mod tests { 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, expected_type).unwrap(); + assert_eq!(desered_literal, expected_literal); + + let expected_json_value: JsonValue = expected_literal.try_into().unwrap(); + let sered_json = serde_json::to_string(&expected_json_value).unwrap(); + let parsed_json_value = serde_json::from_str::(&sered_json).unwrap(); + let raw_json_value = serde_json::from_str::(json).unwrap(); + + assert_eq!(parsed_json_value, raw_json_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( + NaiveDate::from_ymd_opt(2017, 11, 16).unwrap(), + )), + &Type::Primitive(PrimitiveType::Date), + ); + } + + #[test] + fn json_time() { + let record = r#""22:31:08.123456""#; + + check_json_serde( + record, + Literal::Primitive(PrimitiveLiteral::Time( + NaiveTime::from_hms_micro_opt(22, 31, 8, 123456).unwrap(), + )), + &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(NaiveDateTime::new( + NaiveDate::from_ymd_opt(2017, 11, 16).unwrap(), + NaiveTime::from_hms_micro_opt(22, 31, 8, 123456).unwrap(), + ))), + &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(DateTime::::from_utc( + NaiveDateTime::new( + NaiveDate::from_ymd_opt(2017, 11, 16).unwrap(), + NaiveTime::from_hms_micro_opt(22, 31, 8, 123456).unwrap(), + ), + Utc, + ))), + &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] pub fn boolean() { let input = Literal::Primitive(PrimitiveLiteral::Boolean(true)); From ab583d7097b033f15e34166a381434a7e7a81f26 Mon Sep 17 00:00:00 2001 From: Jan Kaul Date: Fri, 4 Aug 2023 11:06:59 +0200 Subject: [PATCH 23/42] remove derive serialize/deserialize --- crates/iceberg/src/spec/values.rs | 345 +----------------------------- 1 file changed, 9 insertions(+), 336 deletions(-) diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index 49938349c..445fe793d 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -19,16 +19,11 @@ * Value in iceberg */ -use std::{any::Any, collections::HashMap, fmt, ops::Deref}; +use std::{any::Any, collections::HashMap}; use chrono::{DateTime, NaiveDate, NaiveDateTime, NaiveTime, Utc}; use ordered_float::OrderedFloat; use rust_decimal::Decimal; -use serde::{ - de::{MapAccess, Visitor}, - ser::SerializeStruct, - Deserialize, Deserializer, Serialize, -}; use serde_bytes::ByteBuf; use serde_json::{Number, Value as JsonValue}; use uuid::Uuid; @@ -38,8 +33,7 @@ use crate::Error; use super::datatypes::{PrimitiveType, Type}; /// Values present in iceberg type -#[derive(Serialize, Deserialize, Clone, Debug, PartialEq)] -#[serde(untagged)] +#[derive(Clone, Debug, PartialEq)] pub enum PrimitiveLiteral { /// 0x00 for false, non-zero byte for true Boolean(bool), @@ -48,17 +42,17 @@ pub enum PrimitiveLiteral { /// Stored as 8-byte little-endian Long(i64), /// Stored as 4-byte little-endian - Float(#[serde(with = "float")] OrderedFloat), + Float(OrderedFloat), /// Stored as 8-byte little-endian - Double(#[serde(with = "double")] OrderedFloat), + Double(OrderedFloat), /// Stores days from the 1970-01-01 in an 4-byte little-endian int - Date(#[serde(with = "date")] NaiveDate), + Date(NaiveDate), /// Stores microseconds from midnight in an 8-byte little-endian long - Time(#[serde(with = "time")] NaiveTime), + Time(NaiveTime), /// Timestamp without timezone - Timestamp(#[serde(with = "timestamp")] NaiveDateTime), + Timestamp(NaiveDateTime), /// Timestamp with timezone - TimestampTZ(#[serde(with = "timestamptz")] DateTime), + TimestampTZ(DateTime), /// UTF-8 bytes (without length) String(String), /// 16-byte big-endian value @@ -73,8 +67,7 @@ pub enum PrimitiveLiteral { } /// Values present in iceberg type -#[derive(Serialize, Deserialize, Clone, Debug, PartialEq)] -#[serde(untagged)] +#[derive(Clone, Debug, PartialEq)] pub enum Literal { /// A primitive value Primitive(PrimitiveLiteral), @@ -199,94 +192,6 @@ pub struct Struct { lookup: HashMap, } -impl Deref for Struct { - type Target = [Option]; - - fn deref(&self) -> &Self::Target { - &self.fields - } -} - -impl Struct { - /// Get reference to partition value - pub fn get(&self, name: &str) -> Option<&Literal> { - self.fields - .get(*self.lookup.get(name)?) - .and_then(|x| x.as_ref()) - } - /// Get mutable reference to partition value - pub fn get_mut(&mut self, name: &str) -> Option<&mut Literal> { - self.fields - .get_mut(*self.lookup.get(name)?) - .and_then(|x| x.as_mut()) - } -} - -impl Serialize for Struct { - fn serialize(&self, serializer: S) -> Result - where - S: serde::Serializer, - { - let mut record = serializer.serialize_struct("r102", self.fields.len())?; - for (i, value) in self.fields.iter().enumerate() { - let (key, _) = self.lookup.iter().find(|(_, value)| **value == i).unwrap(); - record.serialize_field(Box::leak(key.clone().into_boxed_str()), value)?; - } - record.end() - } -} - -impl<'de> Deserialize<'de> for Struct { - fn deserialize(deserializer: D) -> Result - where - D: Deserializer<'de>, - { - struct PartitionStructVisitor; - - impl<'de> Visitor<'de> for PartitionStructVisitor { - type Value = Struct; - - fn expecting(&self, formatter: &mut fmt::Formatter) -> fmt::Result { - formatter.write_str("map") - } - - fn visit_map(self, mut map: V) -> Result - where - V: MapAccess<'de>, - { - let mut fields: Vec> = Vec::new(); - let mut lookup: HashMap = HashMap::new(); - let mut index = 0; - while let Some(key) = map.next_key()? { - fields.push(map.next_value()?); - lookup.insert(key, index); - index += 1; - } - Ok(Struct { fields, lookup }) - } - } - deserializer.deserialize_struct( - "r102", - Box::leak(vec![].into_boxed_slice()), - PartitionStructVisitor, - ) - } -} - -impl FromIterator<(String, Option)> for Struct { - fn from_iter)>>(iter: I) -> Self { - let mut fields = Vec::new(); - let mut lookup = HashMap::new(); - - for (i, (key, value)) in iter.into_iter().enumerate() { - fields.push(value); - lookup.insert(key, i); - } - - Struct { fields, lookup } - } -} - impl Literal { #[inline] /// Create iceberg value from bytes @@ -470,67 +375,11 @@ impl Literal { } } -mod float { - use ordered_float::OrderedFloat; - use serde::{Deserialize, Deserializer, Serialize, Serializer}; - - pub fn serialize(value: &OrderedFloat, serializer: S) -> Result - where - S: Serializer, - { - value.serialize(serializer) - } - - pub fn deserialize<'de, D>(deserializer: D) -> Result, D::Error> - where - D: Deserializer<'de>, - { - f32::deserialize(deserializer).map(OrderedFloat) - } -} - -mod double { - use ordered_float::OrderedFloat; - use serde::{Deserialize, Deserializer, Serialize, Serializer}; - - pub fn serialize(value: &OrderedFloat, serializer: S) -> Result - where - S: Serializer, - { - value.serialize(serializer) - } - - pub fn deserialize<'de, D>(deserializer: D) -> Result, D::Error> - where - D: Deserializer<'de>, - { - f64::deserialize(deserializer).map(OrderedFloat) - } -} - mod date { use chrono::NaiveDate; - use serde::{de, ser, Deserialize, Deserializer, Serialize, Serializer}; use crate::Error; - pub fn serialize(value: &NaiveDate, serializer: S) -> Result - where - S: Serializer, - { - let days = date_to_days(value).map_err(|err| ser::Error::custom(err.to_string()))?; - days.serialize(serializer) - } - - pub fn deserialize<'de, D>(deserializer: D) -> Result - where - D: Deserializer<'de>, - { - let days = i32::deserialize(deserializer)?; - - days_to_date(days).map_err(|err| de::Error::custom(err.to_string())) - } - pub(crate) fn date_to_days(date: &NaiveDate) -> Result { Ok(date .signed_duration_since(NaiveDate::from_ymd_opt(1970, 0, 0).ok_or(Error::new( @@ -550,28 +399,9 @@ mod date { mod time { use chrono::NaiveTime; - use serde::{de, ser, Deserialize, Deserializer, Serialize, Serializer}; use crate::Error; - pub fn serialize(value: &NaiveTime, serializer: S) -> Result - where - S: Serializer, - { - let micros = - time_to_microseconds(value).map_err(|err| ser::Error::custom(err.to_string()))?; - micros.serialize(serializer) - } - - pub fn deserialize<'de, D>(deserializer: D) -> Result - where - D: Deserializer<'de>, - { - let micros = i64::deserialize(deserializer)?; - - microseconds_to_time(micros).map_err(|err| de::Error::custom(err.to_string())) - } - pub(crate) fn time_to_microseconds(time: &NaiveTime) -> Result { time.signed_duration_since(NaiveTime::from_num_seconds_from_midnight_opt(0, 0).ok_or( Error::new( @@ -600,28 +430,9 @@ mod time { mod timestamp { use chrono::NaiveDateTime; - use serde::{de, ser, Deserialize, Deserializer, Serialize, Serializer}; use crate::Error; - pub fn serialize(value: &NaiveDateTime, serializer: S) -> Result - where - S: Serializer, - { - let micros = - datetime_to_microseconds(value).map_err(|err| ser::Error::custom(err.to_string()))?; - micros.serialize(serializer) - } - - pub fn deserialize<'de, D>(deserializer: D) -> Result - where - D: Deserializer<'de>, - { - let micros = i64::deserialize(deserializer)?; - - microseconds_to_datetime(micros).map_err(|err| de::Error::custom(err.to_string())) - } - pub(crate) fn datetime_to_microseconds(time: &NaiveDateTime) -> Result { time.signed_duration_since(NaiveDateTime::from_timestamp_opt(0, 0).ok_or(Error::new( crate::ErrorKind::DataInvalid, @@ -646,28 +457,9 @@ mod timestamp { mod timestamptz { use chrono::{DateTime, NaiveDateTime, Utc}; - use serde::{de, ser, Deserialize, Deserializer, Serialize, Serializer}; use crate::Error; - pub fn serialize(value: &DateTime, serializer: S) -> Result - where - S: Serializer, - { - let micros = - datetimetz_to_microseconds(value).map_err(|err| ser::Error::custom(err.to_string()))?; - micros.serialize(serializer) - } - - pub fn deserialize<'de, D>(deserializer: D) -> Result, D::Error> - where - D: Deserializer<'de>, - { - let micros = i64::deserialize(deserializer)?; - - microseconds_to_datetimetz(micros).map_err(|err| de::Error::custom(err.to_string())) - } - pub(crate) fn datetimetz_to_microseconds(time: &DateTime) -> Result { time.signed_duration_since(DateTime::::from_utc( NaiveDateTime::from_timestamp_opt(0, 0).ok_or(Error::new( @@ -849,123 +641,4 @@ mod tests { &Type::Primitive(PrimitiveType::Uuid), ); } - - #[test] - pub fn boolean() { - let input = Literal::Primitive(PrimitiveLiteral::Boolean(true)); - - let raw_schema = r#""boolean""#; - - let schema = apache_avro::Schema::parse_str(raw_schema).unwrap(); - - let mut writer = apache_avro::Writer::new(&schema, Vec::new()); - - writer.append_ser(input.clone()).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(); - assert_eq!(input, result); - } - } - - #[test] - pub fn int() { - let input = Literal::Primitive(PrimitiveLiteral::Int(42)); - - let raw_schema = r#""int""#; - - let schema = apache_avro::Schema::parse_str(raw_schema).unwrap(); - - let mut writer = apache_avro::Writer::new(&schema, Vec::new()); - - writer.append_ser(input.clone()).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(); - assert_eq!(input, result); - } - } - - #[test] - pub fn float() { - let input = Literal::Primitive(PrimitiveLiteral::Float(OrderedFloat(42.0))); - - let raw_schema = r#""float""#; - - let schema = apache_avro::Schema::parse_str(raw_schema).unwrap(); - - let mut writer = apache_avro::Writer::new(&schema, Vec::new()); - - writer.append_ser(input.clone()).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(); - assert_eq!(input, result); - } - } - - #[test] - pub fn string() { - let input = Literal::Primitive(PrimitiveLiteral::String("test".to_string())); - - let raw_schema = r#""string""#; - - let schema = apache_avro::Schema::parse_str(raw_schema).unwrap(); - - let mut writer = apache_avro::Writer::new(&schema, Vec::new()); - - writer.append_ser(input.clone()).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(); - assert_eq!(input, result); - } - } - - #[test] - pub fn struct_value() { - let input = Literal::Struct(Struct::from_iter(vec![( - "name".to_string(), - Some(Literal::Primitive(PrimitiveLiteral::String( - "Alice".to_string(), - ))), - )])); - - let raw_schema = r#"{"type": "record","name": "r102","fields": [{ - "name": "name", - "type": ["null","string"], - "default": null - }]}"#; - - let schema = apache_avro::Schema::parse_str(raw_schema).unwrap(); - - let mut writer = apache_avro::Writer::new(&schema, Vec::new()); - - writer.append_ser(input.clone()).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(); - assert_eq!(input, result); - } - } } From 6baf6db3bd509a3e277260cf0c8778abae21fb2d Mon Sep 17 00:00:00 2001 From: Jan Kaul Date: Fri, 4 Aug 2023 11:25:26 +0200 Subject: [PATCH 24/42] implement From Literal for ByteBuf --- crates/iceberg/src/spec/values.rs | 103 ++++++++++-------------------- 1 file changed, 34 insertions(+), 69 deletions(-) diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index 445fe793d..38870ba3d 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -85,44 +85,40 @@ pub enum Literal { Map(HashMap>), } -impl TryFrom for ByteBuf { - type Error = Error; - fn try_from(value: Literal) -> Result { +impl From for ByteBuf { + fn from(value: Literal) -> Self { match value { Literal::Primitive(prim) => match prim { PrimitiveLiteral::Boolean(val) => { if val { - Ok(ByteBuf::from([0u8])) + ByteBuf::from([0u8]) } else { - Ok(ByteBuf::from([1u8])) + ByteBuf::from([1u8]) } } - PrimitiveLiteral::Int(val) => Ok(ByteBuf::from(val.to_le_bytes())), - PrimitiveLiteral::Long(val) => Ok(ByteBuf::from(val.to_le_bytes())), - PrimitiveLiteral::Float(val) => Ok(ByteBuf::from(val.to_le_bytes())), - PrimitiveLiteral::Double(val) => Ok(ByteBuf::from(val.to_le_bytes())), + 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) => { - Ok(ByteBuf::from(date::date_to_days(&val)?.to_le_bytes())) + ByteBuf::from(date::date_to_days(&val).to_le_bytes()) } - PrimitiveLiteral::Time(val) => Ok(ByteBuf::from( - time::time_to_microseconds(&val)?.to_le_bytes(), - )), - PrimitiveLiteral::Timestamp(val) => Ok(ByteBuf::from( - timestamp::datetime_to_microseconds(&val)?.to_le_bytes(), - )), - PrimitiveLiteral::TimestampTZ(val) => Ok(ByteBuf::from( - timestamptz::datetimetz_to_microseconds(&val)?.to_le_bytes(), - )), - PrimitiveLiteral::String(val) => Ok(ByteBuf::from(val.as_bytes())), - PrimitiveLiteral::UUID(val) => Ok(ByteBuf::from(val.as_u128().to_be_bytes())), - PrimitiveLiteral::Fixed(val) => Ok(ByteBuf::from(val)), - PrimitiveLiteral::Binary(val) => Ok(ByteBuf::from(val)), + PrimitiveLiteral::Time(val) => { + ByteBuf::from(time::time_to_microseconds(&val).to_le_bytes()) + } + PrimitiveLiteral::Timestamp(val) => { + ByteBuf::from(timestamp::datetime_to_microseconds(&val).to_le_bytes()) + } + PrimitiveLiteral::TimestampTZ(val) => { + ByteBuf::from(timestamptz::datetimetz_to_microseconds(&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!(), }, - _ => Err(Error::new( - crate::ErrorKind::DataInvalid, - "Complex types can't be converted to bytes", - )), + _ => unimplemented!(), } } } @@ -380,13 +376,9 @@ mod date { use crate::Error; - pub(crate) fn date_to_days(date: &NaiveDate) -> Result { - Ok(date - .signed_duration_since(NaiveDate::from_ymd_opt(1970, 0, 0).ok_or(Error::new( - crate::ErrorKind::DataInvalid, - "Failed to get time from midnight", - ))?) - .num_days() as i32) + pub(crate) fn date_to_days(date: &NaiveDate) -> i32 { + date.signed_duration_since(NaiveDate::from_ymd_opt(1970, 0, 0).unwrap()) + .num_days() as i32 } pub(crate) fn days_to_date(days: i32) -> Result { @@ -402,18 +394,10 @@ mod time { use crate::Error; - pub(crate) fn time_to_microseconds(time: &NaiveTime) -> Result { - time.signed_duration_since(NaiveTime::from_num_seconds_from_midnight_opt(0, 0).ok_or( - Error::new( - crate::ErrorKind::DataInvalid, - "Failed to get time from midnight", - ), - )?) - .num_microseconds() - .ok_or(Error::new( - crate::ErrorKind::DataInvalid, - "Failed to convert time to microseconds", - )) + pub(crate) fn time_to_microseconds(time: &NaiveTime) -> i64 { + time.signed_duration_since(NaiveTime::from_num_seconds_from_midnight_opt(0, 0).unwrap()) + .num_microseconds() + .unwrap() } pub(crate) fn microseconds_to_time(micros: i64) -> Result { @@ -433,16 +417,8 @@ mod timestamp { use crate::Error; - pub(crate) fn datetime_to_microseconds(time: &NaiveDateTime) -> Result { - time.signed_duration_since(NaiveDateTime::from_timestamp_opt(0, 0).ok_or(Error::new( - crate::ErrorKind::DataInvalid, - "Failed to get time from midnight", - ))?) - .num_microseconds() - .ok_or(Error::new( - crate::ErrorKind::DataInvalid, - "Failed to convert time to microseconds", - )) + pub(crate) fn datetime_to_microseconds(time: &NaiveDateTime) -> i64 { + time.timestamp_micros() } pub(crate) fn microseconds_to_datetime(micros: i64) -> Result { @@ -460,19 +436,8 @@ mod timestamptz { use crate::Error; - pub(crate) fn datetimetz_to_microseconds(time: &DateTime) -> Result { - time.signed_duration_since(DateTime::::from_utc( - NaiveDateTime::from_timestamp_opt(0, 0).ok_or(Error::new( - crate::ErrorKind::DataInvalid, - "Failed to get time from midnight", - ))?, - Utc, - )) - .num_microseconds() - .ok_or(Error::new( - crate::ErrorKind::DataInvalid, - "Failed to convert time to microseconds", - )) + pub(crate) fn datetimetz_to_microseconds(time: &DateTime) -> i64 { + time.timestamp_micros() } pub(crate) fn microseconds_to_datetimetz(micros: i64) -> Result, Error> { From e84f223ee8289c395146cf996076705e4f74be13 Mon Sep 17 00:00:00 2001 From: Jan Kaul Date: Fri, 4 Aug 2023 11:29:23 +0200 Subject: [PATCH 25/42] implement From Literal for JsonValue --- crates/iceberg/src/spec/values.rs | 74 +++++++++++++------------------ 1 file changed, 31 insertions(+), 43 deletions(-) diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index 38870ba3d..dac573b93 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -123,56 +123,44 @@ impl From for ByteBuf { } } -impl TryFrom for JsonValue { - type Error = Error; - fn try_from(value: Literal) -> Result { +impl From for JsonValue { + fn from(value: Literal) -> Self { match value { 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) => Ok(JsonValue::Number( - Number::from_f64(val.0 as f64).ok_or(Error::new( - crate::ErrorKind::DataInvalid, - "Failed to convert float to json", - ))?, - )), - PrimitiveLiteral::Double(val) => Ok(JsonValue::Number( - Number::from_f64(val.0).ok_or(Error::new( - crate::ErrorKind::DataInvalid, - "Failed to convert float to json", - ))?, - )), - PrimitiveLiteral::Date(val) => Ok(JsonValue::String(val.to_string())), - PrimitiveLiteral::Time(val) => Ok(JsonValue::String(val.to_string())), - PrimitiveLiteral::Timestamp(val) => Ok(JsonValue::String( - val.format("%Y-%m-%dT%H:%M:%S%.f").to_string(), - )), - PrimitiveLiteral::TimestampTZ(val) => Ok(JsonValue::String( - val.format("%Y-%m-%dT%H:%M:%S%.f+00:00").to_string(), - )), - PrimitiveLiteral::String(val) => Ok(JsonValue::String(val)), - PrimitiveLiteral::UUID(val) => Ok(JsonValue::String(val.to_string())), - PrimitiveLiteral::Fixed(val) => Ok(JsonValue::String(val.into_iter().fold( - String::new(), - |mut acc, x| { + PrimitiveLiteral::Boolean(val) => JsonValue::Bool(val), + PrimitiveLiteral::Int(val) => JsonValue::Number(val.into()), + PrimitiveLiteral::Long(val) => JsonValue::Number(val.into()), + PrimitiveLiteral::Float(val) => { + JsonValue::Number(Number::from_f64(val.0 as f64).unwrap()) + } + PrimitiveLiteral::Double(val) => { + JsonValue::Number(Number::from_f64(val.0).unwrap()) + } + PrimitiveLiteral::Date(val) => JsonValue::String(val.to_string()), + PrimitiveLiteral::Time(val) => JsonValue::String(val.to_string()), + PrimitiveLiteral::Timestamp(val) => { + JsonValue::String(val.format("%Y-%m-%dT%H:%M:%S%.f").to_string()) + } + PrimitiveLiteral::TimestampTZ(val) => { + JsonValue::String(val.format("%Y-%m-%dT%H:%M:%S%.f+00:00").to_string()) + } + PrimitiveLiteral::String(val) => JsonValue::String(val), + PrimitiveLiteral::UUID(val) => JsonValue::String(val.to_string()), + PrimitiveLiteral::Fixed(val) => { + JsonValue::String(val.into_iter().fold(String::new(), |mut acc, x| { acc.push_str(&format!("{:x}", x)); acc - }, - ))), - PrimitiveLiteral::Binary(val) => Ok(JsonValue::String(val.into_iter().fold( - String::new(), - |mut acc, x| { + })) + } + PrimitiveLiteral::Binary(val) => { + JsonValue::String(val.into_iter().fold(String::new(), |mut acc, x| { acc.push_str(&format!("{:x}", x)); acc - }, - ))), + })) + } PrimitiveLiteral::Decimal(_) => todo!(), }, - _ => Err(Error::new( - crate::ErrorKind::DataInvalid, - "Complex types can't be converted to bytes", - )), + _ => todo!(), } } } @@ -463,7 +451,7 @@ mod tests { let desered_literal = Literal::try_from_json(raw_json_value, expected_type).unwrap(); assert_eq!(desered_literal, expected_literal); - let expected_json_value: JsonValue = expected_literal.try_into().unwrap(); + let expected_json_value: JsonValue = expected_literal.into(); let sered_json = serde_json::to_string(&expected_json_value).unwrap(); let parsed_json_value = serde_json::from_str::(&sered_json).unwrap(); let raw_json_value = serde_json::from_str::(json).unwrap(); From 72d4b258f826aa7b3040d81bea375f9737372375 Mon Sep 17 00:00:00 2001 From: Jan Kaul Date: Fri, 4 Aug 2023 11:33:28 +0200 Subject: [PATCH 26/42] fix From Literal for JsonValue --- crates/iceberg/src/spec/values.rs | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index dac573b93..db8d14bac 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -130,12 +130,14 @@ impl From for JsonValue { PrimitiveLiteral::Boolean(val) => JsonValue::Bool(val), PrimitiveLiteral::Int(val) => JsonValue::Number(val.into()), PrimitiveLiteral::Long(val) => JsonValue::Number(val.into()), - PrimitiveLiteral::Float(val) => { - JsonValue::Number(Number::from_f64(val.0 as f64).unwrap()) - } - PrimitiveLiteral::Double(val) => { - JsonValue::Number(Number::from_f64(val.0).unwrap()) - } + PrimitiveLiteral::Float(val) => match Number::from_f64(val.0 as f64) { + Some(number) => JsonValue::Number(number), + None => JsonValue::Null, + }, + PrimitiveLiteral::Double(val) => match Number::from_f64(val.0) { + Some(number) => JsonValue::Number(number), + None => JsonValue::Null, + }, PrimitiveLiteral::Date(val) => JsonValue::String(val.to_string()), PrimitiveLiteral::Time(val) => JsonValue::String(val.to_string()), PrimitiveLiteral::Timestamp(val) => { From c6bbaa43811ee39d8342b6218f62aa2a49e48506 Mon Sep 17 00:00:00 2001 From: Jan Kaul Date: Fri, 4 Aug 2023 15:18:59 +0200 Subject: [PATCH 27/42] implement struct --- crates/iceberg/Cargo.toml | 1 + crates/iceberg/src/spec/datatypes.rs | 5 + crates/iceberg/src/spec/values.rs | 150 ++++++++++++++++++++++++--- 3 files changed, 143 insertions(+), 13 deletions(-) diff --git a/crates/iceberg/Cargo.toml b/crates/iceberg/Cargo.toml index f827c12e3..e1b5de2cb 100644 --- a/crates/iceberg/Cargo.toml +++ b/crates/iceberg/Cargo.toml @@ -38,6 +38,7 @@ rust_decimal = "1.31.0" chrono = "0.4" uuid = "1.4.1" ordered-float = "3.7.0" +bitvec = "1.0.1" [dev-dependencies] pretty_assertions = "1.4.0" diff --git a/crates/iceberg/src/spec/datatypes.rs b/crates/iceberg/src/spec/datatypes.rs index 301415cef..6686f60e3 100644 --- a/crates/iceberg/src/spec/datatypes.rs +++ b/crates/iceberg/src/spec/datatypes.rs @@ -22,6 +22,7 @@ use ::serde::de::{MapAccess, Visitor}; use serde::de::{Error, IntoDeserializer}; use serde::{de, Deserialize, Deserializer, Serialize, Serializer}; use std::cell::OnceCell; +use std::slice::Iter; use std::{collections::HashMap, fmt, ops::Index}; /// Field name for list type. @@ -293,6 +294,10 @@ impl StructType { .get(&field_id) .copied() } + /// Returns an iteratorr over the struct fields + pub fn iter(&self) -> Iter { + self.fields.iter() + } } impl PartialEq for StructType { diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index db8d14bac..ed435d9be 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -21,11 +21,12 @@ use std::{any::Any, collections::HashMap}; +use bitvec::vec::BitVec; use chrono::{DateTime, NaiveDate, NaiveDateTime, NaiveTime, Utc}; use ordered_float::OrderedFloat; use rust_decimal::Decimal; use serde_bytes::ByteBuf; -use serde_json::{Number, Value as JsonValue}; +use serde_json::{Map as JsonMap, Number, Value as JsonValue}; use uuid::Uuid; use crate::Error; @@ -123,13 +124,13 @@ impl From for ByteBuf { } } -impl From for JsonValue { - fn from(value: Literal) -> Self { +impl From<&Literal> for JsonValue { + fn from(value: &Literal) -> Self { match value { Literal::Primitive(prim) => match prim { - PrimitiveLiteral::Boolean(val) => JsonValue::Bool(val), - PrimitiveLiteral::Int(val) => JsonValue::Number(val.into()), - PrimitiveLiteral::Long(val) => JsonValue::Number(val.into()), + PrimitiveLiteral::Boolean(val) => JsonValue::Bool(*val), + PrimitiveLiteral::Int(val) => JsonValue::Number((*val).into()), + PrimitiveLiteral::Long(val) => JsonValue::Number((*val).into()), PrimitiveLiteral::Float(val) => match Number::from_f64(val.0 as f64) { Some(number) => JsonValue::Number(number), None => JsonValue::Null, @@ -146,7 +147,7 @@ impl From for JsonValue { PrimitiveLiteral::TimestampTZ(val) => { JsonValue::String(val.format("%Y-%m-%dT%H:%M:%S%.f+00:00").to_string()) } - PrimitiveLiteral::String(val) => JsonValue::String(val), + PrimitiveLiteral::String(val) => JsonValue::String(val.clone()), PrimitiveLiteral::UUID(val) => JsonValue::String(val.to_string()), PrimitiveLiteral::Fixed(val) => { JsonValue::String(val.into_iter().fold(String::new(), |mut acc, x| { @@ -162,6 +163,15 @@ impl From for JsonValue { } PrimitiveLiteral::Decimal(_) => todo!(), }, + Literal::Struct(s) => { + JsonValue::Object(JsonMap::from_iter(s.iter().map(|(id, value, _)| { + let json: JsonValue = match value { + Some(val) => val.into(), + None => JsonValue::Null, + }; + (id.to_string(), json) + }))) + } _ => todo!(), } } @@ -173,9 +183,57 @@ impl From for JsonValue { #[derive(Debug, Clone, PartialEq)] pub struct Struct { /// Vector to store the field values - fields: Vec>, - /// A lookup that matches the field name to the entry in the vector - lookup: HashMap, + fields: Vec, + /// Vector to store the field ids + field_ids: Vec, + /// Vector to store the field names + field_names: Vec, + /// Null bitmap + null_bitmap: BitVec, +} + +impl Struct { + /// Create a iterator to read the field in order of (field_id, field_value, field_name). + pub fn iter(&self) -> impl Iterator, &str)> { + self.null_bitmap + .iter() + .zip(self.fields.iter()) + .zip(self.field_ids.iter()) + .zip(self.field_names.iter()) + .map(|(((null, value), id), name)| { + (id, if *null { None } else { Some(value) }, name.as_str()) + }) + } +} + +impl FromIterator<(i32, Option, String)> for Struct { + fn from_iter, String)>>(iter: I) -> Self { + let mut fields = Vec::new(); + let mut field_ids = Vec::new(); + let mut field_names = Vec::new(); + let mut null_bitmap = BitVec::new(); + + for (id, value, name) in iter.into_iter() { + field_ids.push(id); + field_names.push(name); + 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, + field_ids, + field_names, + null_bitmap, + } + } } impl Literal { @@ -238,6 +296,7 @@ impl Literal { } } + #[inline] /// Create iceberg value from a json value pub fn try_from_json(value: JsonValue, data_type: &Type) -> Result { match data_type { @@ -301,6 +360,26 @@ impl Literal { (PrimitiveType::Binary, JsonValue::String(_)) => todo!(), _ => todo!(), }, + Type::Struct(schema) => { + if let JsonValue::Object(mut object) = value { + Ok(Literal::Struct(Struct::from_iter(schema.iter().map( + |field| { + ( + field.id, + object.remove(&field.id.to_string()).and_then(|value| { + Literal::try_from_json(value, &field.field_type).ok() + }), + field.name.clone(), + ) + }, + )))) + } else { + Err(Error::new( + crate::ErrorKind::DataInvalid, + "The json value for a struct type must be an object.", + )) + } + } _ => Err(Error::new( crate::ErrorKind::DataInvalid, "Converting bytes to non-primitive types is not supported.", @@ -446,17 +525,19 @@ mod timestamptz { #[cfg(test)] mod tests { + use crate::spec::datatypes::{StructField, StructType}; + 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, expected_type).unwrap(); + let desered_literal = + Literal::try_from_json(raw_json_value.clone(), expected_type).unwrap(); assert_eq!(desered_literal, expected_literal); - let expected_json_value: JsonValue = expected_literal.into(); + let expected_json_value: JsonValue = (&expected_literal).into(); let sered_json = serde_json::to_string(&expected_json_value).unwrap(); let parsed_json_value = serde_json::from_str::(&sered_json).unwrap(); - let raw_json_value = serde_json::from_str::(json).unwrap(); assert_eq!(parsed_json_value, raw_json_value); } @@ -596,4 +677,47 @@ mod tests { &Type::Primitive(PrimitiveType::Uuid), ); } + + #[test] + fn json_struct() { + let record = r#"{"1": 1, "2": "bar"}"#; + + check_json_serde( + record, + Literal::Struct(Struct::from_iter(vec![ + ( + 1, + Some(Literal::Primitive(PrimitiveLiteral::Int(1))), + "id".to_string(), + ), + ( + 2, + Some(Literal::Primitive(PrimitiveLiteral::String( + "bar".to_string(), + ))), + "name".to_string(), + ), + ])), + &Type::Struct(StructType::new(vec![ + StructField { + id: 1, + name: "id".to_string(), + required: true, + field_type: Type::Primitive(PrimitiveType::Int), + doc: None, + initial_default: None, + write_default: None, + }, + StructField { + id: 2, + name: "name".to_string(), + required: false, + field_type: Type::Primitive(PrimitiveType::String), + doc: None, + initial_default: None, + write_default: None, + }, + ])), + ); + } } From 61258b5bee33d5408a299e9043b19a8fb7b21a3c Mon Sep 17 00:00:00 2001 From: Jan Kaul Date: Fri, 4 Aug 2023 15:21:51 +0200 Subject: [PATCH 28/42] fix clippy warnings --- crates/iceberg/src/spec/values.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index ed435d9be..a7ef6ec59 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -150,13 +150,13 @@ impl From<&Literal> for JsonValue { PrimitiveLiteral::String(val) => JsonValue::String(val.clone()), PrimitiveLiteral::UUID(val) => JsonValue::String(val.to_string()), PrimitiveLiteral::Fixed(val) => { - JsonValue::String(val.into_iter().fold(String::new(), |mut acc, x| { + JsonValue::String(val.iter().fold(String::new(), |mut acc, x| { acc.push_str(&format!("{:x}", x)); acc })) } PrimitiveLiteral::Binary(val) => { - JsonValue::String(val.into_iter().fold(String::new(), |mut acc, x| { + JsonValue::String(val.iter().fold(String::new(), |mut acc, x| { acc.push_str(&format!("{:x}", x)); acc })) From b484b2e51e8a68aa6087fd8630cc4127e05b9bdf Mon Sep 17 00:00:00 2001 From: Jan Kaul Date: Fri, 4 Aug 2023 16:25:16 +0200 Subject: [PATCH 29/42] add avro tests for some types --- crates/iceberg/src/spec/values.rs | 86 +++++++++++++++++++++++++++++++ 1 file changed, 86 insertions(+) diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index a7ef6ec59..921724261 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -542,6 +542,26 @@ mod tests { 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.clone()); + 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(bytes).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 = Literal::try_from_bytes(&result, expected_type).unwrap(); + assert_eq!(desered_literal, expected_literal); + } + } + #[test] fn json_boolean() { let record = r#"true"#; @@ -720,4 +740,70 @@ mod tests { ])), ); } + + #[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), + ); + } } From e0ed4ff7970b384628f9b1b276e9ae24f0581e27 Mon Sep 17 00:00:00 2001 From: Jan Kaul Date: Fri, 4 Aug 2023 16:25:44 +0200 Subject: [PATCH 30/42] fix clippy warnings --- crates/iceberg/src/spec/values.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index 921724261..f940a4944 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -544,9 +544,9 @@ mod tests { 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 schema = apache_avro::Schema::parse_str(raw_schema).unwrap(); - let bytes = ByteBuf::from(input.clone()); + let bytes = ByteBuf::from(input); let literal = Literal::try_from_bytes(&bytes, expected_type).unwrap(); assert_eq!(literal, expected_literal); From bb379bf2408519c1e9e3702472127b85e2c72ede Mon Sep 17 00:00:00 2001 From: Jan Kaul Date: Fri, 4 Aug 2023 16:32:16 +0200 Subject: [PATCH 31/42] fix nested field --- crates/iceberg/src/spec/datatypes.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/iceberg/src/spec/datatypes.rs b/crates/iceberg/src/spec/datatypes.rs index 6686f60e3..5750ab968 100644 --- a/crates/iceberg/src/spec/datatypes.rs +++ b/crates/iceberg/src/spec/datatypes.rs @@ -295,7 +295,7 @@ impl StructType { .copied() } /// Returns an iteratorr over the struct fields - pub fn iter(&self) -> Iter { + pub fn iter(&self) -> Iter { self.fields.iter() } } From 8292d42bc99996a3a8f993e1c6fd5eef6ab5bae1 Mon Sep 17 00:00:00 2001 From: Jan Kaul Date: Fri, 4 Aug 2023 16:34:33 +0200 Subject: [PATCH 32/42] fix nested field --- crates/iceberg/src/spec/values.rs | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index f940a4944..87face0df 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -525,7 +525,7 @@ mod timestamptz { #[cfg(test)] mod tests { - use crate::spec::datatypes::{StructField, StructType}; + use crate::spec::datatypes::{NestedField, StructType}; use super::*; @@ -719,20 +719,20 @@ mod tests { ), ])), &Type::Struct(StructType::new(vec![ - StructField { + NestedField { id: 1, name: "id".to_string(), required: true, - field_type: Type::Primitive(PrimitiveType::Int), + field_type: Box::new(Type::Primitive(PrimitiveType::Int)), doc: None, initial_default: None, write_default: None, }, - StructField { + NestedField { id: 2, name: "name".to_string(), required: false, - field_type: Type::Primitive(PrimitiveType::String), + field_type: Box::new(Type::Primitive(PrimitiveType::String)), doc: None, initial_default: None, write_default: None, From 7058b9332261c2d6aa2dc464a6d0acd38c1717d1 Mon Sep 17 00:00:00 2001 From: Jan Kaul Date: Fri, 4 Aug 2023 19:52:41 +0200 Subject: [PATCH 33/42] implement list test --- crates/iceberg/src/spec/values.rs | 55 ++++++++++++++++++++++++++++++- 1 file changed, 54 insertions(+), 1 deletion(-) diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index 87face0df..24775133f 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -172,6 +172,14 @@ impl From<&Literal> for JsonValue { (id.to_string(), json) }))) } + Literal::List(list) => JsonValue::Array( + list.into_iter() + .map(|opt| match opt { + Some(literal) => literal.into(), + None => JsonValue::Null, + }) + .collect(), + ), _ => todo!(), } } @@ -380,6 +388,26 @@ impl Literal { )) } } + Type::List(list) => { + if let JsonValue::Array(array) = value { + Ok(Literal::List( + array + .into_iter() + .map(|value| { + Ok(Some(Literal::try_from_json( + value, + &list.element_field.field_type, + )?)) + }) + .collect::, Error>>()?, + )) + } else { + Err(Error::new( + crate::ErrorKind::DataInvalid, + "The json value for a list type must be an array.", + )) + } + } _ => Err(Error::new( crate::ErrorKind::DataInvalid, "Converting bytes to non-primitive types is not supported.", @@ -525,7 +553,7 @@ mod timestamptz { #[cfg(test)] mod tests { - use crate::spec::datatypes::{NestedField, StructType}; + use crate::spec::datatypes::{ListType, NestedField, StructType}; use super::*; @@ -741,6 +769,31 @@ mod tests { ); } + #[test] + fn json_list() { + let record = r#"[1, 2, 3]"#; + + 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))), + ]), + &Type::List(ListType { + element_field: NestedField { + id: 0, + name: "".to_string(), + required: true, + field_type: Box::new(Type::Primitive(PrimitiveType::Int)), + doc: None, + initial_default: None, + write_default: None, + }, + }), + ); + } + #[test] fn avro_bytes_boolean() { let bytes = vec![1u8]; From 7e032b5a06693d714bfc075afee059bb1d739c39 Mon Sep 17 00:00:00 2001 From: Jan Kaul Date: Fri, 4 Aug 2023 20:36:55 +0200 Subject: [PATCH 34/42] implement map test --- crates/iceberg/src/spec/values.rs | 110 ++++++++++++++++++++++++++---- 1 file changed, 97 insertions(+), 13 deletions(-) diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index 24775133f..41d3cb6c5 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -19,7 +19,7 @@ * Value in iceberg */ -use std::{any::Any, collections::HashMap}; +use std::{any::Any, collections::BTreeMap}; use bitvec::vec::BitVec; use chrono::{DateTime, NaiveDate, NaiveDateTime, NaiveTime, Utc}; @@ -34,7 +34,7 @@ use crate::Error; use super::datatypes::{PrimitiveType, Type}; /// Values present in iceberg type -#[derive(Clone, Debug, PartialEq)] +#[derive(Clone, Debug, PartialEq, Hash, Eq, PartialOrd, Ord)] pub enum PrimitiveLiteral { /// 0x00 for false, non-zero byte for true Boolean(bool), @@ -68,7 +68,7 @@ pub enum PrimitiveLiteral { } /// Values present in iceberg type -#[derive(Clone, Debug, PartialEq)] +#[derive(Clone, Debug, PartialEq, Hash, Eq, PartialOrd, Ord)] pub enum Literal { /// A primitive value Primitive(PrimitiveLiteral), @@ -83,7 +83,7 @@ pub enum Literal { /// 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(HashMap>), + Map(BTreeMap>), } impl From for ByteBuf { @@ -180,7 +180,25 @@ impl From<&Literal> for JsonValue { }) .collect(), ), - _ => todo!(), + Literal::Map(map) => { + let mut object = JsonMap::with_capacity(2); + object.insert( + "keys".to_string(), + JsonValue::Array(map.keys().map(|literal| literal.into()).collect()), + ); + object.insert( + "values".to_string(), + JsonValue::Array( + map.values() + .map(|literal| match literal { + Some(literal) => literal.into(), + None => JsonValue::Null, + }) + .collect(), + ), + ); + JsonValue::Object(object) + } } } } @@ -188,7 +206,7 @@ impl From<&Literal> for JsonValue { /// 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)] +#[derive(Debug, Clone, PartialEq, Hash, Eq, PartialOrd, Ord)] pub struct Struct { /// Vector to store the field values fields: Vec, @@ -245,7 +263,6 @@ impl FromIterator<(i32, Option, String)> for Struct { } impl Literal { - #[inline] /// Create iceberg value from bytes pub fn try_from_bytes(bytes: &[u8], data_type: &Type) -> Result { match data_type { @@ -304,7 +321,6 @@ impl Literal { } } - #[inline] /// Create iceberg value from a json value pub fn try_from_json(value: JsonValue, data_type: &Type) -> Result { match data_type { @@ -408,10 +424,39 @@ impl Literal { )) } } - _ => Err(Error::new( - crate::ErrorKind::DataInvalid, - "Converting bytes to non-primitive types is not supported.", - )), + 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(Literal::Map(BTreeMap::from_iter( + keys.into_iter() + .zip(values.into_iter()) + .map(|(key, value)| { + Ok(( + Literal::try_from_json(key, &map.key_field.field_type)?, + Some(Literal::try_from_json( + value, + &map.value_field.field_type, + )?), + )) + }) + .collect::, Error>>()? + .into_iter(), + ))) + } 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.", + )) + } + } } } @@ -553,7 +598,7 @@ mod timestamptz { #[cfg(test)] mod tests { - use crate::spec::datatypes::{ListType, NestedField, StructType}; + use crate::spec::datatypes::{ListType, MapType, NestedField, StructType}; use super::*; @@ -794,6 +839,45 @@ mod tests { ); } + #[test] + fn json_map() { + let record = r#"{ "keys": ["a", "b"], "values": [1, 2] }"#; + + check_json_serde( + record, + Literal::Map(BTreeMap::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))), + ), + ])), + &Type::Map(MapType { + key_field: NestedField { + id: 0, + name: "key".to_string(), + required: true, + field_type: Box::new(Type::Primitive(PrimitiveType::String)), + doc: None, + initial_default: None, + write_default: None, + }, + value_field: NestedField { + id: 1, + name: "value".to_string(), + required: true, + field_type: Box::new(Type::Primitive(PrimitiveType::Int)), + doc: None, + initial_default: None, + write_default: None, + }, + }), + ); + } + #[test] fn avro_bytes_boolean() { let bytes = vec![1u8]; From 9fe7c76c83a1c36b8c01b0024eea642688287401 Mon Sep 17 00:00:00 2001 From: Jan Kaul Date: Fri, 4 Aug 2023 20:53:43 +0200 Subject: [PATCH 35/42] fix error --- crates/iceberg/src/spec/values.rs | 21 ++++++++++++++++++--- 1 file changed, 18 insertions(+), 3 deletions(-) diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index 41d3cb6c5..dd6f10d82 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -312,7 +312,10 @@ impl Literal { PrimitiveType::Binary => Ok(Literal::Primitive(PrimitiveLiteral::Binary( Vec::from(bytes), ))), - _ => todo!(), + PrimitiveType::Decimal { + precision: _, + scale: _, + } => todo!(), }, _ => Err(Error::new( crate::ErrorKind::DataInvalid, @@ -382,7 +385,20 @@ impl Literal { )), (PrimitiveType::Fixed(_), JsonValue::String(_)) => todo!(), (PrimitiveType::Binary, JsonValue::String(_)) => todo!(), - _ => todo!(), + ( + PrimitiveType::Decimal { + precision: _, + scale: _, + }, + JsonValue::String(_), + ) => todo!(), + (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 { @@ -507,7 +523,6 @@ impl Literal { PrimitiveLiteral::UUID(any) => Box::new(any), PrimitiveLiteral::Decimal(any) => Box::new(any), }, - _ => unimplemented!(), } } From 0bf2a7b324240ff751ede2b6b8212d8bd59c1a18 Mon Sep 17 00:00:00 2001 From: Jan Kaul Date: Fri, 4 Aug 2023 20:55:13 +0200 Subject: [PATCH 36/42] fix clippy warnings --- crates/iceberg/src/spec/values.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index dd6f10d82..2b983d1b9 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -173,7 +173,7 @@ impl From<&Literal> for JsonValue { }))) } Literal::List(list) => JsonValue::Array( - list.into_iter() + list.iter() .map(|opt| match opt { Some(literal) => literal.into(), None => JsonValue::Null, From 06b567658a1ba032d938b71d40ed37b202639b7d Mon Sep 17 00:00:00 2001 From: Jan Kaul Date: Wed, 9 Aug 2023 08:03:55 +0200 Subject: [PATCH 37/42] change timestamps to int/long --- crates/iceberg/src/spec/values.rs | 172 +++++++++++++----------------- 1 file changed, 76 insertions(+), 96 deletions(-) diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index 2b983d1b9..f455357eb 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -47,13 +47,13 @@ pub enum PrimitiveLiteral { /// Stored as 8-byte little-endian Double(OrderedFloat), /// Stores days from the 1970-01-01 in an 4-byte little-endian int - Date(NaiveDate), + Date(i32), /// Stores microseconds from midnight in an 8-byte little-endian long - Time(NaiveTime), + Time(i64), /// Timestamp without timezone - Timestamp(NaiveDateTime), + Timestamp(i64), /// Timestamp with timezone - TimestampTZ(DateTime), + TimestampTZ(i64), /// UTF-8 bytes (without length) String(String), /// 16-byte big-endian value @@ -101,18 +101,10 @@ impl From for ByteBuf { 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(date::date_to_days(&val).to_le_bytes()) - } - PrimitiveLiteral::Time(val) => { - ByteBuf::from(time::time_to_microseconds(&val).to_le_bytes()) - } - PrimitiveLiteral::Timestamp(val) => { - ByteBuf::from(timestamp::datetime_to_microseconds(&val).to_le_bytes()) - } - PrimitiveLiteral::TimestampTZ(val) => { - ByteBuf::from(timestamptz::datetimetz_to_microseconds(&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), @@ -139,14 +131,22 @@ impl From<&Literal> for JsonValue { Some(number) => JsonValue::Number(number), None => JsonValue::Null, }, - PrimitiveLiteral::Date(val) => JsonValue::String(val.to_string()), - PrimitiveLiteral::Time(val) => JsonValue::String(val.to_string()), - PrimitiveLiteral::Timestamp(val) => { - JsonValue::String(val.format("%Y-%m-%dT%H:%M:%S%.f").to_string()) + PrimitiveLiteral::Date(val) => { + JsonValue::String(date::days_to_date(*val).to_string()) } - PrimitiveLiteral::TimestampTZ(val) => { - JsonValue::String(val.format("%Y-%m-%dT%H:%M:%S%.f+00:00").to_string()) + PrimitiveLiteral::Time(val) => { + JsonValue::String(time::microseconds_to_time(*val).to_string()) } + PrimitiveLiteral::Timestamp(val) => JsonValue::String( + timestamp::microseconds_to_datetime(*val) + .format("%Y-%m-%dT%H:%M:%S%.f") + .to_string(), + ), + PrimitiveLiteral::TimestampTZ(val) => JsonValue::String( + timestamptz::microseconds_to_datetimetz(*val) + .format("%Y-%m-%dT%H:%M:%S%.f+00:00") + .to_string(), + ), PrimitiveLiteral::String(val) => JsonValue::String(val.clone()), PrimitiveLiteral::UUID(val) => JsonValue::String(val.to_string()), PrimitiveLiteral::Fixed(val) => { @@ -287,18 +287,16 @@ impl Literal { OrderedFloat(f64::from_le_bytes(bytes.try_into()?)), ))), PrimitiveType::Date => Ok(Literal::Primitive(PrimitiveLiteral::Date( - date::days_to_date(i32::from_le_bytes(bytes.try_into()?))?, + i32::from_le_bytes(bytes.try_into()?), ))), PrimitiveType::Time => Ok(Literal::Primitive(PrimitiveLiteral::Time( - time::microseconds_to_time(i64::from_le_bytes(bytes.try_into()?))?, + i64::from_le_bytes(bytes.try_into()?), ))), PrimitiveType::Timestamp => Ok(Literal::Primitive(PrimitiveLiteral::Timestamp( - timestamp::microseconds_to_datetime(i64::from_le_bytes(bytes.try_into()?))?, + i64::from_le_bytes(bytes.try_into()?), ))), PrimitiveType::Timestamptz => Ok(Literal::Primitive( - PrimitiveLiteral::TimestampTZ(timestamptz::microseconds_to_datetimetz( - i64::from_le_bytes(bytes.try_into()?), - )?), + PrimitiveLiteral::TimestampTZ(i64::from_le_bytes(bytes.try_into()?)), )), PrimitiveType::String => Ok(Literal::Primitive(PrimitiveLiteral::String( std::str::from_utf8(bytes)?.to_string(), @@ -360,23 +358,29 @@ impl Literal { "Failed to convert json number to double", ))?)), )), - (PrimitiveType::Date, JsonValue::String(s)) => Ok(Literal::Primitive( - PrimitiveLiteral::Date(NaiveDate::parse_from_str(&s, "%Y-%m-%d")?), - )), - (PrimitiveType::Time, JsonValue::String(s)) => Ok(Literal::Primitive( - PrimitiveLiteral::Time(NaiveTime::parse_from_str(&s, "%H:%M:%S%.f")?), - )), - (PrimitiveType::Timestamp, JsonValue::String(s)) => { - Ok(Literal::Primitive(PrimitiveLiteral::Timestamp( - NaiveDateTime::parse_from_str(&s, "%Y-%m-%dT%H:%M:%S%.f")?, + (PrimitiveType::Date, JsonValue::String(s)) => { + Ok(Literal::Primitive(PrimitiveLiteral::Date( + date::date_to_days(&NaiveDate::parse_from_str(&s, "%Y-%m-%d")?), + ))) + } + (PrimitiveType::Time, JsonValue::String(s)) => { + Ok(Literal::Primitive(PrimitiveLiteral::Time( + time::time_to_microseconds(&NaiveTime::parse_from_str(&s, "%H:%M:%S%.f")?), ))) } - (PrimitiveType::Timestamptz, JsonValue::String(s)) => Ok(Literal::Primitive( - PrimitiveLiteral::TimestampTZ(DateTime::from_utc( - NaiveDateTime::parse_from_str(&s, "%Y-%m-%dT%H:%M:%S%.f+00:00")?, - Utc, + (PrimitiveType::Timestamp, JsonValue::String(s)) => Ok(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(Literal::Primitive(PrimitiveLiteral::TimestampTZ( + timestamptz::datetimetz_to_microseconds(&DateTime::from_utc( + NaiveDateTime::parse_from_str(&s, "%Y-%m-%dT%H:%M:%S%.f+00:00")?, + Utc, + )), + ))) + } (PrimitiveType::String, JsonValue::String(s)) => { Ok(Literal::Primitive(PrimitiveLiteral::String(s))) } @@ -529,84 +533,73 @@ impl Literal { } mod date { - use chrono::NaiveDate; - - use crate::Error; + use chrono::{NaiveDate, NaiveDateTime}; pub(crate) fn date_to_days(date: &NaiveDate) -> i32 { - date.signed_duration_since(NaiveDate::from_ymd_opt(1970, 0, 0).unwrap()) - .num_days() as 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) -> Result { - NaiveDate::from_num_days_from_ce_opt(days).ok_or(Error::new( - crate::ErrorKind::DataInvalid, - "Failed to convert microseconds to time", - )) + pub(crate) fn days_to_date(days: i32) -> NaiveDate { + // This shouldn't fail until the year 262000 + NaiveDateTime::from_timestamp_opt(days as i64 * 86_400, 0) + .unwrap() + .date() } } mod time { use chrono::NaiveTime; - use crate::Error; - pub(crate) fn time_to_microseconds(time: &NaiveTime) -> i64 { - time.signed_duration_since(NaiveTime::from_num_seconds_from_midnight_opt(0, 0).unwrap()) - .num_microseconds() - .unwrap() + 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) -> Result { + 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 * 1000).ok_or( - Error::new( - crate::ErrorKind::DataInvalid, - "Failed to convert microseconds to time", - ), - ) + NaiveTime::from_num_seconds_from_midnight_opt(secs as u32, rem as u32 * 1_000).unwrap() } } mod timestamp { use chrono::NaiveDateTime; - use crate::Error; - pub(crate) fn datetime_to_microseconds(time: &NaiveDateTime) -> i64 { time.timestamp_micros() } - pub(crate) fn microseconds_to_datetime(micros: i64) -> Result { + pub(crate) fn microseconds_to_datetime(micros: i64) -> NaiveDateTime { let (secs, rem) = (micros / 1_000_000, micros % 1_000_000); - NaiveDateTime::from_timestamp_opt(secs, rem as u32 * 1000).ok_or(Error::new( - crate::ErrorKind::DataInvalid, - "Failed to convert microseconds to time", - )) + // This shouldn't fail until the year 262000 + NaiveDateTime::from_timestamp_opt(secs, rem as u32 * 1_000).unwrap() } } mod timestamptz { use chrono::{DateTime, NaiveDateTime, Utc}; - use crate::Error; - pub(crate) fn datetimetz_to_microseconds(time: &DateTime) -> i64 { time.timestamp_micros() } - pub(crate) fn microseconds_to_datetimetz(micros: i64) -> Result, Error> { + pub(crate) fn microseconds_to_datetimetz(micros: i64) -> DateTime { let (secs, rem) = (micros / 1_000_000, micros % 1_000_000); - Ok(DateTime::::from_utc( - NaiveDateTime::from_timestamp_opt(secs, rem as u32 * 1000).ok_or(Error::new( - crate::ErrorKind::DataInvalid, - "Failed to convert microseconds to time", - ))?, + DateTime::::from_utc( + // This shouldn't fail until the year 262000 + NaiveDateTime::from_timestamp_opt(secs, rem as u32 * 1_000).unwrap(), Utc, - )) + ) } } @@ -711,9 +704,7 @@ mod tests { check_json_serde( record, - Literal::Primitive(PrimitiveLiteral::Date( - NaiveDate::from_ymd_opt(2017, 11, 16).unwrap(), - )), + Literal::Primitive(PrimitiveLiteral::Date(17486)), &Type::Primitive(PrimitiveType::Date), ); } @@ -724,9 +715,7 @@ mod tests { check_json_serde( record, - Literal::Primitive(PrimitiveLiteral::Time( - NaiveTime::from_hms_micro_opt(22, 31, 8, 123456).unwrap(), - )), + Literal::Primitive(PrimitiveLiteral::Time(81068123456)), &Type::Primitive(PrimitiveType::Time), ); } @@ -737,10 +726,7 @@ mod tests { check_json_serde( record, - Literal::Primitive(PrimitiveLiteral::Timestamp(NaiveDateTime::new( - NaiveDate::from_ymd_opt(2017, 11, 16).unwrap(), - NaiveTime::from_hms_micro_opt(22, 31, 8, 123456).unwrap(), - ))), + Literal::Primitive(PrimitiveLiteral::Timestamp(1510871468123456)), &Type::Primitive(PrimitiveType::Timestamp), ); } @@ -751,13 +737,7 @@ mod tests { check_json_serde( record, - Literal::Primitive(PrimitiveLiteral::TimestampTZ(DateTime::::from_utc( - NaiveDateTime::new( - NaiveDate::from_ymd_opt(2017, 11, 16).unwrap(), - NaiveTime::from_hms_micro_opt(22, 31, 8, 123456).unwrap(), - ), - Utc, - ))), + Literal::Primitive(PrimitiveLiteral::TimestampTZ(1510871468123456)), &Type::Primitive(PrimitiveType::Timestamptz), ); } From 29488dcf901ae21a984104d613a3c1969eb5d011 Mon Sep 17 00:00:00 2001 From: Jan Kaul Date: Wed, 9 Aug 2023 08:43:04 +0200 Subject: [PATCH 38/42] convert nulls to None --- crates/iceberg/src/error.rs | 3 +++ crates/iceberg/src/spec/values.rs | 31 ++++++++++++++++++++++++++----- 2 files changed, 29 insertions(+), 5 deletions(-) diff --git a/crates/iceberg/src/error.rs b/crates/iceberg/src/error.rs index 6cb41db79..3f6b2433a 100644 --- a/crates/iceberg/src/error.rs +++ b/crates/iceberg/src/error.rs @@ -44,6 +44,8 @@ pub enum ErrorKind { /// /// This error is returned when given iceberg feature is not supported. FeatureUnsupported, + /// This error is returned when we try to convert a JSON null value to an iceberg value + NullConversion, } impl ErrorKind { @@ -59,6 +61,7 @@ impl From for &'static str { ErrorKind::Unexpected => "Unexpected", ErrorKind::DataInvalid => "DataInvalid", ErrorKind::FeatureUnsupported => "FeatureUnsupported", + ErrorKind::NullConversion => "NullConversion", } } } diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index f455357eb..ff0006268 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -29,7 +29,7 @@ use serde_bytes::ByteBuf; use serde_json::{Map as JsonMap, Number, Value as JsonValue}; use uuid::Uuid; -use crate::Error; +use crate::{Error, ErrorKind}; use super::datatypes::{PrimitiveType, Type}; @@ -396,6 +396,10 @@ impl Literal { }, JsonValue::String(_), ) => todo!(), + (_, JsonValue::Null) => Err(Error::new( + crate::ErrorKind::NullConversion, + "Failed to convert null to iceberg value", + )), (i, j) => Err(Error::new( crate::ErrorKind::DataInvalid, format!( @@ -430,10 +434,10 @@ impl Literal { array .into_iter() .map(|value| { - Ok(Some(Literal::try_from_json( + to_optional_literal(Literal::try_from_json( value, &list.element_field.field_type, - )?)) + )) }) .collect::, Error>>()?, )) @@ -455,10 +459,10 @@ impl Literal { .map(|(key, value)| { Ok(( Literal::try_from_json(key, &map.key_field.field_type)?, - Some(Literal::try_from_json( + to_optional_literal(Literal::try_from_json( value, &map.value_field.field_type, - )?), + ))?, )) }) .collect::, Error>>()? @@ -532,6 +536,23 @@ impl Literal { } } +fn to_optional_literal(value: Result) -> Result, Error> { + match value { + Err(err) => { + let is_null_error = match err.kind() { + ErrorKind::NullConversion => true, + _ => false, + }; + if is_null_error { + Ok(None) + } else { + Err(err) + } + } + Ok(x) => Ok(Some(x)), + } +} + mod date { use chrono::{NaiveDate, NaiveDateTime}; From 583e3b18350fc6fbecc2fff5722d3b95f9ef9e36 Mon Sep 17 00:00:00 2001 From: Jan Kaul Date: Wed, 9 Aug 2023 08:44:50 +0200 Subject: [PATCH 39/42] add tests for null --- crates/iceberg/src/spec/values.rs | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index ff0006268..0529539c3 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -832,7 +832,7 @@ mod tests { #[test] fn json_list() { - let record = r#"[1, 2, 3]"#; + let record = r#"[1, 2, 3, null]"#; check_json_serde( record, @@ -840,6 +840,7 @@ mod tests { 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 { @@ -857,7 +858,7 @@ mod tests { #[test] fn json_map() { - let record = r#"{ "keys": ["a", "b"], "values": [1, 2] }"#; + let record = r#"{ "keys": ["a", "b", "c"], "values": [1, 2, null] }"#; check_json_serde( record, @@ -870,6 +871,10 @@ mod tests { 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 { From 927fa7f145bda1352ffd5411b38c013f4e6e127c Mon Sep 17 00:00:00 2001 From: Jan Kaul Date: Wed, 9 Aug 2023 08:47:09 +0200 Subject: [PATCH 40/42] null test for struct --- crates/iceberg/src/spec/values.rs | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index 0529539c3..a43ff2be8 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -789,7 +789,7 @@ mod tests { #[test] fn json_struct() { - let record = r#"{"1": 1, "2": "bar"}"#; + let record = r#"{"1": 1, "2": "bar", "3": null}"#; check_json_serde( record, @@ -806,6 +806,7 @@ mod tests { ))), "name".to_string(), ), + (3, None, "address".to_string()), ])), &Type::Struct(StructType::new(vec![ NestedField { @@ -826,6 +827,15 @@ mod tests { initial_default: None, write_default: None, }, + NestedField { + id: 3, + name: "address".to_string(), + required: false, + field_type: Box::new(Type::Primitive(PrimitiveType::String)), + doc: None, + initial_default: None, + write_default: None, + }, ])), ); } From 451f3d4f4c80981ffd5d795792a3d768fc78e226 Mon Sep 17 00:00:00 2001 From: Jan Kaul Date: Wed, 9 Aug 2023 08:50:34 +0200 Subject: [PATCH 41/42] fix clippy warning --- crates/iceberg/src/spec/values.rs | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index a43ff2be8..adca29cd4 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -539,10 +539,7 @@ impl Literal { fn to_optional_literal(value: Result) -> Result, Error> { match value { Err(err) => { - let is_null_error = match err.kind() { - ErrorKind::NullConversion => true, - _ => false, - }; + let is_null_error = matches!(err.kind(), ErrorKind::NullConversion); if is_null_error { Ok(None) } else { From c3fa7d71a29b6d319024bf74806d4b1771d07dfb Mon Sep 17 00:00:00 2001 From: Jan Kaul Date: Wed, 9 Aug 2023 10:48:06 +0200 Subject: [PATCH 42/42] convert json null to option --- crates/iceberg/src/error.rs | 3 - crates/iceberg/src/spec/values.rs | 102 ++++++++++++++---------------- 2 files changed, 46 insertions(+), 59 deletions(-) diff --git a/crates/iceberg/src/error.rs b/crates/iceberg/src/error.rs index 3f6b2433a..6cb41db79 100644 --- a/crates/iceberg/src/error.rs +++ b/crates/iceberg/src/error.rs @@ -44,8 +44,6 @@ pub enum ErrorKind { /// /// This error is returned when given iceberg feature is not supported. FeatureUnsupported, - /// This error is returned when we try to convert a JSON null value to an iceberg value - NullConversion, } impl ErrorKind { @@ -61,7 +59,6 @@ impl From for &'static str { ErrorKind::Unexpected => "Unexpected", ErrorKind::DataInvalid => "DataInvalid", ErrorKind::FeatureUnsupported => "FeatureUnsupported", - ErrorKind::NullConversion => "NullConversion", } } } diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index adca29cd4..771ffd88e 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -323,14 +323,14 @@ impl Literal { } /// Create iceberg value from a json value - pub fn try_from_json(value: JsonValue, data_type: &Type) -> Result { + pub fn try_from_json(value: JsonValue, data_type: &Type) -> Result, Error> { match data_type { Type::Primitive(primitive) => match (primitive, value) { (PrimitiveType::Boolean, JsonValue::Bool(bool)) => { - Ok(Literal::Primitive(PrimitiveLiteral::Boolean(bool))) + Ok(Some(Literal::Primitive(PrimitiveLiteral::Boolean(bool)))) } (PrimitiveType::Int, JsonValue::Number(number)) => { - Ok(Literal::Primitive(PrimitiveLiteral::Int( + Ok(Some(Literal::Primitive(PrimitiveLiteral::Int( number .as_i64() .ok_or(Error::new( @@ -338,55 +338,55 @@ impl Literal { "Failed to convert json number to int", ))? .try_into()?, - ))) + )))) } - (PrimitiveType::Long, JsonValue::Number(number)) => Ok(Literal::Primitive( + (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(Literal::Primitive( + ))), + (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(Literal::Primitive( + ))), + (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(Literal::Primitive(PrimitiveLiteral::Date( + Ok(Some(Literal::Primitive(PrimitiveLiteral::Date( date::date_to_days(&NaiveDate::parse_from_str(&s, "%Y-%m-%d")?), - ))) + )))) } (PrimitiveType::Time, JsonValue::String(s)) => { - Ok(Literal::Primitive(PrimitiveLiteral::Time( + 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(Literal::Primitive( + (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(Literal::Primitive(PrimitiveLiteral::TimestampTZ( + Ok(Some(Literal::Primitive(PrimitiveLiteral::TimestampTZ( timestamptz::datetimetz_to_microseconds(&DateTime::from_utc( NaiveDateTime::parse_from_str(&s, "%Y-%m-%dT%H:%M:%S%.f+00:00")?, Utc, )), - ))) + )))) } (PrimitiveType::String, JsonValue::String(s)) => { - Ok(Literal::Primitive(PrimitiveLiteral::String(s))) + Ok(Some(Literal::Primitive(PrimitiveLiteral::String(s)))) } - (PrimitiveType::Uuid, JsonValue::String(s)) => Ok(Literal::Primitive( + (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!(), ( @@ -396,10 +396,7 @@ impl Literal { }, JsonValue::String(_), ) => todo!(), - (_, JsonValue::Null) => Err(Error::new( - crate::ErrorKind::NullConversion, - "Failed to convert null to iceberg value", - )), + (_, JsonValue::Null) => Ok(None), (i, j) => Err(Error::new( crate::ErrorKind::DataInvalid, format!( @@ -410,17 +407,24 @@ impl Literal { }, Type::Struct(schema) => { if let JsonValue::Object(mut object) = value { - Ok(Literal::Struct(Struct::from_iter(schema.iter().map( + Ok(Some(Literal::Struct(Struct::from_iter(schema.iter().map( |field| { ( field.id, object.remove(&field.id.to_string()).and_then(|value| { - Literal::try_from_json(value, &field.field_type).ok() + 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() }), field.name.clone(), ) }, - )))) + ))))) } else { Err(Error::new( crate::ErrorKind::DataInvalid, @@ -430,17 +434,14 @@ impl Literal { } Type::List(list) => { if let JsonValue::Array(array) = value { - Ok(Literal::List( + Ok(Some(Literal::List( array .into_iter() .map(|value| { - to_optional_literal(Literal::try_from_json( - value, - &list.element_field.field_type, - )) + Literal::try_from_json(value, &list.element_field.field_type) }) .collect::, Error>>()?, - )) + ))) } else { Err(Error::new( crate::ErrorKind::DataInvalid, @@ -453,21 +454,24 @@ impl Literal { if let (Some(JsonValue::Array(keys)), Some(JsonValue::Array(values))) = (object.remove("keys"), object.remove("values")) { - Ok(Literal::Map(BTreeMap::from_iter( + Ok(Some(Literal::Map(BTreeMap::from_iter( keys.into_iter() .zip(values.into_iter()) .map(|(key, value)| { Ok(( - Literal::try_from_json(key, &map.key_field.field_type)?, - to_optional_literal(Literal::try_from_json( - value, - &map.value_field.field_type, - ))?, + 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::, Error>>()? .into_iter(), - ))) + )))) } else { Err(Error::new( crate::ErrorKind::DataInvalid, @@ -536,20 +540,6 @@ impl Literal { } } -fn to_optional_literal(value: Result) -> Result, Error> { - match value { - Err(err) => { - let is_null_error = matches!(err.kind(), ErrorKind::NullConversion); - if is_null_error { - Ok(None) - } else { - Err(err) - } - } - Ok(x) => Ok(Some(x)), - } -} - mod date { use chrono::{NaiveDate, NaiveDateTime}; @@ -632,7 +622,7 @@ mod tests { 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, expected_literal); + assert_eq!(desered_literal, Some(expected_literal.clone())); let expected_json_value: JsonValue = (&expected_literal).into(); let sered_json = serde_json::to_string(&expected_json_value).unwrap();