From 3629d3511a15d56f1d5c107a8acf6097c0cce099 Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Tue, 14 Mar 2023 13:39:10 +0100 Subject: [PATCH 1/4] cluster: make Cluster cheaply cloneable MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit The Cluster struct by itself only serves as a facade for the ClusterWorker, i.e. it has channels that allow sending requests to the worker, receives the ClusterData via the `data` field etc. Apart from the `_worker_handle` field, all other fields are cloneable. Two tasks working on two copies of the same Cluster object should behave the same as if they shared and operated on a single Cluster object (e.g. via Arc). This commit makes the Cluster object cloneable - the `_worker_handle` is shared via an Arc. This will be very useful in the next commit - we will do a similar thing for the GenericSession object. Co-authored-by: Wojciech Przytuła --- scylla/src/transport/cluster.rs | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/scylla/src/transport/cluster.rs b/scylla/src/transport/cluster.rs index 7cd11dcda..9e7aae4a3 100644 --- a/scylla/src/transport/cluster.rs +++ b/scylla/src/transport/cluster.rs @@ -36,6 +36,18 @@ use super::topology::Strategy; /// Cluster manages up to date information and connections to database nodes. /// All data can be accessed by cloning Arc in the `data` field +// +// NOTE: This structure was intentionally made cloneable. The reason for this +// is to make it possible to use two different Session APIs in the same program +// that share the same session resources. +// +// It is safe to do because the Cluster struct is just a facade for the real, +// "semantic" Cluster object. Cloned instance of this struct will use the same +// ClusterData and worker and will observe the same state. +// +// TODO: revert this commit (one making Cluster clonable) once the legacy +// deserialization API is removed. +#[derive(Clone)] pub(crate) struct Cluster { // `ArcSwap` is wrapped in `Arc` to support sharing cluster data // between `Cluster` and `ClusterWorker` @@ -44,7 +56,7 @@ pub(crate) struct Cluster { refresh_channel: tokio::sync::mpsc::Sender, use_keyspace_channel: tokio::sync::mpsc::Sender, - _worker_handle: RemoteHandle<()>, + _worker_handle: Arc>, } /// Enables printing [Cluster] struct in a neat way, by skipping the rather useless @@ -204,7 +216,7 @@ impl Cluster { data: cluster_data, refresh_channel: refresh_sender, use_keyspace_channel: use_keyspace_sender, - _worker_handle: worker_handle, + _worker_handle: Arc::new(worker_handle), }; Ok(result) From 6baaf62bda7f9623ab5cbbfd7b467e236bf39595 Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Tue, 14 Mar 2023 14:04:54 +0100 Subject: [PATCH 2/4] session: add methods for creating compatible session with different API In order to make migration from the old API easier and allow doing it gradually, some components of the client programs would probably like to use the old API while the new components will use the new API. However, in the current design, Session and LegacySession are two different types and it's not possible to "cast" one to another - even though they have nearly the same fields and implementations. The previous commit made Cluster cloneable, based on the observation that it's perfectly fine to clone Cluster's fields, construct a new one and treat it as a shared facade, handle to the same "semantic" cluster. The same applies to Session, actually - cloning a session would have similar effect (though we encourage users to keep Session in an Arc so that cloning is cheaper). Instead of making GenericSession cloneable, we introduce methods which, in reality, perform a clone but change the kind of session's API. This allows to have two session objects which share the same resources but have different APIs. This should be very useful when migrating large projects to the new API - components that need to use the new API can just "convert" the session to the new interface and use that. --- scylla/src/transport/session.rs | 59 +++++++++++++++++++++++++++- scylla/src/transport/session_test.rs | 41 +++++++++++++++++++ 2 files changed, 98 insertions(+), 2 deletions(-) diff --git a/scylla/src/transport/session.rs b/scylla/src/transport/session.rs index 2ed5ca901..60f9a1d07 100644 --- a/scylla/src/transport/session.rs +++ b/scylla/src/transport/session.rs @@ -186,7 +186,7 @@ where schema_agreement_timeout: Duration, schema_agreement_automatic_waiting: bool, refresh_metadata_on_auto_schema_agreement: bool, - keyspace_name: ArcSwapOption, + keyspace_name: Arc>, tracing_info_fetch_attempts: NonZeroU32, tracing_info_fetch_interval: Duration, tracing_info_fetch_consistency: Consistency, @@ -852,6 +852,32 @@ impl GenericSession { ) -> Result { self.do_batch(batch, values).await } + + /// Creates a new Session instance that shared resources with + /// the current Session but supports the legacy API. + /// + /// This method is provided in order to make migration to the new + /// deserialization API easier. For example, if your program in general uses + /// the new API but you still have some modules left that use the old one, + /// you can use this method to create an instance that supports the old API + /// and pass it to the module that you intend to migrate later. + pub fn make_shared_session_with_legacy_api(&self) -> LegacySession { + LegacySession { + cluster: self.cluster.clone(), + default_execution_profile_handle: self.default_execution_profile_handle.clone(), + metrics: self.metrics.clone(), + refresh_metadata_on_auto_schema_agreement: self + .refresh_metadata_on_auto_schema_agreement, + schema_agreement_interval: self.schema_agreement_interval, + keyspace_name: self.keyspace_name.clone(), + schema_agreement_timeout: self.schema_agreement_timeout, + schema_agreement_automatic_waiting: self.schema_agreement_automatic_waiting, + tracing_info_fetch_attempts: self.tracing_info_fetch_attempts, + tracing_info_fetch_interval: self.tracing_info_fetch_interval, + tracing_info_fetch_consistency: self.tracing_info_fetch_consistency, + _phantom_deser_api: PhantomData, + } + } } impl GenericSession { @@ -928,6 +954,35 @@ impl GenericSession { ) -> Result { Ok(self.do_batch(batch, values).await?.into_legacy_result()?) } + + /// Creates a new Session instance that shares resources with + /// the current Session but supports the new API. + /// + /// This method is provided in order to make migration to the new + /// deserialization API easier. For example, if your program in general uses + /// the old API but you want to migrate some modules to the new one, you + /// can use this method to create an instance that supports the new API + /// and pass it to the module that you intend to migrate. + /// + /// The new session object will use the same connections and cluster + /// metadata. + pub fn make_shared_session_with_new_api(&self) -> Session { + Session { + cluster: self.cluster.clone(), + default_execution_profile_handle: self.default_execution_profile_handle.clone(), + metrics: self.metrics.clone(), + refresh_metadata_on_auto_schema_agreement: self + .refresh_metadata_on_auto_schema_agreement, + schema_agreement_interval: self.schema_agreement_interval, + keyspace_name: self.keyspace_name.clone(), + schema_agreement_timeout: self.schema_agreement_timeout, + schema_agreement_automatic_waiting: self.schema_agreement_automatic_waiting, + tracing_info_fetch_attempts: self.tracing_info_fetch_attempts, + tracing_info_fetch_interval: self.tracing_info_fetch_interval, + tracing_info_fetch_consistency: self.tracing_info_fetch_consistency, + _phantom_deser_api: PhantomData, + } + } } /// Represents a CQL session, which can be used to communicate @@ -1041,7 +1096,7 @@ where schema_agreement_automatic_waiting: config.schema_agreement_automatic_waiting, refresh_metadata_on_auto_schema_agreement: config .refresh_metadata_on_auto_schema_agreement, - keyspace_name: ArcSwapOption::default(), // will be set by use_keyspace + keyspace_name: Arc::new(ArcSwapOption::default()), // will be set by use_keyspace tracing_info_fetch_attempts: config.tracing_info_fetch_attempts, tracing_info_fetch_interval: config.tracing_info_fetch_interval, tracing_info_fetch_consistency: config.tracing_info_fetch_consistency, diff --git a/scylla/src/transport/session_test.rs b/scylla/src/transport/session_test.rs index 8027360a7..923cbd621 100644 --- a/scylla/src/transport/session_test.rs +++ b/scylla/src/transport/session_test.rs @@ -3123,3 +3123,44 @@ async fn test_deserialize_empty_collections() { .await; assert!(map.is_empty()); } + +#[tokio::test] +async fn test_api_migration_session_sharing() { + { + let session = create_new_session_builder().build().await.unwrap(); + let session_shared = session.make_shared_session_with_legacy_api(); + + // If we are unlucky then we will race with metadata fetch/cluster update + // and both invocations will return different cluster data. This should be + // SUPER rare, but in order to reduce the chance of flakiness to a minimum + // we will try it three times in a row. Cluster data is updated once per + // minute, so this should be good enough. + let mut matched = false; + for _ in 0..3 { + let cd1 = session.get_cluster_data(); + let cd2 = session_shared.get_cluster_data(); + + if Arc::ptr_eq(&cd1, &cd2) { + matched = true; + break; + } + } + assert!(matched); + } + { + let session = create_new_session_builder().build_legacy().await.unwrap(); + let session_shared = session.make_shared_session_with_new_api(); + + let mut matched = false; + for _ in 0..3 { + let cd1 = session.get_cluster_data(); + let cd2 = session_shared.get_cluster_data(); + + if Arc::ptr_eq(&cd1, &cd2) { + matched = true; + break; + } + } + assert!(matched); + } +} From 12608151c3264ee1ae0608712eb51756163fccc5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Wojciech=20Przytu=C5=82a?= Date: Tue, 13 Aug 2024 11:21:36 +0200 Subject: [PATCH 3/4] codewide: add #[deprecated] marker for legacy API As we want to phase out the legacy deserialization API, let's yell at users still using it, by use of the amazing #[deprecated] annotation, the greatest friend of a Rust lib maintainer. --- scylla-cql/src/frame/response/cql_to_rust.rs | 10 +++++++++ scylla-cql/src/frame/response/result.rs | 6 +++++ scylla-cql/src/lib.rs | 3 +++ scylla/src/lib.rs | 7 ++++-- scylla/src/macros.rs | 13 +++++++++++ scylla/src/transport/caching_session.rs | 17 ++++++++++++++- scylla/src/transport/iterator.rs | 17 +++++++++++++++ scylla/src/transport/legacy_query_result.rs | 9 +++++++- scylla/src/transport/query_result.rs | 6 +++++ scylla/src/transport/session.rs | 23 ++++++++++++++++++++ scylla/src/transport/session_builder.rs | 6 +++++ scylla/src/transport/session_test.rs | 1 + scylla/tests/integration/hygiene.rs | 1 + 13 files changed, 115 insertions(+), 4 deletions(-) diff --git a/scylla-cql/src/frame/response/cql_to_rust.rs b/scylla-cql/src/frame/response/cql_to_rust.rs index 8f199b83e..a2a450ea9 100644 --- a/scylla-cql/src/frame/response/cql_to_rust.rs +++ b/scylla-cql/src/frame/response/cql_to_rust.rs @@ -1,3 +1,5 @@ +#![allow(deprecated)] + use super::result::{CqlValue, Row}; use crate::frame::value::{ Counter, CqlDate, CqlDecimal, CqlDuration, CqlTime, CqlTimestamp, CqlTimeuuid, CqlVarint, @@ -19,6 +21,10 @@ pub enum FromRowError { /// This trait defines a way to convert CqlValue or `Option` into some rust type // We can't use From trait because impl From> for String {...} // is forbidden since neither From nor String are defined in this crate +#[deprecated( + since = "0.15.0", + note = "Legacy deserialization API is inefficient and is going to be removed soon" +)] pub trait FromCqlVal: Sized { fn from_cql(cql_val: T) -> Result; } @@ -34,6 +40,10 @@ pub enum FromCqlValError { } /// This trait defines a way to convert CQL Row into some rust type +#[deprecated( + since = "0.15.0", + note = "Legacy deserialization API is inefficient and is going to be removed soon" +)] pub trait FromRow: Sized { fn from_row(row: Row) -> Result; } diff --git a/scylla-cql/src/frame/response/result.rs b/scylla-cql/src/frame/response/result.rs index 54a1567c7..4858d0efc 100644 --- a/scylla-cql/src/frame/response/result.rs +++ b/scylla-cql/src/frame/response/result.rs @@ -1,3 +1,4 @@ +#[allow(deprecated)] use crate::cql_to_rust::{FromRow, FromRowError}; use crate::frame::frame_errors::{ ColumnSpecParseError, ColumnSpecParseErrorKind, CqlResultParseError, CqlTypeParseError, @@ -607,6 +608,11 @@ pub struct Row { impl Row { /// Allows converting Row into tuple of rust types or custom struct deriving FromRow + #[deprecated( + since = "0.15.0", + note = "Legacy deserialization API is inefficient and is going to be removed soon" + )] + #[allow(deprecated)] pub fn into_typed(self) -> StdResult { RowT::from_row(self) } diff --git a/scylla-cql/src/lib.rs b/scylla-cql/src/lib.rs index 03337c2d6..228fc43f8 100644 --- a/scylla-cql/src/lib.rs +++ b/scylla-cql/src/lib.rs @@ -13,6 +13,7 @@ pub mod macros { // Reexports for derive(IntoUserType) pub use bytes::{BufMut, Bytes, BytesMut}; + #[allow(deprecated)] pub use crate::impl_from_cql_value_from_method; pub use crate::impl_serialize_row_via_value_list; @@ -22,12 +23,14 @@ pub mod macros { pub mod types; pub use crate::frame::response::cql_to_rust; +#[allow(deprecated)] pub use crate::frame::response::cql_to_rust::FromRow; pub use crate::frame::types::Consistency; #[doc(hidden)] pub mod _macro_internal { + #[allow(deprecated)] pub use crate::frame::response::cql_to_rust::{ FromCqlVal, FromCqlValError, FromRow, FromRowError, }; diff --git a/scylla/src/lib.rs b/scylla/src/lib.rs index 39387f621..0bd45d7a1 100644 --- a/scylla/src/lib.rs +++ b/scylla/src/lib.rs @@ -266,13 +266,16 @@ pub use statement::batch; pub use statement::prepared_statement; pub use statement::query; -pub use frame::response::cql_to_rust; -pub use frame::response::cql_to_rust::FromRow; +#[allow(deprecated)] +pub use frame::response::cql_to_rust::{self, FromRow}; +#[allow(deprecated)] pub use transport::caching_session::{CachingSession, GenericCachingSession, LegacyCachingSession}; pub use transport::execution_profile::ExecutionProfile; +#[allow(deprecated)] pub use transport::legacy_query_result::LegacyQueryResult; pub use transport::query_result::{QueryResult, QueryRowsResult}; +#[allow(deprecated)] pub use transport::session::{IntoTypedRows, LegacySession, Session, SessionConfig}; pub use transport::session_builder::SessionBuilder; diff --git a/scylla/src/macros.rs b/scylla/src/macros.rs index 4851f334d..f5bef190f 100644 --- a/scylla/src/macros.rs +++ b/scylla/src/macros.rs @@ -6,6 +6,10 @@ /// /// --- /// +#[deprecated( + since = "0.15.0", + note = "Legacy deserialization API is inefficient and is going to be removed soon" +)] pub use scylla_cql::macros::FromRow; /// #[derive(FromUserType)] allows to parse struct as a User Defined Type @@ -14,6 +18,10 @@ pub use scylla_cql::macros::FromRow; /// /// --- /// +#[deprecated( + since = "0.15.0", + note = "Legacy deserialization API is inefficient and is going to be removed soon" +)] pub use scylla_cql::macros::FromUserType; /// #[derive(IntoUserType)] allows to pass struct a User Defined Type Value in queries @@ -468,6 +476,11 @@ pub use scylla_macros::DeserializeRow; /// pub use scylla_cql::macros::ValueList; +#[deprecated( + since = "0.15.0", + note = "Legacy deserialization API is inefficient and is going to be removed soon" +)] +#[allow(deprecated)] pub use scylla_cql::macros::impl_from_cql_value_from_method; pub use scylla_cql::macros::impl_serialize_row_via_value_list; diff --git a/scylla/src/transport/caching_session.rs b/scylla/src/transport/caching_session.rs index 108752e4c..79d2c2538 100644 --- a/scylla/src/transport/caching_session.rs +++ b/scylla/src/transport/caching_session.rs @@ -3,9 +3,12 @@ use crate::prepared_statement::PreparedStatement; use crate::query::Query; use crate::statement::{PagingState, PagingStateResponse}; use crate::transport::errors::QueryError; +#[allow(deprecated)] use crate::transport::iterator::LegacyRowIterator; use crate::transport::partitioner::PartitionerName; -use crate::{LegacyQueryResult, QueryResult}; +#[allow(deprecated)] +use crate::LegacyQueryResult; +use crate::QueryResult; use bytes::Bytes; use dashmap::DashMap; use futures::future::try_join_all; @@ -17,6 +20,7 @@ use std::hash::BuildHasher; use std::sync::Arc; use super::iterator::QueryPager; +#[allow(deprecated)] use super::session::{ CurrentDeserializationApi, DeserializationApiKind, GenericSession, LegacyDeserializationApi, }; @@ -50,6 +54,12 @@ where } pub type CachingSession = GenericCachingSession; + +#[deprecated( + since = "0.15.0", + note = "Legacy deserialization API is inefficient and is going to be removed soon" +)] +#[allow(deprecated)] pub type LegacyCachingSession = GenericCachingSession; impl GenericCachingSession @@ -149,6 +159,11 @@ where } } +#[deprecated( + since = "0.15.0", + note = "Legacy deserialization API is inefficient and is going to be removed soon" +)] +#[allow(deprecated)] impl GenericCachingSession where S: BuildHasher + Clone, diff --git a/scylla/src/transport/iterator.rs b/scylla/src/transport/iterator.rs index bda210b31..8d7c07be3 100644 --- a/scylla/src/transport/iterator.rs +++ b/scylla/src/transport/iterator.rs @@ -22,6 +22,7 @@ use tokio::sync::mpsc; use super::execution_profile::ExecutionProfileInner; use super::query_result::ColumnSpecs; use super::session::RequestSpan; +#[allow(deprecated)] use crate::cql_to_rust::{FromRow, FromRowError}; use crate::deserialize::DeserializeOwnedRow; @@ -665,6 +666,11 @@ impl QueryPager { /// using the legacy deserialization framework. /// This is inefficient, because all rows are being eagerly deserialized /// to a middle-man [Row] type. + #[deprecated( + since = "0.15.0", + note = "Legacy deserialization API is inefficient and is going to be removed soon" + )] + #[allow(deprecated)] #[inline] pub fn into_legacy(self) -> LegacyRowIterator { LegacyRowIterator::new(self) @@ -1065,11 +1071,16 @@ pub enum NextRowError { } mod legacy { + #![allow(deprecated)] use super::*; /// Iterator over rows returned by paged queries. /// /// Allows to easily access rows without worrying about handling multiple pages. + #[deprecated( + since = "0.15.0", + note = "Legacy deserialization API is inefficient and is going to be removed soon" + )] pub struct LegacyRowIterator { raw_stream: QueryPager, } @@ -1118,6 +1129,11 @@ mod legacy { /// Iterator over rows returned by paged queries /// where each row is parsed as the given type\ /// Returned by `RowIterator::into_typed` + #[deprecated( + since = "0.15.0", + note = "Legacy deserialization API is inefficient and is going to be removed soon" + )] + #[allow(deprecated)] pub struct LegacyTypedRowIterator { row_iterator: LegacyRowIterator, _phantom_data: std::marker::PhantomData, @@ -1170,4 +1186,5 @@ mod legacy { // LegacyTypedRowIterator can be moved freely for any RowT so it's Unpin impl Unpin for LegacyTypedRowIterator {} } +#[allow(deprecated)] pub use legacy::{LegacyNextRowError, LegacyRowIterator, LegacyTypedRowIterator}; diff --git a/scylla/src/transport/legacy_query_result.rs b/scylla/src/transport/legacy_query_result.rs index 8dd6b7214..6b257ffef 100644 --- a/scylla/src/transport/legacy_query_result.rs +++ b/scylla/src/transport/legacy_query_result.rs @@ -1,3 +1,5 @@ +#![allow(deprecated)] + use crate::frame::response::cql_to_rust::{FromRow, FromRowError}; use crate::frame::response::result::ColumnSpec; use crate::frame::response::result::Row; @@ -41,7 +43,12 @@ impl Iterator for TypedRowIter { /// Result of a single query\ /// Contains all rows returned by the database and some more information -#[derive(Debug)] +#[non_exhaustive] +#[derive(Default, Debug)] +#[deprecated( + since = "0.15.0", + note = "Legacy deserialization API is inefficient and is going to be removed soon" +)] pub struct LegacyQueryResult { /// Rows returned by the database.\ /// Queries like `SELECT` will have `Some(Vec)`, while queries like `INSERT` will have `None`.\ diff --git a/scylla/src/transport/query_result.rs b/scylla/src/transport/query_result.rs index 9a7745c50..114a433f2 100644 --- a/scylla/src/transport/query_result.rs +++ b/scylla/src/transport/query_result.rs @@ -11,6 +11,7 @@ use scylla_cql::types::deserialize::result::TypedRowIterator; use scylla_cql::types::deserialize::row::DeserializeRow; use scylla_cql::types::deserialize::{DeserializationError, TypeCheckError}; +#[allow(deprecated)] use super::legacy_query_result::{IntoLegacyQueryResultError, LegacyQueryResult}; /// A view over specification of a table in the database. @@ -258,6 +259,11 @@ impl QueryResult { /// Transforms itself into the legacy result type, by eagerly deserializing rows /// into the Row type. This is inefficient, and should only be used during transition /// period to the new API. + #[deprecated( + since = "0.15.0", + note = "Legacy deserialization API is inefficient and is going to be removed soon" + )] + #[allow(deprecated)] pub fn into_legacy_result(self) -> Result { if let Some(raw_rows) = self.raw_metadata_and_rows { let raw_rows_with_metadata = raw_rows.deserialize_metadata()?; diff --git a/scylla/src/transport/session.rs b/scylla/src/transport/session.rs index 60f9a1d07..b3efa7e07 100644 --- a/scylla/src/transport/session.rs +++ b/scylla/src/transport/session.rs @@ -4,6 +4,7 @@ use crate::batch::batch_values; #[cfg(feature = "cloud")] use crate::cloud::CloudConfig; +#[allow(deprecated)] use crate::LegacyQueryResult; use crate::history; @@ -64,6 +65,7 @@ use crate::transport::cluster::{Cluster, ClusterData, ClusterNeatDebug}; use crate::transport::connection::{Connection, ConnectionConfig, VerifiedKeyspaceName}; use crate::transport::connection_pool::PoolConfig; use crate::transport::host_filter::HostFilter; +#[allow(deprecated)] use crate::transport::iterator::{LegacyRowIterator, PreparedIteratorConfig}; use crate::transport::load_balancing::{self, RoutingInfo}; use crate::transport::metrics::Metrics; @@ -170,8 +172,14 @@ pub enum CurrentDeserializationApi {} impl sealed::Sealed for CurrentDeserializationApi {} impl DeserializationApiKind for CurrentDeserializationApi {} +#[deprecated( + since = "0.15.0", + note = "Legacy deserialization API is inefficient and is going to be removed soon" +)] pub enum LegacyDeserializationApi {} +#[allow(deprecated)] impl sealed::Sealed for LegacyDeserializationApi {} +#[allow(deprecated)] impl DeserializationApiKind for LegacyDeserializationApi {} /// `Session` manages connections to the cluster and allows to perform queries @@ -194,6 +202,11 @@ where } pub type Session = GenericSession; +#[allow(deprecated)] +#[deprecated( + since = "0.15.0", + note = "Legacy deserialization API is inefficient and is going to be removed soon" +)] pub type LegacySession = GenericSession; /// This implementation deliberately omits some details from Cluster in order @@ -861,6 +874,11 @@ impl GenericSession { /// the new API but you still have some modules left that use the old one, /// you can use this method to create an instance that supports the old API /// and pass it to the module that you intend to migrate later. + #[deprecated( + since = "0.15.0", + note = "Legacy deserialization API is inefficient and is going to be removed soon" + )] + #[allow(deprecated)] pub fn make_shared_session_with_legacy_api(&self) -> LegacySession { LegacySession { cluster: self.cluster.clone(), @@ -880,6 +898,11 @@ impl GenericSession { } } +#[deprecated( + since = "0.15.0", + note = "Legacy deserialization API is inefficient and is going to be removed soon" +)] +#[allow(deprecated)] impl GenericSession { pub async fn query_unpaged( &self, diff --git a/scylla/src/transport/session_builder.rs b/scylla/src/transport/session_builder.rs index 31b653a5c..404e27733 100644 --- a/scylla/src/transport/session_builder.rs +++ b/scylla/src/transport/session_builder.rs @@ -2,6 +2,7 @@ use super::connection::SelfIdentity; use super::execution_profile::ExecutionProfileHandle; +#[allow(deprecated)] use super::session::{ AddressTranslator, CurrentDeserializationApi, GenericSession, LegacyDeserializationApi, SessionConfig, @@ -541,6 +542,11 @@ impl GenericSessionBuilder { /// # Ok(()) /// # } /// ``` + #[deprecated( + since = "0.15.0", + note = "Legacy deserialization API is inefficient and is going to be removed soon" + )] + #[allow(deprecated)] pub async fn build_legacy( &self, ) -> Result, NewSessionError> { diff --git a/scylla/src/transport/session_test.rs b/scylla/src/transport/session_test.rs index 923cbd621..3e3552c23 100644 --- a/scylla/src/transport/session_test.rs +++ b/scylla/src/transport/session_test.rs @@ -3125,6 +3125,7 @@ async fn test_deserialize_empty_collections() { } #[tokio::test] +#[allow(deprecated)] async fn test_api_migration_session_sharing() { { let session = create_new_session_builder().build().await.unwrap(); diff --git a/scylla/tests/integration/hygiene.rs b/scylla/tests/integration/hygiene.rs index 622acf13d..042805243 100644 --- a/scylla/tests/integration/hygiene.rs +++ b/scylla/tests/integration/hygiene.rs @@ -1,4 +1,5 @@ #![no_implicit_prelude] +#![allow(deprecated)] // Macro that is given a crate name and tests it for hygiene macro_rules! test_crate { From b7d1dfeb0116a4c7562d83550d88b207a51a0963 Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Thu, 30 Mar 2023 08:43:12 +0200 Subject: [PATCH 4/4] docs: add migration guide MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Adds a document that should help adjust users to the new deserialization API. Co-authored-by: Wojciech Przytuła --- docs/source/SUMMARY.md | 1 + .../migration-guides/0.15-deserialization.md | 295 ++++++++++++++++++ .../migration-guides/migration-guides.md | 4 +- 3 files changed, 299 insertions(+), 1 deletion(-) create mode 100644 docs/source/migration-guides/0.15-deserialization.md diff --git a/docs/source/SUMMARY.md b/docs/source/SUMMARY.md index 63639364f..c5f65dc92 100644 --- a/docs/source/SUMMARY.md +++ b/docs/source/SUMMARY.md @@ -9,6 +9,7 @@ - [Migration guides](migration-guides/migration-guides.md) - [Adjusting code to changes in serialization API introduced in 0.11](migration-guides/0.11-serialization.md) + - [Adjusting code to changes in deserialization API introduced in 0.15](migration-guides/0.15-deserialization.md) - [Connecting to the cluster](connecting/connecting.md) - [Compression](connecting/compression.md) diff --git a/docs/source/migration-guides/0.15-deserialization.md b/docs/source/migration-guides/0.15-deserialization.md new file mode 100644 index 000000000..3c9d11478 --- /dev/null +++ b/docs/source/migration-guides/0.15-deserialization.md @@ -0,0 +1,295 @@ +# Adjusting code to changes in deserialization API introduced in 0.15 + +In 0.15, a new deserialization API has been introduced. The new API improves type safety and performance of the old one, so it is highly recommended to switch to it. However, deserialization is an area of the API that users frequently interact with: deserialization traits appear in generic code and custom implementations have been written. In order to make migration easier, the driver still offers the old API, which - while opt-in - can be very easily switched to after version upgrade. Furthermore, a number of facilities have been introduced which help migrate the user code to the new API piece-by-piece. + +The old API and migration facilities will be removed in a future major release. + +## Introduction + +### Old traits + +The legacy API works by deserializing rows in the query response to a sequence of `Row`s. The `Row` is just a `Vec>`, where `CqlValue` is an enum that is able to represent any CQL value. + +The user can request this type-erased representation to be converted into something useful. There are two traits that power this: + +__`FromRow`__ + +```rust +# extern crate scylla; +# use scylla::frame::response::cql_to_rust::FromRowError; +# use scylla::frame::response::result::Row; +pub trait FromRow: Sized { + fn from_row(row: Row) -> Result; +} +``` + +__`FromCqlVal`__ + +```rust +# extern crate scylla; +# use scylla::frame::response::cql_to_rust::FromCqlValError; +// The `T` parameter is supposed to be either `CqlValue` or `Option` +pub trait FromCqlVal: Sized { + fn from_cql(cql_val: T) -> Result; +} +``` + +These traits are implemented for some common types: + +- `FromRow` is implemented for tuples up to 16 elements, +- `FromCqlVal` is implemented for a bunch of types, and each CQL type can be converted to one of them. + +While it's possible to implement those manually, the driver provides procedural macros for automatic derivation in some cases: + +- `FromRow` - implements `FromRow` for a struct. +- `FromUserType` - generated an implementation of `FromCqlVal` for the struct, trying to parse the CQL value as a UDT. + +Note: the macros above have a default behavior that is different than what `FromRow` and `FromUserType` do. + +### New traits + +The new API introduce two analogous traits that, instead of consuming pre-parsed `Vec>`, are given raw, serialized data with full information about its type. This leads to better performance and allows for better type safety. + +The new traits are: + +__`DeserializeRow<'frame, 'metadata>`__ + +```rust +# extern crate scylla; +# use scylla::deserialize::row::ColumnIterator; +# use scylla::deserialize::{DeserializationError, TypeCheckError}; +# use scylla::frame::response::result::ColumnSpec; +pub trait DeserializeRow<'frame, 'metadata> +where + Self: Sized, +{ + fn type_check(specs: &[ColumnSpec]) -> Result<(), TypeCheckError>; + fn deserialize(row: ColumnIterator<'frame, 'metadata>) -> Result; +} +``` + +__`DeserializeValue<'frame, 'metadata>`__ + +```rust +# extern crate scylla; +# use scylla::deserialize::row::ColumnIterator; +# use scylla::deserialize::FrameSlice; +# use scylla::deserialize::{DeserializationError, TypeCheckError}; +# use scylla::frame::response::result::ColumnType; +pub trait DeserializeValue<'frame, 'metadata> +where + Self: Sized, +{ + fn type_check(typ: &ColumnType) -> Result<(), TypeCheckError>; + fn deserialize( + typ: &'metadata ColumnType<'metadata>, + v: Option>, + ) -> Result; +} +``` + +The above traits have been implemented for the same set of types as `FromRow` and `FromCqlVal`, respectively. Notably, `DeserializeRow` is implemented for `Row`, and `DeserializeValue` is implemented for `CqlValue`. + +There are also `DeserializeRow` and `DeserializeValue` derive macros, analogous to `FromRow` and `FromUserType`, respectively - but with slightly different defaults (explained later in this doc page). + +## Updating the code to use the new API + +Some of the core types have been updated to use the new traits. Updating the code to use the new API should be straightforward. + +### Basic queries + +Sending queries with the single page API should work similarly as before. The `Session::query_{unpaged,single_page}`, `Session::execute_{unpaged,single_page}` and `Session::batch` functions have the same interface as before, the only exception being that they return a new, updated `QueryResult`. + +Consuming rows from a result will require only minimal changes if you are using helper methods of the `QueryResult`. Now, there is no distinction between "typed" and "non-typed" methods; all methods that return rows need to have the type specified. For example, previously there used to be both `rows(self)` and `rows_typed(self)`, now there is only a single `rows>(&self)`. Another thing worth mentioning is that the returned iterator now _borrows_ from the `QueryResult` instead of consuming it. + +Note that the `QueryResult::rows` field is not available anymore. If you used to access it directly, you need to change your code to use the helper methods instead. + +Before: + +```rust +# extern crate scylla; +# use scylla::LegacySession; +# use std::error::Error; +# async fn check_only_compiles(session: &LegacySession) -> Result<(), Box> { +let iter = session + .query_unpaged("SELECT name, age FROM my_keyspace.people", &[]) + .await? + .rows_typed::<(String, i32)>()?; +for row in iter { + let (name, age) = row?; + println!("{} has age {}", name, age); +} +# Ok(()) +# } +``` + +After: + +```rust +# extern crate scylla; +# use scylla::Session; +# use std::error::Error; +# async fn check_only_compiles(session: &Session) -> Result<(), Box> { +// 1. Note that the result must be converted to a rows result, and only then +// an iterator created. +let result = session + .query_unpaged("SELECT name, age FROM my_keyspace.people", &[]) + .await? + .into_rows_result()?; + +// 2. Note that `rows` is used here, not `rows_typed`. +// 3. Note that the new deserialization framework support deserializing types +// that borrow directly from the result frame; let's use them to avoid +// needless allocations. +for row in result.rows::<(&str, i32)>()? { + let (name, age) = row?; + println!("{} has age {}", name, age); +} +# Ok(()) +# } +``` + +### Iterator queries + +The `Session::query_iter` and `Session::execute_iter` have been adjusted, too. They now return a `QueryPager` - an intermediate object which needs to be converted into `TypedRowStream` first before being actually iterated over. + +Before: + +```rust +# extern crate scylla; +# extern crate futures; +# use scylla::LegacySession; +# use std::error::Error; +# use scylla::IntoTypedRows; +# use futures::stream::StreamExt; +# async fn check_only_compiles(session: &LegacySession) -> Result<(), Box> { +let mut rows_stream = session + .query_iter("SELECT name, age FROM my_keyspace.people", &[]) + .await? + .into_typed::<(String, i32)>(); + +while let Some(next_row_res) = rows_stream.next().await { + let (a, b): (String, i32) = next_row_res?; + println!("a, b: {}, {}", a, b); +} +# Ok(()) +# } +``` + +After: + +```rust +# extern crate scylla; +# extern crate futures; +# use scylla::Session; +# use std::error::Error; +# use futures::stream::StreamExt; +# async fn check_only_compiles(session: &Session) -> Result<(), Box> { +let mut rows_stream = session + .query_iter("SELECT name, age FROM my_keyspace.people", &[]) + .await? + // The type of the TypedRowStream is inferred from further use of it. + // Alternatively, it can be specified using turbofish syntax: + // .rows_stream::<(String, i32)>()?; + .rows_stream()?; + +while let Some(next_row_res) = rows_stream.next().await { + let (a, b): (String, i32) = next_row_res?; + println!("a, b: {}, {}", a, b); +} +# Ok(()) +# } +``` + +Currently, `QueryPager`/`TypedRowStream` do not support deserialization of borrowed types due to limitations of Rust with regard to lending streams. If you want to deserialize borrowed types not to incur additional allocations, use manual paging (`{query/execute}_single_page`) API. + +### Procedural macros + +As mentioned in the Introduction section, the driver provides new procedural macros for the `DeserializeRow` and `DeserializeValue` traits that are meant to replace `FromRow` and `FromUserType`, respectively. The new macros are designed to be slightly more type-safe by matching column/UDT field names to rust field names dynamically. This is a different behavior to what the old macros used to do, but the new macros can be configured with `#[attributes]` to simulate the old behavior. + +__`FromRow` vs. `DeserializeRow`__ + +The impl generated by `FromRow` expects columns to be in the same order as the struct fields. The `FromRow` trait does not have information about column names, so it cannot match them with the struct field names. You can use `enforce_order` and `skip_name_checks` attributes to achieve such behavior via `DeserializeRow` trait. + +__`FromUserType` vs. `DeserializeValue`__ + +The impl generated by `FromUserType` expects UDT fields to be in the same order as the struct fields. Field names should be the same both in the UDT and in the struct. You can use the `enforce_order` attribute to achieve such behavior via the `DeserializeValue` trait. + +### Adjusting custom impls of deserialization traits + +If you have a custom type with a hand-written `impl FromRow` or `impl FromCqlVal`, the best thing to do is to just write a new impl for `DeserializeRow` or `DeserializeValue` manually. Although it's technically possible to implement the new traits by using the existing implementation of the old ones, rolling out a new implementation will avoid performance problems related to the inefficient `CqlValue` representation. + +## Accessing the old API + +Most important types related to deserialization of the old API have been renamed and contain a `Legacy` prefix in their names: + +- `Session` -> `LegacySession` +- `CachingSession` -> `LegacyCachingSession` +- `RowIterator` -> `LegacyRowIterator` +- `TypedRowIterator` -> `LegacyTypedRowIterator` +- `QueryResult` -> `LegacyQueryResult` + +If you intend to quickly migrate your application by using the old API, you can just import the legacy stuff and alias it as the new one, e.g.: + +```rust +# extern crate scylla; +use scylla::LegacySession as Session; +``` + +In order to create the `LegacySession` instead of the new `Session`, you need to use `SessionBuilder`'s `build_legacy()` method instead of `build()`: + +```rust +# extern crate scylla; +# use scylla::{LegacySession, SessionBuilder}; +# use std::error::Error; +# async fn check_only_compiles() -> Result<(), Box> { +let session: LegacySession = SessionBuilder::new() + .known_node("127.0.0.1") + .build_legacy() + .await?; +# Ok(()) +# } +``` + +## Mixing the old and the new API + +It is possible to use different APIs in different parts of the program. The `Session` allows to create a `LegacySession` object that has the old API but shares all resources with the session that has the new API (and vice versa - you can create a new API session from the old API session). + +```rust +# extern crate scylla; +# use scylla::{LegacySession, Session}; +# use std::error::Error; +# async fn check_only_compiles(new_api_session: &Session) -> Result<(), Box> { +// All of the session objects below will use the same resources: connections, +// metadata, current keyspace, etc. +let old_api_session: LegacySession = new_api_session.make_shared_session_with_legacy_api(); +let another_new_api_session: Session = old_api_session.make_shared_session_with_new_api(); +# Ok(()) +# } +``` + +In addition to that, it is possible to convert a `QueryResult` to `LegacyQueryResult`: + +```rust +# extern crate scylla; +# use scylla::{QueryResult, LegacyQueryResult}; +# use std::error::Error; +# async fn check_only_compiles(result: QueryResult) -> Result<(), Box> { +let result: QueryResult = result; +let legacy_result: LegacyQueryResult = result.into_legacy_result()?; +# Ok(()) +# } +``` + +... and `QueryPager` into `LegacyRowIterator`: + +```rust +# extern crate scylla; +# use scylla::transport::iterator::{QueryPager, LegacyRowIterator}; +# use std::error::Error; +# async fn check_only_compiles(pager: QueryPager) -> Result<(), Box> { +let pager: QueryPager = pager; +let legacy_result: LegacyRowIterator = pager.into_legacy(); +# Ok(()) +# } +``` diff --git a/docs/source/migration-guides/migration-guides.md b/docs/source/migration-guides/migration-guides.md index 86887a64c..d18b94659 100644 --- a/docs/source/migration-guides/migration-guides.md +++ b/docs/source/migration-guides/migration-guides.md @@ -1,6 +1,7 @@ # Migration guides - [Serialization changes in version 0.11](0.11-serialization.md) +- [Deserialization changes in version 0.15](0.15-deserialization.md) ```{eval-rst} .. toctree:: @@ -8,4 +9,5 @@ :glob: 0.11-serialization -``` + 0.15-deserialization +``` \ No newline at end of file