Skip to content

Commit

Permalink
argconv: make FFI implementation mutually exclusive
Browse files Browse the repository at this point in the history
Disallow implementing two different APIs for specific type.
  • Loading branch information
muzarski committed Dec 19, 2024
1 parent 739d7fe commit 6225480
Show file tree
Hide file tree
Showing 19 changed files with 181 additions and 43 deletions.
110 changes: 97 additions & 13 deletions scylla-rust-wrapper/src/argconv.rs
Original file line number Diff line number Diff line change
Expand Up @@ -272,12 +272,18 @@ impl<T: Sized> CassPtr<'_, T, (Mut,)> {
}
}

mod origin_sealed {
pub trait FromBoxSealed {}
pub trait FromArcSealed {}
pub trait FromRefSealed {}
}

/// Defines a pointer manipulation API for non-shared heap-allocated data.
///
/// Implement this trait for types that are allocated by the driver via [`Box::new`],
/// and then returned to the user as a pointer. The user is responsible for freeing
/// the memory associated with the pointer using corresponding driver's API function.
pub trait BoxFFI: Sized {
pub trait BoxFFI: Sized + origin_sealed::FromBoxSealed {
/// Consumes the Box and returns a pointer with exclusive ownership.
/// The pointer needs to be freed. See [`BoxFFI::free()`].
fn into_ptr<M: Mutability>(self: Box<Self>) -> CassPtr<'static, Self, (M,)> {
Expand Down Expand Up @@ -342,7 +348,7 @@ pub trait BoxFFI: Sized {
/// The data should be allocated via [`Arc::new`], and then returned to the user as a pointer.
/// The user is responsible for freeing the memory associated
/// with the pointer using corresponding driver's API function.
pub trait ArcFFI: Sized {
pub trait ArcFFI: Sized + origin_sealed::FromArcSealed {
/// Creates a pointer from a valid reference to Arc-allocated data.
/// Holder of the pointer borrows the pointee.
fn as_ptr<'a>(self: &'a Arc<Self>) -> CassPtr<'a, Self, (Const,)> {
Expand Down Expand Up @@ -431,7 +437,7 @@ pub trait ArcFFI: Sized {
/// For example: lifetime of CassRow is bound by the lifetime of CassResult.
/// There is no API function that frees the CassRow. It should be automatically
/// freed when user calls cass_result_free.
pub trait RefFFI: Sized {
pub trait RefFFI: Sized + origin_sealed::FromRefSealed {
/// Creates a borrowed pointer from a valid reference.
fn as_ptr<'a>(&'a self) -> CassPtr<'a, Self, (Const,)> {
// SAFETY:
Expand Down Expand Up @@ -472,11 +478,81 @@ pub trait RefFFI: Sized {
}
}

/// This trait should be implemented for types that are passed between
/// C and Rust API. We currently distinguish 3 kinds of implementors,
/// wrt. the origin of the pointer. The implementor should pick one of the 3 ownership
/// kinds as the associated type:
/// - [`FromBox`]
/// - [`FromArc`]
/// - [`FromRef`]
#[allow(clippy::upper_case_acronyms)]
pub trait FFI {
type Origin;
}

/// Represents types with an exclusive ownership.
///
/// Use this associated type for implementors that require:
/// - owned exclusive pointer manipulation via [`BoxFFI`]
/// - exclusive ownership of the corresponding object
/// - potential mutability of the corresponding object
/// - manual memory freeing
///
/// C API user should be responsible for freeing associated memory manually
/// via corresponding API call.
///
/// An example of such implementor would be [`CassCluster`](crate::cluster::CassCluster):
/// - it is allocated on the heap via [`Box::new`]
/// - user is the exclusive owner of the CassCluster object
/// - there is no API to increase a reference count of CassCluster object
/// - CassCluster is mutable via some API methods (`cass_cluster_set_*`)
/// - user is responsible for freeing the associated memory (`cass_cluster_free`)
pub struct FromBox;
impl<T> origin_sealed::FromBoxSealed for T where T: FFI<Origin = FromBox> {}
impl<T> BoxFFI for T where T: FFI<Origin = FromBox> {}

/// Represents types with a shared ownership.
///
/// Use this associated type for implementors that require:
/// - pointer with shared ownership manipulation via [`ArcFFI`]
/// - shared ownership of the corresponding object
/// - manual memory freeing
///
/// C API user should be responsible for freeing (decreasing reference count of)
/// associated memory manually via corresponding API call.
///
/// An example of such implementor would be [`CassDataType`](crate::cass_types::CassDataType):
/// - it is allocated on the heap via [`Arc::new`]
/// - there are multiple owners of the shared CassDataType object
/// - some API functions require to increase a reference count of the object
/// - user is responsible for freeing (decreasing RC of) the associated memory (`cass_data_type_free`)
pub struct FromArc;
impl<T> origin_sealed::FromArcSealed for T where T: FFI<Origin = FromArc> {}
impl<T> ArcFFI for T where T: FFI<Origin = FromArc> {}

/// Represents borrowed types.
///
/// Use this associated type for implementors that do not require any assumptions
/// about the pointer type (apart from validity).
/// The implementation will enable [`CassBorrowedPtr`] manipulation via [`RefFFI`]
///
/// C API user is not responsible for freeing associated memory manually. The memory
/// should be freed automatically, when the owner is being dropped.
///
/// An example of such implementor would be [`CassRow`](crate::query_result::CassRow):
/// - its lifetime is tied to the lifetime of CassResult
/// - user only "borrows" the pointer - he is not responsible for freeing the memory
pub struct FromRef;
impl<T> origin_sealed::FromRefSealed for T where T: FFI<Origin = FromRef> {}
impl<T> RefFFI for T where T: FFI<Origin = FromRef> {}

/// ```compile_fail,E0499
/// # use scylla_cpp_driver::argconv::{CassOwnedMutPtr, CassBorrowedMutPtr};
/// # use scylla_cpp_driver::argconv::BoxFFI;
/// # use scylla_cpp_driver::argconv::{FFI, BoxFFI, FromBox};
/// struct Foo;
/// impl BoxFFI for Foo {}
/// impl FFI for Foo {
/// type Origin = FromBox;
/// }
///
/// let mut ptr: CassOwnedMutPtr<Foo> = BoxFFI::into_ptr(Box::new(Foo));
/// let borrowed_mut_ptr1: CassBorrowedMutPtr<Foo> = ptr.borrow_mut();
Expand All @@ -488,9 +564,11 @@ fn _test_box_ffi_cannot_have_two_mutable_references() {}

/// ```compile_fail,E0502
/// # use scylla_cpp_driver::argconv::{CassOwnedMutPtr, CassBorrowedPtr, CassBorrowedMutPtr};
/// # use scylla_cpp_driver::argconv::BoxFFI;
/// # use scylla_cpp_driver::argconv::{FFI, BoxFFI, FromBox};
/// struct Foo;
/// impl BoxFFI for Foo {}
/// impl FFI for Foo {
/// type Origin = FromBox;
/// }
///
/// let mut ptr: CassOwnedMutPtr<Foo> = BoxFFI::into_ptr(Box::new(Foo));
/// let borrowed_mut_ptr: CassBorrowedMutPtr<Foo> = ptr.borrow_mut();
Expand All @@ -502,9 +580,11 @@ fn _test_box_ffi_cannot_have_mutable_and_immutable_references_at_the_same_time()

/// ```compile_fail,E0505
/// # use scylla_cpp_driver::argconv::{CassOwnedMutPtr, CassBorrowedPtr};
/// # use scylla_cpp_driver::argconv::BoxFFI;
/// # use scylla_cpp_driver::argconv::{FFI, BoxFFI, FromBox};
/// struct Foo;
/// impl BoxFFI for Foo {}
/// impl FFI for Foo {
/// type Origin = FromBox;
/// }
///
/// let ptr: CassOwnedMutPtr<Foo> = BoxFFI::into_ptr(Box::new(Foo));
/// let borrowed_ptr: CassBorrowedPtr<Foo> = ptr.borrow();
Expand All @@ -515,10 +595,12 @@ fn _test_box_ffi_cannot_free_while_having_borrowed_pointer() {}

/// ```compile_fail,E0505
/// # use scylla_cpp_driver::argconv::{CassOwnedPtr, CassBorrowedPtr};
/// # use scylla_cpp_driver::argconv::ArcFFI;
/// # use scylla_cpp_driver::argconv::{FFI, ArcFFI, FromArc};
/// # use std::sync::Arc;
/// struct Foo;
/// impl ArcFFI for Foo {}
/// impl FFI for Foo {
/// type Origin = FromArc;
/// }
///
/// let ptr: CassOwnedPtr<Foo> = ArcFFI::into_ptr(Arc::new(Foo));
/// let borrowed_ptr: CassBorrowedPtr<Foo> = ptr.borrow();
Expand All @@ -529,10 +611,12 @@ fn _test_arc_ffi_cannot_clone_after_free() {}

/// ```compile_fail,E0505
/// # use scylla_cpp_driver::argconv::{CassBorrowedPtr};
/// # use scylla_cpp_driver::argconv::ArcFFI;
/// # use scylla_cpp_driver::argconv::{FFI, ArcFFI, FromArc};
/// # use std::sync::Arc;
/// struct Foo;
/// impl ArcFFI for Foo {}
/// impl FFI for Foo {
/// type Origin = FromArc;
/// }
///
/// let arc = Arc::new(Foo);
/// let borrowed_ptr: CassBorrowedPtr<Foo> = ArcFFI::as_ptr(&arc);
Expand Down
8 changes: 6 additions & 2 deletions scylla-rust-wrapper/src/batch.rs
Original file line number Diff line number Diff line change
@@ -1,4 +1,6 @@
use crate::argconv::{ArcFFI, BoxFFI, CassBorrowedMutPtr, CassBorrowedPtr, CassOwnedMutPtr};
use crate::argconv::{
ArcFFI, BoxFFI, CassBorrowedMutPtr, CassBorrowedPtr, CassOwnedMutPtr, FromBox, FFI,
};
use crate::cass_error::CassError;
use crate::cass_types::CassConsistency;
use crate::cass_types::{make_batch_type, CassBatchType};
Expand All @@ -19,7 +21,9 @@ pub struct CassBatch {
pub(crate) exec_profile: Option<PerStatementExecProfile>,
}

impl BoxFFI for CassBatch {}
impl FFI for CassBatch {
type Origin = FromBox;
}

#[derive(Clone)]
pub struct CassBatchState {
Expand Down
4 changes: 3 additions & 1 deletion scylla-rust-wrapper/src/cass_types.rs
Original file line number Diff line number Diff line change
Expand Up @@ -175,7 +175,9 @@ pub enum CassDataTypeInner {
Custom(String),
}

impl ArcFFI for CassDataType {}
impl FFI for CassDataType {
type Origin = FromArc;
}

impl CassDataTypeInner {
/// Checks for equality during typechecks.
Expand Down
4 changes: 3 additions & 1 deletion scylla-rust-wrapper/src/cluster.rs
Original file line number Diff line number Diff line change
Expand Up @@ -165,7 +165,9 @@ impl CassCluster {
}
}

impl BoxFFI for CassCluster {}
impl FFI for CassCluster {
type Origin = FromBox;
}

pub struct CassCustomPayload;

Expand Down
4 changes: 3 additions & 1 deletion scylla-rust-wrapper/src/collection.rs
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,9 @@ pub struct CassCollection {
pub items: Vec<CassCqlValue>,
}

impl BoxFFI for CassCollection {}
impl FFI for CassCollection {
type Origin = FromBox;
}

impl CassCollection {
fn typecheck_on_append(&self, value: &Option<CassCqlValue>) -> CassError {
Expand Down
5 changes: 4 additions & 1 deletion scylla-rust-wrapper/src/exec_profile.rs
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ use scylla::statement::Consistency;

use crate::argconv::{
ptr_to_cstr_n, strlen, ArcFFI, BoxFFI, CassBorrowedMutPtr, CassBorrowedPtr, CassOwnedMutPtr,
FromBox, FFI,
};
use crate::batch::CassBatch;
use crate::cass_error::CassError;
Expand All @@ -39,7 +40,9 @@ pub struct CassExecProfile {
load_balancing_config: LoadBalancingConfig,
}

impl BoxFFI for CassExecProfile {}
impl FFI for CassExecProfile {
type Origin = FromBox;
}

impl CassExecProfile {
fn new() -> Self {
Expand Down
4 changes: 3 additions & 1 deletion scylla-rust-wrapper/src/future.rs
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,9 @@ pub struct CassFuture {
wait_for_value: Condvar,
}

impl ArcFFI for CassFuture {}
impl FFI for CassFuture {
type Origin = FromArc;
}

/// An error that can appear during `cass_future_wait_timed`.
enum FutureError {
Expand Down
6 changes: 4 additions & 2 deletions scylla-rust-wrapper/src/logging.rs
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
use crate::argconv::{arr_to_cstr, ptr_to_cstr, str_to_arr, CassBorrowedPtr, RefFFI};
use crate::argconv::{arr_to_cstr, ptr_to_cstr, str_to_arr, CassBorrowedPtr, FromRef, RefFFI, FFI};
use crate::cass_log_types::{CassLogLevel, CassLogMessage};
use crate::types::size_t;
use crate::LOGGER;
Expand All @@ -14,7 +14,9 @@ use tracing_subscriber::layer::Context;
use tracing_subscriber::prelude::*;
use tracing_subscriber::Layer;

impl RefFFI for CassLogMessage {}
impl FFI for CassLogMessage {
type Origin = FromRef;
}

pub type CassLogCallback =
Option<unsafe extern "C" fn(message: CassBorrowedPtr<CassLogMessage>, data: *mut c_void)>;
Expand Down
20 changes: 15 additions & 5 deletions scylla-rust-wrapper/src/metadata.rs
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,9 @@ pub struct CassSchemaMeta {
pub keyspaces: HashMap<String, CassKeyspaceMeta>,
}

impl BoxFFI for CassSchemaMeta {}
impl FFI for CassSchemaMeta {
type Origin = FromBox;
}

pub struct CassKeyspaceMeta {
pub name: String,
Expand All @@ -25,7 +27,9 @@ pub struct CassKeyspaceMeta {
}

// Owned by CassSchemaMeta
impl RefFFI for CassKeyspaceMeta {}
impl FFI for CassKeyspaceMeta {
type Origin = FromRef;
}

pub struct CassTableMeta {
pub name: String,
Expand All @@ -38,7 +42,9 @@ pub struct CassTableMeta {
// Either:
// - owned by CassMaterializedViewMeta - won't be given to user
// - Owned by CassKeyspaceMeta (in Arc), referenced (Weak) by CassMaterializedViewMeta
impl RefFFI for CassTableMeta {}
impl FFI for CassTableMeta {
type Origin = FromRef;
}

pub struct CassMaterializedViewMeta {
pub name: String,
Expand All @@ -47,7 +53,9 @@ pub struct CassMaterializedViewMeta {
}

// Shared ownership by CassKeyspaceMeta and CassTableMeta
impl RefFFI for CassMaterializedViewMeta {}
impl FFI for CassMaterializedViewMeta {
type Origin = FromRef;
}

pub struct CassColumnMeta {
pub name: String,
Expand All @@ -56,7 +64,9 @@ pub struct CassColumnMeta {
}

// Owned by CassTableMeta
impl RefFFI for CassColumnMeta {}
impl FFI for CassColumnMeta {
type Origin = FromRef;
}

pub unsafe fn create_table_metadata(
keyspace_name: &str,
Expand Down
4 changes: 3 additions & 1 deletion scylla-rust-wrapper/src/prepared.rs
Original file line number Diff line number Diff line change
Expand Up @@ -72,7 +72,9 @@ impl CassPrepared {
}
}

impl ArcFFI for CassPrepared {}
impl FFI for CassPrepared {
type Origin = FromArc;
}

#[no_mangle]
pub unsafe extern "C" fn cass_prepared_free(prepared_raw: CassOwnedPtr<CassPrepared>) {
Expand Down
4 changes: 3 additions & 1 deletion scylla-rust-wrapper/src/query_error.rs
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,9 @@ pub enum CassErrorResult {
Deserialization(#[from] DeserializationError),
}

impl ArcFFI for CassErrorResult {}
impl FFI for CassErrorResult {
type Origin = FromArc;
}

impl From<Consistency> for CassConsistency {
fn from(c: Consistency) -> CassConsistency {
Expand Down
16 changes: 12 additions & 4 deletions scylla-rust-wrapper/src/query_result.rs
Original file line number Diff line number Diff line change
Expand Up @@ -95,7 +95,9 @@ impl CassResult {
}
}

impl ArcFFI for CassResult {}
impl FFI for CassResult {
type Origin = FromArc;
}

#[derive(Debug)]
pub struct CassResultMetadata {
Expand Down Expand Up @@ -149,7 +151,9 @@ pub struct CassRow {
pub result_metadata: Arc<CassResultMetadata>,
}

impl RefFFI for CassRow {}
impl FFI for CassRow {
type Origin = FromRef;
}

pub fn create_cass_rows_from_rows(
rows: Vec<Row>,
Expand Down Expand Up @@ -185,7 +189,9 @@ pub struct CassValue {
pub value_type: Arc<CassDataType>,
}

impl RefFFI for CassValue {}
impl FFI for CassValue {
type Origin = FromRef;
}

fn create_cass_row_columns(row: Row, metadata: &Arc<CassResultMetadata>) -> Vec<CassValue> {
row.columns
Expand Down Expand Up @@ -367,7 +373,9 @@ pub enum CassIterator<'result_or_schema> {
CassViewMetaIterator(CassViewMetaIterator<'result_or_schema>),
}

impl BoxFFI for CassIterator<'_> {}
impl FFI for CassIterator<'_> {
type Origin = FromBox;
}

#[no_mangle]
pub unsafe extern "C" fn cass_iterator_free(iterator: CassOwnedMutPtr<CassIterator>) {
Expand Down
Loading

0 comments on commit 6225480

Please sign in to comment.