Skip to content

Commit

Permalink
Merge pull request #894 from muzarski/timeuuid
Browse files Browse the repository at this point in the history
Introduce `CqlTimeuuid` type
  • Loading branch information
piodul authored Jan 26, 2024
2 parents 4feb6fb + 1958216 commit d01b473
Show file tree
Hide file tree
Showing 11 changed files with 343 additions and 31 deletions.
3 changes: 2 additions & 1 deletion docs/source/SUMMARY.md
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,8 @@
- [Counter](data-types/counter.md)
- [Blob](data-types/blob.md)
- [Inet](data-types/inet.md)
- [Uuid, Timeuuid](data-types/uuid.md)
- [Uuid](data-types/uuid.md)
- [Timeuuid](data-types/timeuuid.md)
- [Date](data-types/date.md)
- [Time](data-types/time.md)
- [Timestamp](data-types/timestamp.md)
Expand Down
4 changes: 3 additions & 1 deletion docs/source/data-types/data-types.md
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,8 @@ Database types and their Rust equivalents:
* `Counter` <----> `value::Counter`
* `Blob` <----> `Vec<u8>`
* `Inet` <----> `std::net::IpAddr`
* `Uuid`, `Timeuuid` <----> `uuid::Uuid`
* `Uuid` <----> `uuid::Uuid`
* `Timeuuid` <----> `value::CqlTimeuuid`
* `Date` <----> `value::CqlDate`, `chrono::NaiveDate`, `time::Date`
* `Time` <----> `value::CqlTime`, `chrono::NaiveTime`, `time::Time`
* `Timestamp` <----> `value::CqlTimestamp`, `chrono::DateTime<Utc>`, `time::OffsetDateTime`
Expand All @@ -45,6 +46,7 @@ Database types and their Rust equivalents:
blob
inet
uuid
timeuuid
date
time
timestamp
Expand Down
30 changes: 30 additions & 0 deletions docs/source/data-types/timeuuid.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,30 @@
# Timeuuid

`Timeuuid` is represented as `value::CqlTimeuuid`.
`value::CqlTimeuuid` is a wrapper for `uuid::Uuid` with custom ordering logic
which follows Scylla/Cassandra semantics.

```rust
# extern crate scylla;
# use scylla::Session;
# use std::error::Error;
# use std::str::FromStr;
# async fn check_only_compiles(session: &Session) -> Result<(), Box<dyn Error>> {
use scylla::IntoTypedRows;
use scylla::frame::value::CqlTimeuuid;

// Insert some timeuuid into the table
let to_insert: CqlTimeuuid = CqlTimeuuid::from_str("8e14e760-7fa8-11eb-bc66-000000000001")?;
session
.query("INSERT INTO keyspace.table (a) VALUES(?)", (to_insert,))
.await?;

// Read timeuuid from the table
if let Some(rows) = session.query("SELECT a FROM keyspace.table", &[]).await?.rows {
for row in rows.into_typed::<(CqlTimeuuid,)>() {
let (timeuuid_value,): (CqlTimeuuid,) = row?;
}
}
# Ok(())
# }
```
8 changes: 4 additions & 4 deletions docs/source/data-types/uuid.md
Original file line number Diff line number Diff line change
@@ -1,6 +1,6 @@
# Uuid, Timeuuid
# Uuid

`Uuid` and `Timeuuid` are represented as `uuid::Uuid`
`Uuid` is represented as `uuid::Uuid`.

```rust
# extern crate scylla;
Expand All @@ -11,13 +11,13 @@
use scylla::IntoTypedRows;
use uuid::Uuid;

// Insert some uuid/timeuuid into the table
// Insert some uuid into the table
let to_insert: Uuid = Uuid::parse_str("8e14e760-7fa8-11eb-bc66-000000000001")?;
session
.query("INSERT INTO keyspace.table (a) VALUES(?)", (to_insert,))
.await?;

// Read uuid/timeuuid from the table
// Read uuid from the table
if let Some(rows) = session.query("SELECT a FROM keyspace.table", &[]).await?.rows {
for row in rows.into_typed::<(Uuid,)>() {
let (uuid_value,): (Uuid,) = row?;
Expand Down
13 changes: 8 additions & 5 deletions scylla-cql/src/frame/response/cql_to_rust.rs
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@
use super::result::{CqlValue, Row};
use crate::frame::value::{Counter, CqlDate, CqlDuration, CqlTime, CqlTimestamp};
use crate::frame::value::{Counter, CqlDate, CqlDuration, CqlTime, CqlTimestamp, CqlTimeuuid};
use bigdecimal::BigDecimal;
use num_bigint::BigInt;
use std::collections::{BTreeMap, BTreeSet, HashMap, HashSet};
Expand Down Expand Up @@ -134,6 +134,7 @@ impl_from_cql_value_from_method!(String, into_string); // String::from_cql<CqlVa
impl_from_cql_value_from_method!(Vec<u8>, into_blob); // Vec<u8>::from_cql<CqlValue>
impl_from_cql_value_from_method!(IpAddr, as_inet); // IpAddr::from_cql<CqlValue>
impl_from_cql_value_from_method!(Uuid, as_uuid); // Uuid::from_cql<CqlValue>
impl_from_cql_value_from_method!(CqlTimeuuid, as_timeuuid); // CqlTimeuuid::from_cql<CqlValue>
impl_from_cql_value_from_method!(BigDecimal, into_decimal); // BigDecimal::from_cql<CqlValue>
impl_from_cql_value_from_method!(CqlDuration, as_cql_duration); // CqlDuration::from_cql<CqlValue>
impl_from_cql_value_from_method!(CqlDate, as_cql_date); // CqlDate::from_cql<CqlValue>
Expand Down Expand Up @@ -390,7 +391,7 @@ impl_tuple_from_cql!(T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14
mod tests {
use super::{CqlValue, FromCqlVal, FromCqlValError, FromRow, FromRowError, Row};
use crate as scylla;
use crate::frame::value::{Counter, CqlDate, CqlDuration, CqlTime, CqlTimestamp};
use crate::frame::value::{Counter, CqlDate, CqlDuration, CqlTime, CqlTimestamp, CqlTimeuuid};
use crate::macros::FromRow;
use bigdecimal::BigDecimal;
use num_bigint::{BigInt, ToBigInt};
Expand Down Expand Up @@ -753,16 +754,18 @@ mod tests {

#[test]
fn uuid_from_cql() {
let test_uuid: Uuid = Uuid::parse_str("8e14e760-7fa8-11eb-bc66-000000000001").unwrap();
let uuid_str = "8e14e760-7fa8-11eb-bc66-000000000001";
let test_uuid: Uuid = Uuid::parse_str(uuid_str).unwrap();
let test_time_uuid = CqlTimeuuid::from_str(uuid_str).unwrap();

assert_eq!(
test_uuid,
Uuid::from_cql(CqlValue::Uuid(test_uuid)).unwrap()
);

assert_eq!(
test_uuid,
Uuid::from_cql(CqlValue::Timeuuid(test_uuid)).unwrap()
test_time_uuid,
CqlTimeuuid::from_cql(CqlValue::Timeuuid(test_time_uuid)).unwrap()
);
}

Expand Down
16 changes: 8 additions & 8 deletions scylla-cql/src/frame/response/result.rs
Original file line number Diff line number Diff line change
@@ -1,7 +1,7 @@
use crate::cql_to_rust::{FromRow, FromRowError};
use crate::frame::response::event::SchemaChangeEvent;
use crate::frame::types::vint_decode;
use crate::frame::value::{Counter, CqlDate, CqlDuration, CqlTime, CqlTimestamp};
use crate::frame::value::{Counter, CqlDate, CqlDuration, CqlTime, CqlTimestamp, CqlTimeuuid};
use crate::frame::{frame_errors::ParseError, types};
use bigdecimal::BigDecimal;
use byteorder::{BigEndian, ReadBytesExt};
Expand Down Expand Up @@ -110,7 +110,7 @@ pub enum CqlValue {
TinyInt(i8),
/// Nanoseconds since midnight
Time(CqlTime),
Timeuuid(Uuid),
Timeuuid(CqlTimeuuid),
Tuple(Vec<Option<CqlValue>>),
Uuid(Uuid),
Varint(BigInt),
Expand Down Expand Up @@ -231,7 +231,6 @@ impl CqlValue {
pub fn as_uuid(&self) -> Option<Uuid> {
match self {
Self::Uuid(u) => Some(*u),
Self::Timeuuid(u) => Some(*u),
_ => None,
}
}
Expand Down Expand Up @@ -285,7 +284,7 @@ impl CqlValue {
}
}

pub fn as_timeuuid(&self) -> Option<Uuid> {
pub fn as_timeuuid(&self) -> Option<CqlTimeuuid> {
match self {
Self::Timeuuid(u) => Some(*u),
_ => None,
Expand Down Expand Up @@ -770,7 +769,7 @@ pub fn deser_cql_value(typ: &ColumnType, buf: &mut &[u8]) -> StdResult<CqlValue,
)));
}
let uuid = uuid::Uuid::from_slice(buf).expect("Deserializing Uuid failed.");
CqlValue::Timeuuid(uuid)
CqlValue::Timeuuid(CqlTimeuuid::from(uuid))
}
Duration => {
let months = i32::try_from(vint_decode(buf)?)?;
Expand Down Expand Up @@ -966,7 +965,7 @@ pub fn deserialize(buf: &mut &[u8]) -> StdResult<Result, ParseError> {
#[cfg(test)]
mod tests {
use crate as scylla;
use crate::frame::value::{Counter, CqlDate, CqlDuration, CqlTime, CqlTimestamp};
use crate::frame::value::{Counter, CqlDate, CqlDuration, CqlTime, CqlTimestamp, CqlTimeuuid};
use bigdecimal::BigDecimal;
use num_bigint::BigInt;
use num_bigint::ToBigInt;
Expand All @@ -993,9 +992,10 @@ mod tests {
let uuid_serialize = super::deser_cql_value(&ColumnType::Uuid, uuid_slice).unwrap();
assert_eq!(uuid_serialize, CqlValue::Uuid(my_uuid));

let my_timeuuid = CqlTimeuuid::from_str("00000000000000000000000000000001").unwrap();
let time_uuid_serialize =
super::deser_cql_value(&ColumnType::Timeuuid, uuid_slice).unwrap();
assert_eq!(time_uuid_serialize, CqlValue::Timeuuid(my_uuid));
assert_eq!(time_uuid_serialize, CqlValue::Timeuuid(my_timeuuid));

let my_ip = "::1".parse().unwrap();
let ip_buf: Vec<u8> = vec![0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1];
Expand Down Expand Up @@ -1754,7 +1754,7 @@ mod tests {
match cql_val {
CqlValue::Timeuuid(uuid) => {
assert_eq!(uuid.as_bytes(), uuid_bytes);
assert_eq!(Uuid::parse_str(uuid_str).unwrap(), uuid);
assert_eq!(CqlTimeuuid::from_str(uuid_str).unwrap(), uuid);
}
_ => panic!("Timeuuid parsed as wrong CqlValue"),
}
Expand Down
170 changes: 170 additions & 0 deletions scylla-cql/src/frame/value.rs
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,170 @@ pub enum MaybeUnset<V> {
Set(V),
}

/// Represents timeuuid (uuid V1) value
///
/// This type has custom comparison logic which follows Scylla/Cassandra semantics.
/// For details, see [`Ord` implementation](#impl-Ord-for-CqlTimeuuid).
#[derive(Debug, Clone, Copy, Eq)]
pub struct CqlTimeuuid(Uuid);

/// [`Uuid`] delegate methods
impl CqlTimeuuid {
pub fn as_bytes(&self) -> &[u8; 16] {
self.0.as_bytes()
}

pub fn as_u128(&self) -> u128 {
self.0.as_u128()
}

pub fn as_fields(&self) -> (u32, u16, u16, &[u8; 8]) {
self.0.as_fields()
}

pub fn as_u64_pair(&self) -> (u64, u64) {
self.0.as_u64_pair()
}

pub fn from_slice(b: &[u8]) -> Result<Self, uuid::Error> {
Ok(Self(Uuid::from_slice(b)?))
}

pub fn from_slice_le(b: &[u8]) -> Result<Self, uuid::Error> {
Ok(Self(Uuid::from_slice_le(b)?))
}

pub fn from_bytes(bytes: [u8; 16]) -> Self {
Self(Uuid::from_bytes(bytes))
}

pub fn from_bytes_le(bytes: [u8; 16]) -> Self {
Self(Uuid::from_bytes_le(bytes))
}

pub fn from_fields(d1: u32, d2: u16, d3: u16, d4: &[u8; 8]) -> Self {
Self(Uuid::from_fields(d1, d2, d3, d4))
}

pub fn from_fields_le(d1: u32, d2: u16, d3: u16, d4: &[u8; 8]) -> Self {
Self(Uuid::from_fields_le(d1, d2, d3, d4))
}

pub fn from_u128(v: u128) -> Self {
Self(Uuid::from_u128(v))
}

pub fn from_u128_le(v: u128) -> Self {
Self(Uuid::from_u128_le(v))
}

pub fn from_u64_pair(high_bits: u64, low_bits: u64) -> Self {
Self(Uuid::from_u64_pair(high_bits, low_bits))
}
}

impl CqlTimeuuid {
/// Read 8 most significant bytes of timeuuid from serialized bytes
fn msb(&self) -> u64 {
// Scylla and Cassandra use a standard UUID memory layout for MSB:
// 4 bytes 2 bytes 2 bytes
// time_low - time_mid - time_hi_and_version
let bytes = self.0.as_bytes();
((bytes[6] & 0x0F) as u64) << 56
| (bytes[7] as u64) << 48
| (bytes[4] as u64) << 40
| (bytes[5] as u64) << 32
| (bytes[0] as u64) << 24
| (bytes[1] as u64) << 16
| (bytes[2] as u64) << 8
| (bytes[3] as u64)
}

fn lsb(&self) -> u64 {
let bytes = self.0.as_bytes();
(bytes[8] as u64) << 56
| (bytes[9] as u64) << 48
| (bytes[10] as u64) << 40
| (bytes[11] as u64) << 32
| (bytes[12] as u64) << 24
| (bytes[13] as u64) << 16
| (bytes[14] as u64) << 8
| (bytes[15] as u64)
}

fn lsb_signed(&self) -> u64 {
self.lsb() ^ 0x8080808080808080
}
}

impl std::str::FromStr for CqlTimeuuid {
type Err = uuid::Error;

fn from_str(s: &str) -> Result<Self, Self::Err> {
Ok(Self(Uuid::from_str(s)?))
}
}

impl std::fmt::Display for CqlTimeuuid {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
write!(f, "{}", self.0)
}
}

impl AsRef<Uuid> for CqlTimeuuid {
fn as_ref(&self) -> &Uuid {
&self.0
}
}

impl From<CqlTimeuuid> for Uuid {
fn from(value: CqlTimeuuid) -> Self {
value.0
}
}

impl From<Uuid> for CqlTimeuuid {
fn from(value: Uuid) -> Self {
Self(value)
}
}

/// Compare two values of timeuuid type.
///
/// Cassandra legacy requires:
/// - converting 8 most significant bytes to date, which is then compared.
/// - masking off UUID version from the 8 ms-bytes during compare, to
/// treat possible non-version-1 UUID the same way as UUID.
/// - using signed compare for least significant bits.
impl Ord for CqlTimeuuid {
fn cmp(&self, other: &Self) -> std::cmp::Ordering {
let mut res = self.msb().cmp(&other.msb());
if let std::cmp::Ordering::Equal = res {
res = self.lsb_signed().cmp(&other.lsb_signed());
}
res
}
}

impl PartialOrd for CqlTimeuuid {
fn partial_cmp(&self, other: &Self) -> Option<std::cmp::Ordering> {
Some(self.cmp(other))
}
}

impl PartialEq for CqlTimeuuid {
fn eq(&self, other: &Self) -> bool {
self.cmp(other) == std::cmp::Ordering::Equal
}
}

impl std::hash::Hash for CqlTimeuuid {
fn hash<H: std::hash::Hasher>(&self, state: &mut H) {
self.lsb_signed().hash(state);
self.msb().hash(state);
}
}

/// Native CQL date representation that allows for a bigger range of dates (-262145-1-1 to 262143-12-31).
///
/// Represented as number of days since -5877641-06-23 i.e. 2^31 days before unix epoch.
Expand Down Expand Up @@ -696,6 +860,12 @@ impl Value for Uuid {
}
}

impl Value for CqlTimeuuid {
fn serialize(&self, buf: &mut Vec<u8>) -> Result<(), ValueTooBig> {
self.0.serialize(buf)
}
}

impl Value for BigInt {
fn serialize(&self, buf: &mut Vec<u8>) -> Result<(), ValueTooBig> {
let serialized = self.to_signed_bytes_be();
Expand Down
Loading

0 comments on commit d01b473

Please sign in to comment.