Skip to content

Commit

Permalink
types: serialize: introduce new helpers for writing values and adjust…
Browse files Browse the repository at this point in the history
… interfaces

Currently, `SerializeRow` and `SerializeCql` traits are just given a
mutable reference to a Vec<u8> and asked to append their CQL
representation to the end. While simple, there are some issues with the
interface:

- The serialize method has access to the serialized representation of
  the values that were appended before it. It's not necessary for a
  correct implementation to have access to it.
- Implementors technically can append any byte sequence to the end, but
  actually are expected to produce a CQL [value] containing the
  serialized value.

While the `SerializeRow` and `SerializeCql` traits are not generally
meant to be manually implemented by the users, we can make the interface
easier to use and harder to misuse by making it append-only, restricting
what the users are allowed to append and requiring the users to append
anything by using a dash of type-level magic.

Introduce `RowWriter` and `CellWriter` traits which satisfy the above
wishes and constraints, and pass them instead of Vec<u8> in
`SerializeRow` and `SerializeCql`.

The new traits have two implementations - a Vec<u8> backed one that
actually appends the bytes given to it, and a usize-backed one which
just measures the length of the output without writing anything. Passing
the latter before doing the actual serialization will allow to
preallocate the right amount of bytes and then serialize without
reallocations. It should be measured whether the reallocation cost
always outweighs the calculation cost before implementing this
optimization.
  • Loading branch information
piodul committed Nov 23, 2023
1 parent 5e544c9 commit a00b284
Show file tree
Hide file tree
Showing 4 changed files with 688 additions and 15 deletions.
6 changes: 6 additions & 0 deletions scylla-cql/src/types/serialize/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -2,5 +2,11 @@ use std::{any::Any, sync::Arc};

pub mod row;
pub mod value;
pub mod writers;

pub use writers::{
BufBackedCellValueBuilder, BufBackedCellWriter, BufBackedRowWriter, CellValueBuilder,
CellWriter, CountingWriter, RowWriter,
};

type SerializationError = Arc<dyn Any + Send + Sync>;
161 changes: 151 additions & 10 deletions scylla-cql/src/types/serialize/row.rs
Original file line number Diff line number Diff line change
@@ -1,20 +1,25 @@
use std::sync::Arc;
use std::{collections::HashMap, sync::Arc};

use thiserror::Error;

use crate::frame::response::result::ColumnSpec;
use crate::frame::value::ValueList;
use crate::frame::{response::result::ColumnSpec, types::RawValue};

use super::SerializationError;
use super::{CellWriter, RowWriter, SerializationError};

/// Contains information needed to serialize a row.
pub struct RowSerializationContext<'a> {
columns: &'a [ColumnSpec],
}

impl<'a> RowSerializationContext<'a> {
/// Returns column/bind marker specifications for given query.
#[inline]
pub fn columns(&self) -> &'a [ColumnSpec] {
self.columns
}

/// Looks up and returns a column/bind marker by name.
// TODO: change RowSerializationContext to make this faster
#[inline]
pub fn column_by_name(&self, target: &str) -> Option<&ColumnSpec> {
Expand All @@ -23,11 +28,25 @@ impl<'a> RowSerializationContext<'a> {
}

pub trait SerializeRow {
/// Checks if it _might_ be possible to serialize the row according to the
/// information in the context.
///
/// This function is intended to serve as an optimization in the future,
/// if we were ever to introduce prepared statements parametrized by types.
///
/// Sometimes, a row cannot be fully type checked right away without knowing
/// the exact values of the columns (e.g. when deserializing to `CqlValue`),
/// but it's fine to do full type checking later in `serialize`.
fn preliminary_type_check(ctx: &RowSerializationContext<'_>) -> Result<(), SerializationError>;
fn serialize(

/// Serializes the row according to the information in the given context.
///
/// The function may assume that `preliminary_type_check` was called,
/// though it must not do anything unsafe if this assumption does not hold.
fn serialize<W: RowWriter>(
&self,
ctx: &RowSerializationContext<'_>,
out: &mut Vec<u8>,
writer: &mut W,
) -> Result<(), SerializationError>;
}

Expand All @@ -38,12 +57,134 @@ impl<T: ValueList> SerializeRow for T {
Ok(())
}

fn serialize(
fn serialize<W: RowWriter>(
&self,
_ctx: &RowSerializationContext<'_>,
out: &mut Vec<u8>,
ctx: &RowSerializationContext<'_>,
writer: &mut W,
) -> Result<(), SerializationError> {
self.write_to_request(out)
.map_err(|err| Arc::new(err) as SerializationError)
serialize_legacy_row(self, ctx, writer)
}
}

pub fn serialize_legacy_row<T: ValueList>(
r: &T,
ctx: &RowSerializationContext<'_>,
writer: &mut impl RowWriter,
) -> Result<(), SerializationError> {
let serialized =
<T as ValueList>::serialized(r).map_err(|err| Arc::new(err) as SerializationError)?;

let mut append_value = |value: RawValue| {
let cell_writer = writer.make_cell_writer();
let _proof = match value {
RawValue::Null => cell_writer.set_null(),
RawValue::Unset => cell_writer.set_unset(),
RawValue::Value(v) => cell_writer.set_value(v),
};
};

if !serialized.has_names() {
serialized.iter().for_each(append_value);
} else {
let values_by_name = serialized
.iter_name_value_pairs()
.map(|(k, v)| (k.unwrap(), v))
.collect::<HashMap<_, _>>();

for col in ctx.columns() {
let val = values_by_name.get(col.name.as_str()).ok_or_else(|| {
Arc::new(ValueListToSerializeRowAdapterError::NoBindMarkerWithName {
name: col.name.clone(),
}) as SerializationError
})?;
append_value(*val);
}
}

Ok(())
}

#[derive(Error, Debug)]
pub enum ValueListToSerializeRowAdapterError {
#[error("There is no bind marker with name {name}, but a value for it was provided")]
NoBindMarkerWithName { name: String },
}

#[cfg(test)]
mod tests {
use crate::frame::response::result::{ColumnSpec, ColumnType, TableSpec};
use crate::frame::value::{MaybeUnset, SerializedValues, ValueList};
use crate::types::serialize::BufBackedRowWriter;

use super::{RowSerializationContext, SerializeRow};

fn col_spec(name: &str, typ: ColumnType) -> ColumnSpec {
ColumnSpec {
table_spec: TableSpec {
ks_name: "ks".to_string(),
table_name: "tbl".to_string(),
},
name: name.to_string(),
typ,
}
}

#[test]
fn test_legacy_fallback() {
let row = (
1i32,
"Ala ma kota",
None::<i64>,
MaybeUnset::Unset::<String>,
);

let mut legacy_data = Vec::new();
<_ as ValueList>::write_to_request(&row, &mut legacy_data).unwrap();

let mut new_data = Vec::new();
let mut new_data_writer = BufBackedRowWriter::new(&mut new_data);
let ctx = RowSerializationContext { columns: &[] };
<_ as SerializeRow>::serialize(&row, &ctx, &mut new_data_writer).unwrap();
assert_eq!(new_data_writer.value_count(), 4);

// Skip the value count
assert_eq!(&legacy_data[2..], new_data);
}

#[test]
fn test_legacy_fallback_with_names() {
let sorted_row = (
1i32,
"Ala ma kota",
None::<i64>,
MaybeUnset::Unset::<String>,
);

let mut sorted_row_data = Vec::new();
<_ as ValueList>::write_to_request(&sorted_row, &mut sorted_row_data).unwrap();

let mut unsorted_row = SerializedValues::new();
unsorted_row.add_named_value("a", &1i32).unwrap();
unsorted_row.add_named_value("b", &"Ala ma kota").unwrap();
unsorted_row
.add_named_value("d", &MaybeUnset::Unset::<String>)
.unwrap();
unsorted_row.add_named_value("c", &None::<i64>).unwrap();

let mut unsorted_row_data = Vec::new();
let mut unsorted_row_data_writer = BufBackedRowWriter::new(&mut unsorted_row_data);
let ctx = RowSerializationContext {
columns: &[
col_spec("a", ColumnType::Int),
col_spec("b", ColumnType::Text),
col_spec("c", ColumnType::BigInt),
col_spec("d", ColumnType::Ascii),
],
};
<_ as SerializeRow>::serialize(&unsorted_row, &ctx, &mut unsorted_row_data_writer).unwrap();
assert_eq!(unsorted_row_data_writer.value_count(), 4);

// Skip the value count
assert_eq!(&sorted_row_data[2..], unsorted_row_data);
}
}
110 changes: 105 additions & 5 deletions scylla-cql/src/types/serialize/value.rs
Original file line number Diff line number Diff line change
@@ -1,22 +1,122 @@
use std::sync::Arc;

use thiserror::Error;

use crate::frame::response::result::ColumnType;
use crate::frame::value::Value;

use super::SerializationError;
use super::{CellWriter, SerializationError};

pub trait SerializeCql {
/// Given a CQL type, checks if it _might_ be possible to serialize to that type.
///
/// This function is intended to serve as an optimization in the future,
/// if we were ever to introduce prepared statements parametrized by types.
///
/// Some types cannot be type checked without knowing the exact value,
/// this is the case e.g. for `CqlValue`. It's also fine to do it later in
/// `serialize`.
fn preliminary_type_check(typ: &ColumnType) -> Result<(), SerializationError>;
fn serialize(&self, typ: &ColumnType, buf: &mut Vec<u8>) -> Result<(), SerializationError>;

/// Serializes the value to given CQL type.
///
/// The function may assume that `preliminary_type_check` was called,
/// though it must not do anything unsafe if this assumption does not hold.
fn serialize<W: CellWriter>(
&self,
typ: &ColumnType,
writer: W,
) -> Result<W::WrittenCellProof, SerializationError>;
}

impl<T: Value> SerializeCql for T {
fn preliminary_type_check(_typ: &ColumnType) -> Result<(), SerializationError> {
Ok(())
}

fn serialize(&self, _typ: &ColumnType, buf: &mut Vec<u8>) -> Result<(), SerializationError> {
self.serialize(buf)
.map_err(|err| Arc::new(err) as SerializationError)
fn serialize<W: CellWriter>(
&self,
_typ: &ColumnType,
writer: W,
) -> Result<W::WrittenCellProof, SerializationError> {
serialize_legacy_value(self, writer)
}
}

pub fn serialize_legacy_value<T: Value, W: CellWriter>(
v: &T,
writer: W,
) -> Result<W::WrittenCellProof, SerializationError> {
// It's an inefficient and slightly tricky but correct implementation.
let mut buf = Vec::new();
<T as Value>::serialize(v, &mut buf).map_err(|err| Arc::new(err) as SerializationError)?;

// Analyze the output.
// All this dance shows how unsafe our previous interface was...
if buf.len() < 4 {
return Err(Arc::new(ValueToSerializeCqlAdapterError::TooShort {
size: buf.len(),
}));
}

let (len_bytes, contents) = buf.split_at(4);
let len = i32::from_be_bytes(len_bytes.try_into().unwrap());
match len {
-2 => Ok(writer.set_unset()),
-1 => Ok(writer.set_null()),
len if len >= 0 => {
if contents.len() != len as usize {
Err(Arc::new(
ValueToSerializeCqlAdapterError::DeclaredVsActualSizeMismatch {
declared: len as usize,
actual: contents.len(),
},
))
} else {
Ok(writer.set_value(contents))
}
}
_ => Err(Arc::new(
ValueToSerializeCqlAdapterError::InvalidDeclaredSize { size: len },
)),
}
}

#[derive(Error, Debug)]
pub enum ValueToSerializeCqlAdapterError {
#[error("Output produced by the Value trait is too short to be considered a value: {size} < 4 minimum bytes")]
TooShort { size: usize },

#[error("Mismatch between the declared value size vs. actual size: {declared} != {actual}")]
DeclaredVsActualSizeMismatch { declared: usize, actual: usize },

#[error("Invalid declared value size: {size}")]
InvalidDeclaredSize { size: i32 },
}

#[cfg(test)]
mod tests {
use crate::frame::response::result::ColumnType;
use crate::frame::value::{MaybeUnset, Value};
use crate::types::serialize::BufBackedCellWriter;

use super::SerializeCql;

fn check_compat<V: Value + SerializeCql>(v: V) {
let mut legacy_data = Vec::new();
<V as Value>::serialize(&v, &mut legacy_data).unwrap();

let mut new_data = Vec::new();
let new_data_writer = BufBackedCellWriter::new(&mut new_data);
<V as SerializeCql>::serialize(&v, &ColumnType::Int, new_data_writer).unwrap();

assert_eq!(legacy_data, new_data);
}

#[test]
fn test_legacy_fallback() {
check_compat(123i32);
check_compat(None::<i32>);
check_compat(MaybeUnset::Unset::<i32>);
}
}
Loading

0 comments on commit a00b284

Please sign in to comment.