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 3a351b434..769e501bd 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, @@ -605,6 +606,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 8b62c0c2b..fe3798897 100644 --- a/scylla/src/lib.rs +++ b/scylla/src/lib.rs @@ -269,13 +269,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 192ad6dd4..128a87cd5 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 160819f6c..23bf6875c 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; @@ -675,6 +676,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) @@ -1089,11 +1095,16 @@ where } 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, } @@ -1142,6 +1153,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, @@ -1190,4 +1206,5 @@ mod legacy { // LegacyTypedRowIterator can be moved freely for any RowT so it's Unpin impl Unpin for LegacyTypedRowIterator {} } +#[allow(deprecated)] pub use legacy::{LegacyRowIterator, LegacyTypedRowIterator, NextRowError}; diff --git a/scylla/src/transport/legacy_query_result.rs b/scylla/src/transport/legacy_query_result.rs index 46818a297..2c7fb1237 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; @@ -39,7 +41,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 52326ba32..544bd5795 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::LegacyQueryResult; /// A view over specification of a table in the database. @@ -243,6 +244,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 587e06658..a605e781a 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 @@ -865,6 +878,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(), @@ -884,6 +902,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 2e99087a7..0a10d4629 100644 --- a/scylla/src/transport/session_test.rs +++ b/scylla/src/transport/session_test.rs @@ -3152,6 +3152,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 {