From 4fbb3153d7768839b492a8572f85c978eb6360af Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Wojciech=20Przytu=C5=82a?= Date: Fri, 11 Oct 2024 09:25:45 +0200 Subject: [PATCH 01/42] result: reposition and comment some code This is a tiny refactor, which makes further commits easier on eyes. --- scylla-cql/src/frame/response/result.rs | 47 ++++++++++++++++--------- 1 file changed, 30 insertions(+), 17 deletions(-) diff --git a/scylla-cql/src/frame/response/result.rs b/scylla-cql/src/frame/response/result.rs index 71d91db9a..a7873bb34 100644 --- a/scylla-cql/src/frame/response/result.rs +++ b/scylla-cql/src/frame/response/result.rs @@ -526,23 +526,6 @@ pub struct ResultMetadata<'a> { } impl<'a> ResultMetadata<'a> { - #[inline] - pub fn mock_empty() -> Self { - Self { - col_count: 0, - col_specs: Vec::new(), - } - } - - #[inline] - #[doc(hidden)] - pub fn new_for_test(col_count: usize, col_specs: Vec>) -> Self { - Self { - col_count, - col_specs, - } - } - #[inline] pub fn col_count(&self) -> usize { self.col_count @@ -552,6 +535,16 @@ impl<'a> ResultMetadata<'a> { pub fn col_specs(&self) -> &[ColumnSpec<'a>] { &self.col_specs } + + // Preferred to implementing Default, because users shouldn't be encouraged to create + // empty ResultMetadata. + #[inline] + pub fn mock_empty() -> Self { + Self { + col_count: 0, + col_specs: Vec::new(), + } + } } #[derive(Debug, Copy, Clone)] @@ -1184,6 +1177,26 @@ pub fn deserialize( ) } +// This is not #[cfg(test)], because it is used by scylla crate. +// Unfortunately, this attribute does not apply recursively to +// children item. Therefore, every `pub` item here must use have +// the specifier, too. +#[doc(hidden)] +mod test_utils { + use super::*; + + impl<'a> ResultMetadata<'a> { + #[inline] + #[doc(hidden)] + pub fn new_for_test(col_count: usize, col_specs: Vec>) -> Self { + Self { + col_count, + col_specs, + } + } + } +} + #[cfg(test)] mod tests { use crate as scylla; From 3cf54f80a277d97118bff6eebc6b87237bc929dc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Wojciech=20Przytu=C5=82a?= Date: Tue, 22 Oct 2024 19:49:21 +0200 Subject: [PATCH 02/42] result: fix lifetimes in ColumnSpec When a lifetime is not given explicitly in a value returned from a method, the rules of lifetime elision make the value bounded by the lifetime of Self (the implied 's lifetime in `&'s self`). In case of `ResultMetadata::col_specs`, `ColumnSpec`'s lifetime parameter was unnecessarily bound to the lifetime of `ResultMetadata`. Also, the commit lifts the requirement of `ResultMetadata::new_for_test` for the `ColumnSpec` given as an argument to have `'static` lifetime. --- scylla-cql/src/frame/response/result.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/scylla-cql/src/frame/response/result.rs b/scylla-cql/src/frame/response/result.rs index a7873bb34..6c027b5a1 100644 --- a/scylla-cql/src/frame/response/result.rs +++ b/scylla-cql/src/frame/response/result.rs @@ -1188,7 +1188,7 @@ mod test_utils { impl<'a> ResultMetadata<'a> { #[inline] #[doc(hidden)] - pub fn new_for_test(col_count: usize, col_specs: Vec>) -> Self { + pub fn new_for_test(col_count: usize, col_specs: Vec>) -> Self { Self { col_count, col_specs, From 7e7769362d4e1db1fc8178826ff880a6c09e3e4a Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Wed, 15 Mar 2023 08:51:42 +0100 Subject: [PATCH 03/42] result: introduce RawRows & friends MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Introduces new structs for lazy deserialization of RESULT:Rows frames. Introduces `ResultMetadataHolder`, which is, unsurprisingly, a versatile holder for ResultMetadata. It allows 3 types of ownership: 1) borrowing it from somewhere, be it the RESULT:Rows frame or the cached metadata in PreparedStatement; 2) owning it after deserializing from RESULT:Rows; 3) sharing ownership of metadata cached in PreparedStatement. Introduces new structs representing the RESULT:Rows CQL frame body, in various phases of deserialization: - `RawMetadataAndRawRows` only deserializes (flags and) paging size in order to pass it directly to the user; keeps metadata and rows in a serialized, unparsed form; - `DeserializedMetadataAndRawRows` deserializes metadata, but keeps rows serialized; `DeserializedMetadataAndRawRows` is lifetime-generic and can be deserialized from `RawMetadataAndRawRows` to borrowed or owned form by corresponding methods on `RawMetadataAndRawRows`. `DeserializedMetadataAndRawRows` must abstract over two different ways of storing the frame: - shared ownership (Bytes), - borrowing (FrameSlice). The problem arises with the `rows_iter` method. - in case of `DeserializedMetadataAndRawRows`, the struct itself owns the frame. Therefore, the reference to `self` should have the `'frame` lifetime (and this way bound the lifetime of deserialized items). - in case of `DeserializedMetadataAndRawRows`, the struct borrows the frame with some lifetime 'frame. Therefore, the reference to `self` should only have the `'metadata` lifetime, as the frame is owned independently of Self's lifetime. This discrepancy is not expressible by enums. Therefore, an entirely separate `rows_iter` must be defined for both cases, and thus both cases must be separate types - and this is guaranteed by having a different type parameter (because they are distinct instantiations of a generic type). To restrict the type parameter of `DeserializedMetadataAndRawRows` to the two valid variants (borrowed and owned), a trait `RawRowsKind` is introduced. Credits to Karol Baryła for replacing a macro (the first approach) with a generic function. Co-authored-by: Wojciech Przytuła Co-authored-by: Karol Baryła --- scylla-cql/src/frame/response/result.rs | 343 +++++++++++++++++++++++- 1 file changed, 339 insertions(+), 4 deletions(-) diff --git a/scylla-cql/src/frame/response/result.rs b/scylla-cql/src/frame/response/result.rs index 6c027b5a1..c4d482284 100644 --- a/scylla-cql/src/frame/response/result.rs +++ b/scylla-cql/src/frame/response/result.rs @@ -11,12 +11,15 @@ use crate::frame::value::{ Counter, CqlDate, CqlDecimal, CqlDuration, CqlTime, CqlTimestamp, CqlTimeuuid, CqlVarint, }; use crate::types::deserialize::result::{RowIterator, TypedRowIterator}; +use crate::types::deserialize::row::DeserializeRow; use crate::types::deserialize::value::{ mk_deser_err, BuiltinDeserializationErrorKind, DeserializeValue, MapIterator, UdtIterator, }; -use crate::types::deserialize::{DeserializationError, FrameSlice}; +use crate::types::deserialize::{DeserializationError, FrameSlice, TypeCheckError}; use bytes::{Buf, Bytes}; use std::borrow::Cow; +use std::fmt::Debug; +use std::ops::Deref; use std::sync::Arc; use std::{net::IpAddr, result::Result as StdResult, str}; use uuid::Uuid; @@ -547,6 +550,31 @@ impl<'a> ResultMetadata<'a> { } } +/// Versatile holder for ResultMetadata. Allows 3 types of ownership +/// of ResultMetadata: +/// 1) borrowing it from somewhere, be it the RESULT:Rows frame +/// or the cached metadata in PreparedStatement; +/// 2) owning it after deserializing from RESULT:Rows; +/// 3) sharing ownership of metadata cached in PreparedStatement. +#[derive(Debug, Clone)] +pub enum ResultMetadataHolder<'frame> { + Owned(ResultMetadata<'frame>), + Borrowed(&'frame ResultMetadata<'frame>), + SharedCached(Arc>), +} + +impl<'frame> Deref for ResultMetadataHolder<'frame> { + type Target = ResultMetadata<'frame>; + + fn deref(&self) -> &Self::Target { + match self { + ResultMetadataHolder::Owned(metadata) => metadata, + ResultMetadataHolder::Borrowed(metadata) => metadata, + ResultMetadataHolder::SharedCached(arc) => arc.deref(), + } + } +} + #[derive(Debug, Copy, Clone)] pub struct PartitionKeyIndex { /// index in the serialized values @@ -577,6 +605,172 @@ impl Row { } } +/// RESULT:Rows response, in partially serialized form. +/// +/// Flags and paging state are deserialized, remaining part of metadata +/// as well as rows remain serialized. +#[derive(Debug)] +pub struct RawMetadataAndRawRows { + // Already deserialized part of metadata: + col_count: usize, + global_tables_spec: bool, + no_metadata: bool, + + /// The remaining part of the RESULT frame. + raw_metadata_and_rows: Bytes, + + /// Metadata cached in PreparedStatement, if present. + cached_metadata: Option>>, +} + +/// RESULT:Rows response, in partially serialized form. +/// +/// Paging state and metadata are deserialized, rows remain serialized. +/// +/// See [`RawRowsKind`] for explanation what it is and why it is needed. +#[derive(Debug)] +pub struct DeserializedMetadataAndRawRows<'frame, RawRowsRepr: RawRowsKind> { + metadata: ResultMetadataHolder<'frame>, + rows_count: usize, + raw_rows: RawRowsRepr, +} + +mod sealed { + // This is a sealed trait - its whole purpose is to be unnameable. + // This means we need to disable the check. + #[allow(unknown_lints)] // Rust 1.70 (our MSRV) doesn't know this lint. + #[allow(unnameable_types)] + pub trait Sealed {} +} + +/// This abstracts over two different ways of storing the frame: +/// - shared ownership (Bytes), +/// - borrowing (FrameSlice). +/// +/// Its whole purpose is to restrict the type parameter of `DeserializedMetadataAndRawRows` +/// to the two valid variants. +/// +/// ### Why is a trait used, and not an enum, as it's done for `ResultMetadataHolder`? +/// +/// The problem arises with the `rows_iter` method. +/// - in case of `DeserializedMetadataAndRawRows`, the struct itself +/// owns the frame. Therefore, the reference to `self` should have the `'frame` +/// lifetime (and this way bound the lifetime of deserialized items). +/// - in case of `DeserializedMetadataAndRawRows`, the struct +/// borrows the frame with some lifetime 'frame. Therefore, the reference to +/// `self` should only have the `'metadata` lifetime, as the frame is owned +/// independently of Self's lifetime. +/// +/// This discrepancy is not expressible by enums. Therefore, an entirely separate +/// `rows_iter` must be defined for both cases, and thus both cases must be separate +/// types - and this is guaranteed by having a different type parameter (because they +/// are distinct instantiations of a generic type). +pub trait RawRowsKind: sealed::Sealed + Debug { + fn as_slice(&self) -> &[u8]; +} +#[derive(Debug)] +pub struct RawRowsOwned(Bytes); +impl sealed::Sealed for RawRowsOwned {} +impl RawRowsKind for RawRowsOwned { + fn as_slice(&self) -> &[u8] { + &self.0 + } +} +#[derive(Debug)] +pub struct RawRowsBorrowed<'frame>(FrameSlice<'frame>); +impl<'frame> sealed::Sealed for RawRowsBorrowed<'frame> {} +impl<'frame> RawRowsKind for RawRowsBorrowed<'frame> { + fn as_slice(&self) -> &'frame [u8] { + self.0.as_slice() + } +} + +impl<'frame, RawRowsRepr: RawRowsKind> DeserializedMetadataAndRawRows<'frame, RawRowsRepr> { + /// Returns the metadata associated with this response + /// (table and column specifications). + #[inline] + pub fn metadata(&self) -> &ResultMetadata<'frame> { + self.metadata.deref() + } + + /// Consumes the `DeserializedMetadataAndRawRows` and returns metadata + /// associated with the response (or cached metadata, if used in its stead). + #[inline] + pub fn into_metadata(self) -> ResultMetadataHolder<'frame> { + self.metadata + } + + /// Returns the number of rows that the RESULT:Rows contain. + #[inline] + pub fn rows_count(&self) -> usize { + self.rows_count + } + + /// Returns the serialized size of the raw rows. + #[inline] + pub fn rows_bytes_size(&self) -> usize { + self.raw_rows.as_slice().len() + } +} + +impl DeserializedMetadataAndRawRows<'static, RawRowsOwned> { + // Preferred to implementing Default, because users shouldn't be encouraged to create + // empty DeserializedMetadataAndRawRows. + #[inline] + pub fn mock_empty() -> Self { + Self { + metadata: ResultMetadataHolder::Owned(ResultMetadata::mock_empty()), + rows_count: 0, + raw_rows: RawRowsOwned(Bytes::new()), + } + } + + pub(crate) fn into_inner(self) -> (ResultMetadataHolder<'static>, usize, Bytes) { + (self.metadata, self.rows_count, self.raw_rows.0) + } + + /// Creates a typed iterator over the rows that lazily deserializes + /// rows in the result. + /// + /// Returns Err if the schema of returned result doesn't match R. + #[inline] + pub fn rows_iter<'frame, 'metadata, R: DeserializeRow<'frame, 'metadata>>( + &'frame self, + ) -> StdResult, TypeCheckError> + where + 'frame: 'metadata, + { + let frame_slice = FrameSlice::new(&self.raw_rows.0); + let raw = RowIterator::new(self.rows_count, self.metadata.col_specs(), frame_slice); + TypedRowIterator::new(raw) + } +} + +impl<'frame> DeserializedMetadataAndRawRows<'frame, RawRowsBorrowed<'frame>> { + // Preferred to implementing Default, because users shouldn't be encouraged to create + // empty DeserializedMetadataAndRawRows. + #[inline] + pub fn mock_empty() -> Self { + Self { + metadata: ResultMetadataHolder::Owned(ResultMetadata::mock_empty()), + rows_count: 0, + raw_rows: RawRowsBorrowed(FrameSlice::new_empty()), + } + } + + /// Creates a typed iterator over the rows that lazily deserializes + /// rows in the result. + /// + /// Returns Err if the schema of returned result doesn't match R. + #[inline] + pub fn rows_iter<'metadata, R: DeserializeRow<'frame, 'metadata>>( + &'metadata self, + ) -> StdResult, TypeCheckError> { + let raw = RowIterator::new(self.rows_count, self.metadata.col_specs(), self.raw_rows.0); + TypedRowIterator::new(raw) + } +} + #[derive(Debug)] pub struct Rows { pub metadata: Arc>, @@ -685,7 +879,7 @@ fn deser_type_generic<'frame, 'result, StrT: Into>>( }) } -fn _deser_type_borrowed<'frame>( +fn deser_type_borrowed<'frame>( buf: &mut &'frame [u8], ) -> StdResult, CqlTypeParseError> { deser_type_generic(buf, |buf| types::read_string(buf)) @@ -809,7 +1003,7 @@ fn deser_col_specs_generic<'frame, 'result>( Ok(col_specs) } -fn _deser_col_specs_borrowed<'frame>( +fn deser_col_specs_borrowed<'frame>( buf: &mut &'frame [u8], global_table_spec: Option>, col_count: usize, @@ -819,7 +1013,7 @@ fn _deser_col_specs_borrowed<'frame>( global_table_spec, col_count, ColumnSpec::borrowed, - _deser_type_borrowed, + deser_type_borrowed, ) } @@ -876,6 +1070,147 @@ fn deser_result_metadata( Ok((metadata, paging_state)) } +impl RawMetadataAndRawRows { + /// Deserializes flags and paging state; the other part of result metadata + /// as well as rows remain serialized. + fn deserialize( + frame: &mut FrameSlice, + cached_metadata: Option>>, + ) -> StdResult<(Self, PagingStateResponse), RowsParseError> { + let flags = types::read_int(frame.as_slice_mut()) + .map_err(|err| ResultMetadataParseError::FlagsParseError(err.into()))?; + let global_tables_spec = flags & 0x0001 != 0; + let has_more_pages = flags & 0x0002 != 0; + let no_metadata = flags & 0x0004 != 0; + + let col_count = types::read_int_length(frame.as_slice_mut()) + .map_err(ResultMetadataParseError::ColumnCountParseError)?; + + let raw_paging_state = has_more_pages + .then(|| { + types::read_bytes(frame.as_slice_mut()) + .map_err(ResultMetadataParseError::PagingStateParseError) + }) + .transpose()?; + + let paging_state = PagingStateResponse::new_from_raw_bytes(raw_paging_state); + + let raw_rows = Self { + col_count, + global_tables_spec, + no_metadata, + raw_metadata_and_rows: frame.to_bytes(), + cached_metadata, + }; + + Ok((raw_rows, paging_state)) + } +} + +type DeserColSpecsFn<'this, 'result> = + fn( + &mut &'this [u8], + Option>, + usize, + ) -> StdResult>, ColumnSpecParseError>; + +impl RawMetadataAndRawRows { + /// Deserializes ResultMetadata and deserializes rows count. Keeps rows in the serialized form. + /// + /// If metadata is cached (in the PreparedStatement), it is reused (shared) from cache + /// instead of deserializing. + fn deserialize_metadata_generic<'this, 'result, T: RawRowsKind>( + &'this self, + raw_rows_constructor: fn(FrameSlice<'this>) -> T, + use_cached_metadata: fn( + &'this Arc>, + ) -> ResultMetadataHolder<'result>, + deser_col_specs: DeserColSpecsFn<'this, 'result>, + ) -> StdResult, RowsParseError> { + let mut frame_slice = FrameSlice::new(&self.raw_metadata_and_rows); + + let metadata = match self.cached_metadata.as_ref() { + Some(cached) if self.no_metadata => { + // Server sent no metadata, but we have metadata cached. This means that we asked the server + // not to send metadata in the response as an optimization. We use cached metadata instead. + use_cached_metadata(cached) + } + None if self.no_metadata => { + // Server sent no metadata and we have no metadata cached. Having no metadata cached, + // we wouldn't have asked the server for skipping metadata. Therefore, this is most probably + // not a SELECT, because in such case the server would send empty metadata both in Prepared + // and in Result responses. + ResultMetadataHolder::Owned(ResultMetadata::mock_empty()) + } + Some(_) | None => { + // Two possibilities: + // 1) no cached_metadata provided. Server is supposed to provide the result metadata. + // 2) cached metadata present (so we should have asked for skipping metadata), + // but the server sent result metadata anyway. + // In case 1 we have to deserialize result metadata. In case 2 we choose to do that, + // too, because it's suspicious, so we had better use the new metadata just in case. + // Also, we simply need to advance the buffer pointer past metadata, and this requires + // parsing metadata. + let server_metadata = { + let global_table_spec = self + .global_tables_spec + .then(|| deser_table_spec(frame_slice.as_slice_mut())) + .transpose() + .map_err(ResultMetadataParseError::from)?; + + let col_specs = deser_col_specs( + frame_slice.as_slice_mut(), + global_table_spec, + self.col_count, + ) + .map_err(ResultMetadataParseError::from)?; + + ResultMetadata { + col_count: self.col_count, + col_specs, + } + }; + if server_metadata.col_count() != server_metadata.col_specs().len() { + return Err(RowsParseError::ColumnCountMismatch { + col_count: server_metadata.col_count(), + col_specs_count: server_metadata.col_specs().len(), + }); + } + ResultMetadataHolder::Owned(server_metadata) + } + }; + + let rows_count: usize = types::read_int_length(frame_slice.as_slice_mut()) + .map_err(RowsParseError::RowsCountParseError)?; + + Ok(DeserializedMetadataAndRawRows { + metadata, + rows_count, + raw_rows: raw_rows_constructor(frame_slice), + }) + } + + pub fn deserialize_borrowed_metadata( + &self, + ) -> StdResult>, RowsParseError> { + self.deserialize_metadata_generic( + RawRowsBorrowed, + |cached: &Arc| ResultMetadataHolder::Borrowed(cached), + deser_col_specs_borrowed, + ) + } + + pub fn deserialize_owned_metadata( + &self, + ) -> StdResult, RowsParseError> { + self.deserialize_metadata_generic( + |frame_slice| RawRowsOwned(frame_slice.to_bytes()), + |cached: &Arc| ResultMetadataHolder::SharedCached(Arc::clone(cached)), + deser_col_specs_owned, + ) + } +} + fn deser_prepared_metadata( buf: &mut &[u8], ) -> StdResult { From 5a4973049f9abaac63658f37f4476d9d38ce11ae Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Wojciech=20Przytu=C5=82a?= Date: Wed, 14 Aug 2024 08:50:35 +0200 Subject: [PATCH 04/42] result: introduce RawRowsLendingIterator The iterator is analogous to RowIterator, but instead of borrowing from external frame bytes and metadata with 'frame lifetime, it owns them and lends them from itself. Thus, it cannot implement Iterator trait. It does not, however, prevent us from exposing a `next()` method on it. The iterator is going to be used in new iterator API for queries (i.e., the one connected to `{query,execute}_iter`), where borrowing is not suitable (or even possible) due to the design of that API. Tests of RowIterator are shared with the new RawRowsLendingIterator, by introducing a new LendingIterator trait using GATs. Due to a bug/limitation in the compiler, 'static lifetimes are needed in tests. I'd like to use the recently stabilised (1.80) std::sync::LazyLock, but our MSRV is too old. Instead, I've employed lazy_static. --- Cargo.lock.msrv | 1 + scylla-cql/Cargo.toml | 1 + scylla-cql/src/frame/response/result.rs | 16 ++ scylla-cql/src/types/deserialize/result.rs | 236 ++++++++++++++++++--- 4 files changed, 225 insertions(+), 29 deletions(-) diff --git a/Cargo.lock.msrv b/Cargo.lock.msrv index c464aa689..f46f392c1 100644 --- a/Cargo.lock.msrv +++ b/Cargo.lock.msrv @@ -1531,6 +1531,7 @@ dependencies = [ "bytes", "chrono", "criterion", + "lazy_static", "lz4_flex", "num-bigint 0.3.3", "num-bigint 0.4.4", diff --git a/scylla-cql/Cargo.toml b/scylla-cql/Cargo.toml index 499b1ddd4..1b5ea06cc 100644 --- a/scylla-cql/Cargo.toml +++ b/scylla-cql/Cargo.toml @@ -31,6 +31,7 @@ time-03 = { package = "time", version = "0.3", optional = true } [dev-dependencies] assert_matches = "1.5.0" criterion = "0.4" # Note: v0.5 needs at least rust 1.70.0 +lazy_static = "1" # We can migrate to std::sync::LazyLock once MSRV is bumped to 1.80. # Use large-dates feature to test potential edge cases time-03 = { package = "time", version = "0.3.21", features = ["large-dates"] } uuid = { version = "1.0", features = ["v4"] } diff --git a/scylla-cql/src/frame/response/result.rs b/scylla-cql/src/frame/response/result.rs index c4d482284..aca1cf73d 100644 --- a/scylla-cql/src/frame/response/result.rs +++ b/scylla-cql/src/frame/response/result.rs @@ -1530,6 +1530,22 @@ mod test_utils { } } } + + impl<'frame> DeserializedMetadataAndRawRows<'frame, RawRowsOwned> { + #[inline] + #[doc(hidden)] + pub fn new_for_test( + metadata: ResultMetadata<'frame>, + rows_count: usize, + raw_rows: Bytes, + ) -> Self { + Self { + metadata: ResultMetadataHolder::Owned(metadata), + rows_count, + raw_rows: RawRowsOwned(raw_rows), + } + } + } } #[cfg(test)] diff --git a/scylla-cql/src/types/deserialize/result.rs b/scylla-cql/src/types/deserialize/result.rs index c8c746936..04b3af6c1 100644 --- a/scylla-cql/src/types/deserialize/result.rs +++ b/scylla-cql/src/types/deserialize/result.rs @@ -1,4 +1,8 @@ -use crate::frame::response::result::ColumnSpec; +use bytes::Bytes; + +use crate::frame::response::result::{ + ColumnSpec, DeserializedMetadataAndRawRows, RawRowsOwned, ResultMetadata, ResultMetadataHolder, +}; use super::row::{mk_deser_err, BuiltinDeserializationErrorKind, ColumnIterator, DeserializeRow}; use super::{DeserializationError, FrameSlice, TypeCheckError}; @@ -136,46 +140,220 @@ where } } +// Technically not an iterator because it returns items that borrow from it, +// and the std Iterator interface does not allow for that. +/// A _lending_ iterator over serialized rows. +/// +/// This type is similar to `RowIterator`, but keeps ownership of the serialized +/// result. Because it returns `ColumnIterator`s that need to borrow from it, +/// it does not implement the `Iterator` trait (there is no type in the standard +/// library to represent this concept yet). +#[derive(Debug)] +pub struct RawRowsLendingIterator { + metadata: ResultMetadataHolder<'static>, + remaining: usize, + at: usize, + raw_rows: Bytes, +} + +impl RawRowsLendingIterator { + /// Creates a new `RawRowsLendingIterator`, consuming given `RawRows`. + #[inline] + pub fn new(raw_rows: DeserializedMetadataAndRawRows<'static, RawRowsOwned>) -> Self { + let (metadata, rows_count, raw_rows) = raw_rows.into_inner(); + Self { + metadata, + remaining: rows_count, + at: 0, + raw_rows, + } + } + + /// Returns a `ColumnIterator` that represents the next row. + /// + /// Note: the `ColumnIterator` borrows from the `RawRowsLendingIterator`. + /// The column iterator must be consumed before the rows iterator can + /// continue. + #[inline] + #[allow(clippy::should_implement_trait)] // https://github.com/rust-lang/rust-clippy/issues/5004 + pub fn next(&mut self) -> Option> { + self.remaining = self.remaining.checked_sub(1)?; + + // First create the slice encompassing the whole frame. + let mut remaining_frame = FrameSlice::new(&self.raw_rows); + // Then slice it to encompass the remaining suffix of the frame. + *remaining_frame.as_slice_mut() = &remaining_frame.as_slice()[self.at..]; + // Ideally, we would prefer to preserve the FrameSlice between calls to `next()`, + // but borrowing from oneself is impossible, so we have to recreate it this way. + + let iter = ColumnIterator::new(self.metadata.col_specs(), remaining_frame); + + // Skip the row here, manually + for (column_index, spec) in self.metadata.col_specs().iter().enumerate() { + let remaining_frame_len_before_column_read = remaining_frame.as_slice().len(); + if let Err(err) = remaining_frame.read_cql_bytes() { + return Some(Err(mk_deser_err::( + BuiltinDeserializationErrorKind::RawColumnDeserializationFailed { + column_index, + column_name: spec.name().to_owned(), + err: DeserializationError::new(err), + }, + ))); + } else { + let remaining_frame_len_after_column_read = remaining_frame.as_slice().len(); + self.at += + remaining_frame_len_before_column_read - remaining_frame_len_after_column_read; + } + } + + Some(Ok(iter)) + } + + #[inline] + pub fn size_hint(&self) -> (usize, Option) { + // next() is written in a way that it does not progress on error, so once an error + // is encountered, the same error keeps being returned until `self.remaining` + // elements are yielded in total. + (self.remaining, Some(self.remaining)) + } + + /// Returns the metadata associated with the response (paging state and + /// column specifications). + #[inline] + pub fn metadata(&self) -> &ResultMetadata<'static> { + &self.metadata + } + + /// Returns the remaining number of rows that this iterator is expected + /// to produce. + #[inline] + pub fn rows_remaining(&self) -> usize { + self.remaining + } +} + #[cfg(test)] mod tests { + use bytes::Bytes; + use std::ops::Deref; - use crate::frame::response::result::ColumnType; + use crate::frame::response::result::{ + ColumnSpec, ColumnType, DeserializedMetadataAndRawRows, ResultMetadata, + }; use super::super::tests::{serialize_cells, spec, CELL1, CELL2}; - use super::{FrameSlice, RowIterator, TypedRowIterator}; + use super::{ + ColumnIterator, DeserializationError, FrameSlice, RawRowsLendingIterator, RowIterator, + TypedRowIterator, + }; + + trait LendingIterator { + type Item<'borrow> + where + Self: 'borrow; + fn lend_next(&mut self) -> Option, DeserializationError>>; + } - #[test] - fn test_row_iterator_basic_parse() { - let raw_data = serialize_cells([Some(CELL1), Some(CELL2), Some(CELL2), Some(CELL1)]); - let specs = [spec("b1", ColumnType::Blob), spec("b2", ColumnType::Blob)]; - let mut iter = RowIterator::new(2, &specs, FrameSlice::new(&raw_data)); - - let mut row1 = iter.next().unwrap().unwrap(); - let c11 = row1.next().unwrap().unwrap(); - assert_eq!(c11.slice.unwrap().as_slice(), CELL1); - let c12 = row1.next().unwrap().unwrap(); - assert_eq!(c12.slice.unwrap().as_slice(), CELL2); - assert!(row1.next().is_none()); - - let mut row2 = iter.next().unwrap().unwrap(); - let c21 = row2.next().unwrap().unwrap(); - assert_eq!(c21.slice.unwrap().as_slice(), CELL2); - let c22 = row2.next().unwrap().unwrap(); - assert_eq!(c22.slice.unwrap().as_slice(), CELL1); - assert!(row2.next().is_none()); + impl<'frame, 'metadata> LendingIterator for RowIterator<'frame, 'metadata> { + type Item<'borrow> = ColumnIterator<'borrow, 'borrow> + where + Self: 'borrow; - assert!(iter.next().is_none()); + fn lend_next(&mut self) -> Option, DeserializationError>> { + self.next() + } + } + + impl LendingIterator for RawRowsLendingIterator { + type Item<'borrow> = ColumnIterator<'borrow, 'borrow>; + + fn lend_next(&mut self) -> Option, DeserializationError>> { + self.next() + } } #[test] - fn test_row_iterator_too_few_rows() { - let raw_data = serialize_cells([Some(CELL1), Some(CELL2)]); - let specs = [spec("b1", ColumnType::Blob), spec("b2", ColumnType::Blob)]; - let mut iter = RowIterator::new(2, &specs, FrameSlice::new(&raw_data)); + fn test_row_iterators_basic_parse() { + // Those statics are required because of a compiler bug-limitation about GATs: + // https://blog.rust-lang.org/2022/10/28/gats-stabilization.html#implied-static-requirement-from-higher-ranked-trait-bounds + // the following type higher-ranked lifetime constraint implies 'static lifetime. + // + // I: for<'item> LendingIterator = ColumnIterator<'item>>, + // + // The bug is said to be a lot of effort to fix, so in tests let's just use `lazy_static` + // to obtain 'static references. + // + // `std::sync::LazyLock` is stable since 1.80, so once we bump MSRV enough, + // we will be able to replace `lazy_static` with `LazyLock`. + + static SPECS: &[ColumnSpec<'static>] = + &[spec("b1", ColumnType::Blob), spec("b2", ColumnType::Blob)]; + lazy_static::lazy_static! { + static ref RAW_DATA: Bytes = serialize_cells([Some(CELL1), Some(CELL2), Some(CELL2), Some(CELL1)]); + } + let raw_data = RAW_DATA.deref(); + let specs = SPECS; + + let row_iter = RowIterator::new(2, specs, FrameSlice::new(raw_data)); + let lending_row_iter = + RawRowsLendingIterator::new(DeserializedMetadataAndRawRows::new_for_test( + ResultMetadata::new_for_test(specs.len(), specs.to_vec()), + 2, + raw_data.clone(), + )); + check(row_iter); + check(lending_row_iter); + + fn check(mut iter: I) + where + I: for<'item> LendingIterator = ColumnIterator<'item, 'item>>, + { + let mut row1 = iter.lend_next().unwrap().unwrap(); + let c11 = row1.next().unwrap().unwrap(); + assert_eq!(c11.slice.unwrap().as_slice(), CELL1); + let c12 = row1.next().unwrap().unwrap(); + assert_eq!(c12.slice.unwrap().as_slice(), CELL2); + assert!(row1.next().is_none()); + + let mut row2 = iter.lend_next().unwrap().unwrap(); + let c21 = row2.next().unwrap().unwrap(); + assert_eq!(c21.slice.unwrap().as_slice(), CELL2); + let c22 = row2.next().unwrap().unwrap(); + assert_eq!(c22.slice.unwrap().as_slice(), CELL1); + assert!(row2.next().is_none()); + + assert!(iter.lend_next().is_none()); + } + } - iter.next().unwrap().unwrap(); - assert!(iter.next().unwrap().is_err()); + #[test] + fn test_row_iterators_too_few_rows() { + static SPECS: &[ColumnSpec<'static>] = + &[spec("b1", ColumnType::Blob), spec("b2", ColumnType::Blob)]; + lazy_static::lazy_static! { + static ref RAW_DATA: Bytes = serialize_cells([Some(CELL1), Some(CELL2)]); + } + let raw_data = RAW_DATA.deref(); + let specs = SPECS; + + let row_iter = RowIterator::new(2, specs, FrameSlice::new(raw_data)); + let lending_row_iter = + RawRowsLendingIterator::new(DeserializedMetadataAndRawRows::new_for_test( + ResultMetadata::new_for_test(specs.len(), specs.to_vec()), + 2, + raw_data.clone(), + )); + check(row_iter); + check(lending_row_iter); + + fn check(mut iter: I) + where + I: for<'item> LendingIterator = ColumnIterator<'item, 'item>>, + { + iter.lend_next().unwrap().unwrap(); + iter.lend_next().unwrap().unwrap_err(); + } } #[test] From 5339d279a5f9368c59ea0f91524436f8f97ff5ed Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Tue, 14 Mar 2023 17:53:31 +0100 Subject: [PATCH 05/42] treewide: rename QueryResult -> LegacyQueryResult Soon, a new QueryResult will be introduced with a slightly different API. The old one will be preserved as LegacyQueryResult, in order to make migration easier. This commit renames the existing QueryResult to LegacyQueryResult, as well as the query_result module to legacy_query_result. The new QueryResult will be introduced in later commits. --- examples/cqlsh-rs.rs | 4 +- examples/tower.rs | 2 +- examples/tracing.rs | 8 +- scylla/src/lib.rs | 2 +- scylla/src/transport/caching_session.rs | 8 +- scylla/src/transport/connection.rs | 27 +++--- scylla/src/transport/errors.rs | 2 +- .../transport/large_batch_statements_test.rs | 8 +- ...query_result.rs => legacy_query_result.rs} | 90 +++++++++---------- scylla/src/transport/mod.rs | 2 +- scylla/src/transport/session.rs | 24 ++--- scylla/src/transport/session_test.rs | 37 +++++--- scylla/tests/integration/tablets.rs | 6 +- 13 files changed, 119 insertions(+), 101 deletions(-) rename scylla/src/transport/{query_result.rs => legacy_query_result.rs} (84%) diff --git a/examples/cqlsh-rs.rs b/examples/cqlsh-rs.rs index 8d9ca8ea6..c12b17a76 100644 --- a/examples/cqlsh-rs.rs +++ b/examples/cqlsh-rs.rs @@ -4,7 +4,7 @@ use rustyline::error::ReadlineError; use rustyline::{CompletionType, Config, Context, Editor}; use rustyline_derive::{Helper, Highlighter, Hinter, Validator}; use scylla::transport::Compression; -use scylla::{QueryResult, Session, SessionBuilder}; +use scylla::{LegacyQueryResult, Session, SessionBuilder}; use std::env; #[derive(Helper, Highlighter, Validator, Hinter)] @@ -173,7 +173,7 @@ impl Completer for CqlHelper { } } -fn print_result(result: &QueryResult) { +fn print_result(result: &LegacyQueryResult) { if result.rows.is_none() { println!("OK"); return; diff --git a/examples/tower.rs b/examples/tower.rs index b45b08ae1..0d28407da 100644 --- a/examples/tower.rs +++ b/examples/tower.rs @@ -12,7 +12,7 @@ struct SessionService { // A trivial service implementation for sending parameterless simple string requests to Scylla. impl Service for SessionService { - type Response = scylla::QueryResult; + type Response = scylla::LegacyQueryResult; type Error = scylla::transport::errors::QueryError; type Future = Pin>>>; diff --git a/examples/tracing.rs b/examples/tracing.rs index d742de7e5..a50e8f7e0 100644 --- a/examples/tracing.rs +++ b/examples/tracing.rs @@ -9,7 +9,7 @@ use scylla::statement::{ }; use scylla::tracing::TracingInfo; use scylla::transport::iterator::RowIterator; -use scylla::QueryResult; +use scylla::LegacyQueryResult; use scylla::{Session, SessionBuilder}; use std::env; use std::num::NonZeroU32; @@ -42,7 +42,7 @@ async fn main() -> Result<()> { query.set_serial_consistency(Some(SerialConsistency::LocalSerial)); // QueryResult will contain a tracing_id which can be used to query tracing information - let query_result: QueryResult = session.query_unpaged(query.clone(), &[]).await?; + let query_result: LegacyQueryResult = session.query_unpaged(query.clone(), &[]).await?; let query_tracing_id: Uuid = query_result .tracing_id .ok_or_else(|| anyhow!("Tracing id is None!"))?; @@ -79,7 +79,7 @@ async fn main() -> Result<()> { // To trace execution of a prepared statement tracing must be enabled for it prepared.set_tracing(true); - let execute_result: QueryResult = session.execute_unpaged(&prepared, &[]).await?; + let execute_result: LegacyQueryResult = session.execute_unpaged(&prepared, &[]).await?; println!("Execute tracing id: {:?}", execute_result.tracing_id); // PAGED QUERY_ITER EXECUTE_ITER @@ -105,7 +105,7 @@ async fn main() -> Result<()> { batch.set_tracing(true); // Run the batch and print its tracing_id - let batch_result: QueryResult = session.batch(&batch, ((),)).await?; + let batch_result: LegacyQueryResult = session.batch(&batch, ((),)).await?; println!("Batch tracing id: {:?}\n", batch_result.tracing_id); // CUSTOM diff --git a/scylla/src/lib.rs b/scylla/src/lib.rs index 92e256259..feea88438 100644 --- a/scylla/src/lib.rs +++ b/scylla/src/lib.rs @@ -258,7 +258,7 @@ pub use frame::response::cql_to_rust::FromRow; pub use transport::caching_session::CachingSession; pub use transport::execution_profile::ExecutionProfile; -pub use transport::query_result::QueryResult; +pub use transport::legacy_query_result::LegacyQueryResult; pub use transport::session::{IntoTypedRows, Session, SessionConfig}; pub use transport::session_builder::SessionBuilder; diff --git a/scylla/src/transport/caching_session.rs b/scylla/src/transport/caching_session.rs index e4bca85bd..b920337fe 100644 --- a/scylla/src/transport/caching_session.rs +++ b/scylla/src/transport/caching_session.rs @@ -5,7 +5,7 @@ use crate::statement::{PagingState, PagingStateResponse}; use crate::transport::errors::QueryError; use crate::transport::iterator::RowIterator; use crate::transport::partitioner::PartitionerName; -use crate::{QueryResult, Session}; +use crate::{LegacyQueryResult, Session}; use bytes::Bytes; use dashmap::DashMap; use futures::future::try_join_all; @@ -75,7 +75,7 @@ where &self, query: impl Into, values: impl SerializeRow, - ) -> Result { + ) -> Result { let query = query.into(); let prepared = self.add_prepared_statement_owned(query).await?; self.session.execute_unpaged(&prepared, values).await @@ -98,7 +98,7 @@ where query: impl Into, values: impl SerializeRow, paging_state: PagingState, - ) -> Result<(QueryResult, PagingStateResponse), QueryError> { + ) -> Result<(LegacyQueryResult, PagingStateResponse), QueryError> { let query = query.into(); let prepared = self.add_prepared_statement_owned(query).await?; self.session @@ -112,7 +112,7 @@ where &self, batch: &Batch, values: impl BatchValues, - ) -> Result { + ) -> Result { let all_prepared: bool = batch .statements .iter() diff --git a/scylla/src/transport/connection.rs b/scylla/src/transport/connection.rs index 942e47698..b4c76e547 100644 --- a/scylla/src/transport/connection.rs +++ b/scylla/src/transport/connection.rs @@ -69,7 +69,10 @@ use crate::routing::ShardInfo; use crate::statement::prepared_statement::PreparedStatement; use crate::statement::{Consistency, PageSize, PagingState, PagingStateResponse}; use crate::transport::Compression; -use crate::QueryResult; + +// Existing code imports scylla::transport::connection::LegacyQueryResult because it used to be located in this file. +// Reexport LegacyQueryResult to avoid breaking the existing code. +use crate::LegacyQueryResult; // Queries for schema agreement const LOCAL_VERSION: &str = "SELECT schema_version FROM system.local WHERE key='local'"; @@ -240,12 +243,12 @@ impl QueryResponse { pub(crate) fn into_query_result_and_paging_state( self, - ) -> Result<(QueryResult, PagingStateResponse), UserRequestError> { + ) -> Result<(LegacyQueryResult, PagingStateResponse), UserRequestError> { self.into_non_error_query_response()? .into_query_result_and_paging_state() } - pub(crate) fn into_query_result(self) -> Result { + pub(crate) fn into_query_result(self) -> Result { self.into_non_error_query_response()?.into_query_result() } } @@ -267,7 +270,7 @@ impl NonErrorQueryResponse { pub(crate) fn into_query_result_and_paging_state( self, - ) -> Result<(QueryResult, PagingStateResponse), UserRequestError> { + ) -> Result<(LegacyQueryResult, PagingStateResponse), UserRequestError> { let (rows, paging_state, metadata, serialized_size) = match self.response { NonErrorResponse::Result(result::Result::Rows(rs)) => ( Some(rs.rows), @@ -284,7 +287,7 @@ impl NonErrorQueryResponse { }; Ok(( - QueryResult { + LegacyQueryResult { rows, warnings: self.warnings, tracing_id: self.tracing_id, @@ -295,7 +298,7 @@ impl NonErrorQueryResponse { )) } - pub(crate) fn into_query_result(self) -> Result { + pub(crate) fn into_query_result(self) -> Result { let (result, paging_state) = self.into_query_result_and_paging_state()?; if !paging_state.finished() { @@ -977,7 +980,7 @@ impl Connection { &self, query: impl Into, paging_state: PagingState, - ) -> Result<(QueryResult, PagingStateResponse), UserRequestError> { + ) -> Result<(LegacyQueryResult, PagingStateResponse), UserRequestError> { let query: Query = query.into(); // This method is used only for driver internal queries, so no need to consult execution profile here. @@ -1002,7 +1005,7 @@ impl Connection { paging_state: PagingState, consistency: Consistency, serial_consistency: Option, - ) -> Result<(QueryResult, PagingStateResponse), UserRequestError> { + ) -> Result<(LegacyQueryResult, PagingStateResponse), UserRequestError> { let query: Query = query.into(); let page_size = query.get_validated_page_size(); @@ -1021,7 +1024,7 @@ impl Connection { pub(crate) async fn query_unpaged( &self, query: impl Into, - ) -> Result { + ) -> Result { // This method is used only for driver internal queries, so no need to consult execution profile here. let query: Query = query.into(); @@ -1081,7 +1084,7 @@ impl Connection { &self, prepared: &PreparedStatement, values: SerializedValues, - ) -> Result { + ) -> Result { // This method is used only for driver internal queries, so no need to consult execution profile here. self.execute_raw_unpaged(prepared, values) .await @@ -1228,7 +1231,7 @@ impl Connection { &self, batch: &Batch, values: impl BatchValues, - ) -> Result { + ) -> Result { self.batch_with_consistency( batch, values, @@ -1246,7 +1249,7 @@ impl Connection { values: impl BatchValues, consistency: Consistency, serial_consistency: Option, - ) -> Result { + ) -> Result { let batch = self.prepare_batch(init_batch, &values).await?; let contexts = batch.statements.iter().map(|bs| match bs { diff --git a/scylla/src/transport/errors.rs b/scylla/src/transport/errors.rs index aee6ab1f3..d95383054 100644 --- a/scylla/src/transport/errors.rs +++ b/scylla/src/transport/errors.rs @@ -32,7 +32,7 @@ use thiserror::Error; use crate::{authentication::AuthError, frame::response}; -use super::query_result::{RowsExpectedError, SingleRowTypedError}; +use super::legacy_query_result::{RowsExpectedError, SingleRowTypedError}; /// Error that occurred during query execution #[derive(Error, Debug, Clone)] diff --git a/scylla/src/transport/large_batch_statements_test.rs b/scylla/src/transport/large_batch_statements_test.rs index 7e8fc482c..33628a49d 100644 --- a/scylla/src/transport/large_batch_statements_test.rs +++ b/scylla/src/transport/large_batch_statements_test.rs @@ -7,7 +7,7 @@ use crate::transport::errors::{BadQuery, QueryError}; use crate::{ batch::Batch, test_utils::{create_new_session_builder, unique_keyspace_name}, - QueryResult, Session, + LegacyQueryResult, Session, }; #[tokio::test] @@ -51,7 +51,11 @@ async fn create_test_session(session: Session, ks: &String) -> Session { session } -async fn write_batch(session: &Session, n: usize, ks: &String) -> Result { +async fn write_batch( + session: &Session, + n: usize, + ks: &String, +) -> Result { let mut batch_query = Batch::new(BatchType::Unlogged); let mut batch_values = Vec::new(); let query = format!("INSERT INTO {}.pairs (dummy, k, v) VALUES (0, ?, ?)", ks); diff --git a/scylla/src/transport/query_result.rs b/scylla/src/transport/legacy_query_result.rs similarity index 84% rename from scylla/src/transport/query_result.rs rename to scylla/src/transport/legacy_query_result.rs index b60485afb..a08765d93 100644 --- a/scylla/src/transport/query_result.rs +++ b/scylla/src/transport/legacy_query_result.rs @@ -11,7 +11,7 @@ use uuid::Uuid; /// Result of a single query\ /// Contains all rows returned by the database and some more information #[derive(Debug)] -pub struct QueryResult { +pub struct LegacyQueryResult { /// Rows returned by the database.\ /// Queries like `SELECT` will have `Some(Vec)`, while queries like `INSERT` will have `None`.\ /// Can contain an empty Vec. @@ -26,7 +26,7 @@ pub struct QueryResult { pub serialized_size: usize, } -impl QueryResult { +impl LegacyQueryResult { pub(crate) fn mock_empty() -> Self { Self { rows: None, @@ -38,7 +38,7 @@ impl QueryResult { } /// Returns the number of received rows.\ - /// Fails when the query isn't of a type that could return rows, same as [`rows()`](QueryResult::rows). + /// Fails when the query isn't of a type that could return rows, same as [`rows()`](LegacyQueryResult::rows). pub fn rows_num(&self) -> Result { match &self.rows { Some(rows) => Ok(rows.len()), @@ -47,7 +47,7 @@ impl QueryResult { } /// Returns the received rows when present.\ - /// If `QueryResult.rows` is `None`, which means that this query is not supposed to return rows (e.g `INSERT`), returns an error.\ + /// If `LegacyQueryResult.rows` is `None`, which means that this query is not supposed to return rows (e.g `INSERT`), returns an error.\ /// Can return an empty `Vec`. pub fn rows(self) -> Result, RowsExpectedError> { match self.rows { @@ -58,14 +58,14 @@ impl QueryResult { /// Returns the received rows parsed as the given type.\ /// Equal to `rows()?.into_typed()`.\ - /// Fails when the query isn't of a type that could return rows, same as [`rows()`](QueryResult::rows). + /// Fails when the query isn't of a type that could return rows, same as [`rows()`](LegacyQueryResult::rows). pub fn rows_typed(self) -> Result, RowsExpectedError> { Ok(self.rows()?.into_typed()) } /// Returns `Ok` for a result of a query that shouldn't contain any rows.\ /// Will return `Ok` for `INSERT` result, but a `SELECT` result, even an empty one, will cause an error.\ - /// Opposite of [`rows()`](QueryResult::rows). + /// Opposite of [`rows()`](LegacyQueryResult::rows). pub fn result_not_rows(&self) -> Result<(), RowsNotExpectedError> { match self.rows { Some(_) => Err(RowsNotExpectedError), @@ -73,14 +73,14 @@ impl QueryResult { } } - /// Returns rows when `QueryResult.rows` is `Some`, otherwise an empty Vec.\ + /// Returns rows when `LegacyQueryResult.rows` is `Some`, otherwise an empty Vec.\ /// Equal to `rows().unwrap_or_default()`. pub fn rows_or_empty(self) -> Vec { self.rows.unwrap_or_default() } /// Returns rows parsed as the given type.\ - /// When `QueryResult.rows` is `None`, returns 0 rows.\ + /// When `LegacyQueryResult.rows` is `None`, returns 0 rows.\ /// Equal to `rows_or_empty().into_typed::()`. pub fn rows_typed_or_empty(self) -> TypedRowIter { self.rows_or_empty().into_typed::() @@ -102,13 +102,13 @@ impl QueryResult { } /// Returns `Option` containing the first of a result.\ - /// Fails when the query isn't of a type that could return rows, same as [`rows()`](QueryResult::rows). + /// Fails when the query isn't of a type that could return rows, same as [`rows()`](LegacyQueryResult::rows). pub fn maybe_first_row(self) -> Result, RowsExpectedError> { Ok(self.rows()?.into_iter().next()) } /// Returns `Option` containing the first of a result.\ - /// Fails when the query isn't of a type that could return rows, same as [`rows()`](QueryResult::rows). + /// Fails when the query isn't of a type that could return rows, same as [`rows()`](LegacyQueryResult::rows). pub fn maybe_first_row_typed( self, ) -> Result, MaybeFirstRowTypedError> { @@ -155,57 +155,57 @@ impl QueryResult { } } -/// [`QueryResult::rows()`](QueryResult::rows) or a similar function called on a bad QueryResult.\ -/// Expected `QueryResult.rows` to be `Some`, but it was `None`.\ -/// `QueryResult.rows` is `Some` for queries that can return rows (e.g `SELECT`).\ +/// [`LegacyQueryResult::rows()`](LegacyQueryResult::rows) or a similar function called on a bad LegacyQueryResult.\ +/// Expected `LegacyQueryResult.rows` to be `Some`, but it was `None`.\ +/// `LegacyQueryResult.rows` is `Some` for queries that can return rows (e.g `SELECT`).\ /// It is `None` for queries that can't return rows (e.g `INSERT`). #[derive(Debug, Clone, Error, PartialEq, Eq)] #[error( - "QueryResult::rows() or similar function called on a bad QueryResult. - Expected QueryResult.rows to be Some, but it was None. - QueryResult.rows is Some for queries that can return rows (e.g SELECT). + "LegacyQueryResult::rows() or similar function called on a bad LegacyQueryResult. + Expected LegacyQueryResult.rows to be Some, but it was None. + LegacyQueryResult.rows is Some for queries that can return rows (e.g SELECT). It is None for queries that can't return rows (e.g INSERT)." )] pub struct RowsExpectedError; -/// [`QueryResult::result_not_rows()`](QueryResult::result_not_rows) called on a bad QueryResult.\ -/// Expected `QueryResult.rows` to be `None`, but it was `Some`.\ -/// `QueryResult.rows` is `Some` for queries that can return rows (e.g `SELECT`).\ +/// [`LegacyQueryResult::result_not_rows()`](LegacyQueryResult::result_not_rows) called on a bad LegacyQueryResult.\ +/// Expected `LegacyQueryResult.rows` to be `None`, but it was `Some`.\ +/// `LegacyQueryResult.rows` is `Some` for queries that can return rows (e.g `SELECT`).\ /// It is `None` for queries that can't return rows (e.g `INSERT`). #[derive(Debug, Clone, Error, PartialEq, Eq)] #[error( - "QueryResult::result_not_rows() called on a bad QueryResult. - Expected QueryResult.rows to be None, but it was Some. - QueryResult.rows is Some for queries that can return rows (e.g SELECT). + "LegacyQueryResult::result_not_rows() called on a bad LegacyQueryResult. + Expected LegacyQueryResult.rows to be None, but it was Some. + LegacyQueryResult.rows is Some for queries that can return rows (e.g SELECT). It is None for queries that can't return rows (e.g INSERT)." )] pub struct RowsNotExpectedError; #[derive(Debug, Clone, Error, PartialEq, Eq)] pub enum FirstRowError { - /// [`QueryResult::first_row()`](QueryResult::first_row) called on a bad QueryResult.\ - /// Expected `QueryResult.rows` to be `Some`, but it was `None`.\ - /// `QueryResult.rows` is `Some` for queries that can return rows (e.g `SELECT`).\ + /// [`LegacyQueryResult::first_row()`](LegacyQueryResult::first_row) called on a bad LegacyQueryResult.\ + /// Expected `LegacyQueryResult.rows` to be `Some`, but it was `None`.\ + /// `LegacyQueryResult.rows` is `Some` for queries that can return rows (e.g `SELECT`).\ /// It is `None` for queries that can't return rows (e.g `INSERT`). #[error(transparent)] RowsExpected(#[from] RowsExpectedError), - /// Rows in `QueryResult` are empty - #[error("Rows in QueryResult are empty")] + /// Rows in `LegacyQueryResult` are empty + #[error("Rows in LegacyQueryResult are empty")] RowsEmpty, } #[derive(Debug, Clone, Error, PartialEq, Eq)] pub enum FirstRowTypedError { - /// [`QueryResult::first_row_typed()`](QueryResult::first_row_typed) called on a bad QueryResult.\ - /// Expected `QueryResult.rows` to be `Some`, but it was `None`.\ - /// `QueryResult.rows` is `Some` for queries that can return rows (e.g `SELECT`).\ + /// [`LegacyQueryResult::first_row_typed()`](LegacyQueryResult::first_row_typed) called on a bad LegacyQueryResult.\ + /// Expected `LegacyQueryResult.rows` to be `Some`, but it was `None`.\ + /// `LegacyQueryResult.rows` is `Some` for queries that can return rows (e.g `SELECT`).\ /// It is `None` for queries that can't return rows (e.g `INSERT`). #[error(transparent)] RowsExpected(#[from] RowsExpectedError), - /// Rows in `QueryResult` are empty - #[error("Rows in QueryResult are empty")] + /// Rows in `LegacyQueryResult` are empty + #[error("Rows in LegacyQueryResult are empty")] RowsEmpty, /// Parsing row as the given type failed @@ -215,9 +215,9 @@ pub enum FirstRowTypedError { #[derive(Debug, Clone, Error, PartialEq, Eq)] pub enum MaybeFirstRowTypedError { - /// [`QueryResult::maybe_first_row_typed()`](QueryResult::maybe_first_row_typed) called on a bad QueryResult.\ - /// Expected `QueryResult.rows` to be `Some`, but it was `None`. - /// `QueryResult.rows` is `Some` for queries that can return rows (e.g `SELECT`).\ + /// [`LegacyQueryResult::maybe_first_row_typed()`](LegacyQueryResult::maybe_first_row_typed) called on a bad LegacyQueryResult.\ + /// Expected `LegacyQueryResult.rows` to be `Some`, but it was `None`. + /// `LegacyQueryResult.rows` is `Some` for queries that can return rows (e.g `SELECT`).\ /// It is `None` for queries that can't return rows (e.g `INSERT`). #[error(transparent)] RowsExpected(#[from] RowsExpectedError), @@ -229,9 +229,9 @@ pub enum MaybeFirstRowTypedError { #[derive(Debug, Clone, Error, PartialEq, Eq)] pub enum SingleRowError { - /// [`QueryResult::single_row()`](QueryResult::single_row) called on a bad QueryResult.\ - /// Expected `QueryResult.rows` to be `Some`, but it was `None`.\ - /// `QueryResult.rows` is `Some` for queries that can return rows (e.g `SELECT`).\ + /// [`LegacyQueryResult::single_row()`](LegacyQueryResult::single_row) called on a bad LegacyQueryResult.\ + /// Expected `LegacyQueryResult.rows` to be `Some`, but it was `None`.\ + /// `LegacyQueryResult.rows` is `Some` for queries that can return rows (e.g `SELECT`).\ /// It is `None` for queries that can't return rows (e.g `INSERT`). #[error(transparent)] RowsExpected(#[from] RowsExpectedError), @@ -243,9 +243,9 @@ pub enum SingleRowError { #[derive(Debug, Clone, Error, PartialEq, Eq)] pub enum SingleRowTypedError { - /// [`QueryResult::single_row_typed()`](QueryResult::single_row_typed) called on a bad QueryResult.\ - /// Expected `QueryResult.rows` to be `Some`, but it was `None`.\ - /// `QueryResult.rows` is `Some` for queries that can return rows (e.g `SELECT`).\ + /// [`LegacyQueryResult::single_row_typed()`](LegacyQueryResult::single_row_typed) called on a bad LegacyQueryResult.\ + /// Expected `LegacyQueryResult.rows` to be `Some`, but it was `None`.\ + /// `LegacyQueryResult.rows` is `Some` for queries that can return rows (e.g `SELECT`).\ /// It is `None` for queries that can't return rows (e.g `INSERT`). #[error(transparent)] RowsExpected(#[from] RowsExpectedError), @@ -322,8 +322,8 @@ mod tests { ResultMetadata::new_for_test(1, vec![column_spec]) } - fn make_not_rows_query_result() -> QueryResult { - QueryResult { + fn make_not_rows_query_result() -> LegacyQueryResult { + LegacyQueryResult { rows: None, warnings: vec![], tracing_id: None, @@ -332,14 +332,14 @@ mod tests { } } - fn make_rows_query_result(rows_num: usize) -> QueryResult { + fn make_rows_query_result(rows_num: usize) -> LegacyQueryResult { let mut res = make_not_rows_query_result(); res.rows = Some(make_rows(rows_num)); res.metadata = Some(Arc::new(make_test_metadata())); res } - fn make_string_rows_query_result(rows_num: usize) -> QueryResult { + fn make_string_rows_query_result(rows_num: usize) -> LegacyQueryResult { let mut res = make_not_rows_query_result(); res.rows = Some(make_string_rows(rows_num)); res.metadata = Some(Arc::new(make_test_metadata())); diff --git a/scylla/src/transport/mod.rs b/scylla/src/transport/mod.rs index 45befce15..74cac10b9 100644 --- a/scylla/src/transport/mod.rs +++ b/scylla/src/transport/mod.rs @@ -7,12 +7,12 @@ pub mod errors; pub mod execution_profile; pub mod host_filter; pub mod iterator; +pub mod legacy_query_result; pub mod load_balancing; pub mod locator; pub(crate) mod metrics; mod node; pub mod partitioner; -pub mod query_result; pub mod retry_policy; pub mod session; pub mod session_builder; diff --git a/scylla/src/transport/session.rs b/scylla/src/transport/session.rs index 42933bdaf..38dbb30a9 100644 --- a/scylla/src/transport/session.rs +++ b/scylla/src/transport/session.rs @@ -42,11 +42,11 @@ use super::connection::QueryResponse; use super::connection::SslConfig; use super::errors::TracingProtocolError; use super::execution_profile::{ExecutionProfile, ExecutionProfileHandle, ExecutionProfileInner}; +use super::legacy_query_result::MaybeFirstRowTypedError; #[cfg(feature = "cloud")] use super::node::CloudEndpoint; use super::node::{InternalKnownNode, KnownNode}; use super::partitioner::PartitionerName; -use super::query_result::MaybeFirstRowTypedError; use super::topology::UntranslatedPeer; use super::{NodeRef, SelfIdentity}; use crate::cql_to_rust::FromRow; @@ -62,10 +62,10 @@ use crate::transport::connection::{Connection, ConnectionConfig, VerifiedKeyspac use crate::transport::connection_pool::PoolConfig; use crate::transport::host_filter::HostFilter; use crate::transport::iterator::{PreparedIteratorConfig, RowIterator}; +use crate::transport::legacy_query_result::LegacyQueryResult; use crate::transport::load_balancing::{self, RoutingInfo}; use crate::transport::metrics::Metrics; use crate::transport::node::Node; -use crate::transport::query_result::QueryResult; use crate::transport::retry_policy::{QueryInfo, RetryDecision, RetrySession}; use crate::transport::speculative_execution; use crate::transport::Compression; @@ -646,7 +646,7 @@ impl Session { &self, query: impl Into, values: impl SerializeRow, - ) -> Result { + ) -> Result { let query = query.into(); let (result, paging_state_response) = self .query(&query, values, None, PagingState::start()) @@ -711,7 +711,7 @@ impl Session { query: impl Into, values: impl SerializeRow, paging_state: PagingState, - ) -> Result<(QueryResult, PagingStateResponse), QueryError> { + ) -> Result<(LegacyQueryResult, PagingStateResponse), QueryError> { let query = query.into(); self.query( &query, @@ -739,7 +739,7 @@ impl Session { values: impl SerializeRow, page_size: Option, paging_state: PagingState, - ) -> Result<(QueryResult, PagingStateResponse), QueryError> { + ) -> Result<(LegacyQueryResult, PagingStateResponse), QueryError> { let execution_profile = query .get_execution_profile_handle() .unwrap_or_else(|| self.get_default_execution_profile_handle()) @@ -1078,7 +1078,7 @@ impl Session { &self, prepared: &PreparedStatement, values: impl SerializeRow, - ) -> Result { + ) -> Result { let serialized_values = prepared.serialize_values(&values)?; let (result, paging_state) = self .execute(prepared, &serialized_values, None, PagingState::start()) @@ -1148,7 +1148,7 @@ impl Session { prepared: &PreparedStatement, values: impl SerializeRow, paging_state: PagingState, - ) -> Result<(QueryResult, PagingStateResponse), QueryError> { + ) -> Result<(LegacyQueryResult, PagingStateResponse), QueryError> { let serialized_values = prepared.serialize_values(&values)?; let page_size = prepared.get_validated_page_size(); self.execute(prepared, &serialized_values, Some(page_size), paging_state) @@ -1171,7 +1171,7 @@ impl Session { serialized_values: &SerializedValues, page_size: Option, paging_state: PagingState, - ) -> Result<(QueryResult, PagingStateResponse), QueryError> { + ) -> Result<(LegacyQueryResult, PagingStateResponse), QueryError> { let values_ref = &serialized_values; let paging_state_ref = &paging_state; @@ -1377,7 +1377,7 @@ impl Session { &self, batch: &Batch, values: impl BatchValues, - ) -> Result { + ) -> Result { // Shard-awareness behavior for batch will be to pick shard based on first batch statement's shard // If users batch statements by shard, they will be rewarded with full shard awareness @@ -1454,7 +1454,7 @@ impl Session { .await?; let result = match run_query_result { - RunQueryResult::IgnoredWriteError => QueryResult::mock_empty(), + RunQueryResult::IgnoredWriteError => LegacyQueryResult::mock_empty(), RunQueryResult::Completed(response) => response, }; span.record_result_fields(&result); @@ -2020,7 +2020,7 @@ impl Session { pub(crate) trait AllowedRunQueryResTType {} impl AllowedRunQueryResTType for Uuid {} -impl AllowedRunQueryResTType for QueryResult {} +impl AllowedRunQueryResTType for LegacyQueryResult {} impl AllowedRunQueryResTType for NonErrorQueryResponse {} struct ExecuteQueryContext<'a> { @@ -2176,7 +2176,7 @@ impl RequestSpan { } } - pub(crate) fn record_result_fields(&self, result: &QueryResult) { + pub(crate) fn record_result_fields(&self, result: &LegacyQueryResult) { self.span.record("result_size", result.serialized_size); if let Some(rows) = result.rows.as_ref() { self.span.record("result_rows", rows.len()); diff --git a/scylla/src/transport/session_test.rs b/scylla/src/transport/session_test.rs index bfaed4d51..d4222d3b5 100644 --- a/scylla/src/transport/session_test.rs +++ b/scylla/src/transport/session_test.rs @@ -21,7 +21,7 @@ use crate::utils::test_utils::{ }; use crate::CachingSession; use crate::ExecutionProfile; -use crate::QueryResult; +use crate::LegacyQueryResult; use crate::{Session, SessionBuilder}; use assert_matches::assert_matches; use futures::{FutureExt, StreamExt, TryStreamExt}; @@ -959,7 +959,7 @@ async fn test_tracing() { async fn test_tracing_query(session: &Session, ks: String) { // A query without tracing enabled has no tracing uuid in result let untraced_query: Query = Query::new(format!("SELECT * FROM {}.tab", ks)); - let untraced_query_result: QueryResult = + let untraced_query_result: LegacyQueryResult = session.query_unpaged(untraced_query, &[]).await.unwrap(); assert!(untraced_query_result.tracing_id.is_none()); @@ -968,7 +968,8 @@ async fn test_tracing_query(session: &Session, ks: String) { let mut traced_query: Query = Query::new(format!("SELECT * FROM {}.tab", ks)); traced_query.config.tracing = true; - let traced_query_result: QueryResult = session.query_unpaged(traced_query, &[]).await.unwrap(); + let traced_query_result: LegacyQueryResult = + session.query_unpaged(traced_query, &[]).await.unwrap(); assert!(traced_query_result.tracing_id.is_some()); // Querying this uuid from tracing table gives some results @@ -982,7 +983,7 @@ async fn test_tracing_execute(session: &Session, ks: String) { .await .unwrap(); - let untraced_prepared_result: QueryResult = session + let untraced_prepared_result: LegacyQueryResult = session .execute_unpaged(&untraced_prepared, &[]) .await .unwrap(); @@ -997,7 +998,7 @@ async fn test_tracing_execute(session: &Session, ks: String) { traced_prepared.config.tracing = true; - let traced_prepared_result: QueryResult = session + let traced_prepared_result: LegacyQueryResult = session .execute_unpaged(&traced_prepared, &[]) .await .unwrap(); @@ -1034,7 +1035,8 @@ async fn test_get_tracing_info(session: &Session, ks: String) { let mut traced_query: Query = Query::new(format!("SELECT * FROM {}.tab", ks)); traced_query.config.tracing = true; - let traced_query_result: QueryResult = session.query_unpaged(traced_query, &[]).await.unwrap(); + let traced_query_result: LegacyQueryResult = + session.query_unpaged(traced_query, &[]).await.unwrap(); let tracing_id: Uuid = traced_query_result.tracing_id.unwrap(); // Getting tracing info from session using this uuid works @@ -1124,7 +1126,8 @@ async fn test_tracing_batch(session: &Session, ks: String) { let mut untraced_batch: Batch = Default::default(); untraced_batch.append_statement(&format!("INSERT INTO {}.tab (a) VALUES('a')", ks)[..]); - let untraced_batch_result: QueryResult = session.batch(&untraced_batch, ((),)).await.unwrap(); + let untraced_batch_result: LegacyQueryResult = + session.batch(&untraced_batch, ((),)).await.unwrap(); assert!(untraced_batch_result.tracing_id.is_none()); // Batch with tracing enabled has a tracing uuid in result @@ -1132,7 +1135,7 @@ async fn test_tracing_batch(session: &Session, ks: String) { traced_batch.append_statement(&format!("INSERT INTO {}.tab (a) VALUES('a')", ks)[..]); traced_batch.config.tracing = true; - let traced_batch_result: QueryResult = session.batch(&traced_batch, ((),)).await.unwrap(); + let traced_batch_result: LegacyQueryResult = session.batch(&traced_batch, ((),)).await.unwrap(); assert!(traced_batch_result.tracing_id.is_some()); assert_in_tracing_table(session, traced_batch_result.tracing_id.unwrap()).await; @@ -2567,7 +2570,7 @@ async fn test_batch_lwts() { batch.append_statement("INSERT INTO tab (p1, c1, r1, r2) VALUES (0, 123, 321, 312)"); batch.append_statement("UPDATE tab SET r1 = 1 WHERE p1 = 0 AND c1 = 0 IF r2 = 0"); - let batch_res: QueryResult = session.batch(&batch, ((), (), ())).await.unwrap(); + let batch_res: LegacyQueryResult = session.batch(&batch, ((), (), ())).await.unwrap(); // Scylla returns 5 columns, but Cassandra returns only 1 let is_scylla: bool = batch_res.col_specs().len() == 5; @@ -2579,7 +2582,11 @@ async fn test_batch_lwts() { } } -async fn test_batch_lwts_for_scylla(session: &Session, batch: &Batch, batch_res: QueryResult) { +async fn test_batch_lwts_for_scylla( + session: &Session, + batch: &Batch, + batch_res: LegacyQueryResult, +) { // Alias required by clippy type IntOrNull = Option; @@ -2600,7 +2607,7 @@ async fn test_batch_lwts_for_scylla(session: &Session, batch: &Batch, batch_res: assert_eq!(batch_res_rows, expected_batch_res_rows); let prepared_batch: Batch = session.prepare_batch(batch).await.unwrap(); - let prepared_batch_res: QueryResult = + let prepared_batch_res: LegacyQueryResult = session.batch(&prepared_batch, ((), (), ())).await.unwrap(); let prepared_batch_res_rows: Vec<(bool, IntOrNull, IntOrNull, IntOrNull, IntOrNull)> = @@ -2619,7 +2626,11 @@ async fn test_batch_lwts_for_scylla(session: &Session, batch: &Batch, batch_res: assert_eq!(prepared_batch_res_rows, expected_prepared_batch_res_rows); } -async fn test_batch_lwts_for_cassandra(session: &Session, batch: &Batch, batch_res: QueryResult) { +async fn test_batch_lwts_for_cassandra( + session: &Session, + batch: &Batch, + batch_res: LegacyQueryResult, +) { // Alias required by clippy type IntOrNull = Option; @@ -2636,7 +2647,7 @@ async fn test_batch_lwts_for_cassandra(session: &Session, batch: &Batch, batch_r assert_eq!(batch_res_rows, expected_batch_res_rows); let prepared_batch: Batch = session.prepare_batch(batch).await.unwrap(); - let prepared_batch_res: QueryResult = + let prepared_batch_res: LegacyQueryResult = session.batch(&prepared_batch, ((), (), ())).await.unwrap(); // Returned columns are: diff --git a/scylla/tests/integration/tablets.rs b/scylla/tests/integration/tablets.rs index 3f1356840..2bdf96987 100644 --- a/scylla/tests/integration/tablets.rs +++ b/scylla/tests/integration/tablets.rs @@ -17,7 +17,7 @@ use scylla::transport::ClusterData; use scylla::transport::Node; use scylla::transport::NodeRef; use scylla::ExecutionProfile; -use scylla::QueryResult; +use scylla::LegacyQueryResult; use scylla::Session; use scylla::transport::errors::QueryError; @@ -185,7 +185,7 @@ async fn send_statement_everywhere( cluster: &ClusterData, statement: &PreparedStatement, values: &dyn SerializeRow, -) -> Result, QueryError> { +) -> Result, QueryError> { let tasks = cluster.get_nodes_info().iter().flat_map(|node| { let shard_count: u16 = node.sharder().unwrap().nr_shards.into(); (0..shard_count).map(|shard| { @@ -210,7 +210,7 @@ async fn send_unprepared_query_everywhere( session: &Session, cluster: &ClusterData, query: &Query, -) -> Result, QueryError> { +) -> Result, QueryError> { let tasks = cluster.get_nodes_info().iter().flat_map(|node| { let shard_count: u16 = node.sharder().unwrap().nr_shards.into(); (0..shard_count).map(|shard| { From 8020cfd32ddd036e7df1bba26b590944dee61f05 Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Mon, 13 Mar 2023 15:46:32 +0100 Subject: [PATCH 06/42] transport: introduce new QueryResult MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit (Re)-introduces QueryResult. It is quite similar to the old (Legacy)QueryResult, but it keeps rows and metadata in an unparsed state (via RawRows) and has methods that allow parsing the contents using the new API. Helper method names are similar to what the old QueryResult had, just with the `_typed` suffix dropped - as now it is always required to provide the type of rows when parsing them, this suffix sounded redundant. There is one crucial change to the API. Motivation is as follows: 1) `QueryResult` can represent a non-Rows response, so every rows-related operation on `QueryResult` may return "NonRowsResponse" error, which is inconvenient; 2) `QueryResult` is an owned type, so it cannot deserialize metadata in the borrowed flavour (i.e., using strings from the frame bytes directly) and it must allocate metadata (mainly ColumnSpecs) on the heap. The solution for both is to extract a new struct, `RowsDeserializer`, which is parametrized by a lifetime and hence can borrow metadata from the frame. Moreover, one has to handle "NonRowsResponse" error only once, upon `RowsDeserializer` creation. All further methods (`rows(), `single_row()`, etc.) may no longer fail with that error, which provides a clean step of conversion from any Result frame to Result:Rows frame. The drawback is that now there is a new call required in the call chain to deserialize a result, namely `.row_deserializer()`. RowsDeserializer is parametrized by the representation of raw rows (Owned or Borrowed), analogously to how DeserializedMetadataAndRawRows are. Co-authored-by: Wojciech Przytuła --- scylla/src/lib.rs | 1 + scylla/src/transport/mod.rs | 1 + scylla/src/transport/query_result.rs | 603 +++++++++++++++++++++++++++ 3 files changed, 605 insertions(+) create mode 100644 scylla/src/transport/query_result.rs diff --git a/scylla/src/lib.rs b/scylla/src/lib.rs index feea88438..289460ba9 100644 --- a/scylla/src/lib.rs +++ b/scylla/src/lib.rs @@ -259,6 +259,7 @@ pub use frame::response::cql_to_rust::FromRow; pub use transport::caching_session::CachingSession; pub use transport::execution_profile::ExecutionProfile; pub use transport::legacy_query_result::LegacyQueryResult; +pub use transport::query_result::QueryResult; pub use transport::session::{IntoTypedRows, Session, SessionConfig}; pub use transport::session_builder::SessionBuilder; diff --git a/scylla/src/transport/mod.rs b/scylla/src/transport/mod.rs index 74cac10b9..be4cfa37b 100644 --- a/scylla/src/transport/mod.rs +++ b/scylla/src/transport/mod.rs @@ -13,6 +13,7 @@ pub mod locator; pub(crate) mod metrics; mod node; pub mod partitioner; +pub mod query_result; pub mod retry_policy; pub mod session; pub mod session_builder; diff --git a/scylla/src/transport/query_result.rs b/scylla/src/transport/query_result.rs new file mode 100644 index 000000000..d9bc9a26f --- /dev/null +++ b/scylla/src/transport/query_result.rs @@ -0,0 +1,603 @@ +use std::fmt::Debug; +use std::sync::Arc; + +use thiserror::Error; +use uuid::Uuid; + +use scylla_cql::frame::frame_errors::RowsParseError; +use scylla_cql::frame::response::result::{ + ColumnSpec, ColumnType, DeserializedMetadataAndRawRows, RawMetadataAndRawRows, RawRowsBorrowed, + RawRowsKind, RawRowsOwned, Row, TableSpec, +}; +use scylla_cql::types::deserialize::result::TypedRowIterator; +use scylla_cql::types::deserialize::row::DeserializeRow; +use scylla_cql::types::deserialize::{DeserializationError, TypeCheckError}; + +use super::legacy_query_result::LegacyQueryResult; + +/// A view over specification of a table in the database. +#[derive(Debug, Clone, Copy)] +#[cfg_attr(test, derive(PartialEq, Eq))] +pub struct TableSpecView<'res> { + table_name: &'res str, + ks_name: &'res str, +} + +impl<'res> TableSpecView<'res> { + pub(crate) fn new_from_table_spec(spec: &'res TableSpec) -> Self { + Self { + table_name: spec.table_name(), + ks_name: spec.ks_name(), + } + } + + /// The name of the table. + #[inline] + pub fn table_name(&self) -> &'res str { + self.table_name + } + + /// The name of the keyspace the table resides in. + #[inline] + pub fn ks_name(&self) -> &'res str { + self.ks_name + } +} + +/// A view over specification of a column returned by the database. +#[derive(Debug, Clone, Copy)] +#[cfg_attr(test, derive(PartialEq, Eq))] +pub struct ColumnSpecView<'res> { + table_spec: TableSpecView<'res>, + name: &'res str, + typ: &'res ColumnType<'res>, +} + +impl<'res> ColumnSpecView<'res> { + pub(crate) fn new_from_column_spec(spec: &'res ColumnSpec) -> Self { + Self { + table_spec: TableSpecView::new_from_table_spec(spec.table_spec()), + name: spec.name(), + typ: spec.typ(), + } + } + + /// Returns a view over specification of the table the column is part of. + #[inline] + pub fn table_spec(&self) -> TableSpecView<'res> { + self.table_spec + } + + /// The column's name. + #[inline] + pub fn name(&self) -> &'res str { + self.name + } + + /// The column's CQL type. + #[inline] + pub fn typ(&self) -> &'res ColumnType { + self.typ + } +} + +/// A view over specification of columns returned by the database. +#[derive(Debug, Clone, Copy)] +pub struct ColumnSpecs<'res> { + specs: &'res [ColumnSpec<'res>], +} + +impl<'res> ColumnSpecs<'res> { + pub(crate) fn new(specs: &'res [ColumnSpec<'res>]) -> Self { + Self { specs } + } + + pub(crate) fn inner(&self) -> &'res [ColumnSpec<'res>] { + self.specs + } + + /// Returns number of columns. + #[allow(clippy::len_without_is_empty)] + #[inline] + pub fn len(&self) -> usize { + self.specs.len() + } + + /// Returns specification of k-th column returned from the database. + #[inline] + pub fn get_by_index(&self, k: usize) -> Option> { + self.specs.get(k).map(ColumnSpecView::new_from_column_spec) + } + + /// Returns specification of the column with given name returned from the database. + #[inline] + pub fn get_by_name(&self, name: &str) -> Option<(usize, ColumnSpecView<'res>)> { + self.specs + .iter() + .enumerate() + .find(|(_idx, spec)| spec.name() == name) + .map(|(idx, spec)| (idx, ColumnSpecView::new_from_column_spec(spec))) + } + + /// Returns iterator over specification of columns returned from the database, + /// ordered by column order in the response. + #[inline] + pub fn iter(&self) -> impl Iterator> { + self.specs.iter().map(ColumnSpecView::new_from_column_spec) + } +} + +/// Result of a single request to the database. It represents any kind of Result frame. +/// +/// The received rows and metadata, which are present if the frame is of Result:Rows kind, +/// are kept in a raw binary form. To deserialize and access them, this struct works +/// in tandem with [`RowsDeserializer`] struct, which borrows from [`QueryResult`]. +/// By borrowing, [`RowsDeserializer`] can avoid heap allocations of metadata strings, +/// borrowing them from the Result frame instead. +/// To create a [`RowsDeserializer`], use [`QueryResult::rows_deserializer`] method. +/// Upon creation, [`RowsDeserializer`] deserializes result metadata and allocates it, +/// so this should be considered a moderately costly operation and performed only once. +/// +/// NOTE: this is a result of a single CQL request. If you use paging for your query, +/// this will contain exactly one page. +#[derive(Debug)] +pub struct QueryResult { + raw_metadata_and_rows: Option, + tracing_id: Option, + warnings: Vec, +} + +impl QueryResult { + pub(crate) fn new( + raw_rows: Option, + tracing_id: Option, + warnings: Vec, + ) -> Self { + Self { + raw_metadata_and_rows: raw_rows, + tracing_id, + warnings, + } + } + + // Preferred to implementing Default, because users shouldn't be able to create + // an empty QueryResult. + // + // For now unused, but it will be used once Session's API is migrated + // to the new QueryResult. + #[allow(dead_code)] + pub(crate) fn mock_empty() -> Self { + Self { + raw_metadata_and_rows: None, + tracing_id: None, + warnings: Vec::new(), + } + } + + /// Warnings emitted by the database. + #[inline] + pub fn warnings(&self) -> impl Iterator { + self.warnings.iter().map(String::as_str) + } + + /// Tracing ID associated with this CQL request. + #[inline] + pub fn tracing_id(&self) -> Option { + self.tracing_id + } + + /// Returns a bool indicating the current response is of Rows type. + #[inline] + pub fn is_rows(&self) -> bool { + self.raw_metadata_and_rows.is_some() + } + + /// Returns `Ok` for a request's result that shouldn't contain any rows.\ + /// Will return `Ok` for `INSERT` result, but a `SELECT` result, even an empty one, will cause an error.\ + /// Opposite of [`rows_deserializer()`](QueryResult::rows_deserializer). + #[inline] + pub fn result_not_rows(&self) -> Result<(), ResultNotRowsError> { + match &self.raw_metadata_and_rows { + Some(_) => Err(ResultNotRowsError), + None => Ok(()), + } + } + + /// Creates a lifetime-bound [`RowsDeserializer`] to enable deserializing rows contained + /// in this [`QueryResult`]'s frame. Deserializes result metadata and allocates it, + /// so **this should be considered a moderately costly operation and performed only once**. + /// + /// Returns `None` if the response is not of Rows kind. + /// + /// The created [`RowsDeserializer`] borrows from the [`QueryResult`], which saves some + /// string heap allocations, but limits flexibility (e.g., such borrowing [`RowsDeserializer`] + /// can't be stored aside on a heap due to lifetime issues). + /// To gain more flexibility on cost of additional allocations, + /// use [`QueryResult::rows_deserializer_owned`]. + /// + /// ```rust + /// # use scylla::transport::query_result::{QueryResult, RowsDeserializer}; + /// # fn example(query_result: QueryResult) -> Result<(), Box> { + /// let rows_deserializer = query_result.rows_deserializer()?; + /// if let Some(rows_result) = rows_deserializer { + /// let mut rows_iter = rows_result.rows::<(i32, &str)>()?; + /// while let Some((num, text)) = rows_iter.next().transpose()? { + /// // do something with `num` and `text`` + /// } + /// } else { + /// // Response was not Result:Rows, but some other kind of Result. + /// } + /// + /// Ok(()) + /// # } + /// + /// ``` + pub fn rows_deserializer( + &self, + ) -> Result>>, RowsParseError> { + self.raw_metadata_and_rows + .as_ref() + .map(|raw_rows| { + let raw_rows_with_metadata = raw_rows.deserialize_borrowed_metadata()?; + Ok(RowsDeserializer { + raw_rows_with_metadata, + }) + }) + .transpose() + } + + /// Creates an owned [`RowsDeserializer`] to enable deserializing rows contained + /// in this [`QueryResult`]'s frame. Deserializes result metadata and allocates it, + /// so this should be considered a moderately costly operation and performed only once. + /// + /// Returns `None` if the response is not of Rows kind. + /// + /// The created [`RowsDeserializer`] does not borrow from the [`QueryResult`], + /// so it does not not limit flexibility. However, the cost involves more string + /// heap allocations. + /// If you don't need that flexibility, use cheaper [`QueryResult::rows_deserializer`]. + /// + /// ```compile_fail + /// # use scylla::transport::QueryResult; + /// fn example(query: impl FnOnce() -> QueryResult) -> Result<(), Box> { + /// let deserializer = query().rows_deserializer()?.unwrap(); + /// + /// // Compiler complains: "Temporary value dropped when borrowed". + /// let col_specs = deserializer.column_specs(); + /// + /// Ok(()) + /// } + /// ``` + /// + /// ```rust + /// # use scylla::transport::query_result::{QueryResult, RowsDeserializerOwning}; + /// fn example( + /// query: impl FnOnce() -> QueryResult + /// ) -> Result> { + /// let deserializer = query().rows_deserializer_owned()?.unwrap(); + /// + /// // This compiles. + /// let col_specs = deserializer.column_specs(); + /// + /// // RowsDeserializer is fully owned and independent, but at cost + /// // of moderately more expensive metadata deserialization. + /// Ok(deserializer) + /// } + /// ``` + pub fn rows_deserializer_owned( + &self, + ) -> Result>, RowsParseError> { + self.raw_metadata_and_rows + .as_ref() + .map(|raw_rows| { + let raw_rows_with_metadata = raw_rows.deserialize_owned_metadata()?; + Ok(RowsDeserializer { + raw_rows_with_metadata, + }) + }) + .transpose() + } + + /// 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. + 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_owned_metadata()?; + + let deserialized_rows = raw_rows_with_metadata + .rows_iter::()? + .collect::, DeserializationError>>()?; + let serialized_size = raw_rows_with_metadata.rows_bytes_size(); + let metadata = raw_rows_with_metadata.into_metadata(); + + Ok(LegacyQueryResult { + rows: Some(deserialized_rows), + warnings: self.warnings, + tracing_id: self.tracing_id, + metadata: Some(Arc::new((*metadata).clone())), + serialized_size, + }) + } else { + Ok(LegacyQueryResult { + rows: None, + warnings: self.warnings, + tracing_id: self.tracing_id, + metadata: None, + serialized_size: 0, + }) + } + } +} + +/// Enables deserialization of rows contained in a [`QueryResult`]. +/// +/// Upon creation, it deserializes result metadata and allocates it, +/// so this should be considered a moderately costly operation and performed +/// only once. +/// +/// This struct provides generic methods which enable typed access to the data, +/// by deserializing rows on the fly to the type provided as a type parameter. +/// Those methods are: +/// - rows() - for iterating through rows, +/// - first_row() and maybe_first_row() - for accessing the first row first, +/// - single_row() - for accessing the first row, additionally asserting +/// that it's the only one in the response. +/// +/// ```rust +/// # use scylla::transport::query_result::QueryResult; +/// # fn example(query_result: QueryResult) -> Result<(), Box> { +/// let rows_deserializer = query_result.rows_deserializer()?; +/// if let Some(rows_result) = rows_deserializer { +/// let mut rows_iter = rows_result.rows::<(i32, &str)>()?; +/// while let Some((num, text)) = rows_iter.next().transpose()? { +/// // do something with `num` and `text`` +/// } +/// } else { +/// // Response was not Result:Rows, but some other kind of Result. +/// } +/// +/// Ok(()) +/// # } +/// +/// ``` +#[derive(Debug)] +pub struct RowsDeserializer<'frame, Kind: RawRowsKind> { + raw_rows_with_metadata: DeserializedMetadataAndRawRows<'frame, Kind>, +} + +pub type RowsDeserializerOwning = RowsDeserializer<'static, RawRowsOwned>; +pub type RowsDeserializerBorrowing<'frame> = RowsDeserializer<'frame, RawRowsBorrowed<'frame>>; + +impl<'frame, RawRows: RawRowsKind> RowsDeserializer<'frame, RawRows> { + /// Returns the number of received rows. + #[inline] + pub fn rows_num(&self) -> usize { + self.raw_rows_with_metadata.rows_count() + } + + /// Returns the size of the serialized rows. + #[inline] + pub fn rows_bytes_size(&self) -> usize { + self.raw_rows_with_metadata.rows_bytes_size() + } + + /// Returns column specifications. + #[inline] + pub fn column_specs(&self) -> ColumnSpecs { + ColumnSpecs::new(self.raw_rows_with_metadata.metadata().col_specs()) + } +} + +impl<'frame> RowsDeserializer<'frame, RawRowsBorrowed<'frame>> { + /// Returns the received rows when present. + /// + /// Returns an error if the rows in the response are of incorrect type. + #[inline] + pub fn rows<'metadata, R: DeserializeRow<'frame, 'metadata>>( + &'metadata self, + ) -> Result, RowsError> { + self.raw_rows_with_metadata + .rows_iter() + .map_err(RowsError::TypeCheckFailed) + } + + /// Returns `Option` containing the first of a result. + /// + /// Fails when the the rows in the response are of incorrect type, + /// or when the deserialization fails. + pub fn maybe_first_row<'metadata, R: DeserializeRow<'frame, 'metadata>>( + &'metadata self, + ) -> Result, MaybeFirstRowError> { + self.rows::() + .map_err(|err| match err { + RowsError::TypeCheckFailed(typck_err) => { + MaybeFirstRowError::TypeCheckFailed(typck_err) + } + })? + .next() + .transpose() + .map_err(MaybeFirstRowError::DeserializationFailed) + } + + /// Returns first row from the received rows. + /// + /// When the first row is not available, returns an error. + /// Fails when the the rows in the response are of incorrect type, + /// or when the deserialization fails. + pub fn first_row<'metadata, R: DeserializeRow<'frame, 'metadata>>( + &'metadata self, + ) -> Result { + match self.maybe_first_row::() { + Ok(Some(row)) => Ok(row), + Ok(None) => Err(FirstRowError::RowsEmpty), + Err(MaybeFirstRowError::TypeCheckFailed(err)) => { + Err(FirstRowError::TypeCheckFailed(err)) + } + Err(MaybeFirstRowError::DeserializationFailed(err)) => { + Err(FirstRowError::DeserializationFailed(err)) + } + } + } + + /// Returns the only received row. + /// + /// Fails if the result is anything else than a single row. + /// Fails when the the rows in the response are of incorrect type, + /// or when the deserialization fails. + pub fn single_row<'metadata, R: DeserializeRow<'frame, 'metadata>>( + &'metadata self, + ) -> Result { + match self.rows::() { + Ok(mut rows) => match rows.next() { + Some(Ok(row)) => { + if rows.rows_remaining() != 0 { + return Err(SingleRowError::UnexpectedRowCount( + rows.rows_remaining() + 1, + )); + } + Ok(row) + } + Some(Err(err)) => Err(SingleRowError::DeserializationFailed(err)), + None => Err(SingleRowError::UnexpectedRowCount(0)), + }, + Err(RowsError::TypeCheckFailed(err)) => Err(SingleRowError::TypeCheckFailed(err)), + } + } +} + +impl RowsDeserializer<'static, RawRowsOwned> { + /// Returns the received rows when present. + /// + /// Returns an error if the rows in the response are of incorrect type. + #[inline] + pub fn rows<'frame, R: DeserializeRow<'frame, 'frame>>( + &'frame self, + ) -> Result, RowsError> { + self.raw_rows_with_metadata + .rows_iter() + .map_err(RowsError::TypeCheckFailed) + } + + /// Returns `Option` containing the first of a result. + /// + /// Fails when the the rows in the response are of incorrect type, + /// or when the deserialization fails. + pub fn maybe_first_row<'frame, R: DeserializeRow<'frame, 'frame>>( + &'frame self, + ) -> Result, MaybeFirstRowError> { + self.rows::() + .map_err(|err| match err { + RowsError::TypeCheckFailed(typck_err) => { + MaybeFirstRowError::TypeCheckFailed(typck_err) + } + })? + .next() + .transpose() + .map_err(MaybeFirstRowError::DeserializationFailed) + } + + /// Returns first row from the received rows. + /// + /// When the first row is not available, returns an error. + /// Fails when the the rows in the response are of incorrect type, + /// or when the deserialization fails. + pub fn first_row<'frame, R: DeserializeRow<'frame, 'frame>>( + &'frame self, + ) -> Result { + match self.maybe_first_row::() { + Ok(Some(row)) => Ok(row), + Ok(None) => Err(FirstRowError::RowsEmpty), + Err(MaybeFirstRowError::TypeCheckFailed(err)) => { + Err(FirstRowError::TypeCheckFailed(err)) + } + Err(MaybeFirstRowError::DeserializationFailed(err)) => { + Err(FirstRowError::DeserializationFailed(err)) + } + } + } + + /// Returns the only received row. + /// + /// Fails if the result is anything else than a single row. + /// Fails when the the rows in the response are of incorrect type, + /// or when the deserialization fails. + pub fn single_row<'frame, R: DeserializeRow<'frame, 'frame>>( + &'frame self, + ) -> Result { + match self.rows::() { + Ok(mut rows) => match rows.next() { + Some(Ok(row)) => { + if rows.rows_remaining() != 0 { + return Err(SingleRowError::UnexpectedRowCount( + rows.rows_remaining() + 1, + )); + } + Ok(row) + } + Some(Err(err)) => Err(SingleRowError::DeserializationFailed(err)), + None => Err(SingleRowError::UnexpectedRowCount(0)), + }, + Err(RowsError::TypeCheckFailed(err)) => Err(SingleRowError::TypeCheckFailed(err)), + } + } +} + +/// An error returned by [`RowsDeserializer::rows`]. +#[derive(Debug, Error)] +pub enum RowsError { + /// Type check failed + #[error("Type check failed: {0}")] + TypeCheckFailed(#[from] TypeCheckError), +} + +/// An error returned by [`RowsDeserializer::maybe_first_row`]. +#[derive(Debug, Error)] +pub enum MaybeFirstRowError { + /// Type check failed + #[error("Type check failed: {0}")] + TypeCheckFailed(#[from] TypeCheckError), + + /// Deserialization failed + #[error("Deserialization failed: {0}")] + DeserializationFailed(#[from] DeserializationError), +} + +/// An error returned by [`RowsDeserializer::first_row`]. +#[derive(Debug, Error)] +pub enum FirstRowError { + /// The request response was of Rows type, but no rows were returned + #[error("The request response was of Rows type, but no rows were returned")] + RowsEmpty, + + /// Type check failed + #[error("Type check failed: {0}")] + TypeCheckFailed(#[from] TypeCheckError), + + /// Deserialization failed + #[error("Deserialization failed: {0}")] + DeserializationFailed(#[from] DeserializationError), +} + +/// An error returned by [`RowsDeserializer::single_row`]. +#[derive(Debug, Error, Clone)] +pub enum SingleRowError { + /// Expected one row, but got a different count + #[error("Expected a single row, but got {0} rows")] + UnexpectedRowCount(usize), + + /// Type check failed + #[error("Type check failed: {0}")] + TypeCheckFailed(#[from] TypeCheckError), + + /// Deserialization failed + #[error("Deserialization failed: {0}")] + DeserializationFailed(#[from] DeserializationError), +} + +/// An error returned by [`QueryResult::result_not_rows`]. +/// +/// It indicates that response to the request was, unexpectedly, of Rows kind. +#[derive(Debug, Error)] +#[error("The request response was, unexpectedly, of Rows kind")] +pub struct ResultNotRowsError; From d7ce383de2933414acb6d4fb20c51a3f074f6456 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Wojciech=20Przytu=C5=82a?= Date: Wed, 16 Oct 2024 13:48:04 +0200 Subject: [PATCH 07/42] result: metadata serialization utils for tests New QueryResult tests are going to require result metadata serialization capabilities, as RawRows keep result metadata in a serialized form. --- scylla-cql/src/frame/response/result.rs | 150 ++++++++++++++++++++++++ scylla-cql/src/frame/types.rs | 10 +- 2 files changed, 158 insertions(+), 2 deletions(-) diff --git a/scylla-cql/src/frame/response/result.rs b/scylla-cql/src/frame/response/result.rs index aca1cf73d..ae5633bbb 100644 --- a/scylla-cql/src/frame/response/result.rs +++ b/scylla-cql/src/frame/response/result.rs @@ -1518,8 +1518,117 @@ pub fn deserialize( // the specifier, too. #[doc(hidden)] mod test_utils { + use std::num::TryFromIntError; + + use bytes::BufMut; + use super::*; + impl TableSpec<'_> { + pub(crate) fn serialize(&self, buf: &mut impl BufMut) -> StdResult<(), TryFromIntError> { + types::write_string(&self.ks_name, buf)?; + types::write_string(&self.table_name, buf)?; + + Ok(()) + } + } + + impl ColumnType<'_> { + fn id(&self) -> u16 { + match self { + Self::Custom(_) => 0x0000, + Self::Ascii => 0x0001, + Self::BigInt => 0x0002, + Self::Blob => 0x0003, + Self::Boolean => 0x0004, + Self::Counter => 0x0005, + Self::Decimal => 0x0006, + Self::Double => 0x0007, + Self::Float => 0x0008, + Self::Int => 0x0009, + Self::Timestamp => 0x000B, + Self::Uuid => 0x000C, + Self::Text => 0x000D, + Self::Varint => 0x000E, + Self::Timeuuid => 0x000F, + Self::Inet => 0x0010, + Self::Date => 0x0011, + Self::Time => 0x0012, + Self::SmallInt => 0x0013, + Self::TinyInt => 0x0014, + Self::Duration => 0x0015, + Self::List(_) => 0x0020, + Self::Map(_, _) => 0x0021, + Self::Set(_) => 0x0022, + Self::UserDefinedType { .. } => 0x0030, + Self::Tuple(_) => 0x0031, + } + } + + // Only for use in tests + pub(crate) fn serialize(&self, buf: &mut impl BufMut) -> StdResult<(), TryFromIntError> { + let id = self.id(); + types::write_short(id, buf); + + match self { + ColumnType::Custom(type_name) => { + types::write_string(type_name, buf)?; + } + + // Simple types + ColumnType::Ascii + | ColumnType::Boolean + | ColumnType::Blob + | ColumnType::Counter + | ColumnType::Date + | ColumnType::Decimal + | ColumnType::Double + | ColumnType::Duration + | ColumnType::Float + | ColumnType::Int + | ColumnType::BigInt + | ColumnType::Text + | ColumnType::Timestamp + | ColumnType::Inet + | ColumnType::SmallInt + | ColumnType::TinyInt + | ColumnType::Time + | ColumnType::Timeuuid + | ColumnType::Uuid + | ColumnType::Varint => (), + + ColumnType::List(elem_type) | ColumnType::Set(elem_type) => { + elem_type.serialize(buf)?; + } + ColumnType::Map(key_type, value_type) => { + key_type.serialize(buf)?; + value_type.serialize(buf)?; + } + ColumnType::Tuple(types) => { + types::write_short_length(types.len(), buf)?; + for typ in types.iter() { + typ.serialize(buf)?; + } + } + ColumnType::UserDefinedType { + type_name, + keyspace, + field_types, + } => { + types::write_string(keyspace, buf)?; + types::write_string(type_name, buf)?; + types::write_short_length(field_types.len(), buf)?; + for (field_name, field_type) in field_types { + types::write_string(field_name, buf)?; + field_type.serialize(buf)?; + } + } + } + + Ok(()) + } + } + impl<'a> ResultMetadata<'a> { #[inline] #[doc(hidden)] @@ -1529,6 +1638,47 @@ mod test_utils { col_specs, } } + + pub(crate) fn serialize( + &self, + buf: &mut impl BufMut, + no_metadata: bool, + global_tables_spec: bool, + ) -> StdResult<(), TryFromIntError> { + let global_table_spec = global_tables_spec + .then(|| self.col_specs.first().map(|col_spec| col_spec.table_spec())) + .flatten(); + + let mut flags = 0; + if global_table_spec.is_some() { + flags |= 0x0001; + } + if no_metadata { + flags |= 0x0004; + } + types::write_int(flags, buf); + + types::write_int_length(self.col_count, buf)?; + + // No paging state. + + if !no_metadata { + if let Some(spec) = global_table_spec { + spec.serialize(buf)?; + } + + for col_spec in self.col_specs() { + if global_table_spec.is_none() { + col_spec.table_spec().serialize(buf)?; + } + + types::write_string(col_spec.name(), buf)?; + col_spec.typ().serialize(buf)?; + } + } + + Ok(()) + } } impl<'frame> DeserializedMetadataAndRawRows<'frame, RawRowsOwned> { diff --git a/scylla-cql/src/frame/types.rs b/scylla-cql/src/frame/types.rs index e73347039..70f28f6c2 100644 --- a/scylla-cql/src/frame/types.rs +++ b/scylla-cql/src/frame/types.rs @@ -173,7 +173,10 @@ pub fn read_int_length(buf: &mut &[u8]) -> Result Result<(), std::num::TryFromIntError> { +pub(crate) fn write_int_length( + v: usize, + buf: &mut impl BufMut, +) -> Result<(), std::num::TryFromIntError> { let v: i32 = v.try_into()?; write_int(v, buf); @@ -224,7 +227,10 @@ pub(crate) fn read_short_length(buf: &mut &[u8]) -> Result Result<(), std::num::TryFromIntError> { +pub(crate) fn write_short_length( + v: usize, + buf: &mut impl BufMut, +) -> Result<(), std::num::TryFromIntError> { let v: u16 = v.try_into()?; write_short(v, buf); Ok(()) From 0af797fabc3862360a7aeca62bc4e038a49af7f6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Wojciech=20Przytu=C5=82a?= Date: Wed, 16 Oct 2024 13:54:41 +0200 Subject: [PATCH 08/42] transport: add tests for new QueryResult --- scylla-cql/src/frame/response/result.rs | 37 ++- scylla/src/transport/query_result.rs | 296 ++++++++++++++++++++++++ 2 files changed, 332 insertions(+), 1 deletion(-) diff --git a/scylla-cql/src/frame/response/result.rs b/scylla-cql/src/frame/response/result.rs index ae5633bbb..bf2ee9786 100644 --- a/scylla-cql/src/frame/response/result.rs +++ b/scylla-cql/src/frame/response/result.rs @@ -1520,7 +1520,7 @@ pub fn deserialize( mod test_utils { use std::num::TryFromIntError; - use bytes::BufMut; + use bytes::{BufMut, BytesMut}; use super::*; @@ -1681,6 +1681,41 @@ mod test_utils { } } + impl RawMetadataAndRawRows { + #[doc(hidden)] + #[inline] + pub fn new_for_test( + cached_metadata: Option>>, + metadata: Option, + global_tables_spec: bool, + rows_count: usize, + raw_rows: &[u8], + ) -> StdResult { + let no_metadata = metadata.is_none(); + let empty_metadata = ResultMetadata::mock_empty(); + let used_metadata = metadata + .as_ref() + .or(cached_metadata.as_deref()) + .unwrap_or(&empty_metadata); + + let raw_result_rows = { + let mut buf = BytesMut::new(); + used_metadata.serialize(&mut buf, global_tables_spec, no_metadata)?; + types::write_int_length(rows_count, &mut buf)?; + buf.extend_from_slice(raw_rows); + + buf.freeze() + }; + + let (raw_rows, _paging_state_response) = + Self::deserialize(&mut FrameSlice::new(&raw_result_rows), cached_metadata).expect( + "Ill-formed serialized metadata for tests - likely bug in serialization code", + ); + + Ok(raw_rows) + } + } + impl<'frame> DeserializedMetadataAndRawRows<'frame, RawRowsOwned> { #[inline] #[doc(hidden)] diff --git a/scylla/src/transport/query_result.rs b/scylla/src/transport/query_result.rs index d9bc9a26f..816bc7e7e 100644 --- a/scylla/src/transport/query_result.rs +++ b/scylla/src/transport/query_result.rs @@ -601,3 +601,299 @@ pub enum SingleRowError { #[derive(Debug, Error)] #[error("The request response was, unexpectedly, of Rows kind")] pub struct ResultNotRowsError; + +#[cfg(test)] +mod tests { + use assert_matches::assert_matches; + use bytes::{Bytes, BytesMut}; + use itertools::Itertools as _; + use scylla_cql::frame::response::result::ResultMetadata; + use scylla_cql::frame::types; + + use super::*; + + const TABLE_SPEC: TableSpec<'static> = TableSpec::borrowed("ks", "tbl"); + + fn column_spec_infinite_iter() -> impl Iterator> { + (0..).map(|k| { + ColumnSpec::owned( + format!("col_{}", k), + match k % 3 { + 0 => ColumnType::Ascii, + 1 => ColumnType::Boolean, + 2 => ColumnType::Float, + _ => unreachable!(), + }, + TABLE_SPEC, + ) + }) + } + + #[test] + fn test_query_result() { + fn serialize_cells(cells: impl IntoIterator>>) -> Bytes { + let mut bytes = BytesMut::new(); + for cell in cells { + types::write_bytes_opt(cell, &mut bytes).unwrap(); + } + bytes.freeze() + } + + fn sample_result_metadata(cols: usize) -> ResultMetadata<'static> { + ResultMetadata::new_for_test(cols, column_spec_infinite_iter().take(cols).collect()) + } + + fn sample_raw_rows(cols: usize, rows: usize) -> RawMetadataAndRawRows { + let metadata = sample_result_metadata(cols); + + static STRING: &[u8] = "MOCK".as_bytes(); + static BOOLEAN: &[u8] = &(true as i8).to_be_bytes(); + static FLOAT: &[u8] = &12341_i32.to_be_bytes(); + let cells = metadata.col_specs().iter().map(|spec| match spec.typ() { + ColumnType::Ascii => STRING, + ColumnType::Boolean => BOOLEAN, + ColumnType::Float => FLOAT, + _ => unreachable!(), + }); + let bytes = serialize_cells(cells.map(Some)); + RawMetadataAndRawRows::new_for_test(None, Some(metadata), false, rows, &bytes).unwrap() + } + + // Used to trigger DeserializationError. + fn sample_raw_rows_invalid_bytes(cols: usize, rows: usize) -> RawMetadataAndRawRows { + let metadata = sample_result_metadata(cols); + + RawMetadataAndRawRows::new_for_test(None, Some(metadata), false, rows, &[]).unwrap() + } + + // Check tracing ID + for tracing_id in [None, Some(Uuid::from_u128(0x_feed_dead))] { + for raw_rows in [None, Some(sample_raw_rows(7, 6))] { + let qr = QueryResult::new(raw_rows, tracing_id, vec![]); + assert_eq!(qr.tracing_id(), tracing_id); + } + } + + // Check warnings + for raw_rows in [None, Some(sample_raw_rows(7, 6))] { + let warnings = &["Ooops", "Meltdown..."]; + let qr = QueryResult::new( + raw_rows, + None, + warnings.iter().copied().map(String::from).collect(), + ); + assert_eq!(qr.warnings().collect_vec(), warnings); + } + + // Check col specs + { + // Not RESULT::Rows response -> no column specs + { + let rqr = QueryResult::new(None, None, Vec::new()); + let qr = rqr.rows_deserializer().unwrap(); + assert_matches!(qr, None); + } + + // RESULT::Rows response -> some column specs + { + let n = 5; + let metadata = sample_result_metadata(n); + let rr = RawMetadataAndRawRows::new_for_test(None, Some(metadata), false, 0, &[]) + .unwrap(); + let rqr = QueryResult::new(Some(rr), None, Vec::new()); + let qr = rqr.rows_deserializer().unwrap().unwrap(); + let column_specs = qr.column_specs(); + assert_eq!(column_specs.len(), n); + + // By index + { + for (i, expected_col_spec) in column_spec_infinite_iter().enumerate().take(n) { + let expected_view = + ColumnSpecView::new_from_column_spec(&expected_col_spec); + assert_eq!(column_specs.get_by_index(i), Some(expected_view)); + } + + assert_matches!(column_specs.get_by_index(n), None); + } + + // By name + { + for (idx, expected_col_spec) in column_spec_infinite_iter().enumerate().take(n) + { + let name = expected_col_spec.name(); + let expected_view = + ColumnSpecView::new_from_column_spec(&expected_col_spec); + assert_eq!(column_specs.get_by_name(name), Some((idx, expected_view))); + } + + assert_matches!(column_specs.get_by_name("ala ma kota"), None); + } + + // By iter + { + for (got_view, expected_col_spec) in + column_specs.iter().zip(column_spec_infinite_iter()) + { + let expected_view = + ColumnSpecView::new_from_column_spec(&expected_col_spec); + assert_eq!(got_view, expected_view); + } + } + } + } + + // rows(), maybe_rows(), result_not_rows(), first_row(), maybe_first_row(), single_row() + // All errors are checked. + { + // Not RESULT::Rows + { + let rqr = QueryResult::new(None, None, Vec::new()); + let qr = rqr.rows_deserializer().unwrap(); + assert_matches!(qr, None); + } + + // RESULT::Rows with 0 rows + { + let rr = sample_raw_rows(1, 0); + let rqr = QueryResult::new(Some(rr), None, Vec::new()); + let qr = rqr.rows_deserializer().unwrap().unwrap(); + + assert_matches!(rqr.result_not_rows(), Err(ResultNotRowsError)); + + // Type check error + { + assert_matches!(qr.rows::<(i32,)>(), Err(RowsError::TypeCheckFailed(_))); + + assert_matches!( + qr.first_row::<(i32,)>(), + Err(FirstRowError::TypeCheckFailed(_)) + ); + assert_matches!( + qr.maybe_first_row::<(i32,)>(), + Err(MaybeFirstRowError::TypeCheckFailed(_)) + ); + + assert_matches!( + qr.single_row::<(i32,)>(), + Err(SingleRowError::TypeCheckFailed(_)) + ); + } + + // Correct type + { + assert_matches!(qr.rows::<(&str,)>(), Ok(_)); + + assert_matches!(qr.first_row::<(&str,)>(), Err(FirstRowError::RowsEmpty)); + assert_matches!(qr.maybe_first_row::<(&str,)>(), Ok(None)); + + assert_matches!( + qr.single_row::<(&str,)>(), + Err(SingleRowError::UnexpectedRowCount(0)) + ); + } + } + + // RESULT::Rows with 1 row + { + let rr_good_data = sample_raw_rows(2, 1); + let rr_bad_data = sample_raw_rows_invalid_bytes(2, 1); + let rqr_good_data = QueryResult::new(Some(rr_good_data), None, Vec::new()); + let qr_good_data = rqr_good_data.rows_deserializer().unwrap().unwrap(); + let rqr_bad_data = QueryResult::new(Some(rr_bad_data), None, Vec::new()); + let qr_bad_data = rqr_bad_data.rows_deserializer().unwrap().unwrap(); + + for rqr in [&rqr_good_data, &rqr_bad_data] { + assert_matches!(rqr.result_not_rows(), Err(ResultNotRowsError)); + } + + for qr in [&qr_good_data, &qr_bad_data] { + // Type check error + { + assert_matches!( + qr.rows::<(i32, i32)>(), + Err(RowsError::TypeCheckFailed(_)) + ); + + assert_matches!( + qr.first_row::<(i32, i32)>(), + Err(FirstRowError::TypeCheckFailed(_)) + ); + assert_matches!( + qr.maybe_first_row::<(i32, i32)>(), + Err(MaybeFirstRowError::TypeCheckFailed(_)) + ); + + assert_matches!( + qr.single_row::<(i32, i32)>(), + Err(SingleRowError::TypeCheckFailed(_)) + ); + } + } + + // Correct type + { + assert_matches!(qr_good_data.rows::<(&str, bool)>(), Ok(_)); + assert_matches!(qr_bad_data.rows::<(&str, bool)>(), Ok(_)); + + assert_matches!(qr_good_data.first_row::<(&str, bool)>(), Ok(_)); + assert_matches!( + qr_bad_data.first_row::<(&str, bool)>(), + Err(FirstRowError::DeserializationFailed(_)) + ); + assert_matches!(qr_good_data.maybe_first_row::<(&str, bool)>(), Ok(_)); + assert_matches!( + qr_bad_data.maybe_first_row::<(&str, bool)>(), + Err(MaybeFirstRowError::DeserializationFailed(_)) + ); + + assert_matches!(qr_good_data.single_row::<(&str, bool)>(), Ok(_)); + assert_matches!( + qr_bad_data.single_row::<(&str, bool)>(), + Err(SingleRowError::DeserializationFailed(_)) + ); + } + } + + // RESULT::Rows with 2 rows + { + let rr = sample_raw_rows(2, 2); + let rqr = QueryResult::new(Some(rr), None, Vec::new()); + let qr = rqr.rows_deserializer().unwrap().unwrap(); + + assert_matches!(rqr.result_not_rows(), Err(ResultNotRowsError)); + + // Type check error + { + assert_matches!(qr.rows::<(i32, i32)>(), Err(RowsError::TypeCheckFailed(_))); + + assert_matches!( + qr.first_row::<(i32, i32)>(), + Err(FirstRowError::TypeCheckFailed(_)) + ); + assert_matches!( + qr.maybe_first_row::<(i32, i32)>(), + Err(MaybeFirstRowError::TypeCheckFailed(_)) + ); + + assert_matches!( + qr.single_row::<(i32, i32)>(), + Err(SingleRowError::TypeCheckFailed(_)) + ); + } + + // Correct type + { + assert_matches!(qr.rows::<(&str, bool)>(), Ok(_)); + + assert_matches!(qr.first_row::<(&str, bool)>(), Ok(_)); + assert_matches!(qr.maybe_first_row::<(&str, bool)>(), Ok(_)); + + assert_matches!( + qr.single_row::<(&str, bool)>(), + Err(SingleRowError::UnexpectedRowCount(2)) + ); + } + } + } + } +} From f2556483e893f23ce14b0a55506e05db13c5e8b9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Wojciech=20Przytu=C5=82a?= Date: Fri, 11 Oct 2024 10:32:48 +0200 Subject: [PATCH 09/42] session,iterator: no longer record rows size & count After the deserialization refactor with lazy result metadata deserialization, we will no longer have access to rows serialized size and rows count in Session and in RowIteratorWorker. Therefore, traces can no longer record that information. A further commit in this PR brings back serialized size to the span, but there is an important change: now the size is of both raw metadata and raw rows; before, metadata was not accounted. --- scylla/src/transport/iterator.rs | 2 -- scylla/src/transport/session.rs | 17 +---------------- 2 files changed, 1 insertion(+), 18 deletions(-) diff --git a/scylla/src/transport/iterator.rs b/scylla/src/transport/iterator.rs index 7ae248d80..89ca911e8 100644 --- a/scylla/src/transport/iterator.rs +++ b/scylla/src/transport/iterator.rs @@ -675,8 +675,6 @@ where let paging_state_response = rows.paging_state_response.take(); - request_span.record_rows_fields(&rows); - let received_page = ReceivedPage { rows, tracing_id }; // Send next page to RowIterator diff --git a/scylla/src/transport/session.rs b/scylla/src/transport/session.rs index 38dbb30a9..a12dcbd64 100644 --- a/scylla/src/transport/session.rs +++ b/scylla/src/transport/session.rs @@ -17,7 +17,7 @@ use async_trait::async_trait; use futures::future::join_all; use futures::future::try_join_all; use itertools::{Either, Itertools}; -use scylla_cql::frame::response::result::{deser_cql_value, ColumnSpec, Rows}; +use scylla_cql::frame::response::result::{deser_cql_value, ColumnSpec}; use scylla_cql::frame::response::NonErrorResponse; use scylla_cql::types::serialize::batch::BatchValues; use scylla_cql::types::serialize::row::{SerializeRow, SerializedValues}; @@ -826,7 +826,6 @@ impl Session { self.handle_auto_await_schema_agreement(&response).await?; let (result, paging_state) = response.into_query_result_and_paging_state()?; - span.record_result_fields(&result); Ok((result, paging_state)) } @@ -1261,7 +1260,6 @@ impl Session { self.handle_auto_await_schema_agreement(&response).await?; let (result, paging_state) = response.into_query_result_and_paging_state()?; - span.record_result_fields(&result); Ok((result, paging_state)) } @@ -1457,7 +1455,6 @@ impl Session { RunQueryResult::IgnoredWriteError => LegacyQueryResult::mock_empty(), RunQueryResult::Completed(response) => response, }; - span.record_result_fields(&result); Ok(result) } @@ -2176,18 +2173,6 @@ impl RequestSpan { } } - pub(crate) fn record_result_fields(&self, result: &LegacyQueryResult) { - self.span.record("result_size", result.serialized_size); - if let Some(rows) = result.rows.as_ref() { - self.span.record("result_rows", rows.len()); - } - } - - pub(crate) fn record_rows_fields(&self, rows: &Rows) { - self.span.record("result_size", rows.serialized_size); - self.span.record("result_rows", rows.rows.len()); - } - pub(crate) fn record_replicas<'a>(&'a self, replicas: &'a [(impl Borrow>, Shard)]) { struct ReplicaIps<'a, N>(&'a [(N, Shard)]); impl<'a, N> Display for ReplicaIps<'a, N> From 691619d33cfc48cc5b161f7f7411d75eec340457 Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Mon, 13 Mar 2023 15:42:41 +0100 Subject: [PATCH 10/42] treewide: propagate RawRows/new QueryResult MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Now, `result::deser_rows` returns RawRows instead of Rows, postponing any actual deserialization of response contents to a later time. The RawRows are pushed down the call stack, converted to the new QueryResult at some point and only converted to LegacyQueryResult where the API requires it. Co-authored-by: Wojciech Przytuła --- scylla-cql/src/frame/response/result.rs | 91 +++++++++++---------- scylla/src/transport/connection.rs | 51 +++++------- scylla/src/transport/iterator.rs | 45 +++++----- scylla/src/transport/legacy_query_result.rs | 12 ++- scylla/src/transport/query_result.rs | 4 +- scylla/src/transport/session.rs | 7 +- 6 files changed, 105 insertions(+), 105 deletions(-) diff --git a/scylla-cql/src/frame/response/result.rs b/scylla-cql/src/frame/response/result.rs index bf2ee9786..44de5ea38 100644 --- a/scylla-cql/src/frame/response/result.rs +++ b/scylla-cql/src/frame/response/result.rs @@ -623,6 +623,26 @@ pub struct RawMetadataAndRawRows { cached_metadata: Option>>, } +impl RawMetadataAndRawRows { + // Preferred to implementing Default, because users shouldn't be encouraged to create + // empty RawMetadataAndRawRows. + #[inline] + pub fn mock_empty() -> Self { + // Minimal correct `raw_metadata_and_rows` looks like this: + // Empty metadata (0 bytes), rows_count=0 (i32 big endian), empty rows (0 bytes). + static EMPTY_METADATA_ZERO_ROWS: &[u8] = &0_i32.to_be_bytes(); + let raw_metadata_and_rows = Bytes::from_static(EMPTY_METADATA_ZERO_ROWS); + + Self { + col_count: 0, + global_tables_spec: false, + no_metadata: false, + raw_metadata_and_rows, + cached_metadata: None, + } + } +} + /// RESULT:Rows response, in partially serialized form. /// /// Paging state and metadata are deserialized, rows remain serialized. @@ -773,7 +793,7 @@ impl<'frame> DeserializedMetadataAndRawRows<'frame, RawRowsBorrowed<'frame>> { #[derive(Debug)] pub struct Rows { - pub metadata: Arc>, + pub metadata: ResultMetadataHolder<'static>, pub paging_state_response: PagingStateResponse, pub rows_count: usize, pub rows: Vec, @@ -784,7 +804,7 @@ pub struct Rows { #[derive(Debug)] pub enum Result { Void, - Rows(Rows), + Rows((RawMetadataAndRawRows, PagingStateResponse)), SetKeyspace(SetKeyspace), Prepared(Prepared), SchemaChange(SchemaChange), @@ -1209,6 +1229,29 @@ impl RawMetadataAndRawRows { deser_col_specs_owned, ) } + + pub fn into_legacy_rows( + self, + paging_state_response: PagingStateResponse, + ) -> StdResult { + let raw_rows_with_deserialized_metadata = self.deserialize_owned_metadata()?; + + let rows_size = raw_rows_with_deserialized_metadata.rows_bytes_size(); + let rows_count = raw_rows_with_deserialized_metadata.rows_count(); + let rows = raw_rows_with_deserialized_metadata + .rows_iter::()? + .collect::>()?; + + let metadata = raw_rows_with_deserialized_metadata.into_metadata(); + + Ok(Rows { + metadata, + paging_state_response, + rows_count, + rows, + serialized_size: rows_size, + }) + } } fn deser_prepared_metadata( @@ -1412,46 +1455,10 @@ pub fn deser_cql_value( fn deser_rows( buf_bytes: Bytes, cached_metadata: Option<&Arc>>, -) -> StdResult { - let buf = &mut &*buf_bytes; - let (server_metadata, paging_state_response) = deser_result_metadata(buf)?; - - let metadata = match cached_metadata { - Some(cached) => Arc::clone(cached), - None => { - // No cached_metadata provided. Server is supposed to provide the result metadata. - if server_metadata.col_count != server_metadata.col_specs.len() { - return Err(RowsParseError::ColumnCountMismatch { - col_count: server_metadata.col_count, - col_specs_count: server_metadata.col_specs.len(), - }); - } - Arc::new(server_metadata) - } - }; - - let original_size = buf.len(); - - let rows_count: usize = - types::read_int_length(buf).map_err(RowsParseError::RowsCountParseError)?; - - let raw_rows_iter = RowIterator::new( - rows_count, - &metadata.col_specs, - FrameSlice::new_borrowed(buf), - ); - let rows_iter = TypedRowIterator::::new(raw_rows_iter) - .map_err(|err| DeserializationError::new(err.0))?; - - let rows = rows_iter.collect::>()?; - - Ok(Rows { - metadata, - paging_state_response, - rows_count, - rows, - serialized_size: original_size - buf.len(), - }) +) -> StdResult<(RawMetadataAndRawRows, PagingStateResponse), RowsParseError> { + let mut frame_slice = FrameSlice::new(&buf_bytes); + RawMetadataAndRawRows::deserialize(&mut frame_slice, cached_metadata.cloned()) + .map_err(Into::into) } fn deser_set_keyspace(buf: &mut &[u8]) -> StdResult { diff --git a/scylla/src/transport/connection.rs b/scylla/src/transport/connection.rs index b4c76e547..359cc9664 100644 --- a/scylla/src/transport/connection.rs +++ b/scylla/src/transport/connection.rs @@ -49,6 +49,7 @@ use std::{ use super::errors::{ProtocolError, UseKeyspaceProtocolError}; use super::iterator::RowIterator; use super::locator::tablets::{RawTablet, TabletParsingError}; +use super::query_result::QueryResult; use super::session::AddressTranslator; use super::topology::{PeerEndpoint, UntranslatedEndpoint, UntranslatedPeer}; use super::NodeAddr; @@ -70,10 +71,6 @@ use crate::statement::prepared_statement::PreparedStatement; use crate::statement::{Consistency, PageSize, PagingState, PagingStateResponse}; use crate::transport::Compression; -// Existing code imports scylla::transport::connection::LegacyQueryResult because it used to be located in this file. -// Reexport LegacyQueryResult to avoid breaking the existing code. -use crate::LegacyQueryResult; - // Queries for schema agreement const LOCAL_VERSION: &str = "SELECT schema_version FROM system.local WHERE key='local'"; @@ -243,12 +240,12 @@ impl QueryResponse { pub(crate) fn into_query_result_and_paging_state( self, - ) -> Result<(LegacyQueryResult, PagingStateResponse), UserRequestError> { + ) -> Result<(QueryResult, PagingStateResponse), UserRequestError> { self.into_non_error_query_response()? .into_query_result_and_paging_state() } - pub(crate) fn into_query_result(self) -> Result { + pub(crate) fn into_query_result(self) -> Result { self.into_non_error_query_response()?.into_query_result() } } @@ -270,15 +267,12 @@ impl NonErrorQueryResponse { pub(crate) fn into_query_result_and_paging_state( self, - ) -> Result<(LegacyQueryResult, PagingStateResponse), UserRequestError> { - let (rows, paging_state, metadata, serialized_size) = match self.response { - NonErrorResponse::Result(result::Result::Rows(rs)) => ( - Some(rs.rows), - rs.paging_state_response, - Some(rs.metadata), - rs.serialized_size, - ), - NonErrorResponse::Result(_) => (None, PagingStateResponse::NoMorePages, None, 0), + ) -> Result<(QueryResult, PagingStateResponse), UserRequestError> { + let (raw_rows, paging_state_response) = match self.response { + NonErrorResponse::Result(result::Result::Rows((rs, paging_state_response))) => { + (Some(rs), paging_state_response) + } + NonErrorResponse::Result(_) => (None, PagingStateResponse::NoMorePages), _ => { return Err(UserRequestError::UnexpectedResponse( self.response.to_response_kind(), @@ -287,18 +281,12 @@ impl NonErrorQueryResponse { }; Ok(( - LegacyQueryResult { - rows, - warnings: self.warnings, - tracing_id: self.tracing_id, - metadata, - serialized_size, - }, - paging_state, + QueryResult::new(raw_rows, self.tracing_id, self.warnings), + paging_state_response, )) } - pub(crate) fn into_query_result(self) -> Result { + pub(crate) fn into_query_result(self) -> Result { let (result, paging_state) = self.into_query_result_and_paging_state()?; if !paging_state.finished() { @@ -980,7 +968,7 @@ impl Connection { &self, query: impl Into, paging_state: PagingState, - ) -> Result<(LegacyQueryResult, PagingStateResponse), UserRequestError> { + ) -> Result<(QueryResult, PagingStateResponse), UserRequestError> { let query: Query = query.into(); // This method is used only for driver internal queries, so no need to consult execution profile here. @@ -1005,7 +993,7 @@ impl Connection { paging_state: PagingState, consistency: Consistency, serial_consistency: Option, - ) -> Result<(LegacyQueryResult, PagingStateResponse), UserRequestError> { + ) -> Result<(QueryResult, PagingStateResponse), UserRequestError> { let query: Query = query.into(); let page_size = query.get_validated_page_size(); @@ -1024,7 +1012,7 @@ impl Connection { pub(crate) async fn query_unpaged( &self, query: impl Into, - ) -> Result { + ) -> Result { // This method is used only for driver internal queries, so no need to consult execution profile here. let query: Query = query.into(); @@ -1084,7 +1072,7 @@ impl Connection { &self, prepared: &PreparedStatement, values: SerializedValues, - ) -> Result { + ) -> Result { // This method is used only for driver internal queries, so no need to consult execution profile here. self.execute_raw_unpaged(prepared, values) .await @@ -1231,7 +1219,7 @@ impl Connection { &self, batch: &Batch, values: impl BatchValues, - ) -> Result { + ) -> Result { self.batch_with_consistency( batch, values, @@ -1249,7 +1237,7 @@ impl Connection { values: impl BatchValues, consistency: Consistency, serial_consistency: Option, - ) -> Result { + ) -> Result { let batch = self.prepare_batch(init_batch, &values).await?; let contexts = batch.statements.iter().map(|bs| match bs { @@ -1446,6 +1434,7 @@ impl Connection { let (version_id,) = self .query_unpaged(LOCAL_VERSION) .await? + .into_legacy_result()? .single_row_typed() .map_err(ProtocolError::SchemaVersionFetch)?; Ok(version_id) @@ -2618,6 +2607,8 @@ mod tests { .query_unpaged("SELECT p, v FROM t") .await .unwrap() + .into_legacy_result() + .unwrap() .rows_typed::<(i32, Vec)>() .unwrap() .collect::, _>>() diff --git a/scylla/src/transport/iterator.rs b/scylla/src/transport/iterator.rs index 89ca911e8..775f6e506 100644 --- a/scylla/src/transport/iterator.rs +++ b/scylla/src/transport/iterator.rs @@ -9,6 +9,8 @@ use std::sync::Arc; use std::task::{Context, Poll}; use futures::Stream; +use scylla_cql::frame::request::query::PagingStateResponse; +use scylla_cql::frame::response::result::RawMetadataAndRawRows; use scylla_cql::frame::response::NonErrorResponse; use scylla_cql::types::serialize::row::SerializedValues; use std::result::Result; @@ -46,7 +48,7 @@ pub struct RowIterator { } struct ReceivedPage { - rows: Rows, + rows: RawMetadataAndRawRows, tracing_id: Option, } @@ -69,7 +71,17 @@ impl Stream for RowIterator { if s.is_current_page_exhausted() { match Pin::new(&mut s.page_receiver).poll_recv(cx) { Poll::Ready(Some(Ok(received_page))) => { - s.current_page = received_page.rows; + let rows = match received_page + .rows + // As RowIteratorWorker manages paging itself, the paging state response + // returned to the user is always NoMorePages. It used to be so before + // the deserialization refactor, too. + .into_legacy_rows(PagingStateResponse::NoMorePages) + { + Ok(rows) => rows, + Err(err) => return Poll::Ready(Some(Err(err.into()))), + }; + s.current_page = rows; s.current_row_idx = 0; if let Some(tracing_id) = received_page.tracing_id { @@ -377,10 +389,13 @@ impl RowIterator { // - That future is polled in a tokio::task which isn't going to be // cancelled let pages_received = receiver.recv().await.unwrap()?; + let rows = pages_received + .rows + .into_legacy_rows(PagingStateResponse::NoMorePages)?; Ok(RowIterator { current_row_idx: 0, - current_page: pages_received.rows, + current_page: rows, page_receiver: receiver, tracing_ids: if let Some(tracing_id) = pages_received.tracing_id { vec![tracing_id] @@ -408,11 +423,8 @@ impl RowIterator { // A separate module is used here so that the parent module cannot construct // SendAttemptedProof directly. mod checked_channel_sender { - use scylla_cql::frame::{ - request::query::PagingStateResponse, - response::result::{ResultMetadata, Rows}, - }; - use std::{marker::PhantomData, sync::Arc}; + use scylla_cql::frame::response::result::RawMetadataAndRawRows; + use std::marker::PhantomData; use tokio::sync::mpsc; use uuid::Uuid; @@ -454,13 +466,7 @@ mod checked_channel_sender { Result<(), mpsc::error::SendError>, ) { let empty_page = ReceivedPage { - rows: Rows { - metadata: Arc::new(ResultMetadata::mock_empty()), - paging_state_response: PagingStateResponse::NoMorePages, - rows_count: 0, - rows: Vec::new(), - serialized_size: 0, - }, + rows: RawMetadataAndRawRows::mock_empty(), tracing_id, }; self.send(Ok(empty_page)).await @@ -662,7 +668,8 @@ where match query_response { Ok(NonErrorQueryResponse { - response: NonErrorResponse::Result(result::Result::Rows(mut rows)), + response: + NonErrorResponse::Result(result::Result::Rows((rows, paging_state_response))), tracing_id, .. }) => { @@ -673,8 +680,6 @@ where .load_balancing_policy .on_query_success(&self.statement_info, elapsed, node); - let paging_state_response = rows.paging_state_response.take(); - let received_page = ReceivedPage { rows, tracing_id }; // Send next page to RowIterator @@ -842,9 +847,7 @@ where let result = (self.fetcher)(paging_state).await?; let response = result.into_non_error_query_response()?; match response.response { - NonErrorResponse::Result(result::Result::Rows(mut rows)) => { - let paging_state_response = rows.paging_state_response.take(); - + NonErrorResponse::Result(result::Result::Rows((rows, paging_state_response))) => { let (proof, send_result) = self .sender .send(Ok(ReceivedPage { diff --git a/scylla/src/transport/legacy_query_result.rs b/scylla/src/transport/legacy_query_result.rs index a08765d93..833c95c23 100644 --- a/scylla/src/transport/legacy_query_result.rs +++ b/scylla/src/transport/legacy_query_result.rs @@ -1,10 +1,8 @@ -use std::sync::Arc; - use crate::frame::response::cql_to_rust::{FromRow, FromRowError}; use crate::frame::response::result::ColumnSpec; use crate::frame::response::result::Row; use crate::transport::session::{IntoTypedRows, TypedRowIter}; -use scylla_cql::frame::response::result::ResultMetadata; +use scylla_cql::frame::response::result::ResultMetadataHolder; use thiserror::Error; use uuid::Uuid; @@ -21,7 +19,7 @@ pub struct LegacyQueryResult { /// CQL Tracing uuid - can only be Some if tracing is enabled for this query pub tracing_id: Option, /// Metadata returned along with this response. - pub(crate) metadata: Option>>, + pub(crate) metadata: Option>, /// The original size of the serialized rows in request pub serialized_size: usize, } @@ -287,7 +285,7 @@ mod tests { use std::convert::TryInto; use assert_matches::assert_matches; - use scylla_cql::frame::response::result::{ColumnType, TableSpec}; + use scylla_cql::frame::response::result::{ColumnType, ResultMetadata, TableSpec}; // Returns specified number of rows, each one containing one int32 value. // Values are 0, 1, 2, 3, 4, ... @@ -335,14 +333,14 @@ mod tests { fn make_rows_query_result(rows_num: usize) -> LegacyQueryResult { let mut res = make_not_rows_query_result(); res.rows = Some(make_rows(rows_num)); - res.metadata = Some(Arc::new(make_test_metadata())); + res.metadata = Some(ResultMetadataHolder::Owned(make_test_metadata())); res } fn make_string_rows_query_result(rows_num: usize) -> LegacyQueryResult { let mut res = make_not_rows_query_result(); res.rows = Some(make_string_rows(rows_num)); - res.metadata = Some(Arc::new(make_test_metadata())); + res.metadata = Some(ResultMetadataHolder::Owned(make_test_metadata())); res } diff --git a/scylla/src/transport/query_result.rs b/scylla/src/transport/query_result.rs index 816bc7e7e..3f7249b31 100644 --- a/scylla/src/transport/query_result.rs +++ b/scylla/src/transport/query_result.rs @@ -1,6 +1,4 @@ use std::fmt::Debug; -use std::sync::Arc; - use thiserror::Error; use uuid::Uuid; @@ -315,7 +313,7 @@ impl QueryResult { rows: Some(deserialized_rows), warnings: self.warnings, tracing_id: self.tracing_id, - metadata: Some(Arc::new((*metadata).clone())), + metadata: Some(metadata), serialized_size, }) } else { diff --git a/scylla/src/transport/session.rs b/scylla/src/transport/session.rs index a12dcbd64..2fcb0f1db 100644 --- a/scylla/src/transport/session.rs +++ b/scylla/src/transport/session.rs @@ -66,6 +66,7 @@ use crate::transport::legacy_query_result::LegacyQueryResult; use crate::transport::load_balancing::{self, RoutingInfo}; use crate::transport::metrics::Metrics; use crate::transport::node::Node; +use crate::transport::query_result::QueryResult; use crate::transport::retry_policy::{QueryInfo, RetryDecision, RetrySession}; use crate::transport::speculative_execution; use crate::transport::Compression; @@ -826,6 +827,7 @@ impl Session { self.handle_auto_await_schema_agreement(&response).await?; let (result, paging_state) = response.into_query_result_and_paging_state()?; + let result = result.into_legacy_result()?; Ok((result, paging_state)) } @@ -1260,6 +1262,7 @@ impl Session { self.handle_auto_await_schema_agreement(&response).await?; let (result, paging_state) = response.into_query_result_and_paging_state()?; + let result = result.into_legacy_result()?; Ok((result, paging_state)) } @@ -1453,7 +1456,7 @@ impl Session { let result = match run_query_result { RunQueryResult::IgnoredWriteError => LegacyQueryResult::mock_empty(), - RunQueryResult::Completed(response) => response, + RunQueryResult::Completed(response) => response.into_legacy_result()?, }; Ok(result) } @@ -2017,7 +2020,7 @@ impl Session { pub(crate) trait AllowedRunQueryResTType {} impl AllowedRunQueryResTType for Uuid {} -impl AllowedRunQueryResTType for LegacyQueryResult {} +impl AllowedRunQueryResTType for QueryResult {} impl AllowedRunQueryResTType for NonErrorQueryResponse {} struct ExecuteQueryContext<'a> { From cae545c12a79fc28d6b78031a3b329c5dd16a5ab Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Mon, 13 Mar 2023 14:47:48 +0100 Subject: [PATCH 11/42] iterator: rename (Typed)RowIterator to Legacy(Typed)RowIterator MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Like in the case of LegacyQueryResult, TypedRowIterator will be replaced with a new one, and the old one preserved for compatibility reasons. As a first step, it is renamed to LegacyTypedRowIterator. Similarly, RowIterator is renamed to LegacyRowIterator to mark that it is the part of the old API. There won't be a new RowIterator, though. Co-authored-by: Wojciech Przytuła --- examples/tracing.rs | 4 +-- scylla/src/transport/caching_session.rs | 4 +-- scylla/src/transport/connection.rs | 17 +++++++----- scylla/src/transport/iterator.rs | 36 ++++++++++++------------- scylla/src/transport/session.rs | 17 ++++++------ 5 files changed, 41 insertions(+), 37 deletions(-) diff --git a/examples/tracing.rs b/examples/tracing.rs index a50e8f7e0..12767de5b 100644 --- a/examples/tracing.rs +++ b/examples/tracing.rs @@ -8,7 +8,7 @@ use scylla::statement::{ prepared_statement::PreparedStatement, query::Query, Consistency, SerialConsistency, }; use scylla::tracing::TracingInfo; -use scylla::transport::iterator::RowIterator; +use scylla::transport::iterator::LegacyRowIterator; use scylla::LegacyQueryResult; use scylla::{Session, SessionBuilder}; use std::env; @@ -86,7 +86,7 @@ async fn main() -> Result<()> { // It's also possible to trace paged queries like query_iter or execute_iter // After iterating through all rows iterator.get_tracing_ids() will give tracing ids // for all page queries - let mut row_iterator: RowIterator = session.query_iter(query, &[]).await?; + let mut row_iterator: LegacyRowIterator = session.query_iter(query, &[]).await?; while let Some(_row) = row_iterator.next().await { // Receive rows diff --git a/scylla/src/transport/caching_session.rs b/scylla/src/transport/caching_session.rs index b920337fe..cbf9d3c6d 100644 --- a/scylla/src/transport/caching_session.rs +++ b/scylla/src/transport/caching_session.rs @@ -3,7 +3,7 @@ use crate::prepared_statement::PreparedStatement; use crate::query::Query; use crate::statement::{PagingState, PagingStateResponse}; use crate::transport::errors::QueryError; -use crate::transport::iterator::RowIterator; +use crate::transport::iterator::LegacyRowIterator; use crate::transport::partitioner::PartitionerName; use crate::{LegacyQueryResult, Session}; use bytes::Bytes; @@ -86,7 +86,7 @@ where &self, query: impl Into, values: impl SerializeRow, - ) -> Result { + ) -> Result { let query = query.into(); let prepared = self.add_prepared_statement_owned(query).await?; self.session.execute_iter(prepared, values).await diff --git a/scylla/src/transport/connection.rs b/scylla/src/transport/connection.rs index 359cc9664..1a7fce16d 100644 --- a/scylla/src/transport/connection.rs +++ b/scylla/src/transport/connection.rs @@ -47,7 +47,7 @@ use std::{ }; use super::errors::{ProtocolError, UseKeyspaceProtocolError}; -use super::iterator::RowIterator; +use super::iterator::LegacyRowIterator; use super::locator::tablets::{RawTablet, TabletParsingError}; use super::query_result::QueryResult; use super::session::AddressTranslator; @@ -1182,14 +1182,19 @@ impl Connection { pub(crate) async fn query_iter( self: Arc, query: Query, - ) -> Result { + ) -> Result { let consistency = query .config .determine_consistency(self.config.default_consistency); let serial_consistency = query.config.serial_consistency.flatten(); - RowIterator::new_for_connection_query_iter(query, self, consistency, serial_consistency) - .await + LegacyRowIterator::new_for_connection_query_iter( + query, + self, + consistency, + serial_consistency, + ) + .await } /// Executes a prepared statements and fetches its results over multiple pages, using @@ -1198,13 +1203,13 @@ impl Connection { self: Arc, prepared_statement: PreparedStatement, values: SerializedValues, - ) -> Result { + ) -> Result { let consistency = prepared_statement .config .determine_consistency(self.config.default_consistency); let serial_consistency = prepared_statement.config.serial_consistency.flatten(); - RowIterator::new_for_connection_execute_iter( + LegacyRowIterator::new_for_connection_execute_iter( prepared_statement, values, self, diff --git a/scylla/src/transport/iterator.rs b/scylla/src/transport/iterator.rs index 775f6e506..e4f868781 100644 --- a/scylla/src/transport/iterator.rs +++ b/scylla/src/transport/iterator.rs @@ -40,7 +40,7 @@ use uuid::Uuid; /// Iterator over rows returned by paged queries\ /// Allows to easily access rows without worrying about handling multiple pages -pub struct RowIterator { +pub struct LegacyRowIterator { current_row_idx: usize, current_page: Rows, page_receiver: mpsc::Receiver>, @@ -62,7 +62,7 @@ pub(crate) struct PreparedIteratorConfig { /// Fetching pages is asynchronous so `RowIterator` does not implement the `Iterator` trait.\ /// Instead it uses the asynchronous `Stream` trait -impl Stream for RowIterator { +impl Stream for LegacyRowIterator { type Item = Result; fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { @@ -108,10 +108,10 @@ impl Stream for RowIterator { } } -impl RowIterator { +impl LegacyRowIterator { /// Converts this iterator into an iterator over rows parsed as given type - pub fn into_typed(self) -> TypedRowIterator { - TypedRowIterator { + pub fn into_typed(self) -> LegacyTypedRowIterator { + LegacyTypedRowIterator { row_iterator: self, phantom_data: Default::default(), } @@ -122,7 +122,7 @@ impl RowIterator { execution_profile: Arc, cluster_data: Arc, metrics: Arc, - ) -> Result { + ) -> Result { let (sender, receiver) = mpsc::channel(1); let consistency = query @@ -201,7 +201,7 @@ impl RowIterator { pub(crate) async fn new_for_prepared_statement( config: PreparedIteratorConfig, - ) -> Result { + ) -> Result { let (sender, receiver) = mpsc::channel(1); let consistency = config @@ -322,7 +322,7 @@ impl RowIterator { connection: Arc, consistency: Consistency, serial_consistency: Option, - ) -> Result { + ) -> Result { let (sender, receiver) = mpsc::channel::>(1); let page_size = query.get_validated_page_size(); @@ -352,7 +352,7 @@ impl RowIterator { connection: Arc, consistency: Consistency, serial_consistency: Option, - ) -> Result { + ) -> Result { let (sender, receiver) = mpsc::channel::>(1); let page_size = prepared.get_validated_page_size(); @@ -380,7 +380,7 @@ impl RowIterator { async fn new_from_worker_future( worker_task: impl Future + Send + 'static, mut receiver: mpsc::Receiver>, - ) -> Result { + ) -> Result { tokio::task::spawn(worker_task); // This unwrap is safe because: @@ -393,7 +393,7 @@ impl RowIterator { .rows .into_legacy_rows(PagingStateResponse::NoMorePages)?; - Ok(RowIterator { + Ok(Self { current_row_idx: 0, current_page: rows, page_receiver: receiver, @@ -893,12 +893,12 @@ where /// Iterator over rows returned by paged queries /// where each row is parsed as the given type\ /// Returned by `RowIterator::into_typed` -pub struct TypedRowIterator { - row_iterator: RowIterator, +pub struct LegacyTypedRowIterator { + row_iterator: LegacyRowIterator, phantom_data: std::marker::PhantomData, } -impl TypedRowIterator { +impl LegacyTypedRowIterator { /// If tracing was enabled returns tracing ids of all finished page queries pub fn get_tracing_ids(&self) -> &[Uuid] { self.row_iterator.get_tracing_ids() @@ -922,9 +922,9 @@ pub enum NextRowError { FromRowError(#[from] FromRowError), } -/// Fetching pages is asynchronous so `TypedRowIterator` does not implement the `Iterator` trait.\ +/// Fetching pages is asynchronous so `LegacyTypedRowIterator` does not implement the `Iterator` trait.\ /// Instead it uses the asynchronous `Stream` trait -impl Stream for TypedRowIterator { +impl Stream for LegacyTypedRowIterator { type Item = Result; fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { @@ -946,5 +946,5 @@ impl Stream for TypedRowIterator { } } -// TypedRowIterator can be moved freely for any RowT so it's Unpin -impl Unpin for TypedRowIterator {} +// LegacyTypedRowIterator can be moved freely for any RowT so it's Unpin +impl Unpin for LegacyTypedRowIterator {} diff --git a/scylla/src/transport/session.rs b/scylla/src/transport/session.rs index 2fcb0f1db..d9b19617b 100644 --- a/scylla/src/transport/session.rs +++ b/scylla/src/transport/session.rs @@ -42,7 +42,7 @@ use super::connection::QueryResponse; use super::connection::SslConfig; use super::errors::TracingProtocolError; use super::execution_profile::{ExecutionProfile, ExecutionProfileHandle, ExecutionProfileInner}; -use super::legacy_query_result::MaybeFirstRowTypedError; +use super::legacy_query_result::{LegacyQueryResult, MaybeFirstRowTypedError}; #[cfg(feature = "cloud")] use super::node::CloudEndpoint; use super::node::{InternalKnownNode, KnownNode}; @@ -61,8 +61,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; -use crate::transport::iterator::{PreparedIteratorConfig, RowIterator}; -use crate::transport::legacy_query_result::LegacyQueryResult; +use crate::transport::iterator::{LegacyRowIterator, PreparedIteratorConfig}; use crate::transport::load_balancing::{self, RoutingInfo}; use crate::transport::metrics::Metrics; use crate::transport::node::Node; @@ -907,7 +906,7 @@ impl Session { &self, query: impl Into, values: impl SerializeRow, - ) -> Result { + ) -> Result { let query: Query = query.into(); let execution_profile = query @@ -916,7 +915,7 @@ impl Session { .access(); if values.is_empty() { - RowIterator::new_for_query( + LegacyRowIterator::new_for_query( query, execution_profile, self.cluster.get_data(), @@ -924,12 +923,12 @@ impl Session { ) .await } else { - // Making RowIterator::new_for_query work with values is too hard (if even possible) + // Making LegacyRowIterator::new_for_query work with values is too hard (if even possible) // so instead of sending one prepare to a specific connection on each iterator query, // we fully prepare a statement beforehand. let prepared = self.prepare(query).await?; let values = prepared.serialize_values(&values)?; - RowIterator::new_for_prepared_statement(PreparedIteratorConfig { + LegacyRowIterator::new_for_prepared_statement(PreparedIteratorConfig { prepared, values, execution_profile, @@ -1310,7 +1309,7 @@ impl Session { &self, prepared: impl Into, values: impl SerializeRow, - ) -> Result { + ) -> Result { let prepared = prepared.into(); let serialized_values = prepared.serialize_values(&values)?; @@ -1319,7 +1318,7 @@ impl Session { .unwrap_or_else(|| self.get_default_execution_profile_handle()) .access(); - RowIterator::new_for_prepared_statement(PreparedIteratorConfig { + LegacyRowIterator::new_for_prepared_statement(PreparedIteratorConfig { prepared, values: serialized_values, execution_profile, From e2d791f11742c1157d85de42d3fb7ae3552656de Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Tue, 28 Feb 2023 17:58:48 +0100 Subject: [PATCH 12/42] iterator: introduce poll_next_internal MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit In order to make further commits easier on the eyes, contents of the LegacyRowIterator::poll_next method are moved to the new poll_next_internal method. Co-authored-by: Wojciech Przytuła --- scylla/src/transport/iterator.rs | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/scylla/src/transport/iterator.rs b/scylla/src/transport/iterator.rs index e4f868781..0413d5bce 100644 --- a/scylla/src/transport/iterator.rs +++ b/scylla/src/transport/iterator.rs @@ -65,7 +65,16 @@ pub(crate) struct PreparedIteratorConfig { impl Stream for LegacyRowIterator { type Item = Result; - fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + self.poll_next_internal(cx) + } +} + +impl LegacyRowIterator { + fn poll_next_internal( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll>> { let mut s = self.as_mut(); if s.is_current_page_exhausted() { @@ -106,9 +115,7 @@ impl Stream for LegacyRowIterator { cx.waker().wake_by_ref(); Poll::Pending } -} -impl LegacyRowIterator { /// Converts this iterator into an iterator over rows parsed as given type pub fn into_typed(self) -> LegacyTypedRowIterator { LegacyTypedRowIterator { From fd905775c2928f8442e6d7597dfce2f483a44639 Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Mon, 13 Mar 2023 16:19:52 +0100 Subject: [PATCH 13/42] iterator: introduce ready_some_ok! macro MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit The iterator module has a small number of functions which return Poll>>, call other functions that return such a stacked type and are support to continue if value obtained is not of form Ready(Some(Ok(x))). This requires a multiline pattern match instruction, which is basically boilerplate. This commit introduces the ready_some_ok! macro which hides the aforementioned boilerplate. Conceptually, it is similar to the existing std::task::ready! macro. Co-authored-by: Wojciech Przytuła --- scylla/src/transport/iterator.rs | 68 ++++++++++++++++---------------- 1 file changed, 33 insertions(+), 35 deletions(-) diff --git a/scylla/src/transport/iterator.rs b/scylla/src/transport/iterator.rs index 0413d5bce..e8d1ce1f0 100644 --- a/scylla/src/transport/iterator.rs +++ b/scylla/src/transport/iterator.rs @@ -38,6 +38,20 @@ use crate::transport::NodeRef; use tracing::{trace, trace_span, warn, Instrument}; use uuid::Uuid; +// Like std::task::ready!, but handles the whole stack of Poll>>. +// If it matches Poll::Ready(Some(Ok(_))), then it returns the innermost value, +// otherwise it returns from the surrounding function. +macro_rules! ready_some_ok { + ($e:expr) => { + match $e { + Poll::Ready(Some(Ok(x))) => x, + Poll::Ready(Some(Err(err))) => return Poll::Ready(Some(Err(err.into()))), + Poll::Ready(None) => return Poll::Ready(None), + Poll::Pending => return Poll::Pending, + } + }; +} + /// Iterator over rows returned by paged queries\ /// Allows to easily access rows without worrying about handling multiple pages pub struct LegacyRowIterator { @@ -78,28 +92,22 @@ impl LegacyRowIterator { let mut s = self.as_mut(); if s.is_current_page_exhausted() { - match Pin::new(&mut s.page_receiver).poll_recv(cx) { - Poll::Ready(Some(Ok(received_page))) => { - let rows = match received_page - .rows - // As RowIteratorWorker manages paging itself, the paging state response - // returned to the user is always NoMorePages. It used to be so before - // the deserialization refactor, too. - .into_legacy_rows(PagingStateResponse::NoMorePages) - { - Ok(rows) => rows, - Err(err) => return Poll::Ready(Some(Err(err.into()))), - }; - s.current_page = rows; - s.current_row_idx = 0; - - if let Some(tracing_id) = received_page.tracing_id { - s.tracing_ids.push(tracing_id); - } - } - Poll::Ready(Some(Err(err))) => return Poll::Ready(Some(Err(err))), - Poll::Ready(None) => return Poll::Ready(None), - Poll::Pending => return Poll::Pending, + let received_page = ready_some_ok!(Pin::new(&mut s.page_receiver).poll_recv(cx)); + let rows = match received_page + .rows + // As RowIteratorWorker manages paging itself, the paging state response + // returned to the user is always NoMorePages. It used to be so before + // the deserialization refactor, too. + .into_legacy_rows(PagingStateResponse::NoMorePages) + { + Ok(rows) => rows, + Err(err) => return Poll::Ready(Some(Err(err.into()))), + }; + s.current_page = rows; + s.current_row_idx = 0; + + if let Some(tracing_id) = received_page.tracing_id { + s.tracing_ids.push(tracing_id); } } @@ -937,19 +945,9 @@ impl Stream for LegacyTypedRowIterator { fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let mut s = self.as_mut(); - let next_elem: Option> = - match Pin::new(&mut s.row_iterator).poll_next(cx) { - Poll::Ready(next_elem) => next_elem, - Poll::Pending => return Poll::Pending, - }; - - let next_ready: Option = match next_elem { - Some(Ok(next_row)) => Some(RowT::from_row(next_row).map_err(|e| e.into())), - Some(Err(e)) => Some(Err(e.into())), - None => None, - }; - - Poll::Ready(next_ready) + let next_row = ready_some_ok!(Pin::new(&mut s.row_iterator).poll_next(cx)); + let typed_row_res = RowT::from_row(next_row).map_err(|e| e.into()); + Poll::Ready(Some(typed_row_res)) } } From 1fa8213a5d1bc89a8df555a2409c243dfd31909c Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Mon, 13 Mar 2023 15:31:12 +0100 Subject: [PATCH 14/42] iterator: introduce poll_next_page MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This commit refactors the part responsible for acquiring the next page by the LegacyRowIterator to a different function. This change is a preparation necessary to support the new deserialization interface - there will be a method that can return deserialized type that borrows from the current iterator, and - for "lifetimes reasons" - acquiring the next page must be put into a separate method. Co-authored-by: Wojciech Przytuła --- scylla/src/transport/iterator.rs | 57 ++++++++++++++++++++------------ 1 file changed, 36 insertions(+), 21 deletions(-) diff --git a/scylla/src/transport/iterator.rs b/scylla/src/transport/iterator.rs index e8d1ce1f0..92423e02f 100644 --- a/scylla/src/transport/iterator.rs +++ b/scylla/src/transport/iterator.rs @@ -89,41 +89,56 @@ impl LegacyRowIterator { mut self: Pin<&mut Self>, cx: &mut Context<'_>, ) -> Poll>> { - let mut s = self.as_mut(); - - if s.is_current_page_exhausted() { - let received_page = ready_some_ok!(Pin::new(&mut s.page_receiver).poll_recv(cx)); - let rows = match received_page - .rows - // As RowIteratorWorker manages paging itself, the paging state response - // returned to the user is always NoMorePages. It used to be so before - // the deserialization refactor, too. - .into_legacy_rows(PagingStateResponse::NoMorePages) - { - Ok(rows) => rows, - Err(err) => return Poll::Ready(Some(Err(err.into()))), - }; - s.current_page = rows; - s.current_row_idx = 0; - - if let Some(tracing_id) = received_page.tracing_id { - s.tracing_ids.push(tracing_id); - } + if self.as_ref().is_current_page_exhausted() { + ready_some_ok!(self.as_mut().poll_next_page(cx)); } + let mut s = self.as_mut(); + let idx = s.current_row_idx; if idx < s.current_page.rows.len() { let row = mem::take(&mut s.current_page.rows[idx]); s.current_row_idx += 1; return Poll::Ready(Some(Ok(row))); } - // We probably got a zero-sized page // Yield, but tell that we are ready cx.waker().wake_by_ref(); Poll::Pending } + /// Makes an attempt to acquire the next page (which may be empty). + /// + /// On success, returns Some(Ok()). + /// On failure, returns Some(Err()). + /// If there are no more pages, returns None. + fn poll_next_page<'r>( + mut self: Pin<&'r mut Self>, + cx: &mut Context<'_>, + ) -> Poll>> { + let mut s = self.as_mut(); + + let received_page = ready_some_ok!(Pin::new(&mut s.page_receiver).poll_recv(cx)); + let rows = match received_page + .rows + // As RowIteratorWorker manages paging itself, the paging state response + // returned to the user is always NoMorePages. It used to be so before + // the deserialization refactor, too. + .into_legacy_rows(PagingStateResponse::NoMorePages) + { + Ok(rows) => rows, + Err(err) => return Poll::Ready(Some(Err(err.into()))), + }; + s.current_page = rows; + s.current_row_idx = 0; + + if let Some(tracing_id) = received_page.tracing_id { + s.tracing_ids.push(tracing_id); + } + + Poll::Ready(Some(Ok(()))) + } + /// Converts this iterator into an iterator over rows parsed as given type pub fn into_typed(self) -> LegacyTypedRowIterator { LegacyTypedRowIterator { From 920244ea49ec000d78744c788d6fd8688d3be9db Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Wojciech=20Przytu=C5=82a?= Date: Tue, 13 Aug 2024 17:21:36 +0200 Subject: [PATCH 15/42] iterator: reorder code for better grouping Now, worker-related code comes strictly before iterator/stream-related code. This locality aid readability. --- scylla/src/transport/iterator.rs | 1452 +++++++++++++++--------------- 1 file changed, 726 insertions(+), 726 deletions(-) diff --git a/scylla/src/transport/iterator.rs b/scylla/src/transport/iterator.rs index 92423e02f..232e0a943 100644 --- a/scylla/src/transport/iterator.rs +++ b/scylla/src/transport/iterator.rs @@ -52,15 +52,6 @@ macro_rules! ready_some_ok { }; } -/// Iterator over rows returned by paged queries\ -/// Allows to easily access rows without worrying about handling multiple pages -pub struct LegacyRowIterator { - current_row_idx: usize, - current_page: Rows, - page_receiver: mpsc::Receiver>, - tracing_ids: Vec, -} - struct ReceivedPage { rows: RawMetadataAndRawRows, tracing_id: Option, @@ -74,849 +65,858 @@ pub(crate) struct PreparedIteratorConfig { pub(crate) metrics: Arc, } -/// Fetching pages is asynchronous so `RowIterator` does not implement the `Iterator` trait.\ -/// Instead it uses the asynchronous `Stream` trait -impl Stream for LegacyRowIterator { - type Item = Result; +// A separate module is used here so that the parent module cannot construct +// SendAttemptedProof directly. +mod checked_channel_sender { + use scylla_cql::frame::response::result::RawMetadataAndRawRows; + use std::marker::PhantomData; + use tokio::sync::mpsc; + use uuid::Uuid; - fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - self.poll_next_internal(cx) - } -} + use crate::transport::errors::QueryError; -impl LegacyRowIterator { - fn poll_next_internal( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll>> { - if self.as_ref().is_current_page_exhausted() { - ready_some_ok!(self.as_mut().poll_next_page(cx)); - } + use super::ReceivedPage; - let mut s = self.as_mut(); + /// A value whose existence proves that there was an attempt + /// to send an item of type T through a channel. + /// Can only be constructed by ProvingSender::send. + pub(crate) struct SendAttemptedProof(PhantomData); - let idx = s.current_row_idx; - if idx < s.current_page.rows.len() { - let row = mem::take(&mut s.current_page.rows[idx]); - s.current_row_idx += 1; - return Poll::Ready(Some(Ok(row))); + /// An mpsc::Sender which returns proofs that it attempted to send items. + pub(crate) struct ProvingSender(mpsc::Sender); + + impl From> for ProvingSender { + fn from(s: mpsc::Sender) -> Self { + Self(s) } - // We probably got a zero-sized page - // Yield, but tell that we are ready - cx.waker().wake_by_ref(); - Poll::Pending } - /// Makes an attempt to acquire the next page (which may be empty). - /// - /// On success, returns Some(Ok()). - /// On failure, returns Some(Err()). - /// If there are no more pages, returns None. - fn poll_next_page<'r>( - mut self: Pin<&'r mut Self>, - cx: &mut Context<'_>, - ) -> Poll>> { - let mut s = self.as_mut(); - - let received_page = ready_some_ok!(Pin::new(&mut s.page_receiver).poll_recv(cx)); - let rows = match received_page - .rows - // As RowIteratorWorker manages paging itself, the paging state response - // returned to the user is always NoMorePages. It used to be so before - // the deserialization refactor, too. - .into_legacy_rows(PagingStateResponse::NoMorePages) - { - Ok(rows) => rows, - Err(err) => return Poll::Ready(Some(Err(err.into()))), - }; - s.current_page = rows; - s.current_row_idx = 0; - - if let Some(tracing_id) = received_page.tracing_id { - s.tracing_ids.push(tracing_id); + impl ProvingSender { + pub(crate) async fn send( + &self, + value: T, + ) -> (SendAttemptedProof, Result<(), mpsc::error::SendError>) { + (SendAttemptedProof(PhantomData), self.0.send(value).await) } - - Poll::Ready(Some(Ok(()))) } - /// Converts this iterator into an iterator over rows parsed as given type - pub fn into_typed(self) -> LegacyTypedRowIterator { - LegacyTypedRowIterator { - row_iterator: self, - phantom_data: Default::default(), + type ResultPage = Result; + + impl ProvingSender { + pub(crate) async fn send_empty_page( + &self, + tracing_id: Option, + ) -> ( + SendAttemptedProof, + Result<(), mpsc::error::SendError>, + ) { + let empty_page = ReceivedPage { + rows: RawMetadataAndRawRows::mock_empty(), + tracing_id, + }; + self.send(Ok(empty_page)).await } } +} - pub(crate) async fn new_for_query( - query: Query, - execution_profile: Arc, - cluster_data: Arc, - metrics: Arc, - ) -> Result { - let (sender, receiver) = mpsc::channel(1); +use checked_channel_sender::{ProvingSender, SendAttemptedProof}; - let consistency = query - .config - .consistency - .unwrap_or(execution_profile.consistency); - let serial_consistency = query - .config - .serial_consistency - .unwrap_or(execution_profile.serial_consistency); +type PageSendAttemptedProof = SendAttemptedProof>; - let page_size = query.get_validated_page_size(); +// RowIteratorWorker works in the background to fetch pages +// RowIterator receives them through a channel +struct RowIteratorWorker<'a, QueryFunc, SpanCreatorFunc> { + sender: ProvingSender>, - let routing_info = RoutingInfo { - consistency, - serial_consistency, - ..Default::default() - }; + // Closure used to perform a single page query + // AsyncFn(Arc, Option>) -> Result + page_query: QueryFunc, - let retry_session = query - .get_retry_policy() - .map(|rp| &**rp) - .unwrap_or(&*execution_profile.retry_policy) - .new_session(); + statement_info: RoutingInfo<'a>, + query_is_idempotent: bool, + query_consistency: Consistency, + retry_session: Box, + execution_profile: Arc, + metrics: Arc, - let parent_span = tracing::Span::current(); - let worker_task = async move { - let query_ref = &query; + paging_state: PagingState, - let page_query = |connection: Arc, - consistency: Consistency, - paging_state: PagingState| { - async move { - connection - .query_raw_with_consistency( - query_ref, - consistency, - serial_consistency, - Some(page_size), - paging_state, - ) - .await - } - }; + history_listener: Option>, + current_query_id: Option, + current_attempt_id: Option, - let query_ref = &query; + parent_span: tracing::Span, + span_creator: SpanCreatorFunc, +} - let span_creator = move || { - let span = RequestSpan::new_query(&query_ref.contents); - span.record_request_size(0); - span - }; +impl RowIteratorWorker<'_, QueryFunc, SpanCreator> +where + QueryFunc: Fn(Arc, Consistency, PagingState) -> QueryFut, + QueryFut: Future>, + SpanCreator: Fn() -> RequestSpan, +{ + // Contract: this function MUST send at least one item through self.sender + async fn work(mut self, cluster_data: Arc) -> PageSendAttemptedProof { + let load_balancer = self.execution_profile.load_balancing_policy.clone(); + let statement_info = self.statement_info.clone(); + let query_plan = + load_balancing::Plan::new(load_balancer.as_ref(), &statement_info, &cluster_data); - let worker = RowIteratorWorker { - sender: sender.into(), - page_query, - statement_info: routing_info, - query_is_idempotent: query.config.is_idempotent, - query_consistency: consistency, - retry_session, - execution_profile, - metrics, - paging_state: PagingState::start(), - history_listener: query.config.history_listener.clone(), - current_query_id: None, - current_attempt_id: None, - parent_span, - span_creator, - }; + let mut last_error: QueryError = QueryError::EmptyPlan; + let mut current_consistency: Consistency = self.query_consistency; - worker.work(cluster_data).await - }; + self.log_query_start(); - Self::new_from_worker_future(worker_task, receiver).await - } + 'nodes_in_plan: for (node, shard) in query_plan { + let span = + trace_span!(parent: &self.parent_span, "Executing query", node = %node.address); + // For each node in the plan choose a connection to use + // This connection will be reused for same node retries to preserve paging cache on the shard + let connection: Arc = match node + .connection_for_shard(shard) + .instrument(span.clone()) + .await + { + Ok(connection) => connection, + Err(e) => { + trace!( + parent: &span, + error = %e, + "Choosing connection failed" + ); + last_error = e.into(); + // Broken connection doesn't count as a failed query, don't log in metrics + continue 'nodes_in_plan; + } + }; - pub(crate) async fn new_for_prepared_statement( - config: PreparedIteratorConfig, - ) -> Result { - let (sender, receiver) = mpsc::channel(1); + 'same_node_retries: loop { + trace!(parent: &span, "Execution started"); + // Query pages until an error occurs + let queries_result: Result = self + .query_pages(&connection, current_consistency, node) + .instrument(span.clone()) + .await; - let consistency = config - .prepared - .config - .consistency - .unwrap_or(config.execution_profile.consistency); - let serial_consistency = config - .prepared - .config - .serial_consistency - .unwrap_or(config.execution_profile.serial_consistency); - - let page_size = config.prepared.get_validated_page_size(); - - let retry_session = config - .prepared - .get_retry_policy() - .map(|rp| &**rp) - .unwrap_or(&*config.execution_profile.retry_policy) - .new_session(); - - let parent_span = tracing::Span::current(); - let worker_task = async move { - let prepared_ref = &config.prepared; - let values_ref = &config.values; - - let (partition_key, token) = match prepared_ref - .extract_partition_key_and_calculate_token( - prepared_ref.get_partitioner_name(), - values_ref, - ) { - Ok(res) => res.unzip(), - Err(err) => { - let (proof, _res) = ProvingSender::from(sender).send(Err(err)).await; - return proof; - } - }; - - let table_spec = config.prepared.get_table_spec(); - let statement_info = RoutingInfo { - consistency, - serial_consistency, - token, - table: table_spec, - is_confirmed_lwt: config.prepared.is_confirmed_lwt(), - }; - - let page_query = |connection: Arc, - consistency: Consistency, - paging_state: PagingState| async move { - connection - .execute_raw_with_consistency( - prepared_ref, - values_ref, - consistency, - serial_consistency, - Some(page_size), - paging_state, - ) - .await - }; - - let serialized_values_size = config.values.buffer_size(); + last_error = match queries_result { + Ok(proof) => { + trace!(parent: &span, "Query succeeded"); + // query_pages returned Ok, so we are guaranteed + // that it attempted to send at least one page + // through self.sender and we can safely return now. + return proof; + } + Err(error) => { + trace!( + parent: &span, + error = %error, + "Query failed" + ); + error + } + }; - let replicas: Option> = - if let (Some(table_spec), Some(token)) = - (statement_info.table, statement_info.token) - { - Some( - config - .cluster_data - .get_token_endpoints_iter(table_spec, token) - .map(|(node, shard)| (node.clone(), shard)) - .collect(), - ) - } else { - None + // Use retry policy to decide what to do next + let query_info = QueryInfo { + error: &last_error, + is_idempotent: self.query_is_idempotent, + consistency: self.query_consistency, }; - let span_creator = move || { - let span = RequestSpan::new_prepared( - partition_key.as_ref().map(|pk| pk.iter()), - token, - serialized_values_size, + let retry_decision = self.retry_session.decide_should_retry(query_info); + trace!( + parent: &span, + retry_decision = format!("{:?}", retry_decision).as_str() ); - if let Some(replicas) = replicas.as_ref() { - span.record_replicas(replicas); - } - span - }; - - let worker = RowIteratorWorker { - sender: sender.into(), - page_query, - statement_info, - query_is_idempotent: config.prepared.config.is_idempotent, - query_consistency: consistency, - retry_session, - execution_profile: config.execution_profile, - metrics: config.metrics, - paging_state: PagingState::start(), - history_listener: config.prepared.config.history_listener.clone(), - current_query_id: None, - current_attempt_id: None, - parent_span, - span_creator, - }; + self.log_attempt_error(&last_error, &retry_decision); + match retry_decision { + RetryDecision::RetrySameNode(cl) => { + self.metrics.inc_retries_num(); + current_consistency = cl.unwrap_or(current_consistency); + continue 'same_node_retries; + } + RetryDecision::RetryNextNode(cl) => { + self.metrics.inc_retries_num(); + current_consistency = cl.unwrap_or(current_consistency); + continue 'nodes_in_plan; + } + RetryDecision::DontRetry => break 'nodes_in_plan, + RetryDecision::IgnoreWriteError => { + warn!("Ignoring error during fetching pages; stopping fetching."); + // If we are here then, most likely, we didn't send + // anything through the self.sender channel. + // Although we are in an awkward situation (_iter + // interface isn't meant for sending writes), + // we must attempt to send something because + // the iterator expects it. + let (proof, _) = self.sender.send_empty_page(None).await; + return proof; + } + }; + } + } - worker.work(config.cluster_data).await - }; + // Send last_error to RowIterator - query failed fully + self.log_query_error(&last_error); + let (proof, _) = self.sender.send(Err(last_error)).await; + proof + } - Self::new_from_worker_future(worker_task, receiver).await + // Given a working connection query as many pages as possible until the first error. + // + // Contract: this function must either: + // - Return an error + // - Return Ok but have attempted to send a page via self.sender + async fn query_pages( + &mut self, + connection: &Arc, + consistency: Consistency, + node: NodeRef<'_>, + ) -> Result { + loop { + let request_span = (self.span_creator)(); + match self + .query_one_page(connection, consistency, node, &request_span) + .instrument(request_span.span().clone()) + .await? + { + ControlFlow::Break(proof) => return Ok(proof), + ControlFlow::Continue(_) => {} + } + } } - pub(crate) async fn new_for_connection_query_iter( - query: Query, - connection: Arc, + async fn query_one_page( + &mut self, + connection: &Arc, consistency: Consistency, - serial_consistency: Option, - ) -> Result { - let (sender, receiver) = mpsc::channel::>(1); + node: NodeRef<'_>, + request_span: &RequestSpan, + ) -> Result, QueryError> { + self.metrics.inc_total_paged_queries(); + let query_start = std::time::Instant::now(); - let page_size = query.get_validated_page_size(); + trace!( + connection = %connection.get_connect_address(), + "Sending" + ); + self.log_attempt_start(connection.get_connect_address()); - let worker_task = async move { - let worker = SingleConnectionRowIteratorWorker { - sender: sender.into(), - fetcher: |paging_state| { - connection.query_raw_with_consistency( - &query, - consistency, - serial_consistency, - Some(page_size), - paging_state, - ) - }, - }; - worker.work().await - }; + let query_response = + (self.page_query)(connection.clone(), consistency, self.paging_state.clone()) + .await + .and_then(QueryResponse::into_non_error_query_response); - Self::new_from_worker_future(worker_task, receiver).await - } + let elapsed = query_start.elapsed(); - pub(crate) async fn new_for_connection_execute_iter( - prepared: PreparedStatement, - values: SerializedValues, - connection: Arc, - consistency: Consistency, - serial_consistency: Option, - ) -> Result { - let (sender, receiver) = mpsc::channel::>(1); + request_span.record_shard_id(connection); - let page_size = prepared.get_validated_page_size(); + match query_response { + Ok(NonErrorQueryResponse { + response: + NonErrorResponse::Result(result::Result::Rows((rows, paging_state_response))), + tracing_id, + .. + }) => { + let _ = self.metrics.log_query_latency(elapsed.as_millis() as u64); + self.log_attempt_success(); + self.log_query_success(); + self.execution_profile + .load_balancing_policy + .on_query_success(&self.statement_info, elapsed, node); - let worker_task = async move { - let worker = SingleConnectionRowIteratorWorker { - sender: sender.into(), - fetcher: |paging_state| { - connection.execute_raw_with_consistency( - &prepared, - &values, - consistency, - serial_consistency, - Some(page_size), - paging_state, - ) - }, - }; - worker.work().await - }; + let received_page = ReceivedPage { rows, tracing_id }; - Self::new_from_worker_future(worker_task, receiver).await - } + // Send next page to RowIterator + let (proof, res) = self.sender.send(Ok(received_page)).await; + if res.is_err() { + // channel was closed, RowIterator was dropped - should shutdown + return Ok(ControlFlow::Break(proof)); + } - async fn new_from_worker_future( - worker_task: impl Future + Send + 'static, - mut receiver: mpsc::Receiver>, - ) -> Result { - tokio::task::spawn(worker_task); + match paging_state_response.into_paging_control_flow() { + ControlFlow::Continue(paging_state) => { + self.paging_state = paging_state; + } + ControlFlow::Break(()) => { + // Reached the last query, shutdown + return Ok(ControlFlow::Break(proof)); + } + } - // This unwrap is safe because: - // - The future returned by worker.work sends at least one item - // to the channel (the PageSendAttemptedProof helps enforce this) - // - That future is polled in a tokio::task which isn't going to be - // cancelled - let pages_received = receiver.recv().await.unwrap()?; - let rows = pages_received - .rows - .into_legacy_rows(PagingStateResponse::NoMorePages)?; + // Query succeeded, reset retry policy for future retries + self.retry_session.reset(); + self.log_query_start(); - Ok(Self { - current_row_idx: 0, - current_page: rows, - page_receiver: receiver, - tracing_ids: if let Some(tracing_id) = pages_received.tracing_id { - vec![tracing_id] - } else { - Vec::new() - }, - }) - } + Ok(ControlFlow::Continue(())) + } + Err(err) => { + let err = err.into(); + self.metrics.inc_failed_paged_queries(); + self.execution_profile + .load_balancing_policy + .on_query_failure(&self.statement_info, elapsed, node, &err); + Err(err) + } + Ok(NonErrorQueryResponse { + response: NonErrorResponse::Result(_), + tracing_id, + .. + }) => { + // We have most probably sent a modification statement (e.g. INSERT or UPDATE), + // so let's return an empty iterator as suggested in #631. - /// If tracing was enabled returns tracing ids of all finished page queries - pub fn get_tracing_ids(&self) -> &[Uuid] { - &self.tracing_ids + // We must attempt to send something because the iterator expects it. + let (proof, _) = self.sender.send_empty_page(tracing_id).await; + Ok(ControlFlow::Break(proof)) + } + Ok(response) => { + self.metrics.inc_failed_paged_queries(); + let err = + ProtocolError::UnexpectedResponse(response.response.to_response_kind()).into(); + self.execution_profile + .load_balancing_policy + .on_query_failure(&self.statement_info, elapsed, node, &err); + Err(err) + } + } } - /// Returns specification of row columns - pub fn get_column_specs(&self) -> &[ColumnSpec<'static>] { - self.current_page.metadata.col_specs() - } + fn log_query_start(&mut self) { + let history_listener: &dyn HistoryListener = match &self.history_listener { + Some(hl) => &**hl, + None => return, + }; - fn is_current_page_exhausted(&self) -> bool { - self.current_row_idx >= self.current_page.rows.len() + self.current_query_id = Some(history_listener.log_query_start()); } -} - -// A separate module is used here so that the parent module cannot construct -// SendAttemptedProof directly. -mod checked_channel_sender { - use scylla_cql::frame::response::result::RawMetadataAndRawRows; - use std::marker::PhantomData; - use tokio::sync::mpsc; - use uuid::Uuid; - use crate::transport::errors::QueryError; + fn log_query_success(&mut self) { + let history_listener: &dyn HistoryListener = match &self.history_listener { + Some(hl) => &**hl, + None => return, + }; - use super::ReceivedPage; + let query_id: history::QueryId = match &self.current_query_id { + Some(id) => *id, + None => return, + }; - /// A value whose existence proves that there was an attempt - /// to send an item of type T through a channel. - /// Can only be constructed by ProvingSender::send. - pub(crate) struct SendAttemptedProof(PhantomData); + history_listener.log_query_success(query_id); + } - /// An mpsc::Sender which returns proofs that it attempted to send items. - pub(crate) struct ProvingSender(mpsc::Sender); + fn log_query_error(&mut self, error: &QueryError) { + let history_listener: &dyn HistoryListener = match &self.history_listener { + Some(hl) => &**hl, + None => return, + }; - impl From> for ProvingSender { - fn from(s: mpsc::Sender) -> Self { - Self(s) - } - } + let query_id: history::QueryId = match &self.current_query_id { + Some(id) => *id, + None => return, + }; - impl ProvingSender { - pub(crate) async fn send( - &self, - value: T, - ) -> (SendAttemptedProof, Result<(), mpsc::error::SendError>) { - (SendAttemptedProof(PhantomData), self.0.send(value).await) - } + history_listener.log_query_error(query_id, error); } - type ResultPage = Result; + fn log_attempt_start(&mut self, node_addr: SocketAddr) { + let history_listener: &dyn HistoryListener = match &self.history_listener { + Some(hl) => &**hl, + None => return, + }; - impl ProvingSender { - pub(crate) async fn send_empty_page( - &self, - tracing_id: Option, - ) -> ( - SendAttemptedProof, - Result<(), mpsc::error::SendError>, - ) { - let empty_page = ReceivedPage { - rows: RawMetadataAndRawRows::mock_empty(), - tracing_id, - }; - self.send(Ok(empty_page)).await - } - } -} + let query_id: history::QueryId = match &self.current_query_id { + Some(id) => *id, + None => return, + }; -use checked_channel_sender::{ProvingSender, SendAttemptedProof}; + self.current_attempt_id = + Some(history_listener.log_attempt_start(query_id, None, node_addr)); + } -type PageSendAttemptedProof = SendAttemptedProof>; + fn log_attempt_success(&mut self) { + let history_listener: &dyn HistoryListener = match &self.history_listener { + Some(hl) => &**hl, + None => return, + }; -// RowIteratorWorker works in the background to fetch pages -// RowIterator receives them through a channel -struct RowIteratorWorker<'a, QueryFunc, SpanCreatorFunc> { - sender: ProvingSender>, + let attempt_id: history::AttemptId = match &self.current_attempt_id { + Some(id) => *id, + None => return, + }; - // Closure used to perform a single page query - // AsyncFn(Arc, Option>) -> Result - page_query: QueryFunc, + history_listener.log_attempt_success(attempt_id); + } - statement_info: RoutingInfo<'a>, - query_is_idempotent: bool, - query_consistency: Consistency, - retry_session: Box, - execution_profile: Arc, - metrics: Arc, + fn log_attempt_error(&mut self, error: &QueryError, retry_decision: &RetryDecision) { + let history_listener: &dyn HistoryListener = match &self.history_listener { + Some(hl) => &**hl, + None => return, + }; - paging_state: PagingState, + let attempt_id: history::AttemptId = match &self.current_attempt_id { + Some(id) => *id, + None => return, + }; - history_listener: Option>, - current_query_id: Option, - current_attempt_id: Option, + history_listener.log_attempt_error(attempt_id, error, retry_decision); + } +} - parent_span: tracing::Span, - span_creator: SpanCreatorFunc, +/// A massively simplified version of the RowIteratorWorker. It does not have +/// any complicated logic related to retries, it just fetches pages from +/// a single connection. +struct SingleConnectionRowIteratorWorker { + sender: ProvingSender>, + fetcher: Fetcher, } -impl RowIteratorWorker<'_, QueryFunc, SpanCreator> +impl SingleConnectionRowIteratorWorker where - QueryFunc: Fn(Arc, Consistency, PagingState) -> QueryFut, - QueryFut: Future>, - SpanCreator: Fn() -> RequestSpan, + Fetcher: Fn(PagingState) -> FetchFut + Send + Sync, + FetchFut: Future> + Send, { - // Contract: this function MUST send at least one item through self.sender - async fn work(mut self, cluster_data: Arc) -> PageSendAttemptedProof { - let load_balancer = self.execution_profile.load_balancing_policy.clone(); - let statement_info = self.statement_info.clone(); - let query_plan = - load_balancing::Plan::new(load_balancer.as_ref(), &statement_info, &cluster_data); + async fn work(mut self) -> PageSendAttemptedProof { + match self.do_work().await { + Ok(proof) => proof, + Err(err) => { + let (proof, _) = self.sender.send(Err(err)).await; + proof + } + } + } - let mut last_error: QueryError = QueryError::EmptyPlan; - let mut current_consistency: Consistency = self.query_consistency; + async fn do_work(&mut self) -> Result { + let mut paging_state = PagingState::start(); + loop { + let result = (self.fetcher)(paging_state).await?; + let response = result.into_non_error_query_response()?; + match response.response { + NonErrorResponse::Result(result::Result::Rows((rows, paging_state_response))) => { + let (proof, send_result) = self + .sender + .send(Ok(ReceivedPage { + rows, + tracing_id: response.tracing_id, + })) + .await; - self.log_query_start(); + if send_result.is_err() { + // channel was closed, RowIterator was dropped - should shutdown + return Ok(proof); + } - 'nodes_in_plan: for (node, shard) in query_plan { - let span = - trace_span!(parent: &self.parent_span, "Executing query", node = %node.address); - // For each node in the plan choose a connection to use - // This connection will be reused for same node retries to preserve paging cache on the shard - let connection: Arc = match node - .connection_for_shard(shard) - .instrument(span.clone()) - .await - { - Ok(connection) => connection, - Err(e) => { - trace!( - parent: &span, - error = %e, - "Choosing connection failed" - ); - last_error = e.into(); - // Broken connection doesn't count as a failed query, don't log in metrics - continue 'nodes_in_plan; + match paging_state_response.into_paging_control_flow() { + ControlFlow::Continue(new_paging_state) => { + paging_state = new_paging_state; + } + ControlFlow::Break(()) => { + // Reached the last query, shutdown + return Ok(proof); + } + } } - }; + NonErrorResponse::Result(_) => { + // We have most probably sent a modification statement (e.g. INSERT or UPDATE), + // so let's return an empty iterator as suggested in #631. - 'same_node_retries: loop { - trace!(parent: &span, "Execution started"); - // Query pages until an error occurs - let queries_result: Result = self - .query_pages(&connection, current_consistency, node) - .instrument(span.clone()) - .await; + // We must attempt to send something because the iterator expects it. + let (proof, _) = self.sender.send_empty_page(response.tracing_id).await; + return Ok(proof); + } + _ => { + return Err(ProtocolError::UnexpectedResponse( + response.response.to_response_kind(), + ) + .into()); + } + } + } + } +} - last_error = match queries_result { - Ok(proof) => { - trace!(parent: &span, "Query succeeded"); - // query_pages returned Ok, so we are guaranteed - // that it attempted to send at least one page - // through self.sender and we can safely return now. - return proof; - } - Err(error) => { - trace!( - parent: &span, - error = %error, - "Query failed" - ); - error - } - }; +/// Iterator over rows returned by paged queries\ +/// Allows to easily access rows without worrying about handling multiple pages +pub struct LegacyRowIterator { + current_row_idx: usize, + current_page: Rows, + page_receiver: mpsc::Receiver>, + tracing_ids: Vec, +} - // Use retry policy to decide what to do next - let query_info = QueryInfo { - error: &last_error, - is_idempotent: self.query_is_idempotent, - consistency: self.query_consistency, - }; +/// Fetching pages is asynchronous so `RowIterator` does not implement the `Iterator` trait.\ +/// Instead it uses the asynchronous `Stream` trait +impl Stream for LegacyRowIterator { + type Item = Result; - let retry_decision = self.retry_session.decide_should_retry(query_info); - trace!( - parent: &span, - retry_decision = format!("{:?}", retry_decision).as_str() - ); - self.log_attempt_error(&last_error, &retry_decision); - match retry_decision { - RetryDecision::RetrySameNode(cl) => { - self.metrics.inc_retries_num(); - current_consistency = cl.unwrap_or(current_consistency); - continue 'same_node_retries; - } - RetryDecision::RetryNextNode(cl) => { - self.metrics.inc_retries_num(); - current_consistency = cl.unwrap_or(current_consistency); - continue 'nodes_in_plan; - } - RetryDecision::DontRetry => break 'nodes_in_plan, - RetryDecision::IgnoreWriteError => { - warn!("Ignoring error during fetching pages; stopping fetching."); - // If we are here then, most likely, we didn't send - // anything through the self.sender channel. - // Although we are in an awkward situation (_iter - // interface isn't meant for sending writes), - // we must attempt to send something because - // the iterator expects it. - let (proof, _) = self.sender.send_empty_page(None).await; - return proof; - } - }; - } + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + self.poll_next_internal(cx) + } +} + +impl LegacyRowIterator { + fn poll_next_internal( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll>> { + if self.as_ref().is_current_page_exhausted() { + ready_some_ok!(self.as_mut().poll_next_page(cx)); + } + + let mut s = self.as_mut(); + + let idx = s.current_row_idx; + if idx < s.current_page.rows.len() { + let row = mem::take(&mut s.current_page.rows[idx]); + s.current_row_idx += 1; + return Poll::Ready(Some(Ok(row))); + } + // We probably got a zero-sized page + // Yield, but tell that we are ready + cx.waker().wake_by_ref(); + Poll::Pending + } + + /// Makes an attempt to acquire the next page (which may be empty). + /// + /// On success, returns Some(Ok()). + /// On failure, returns Some(Err()). + /// If there are no more pages, returns None. + fn poll_next_page<'r>( + mut self: Pin<&'r mut Self>, + cx: &mut Context<'_>, + ) -> Poll>> { + let mut s = self.as_mut(); + + let received_page = ready_some_ok!(Pin::new(&mut s.page_receiver).poll_recv(cx)); + let rows = match received_page + .rows + // As RowIteratorWorker manages paging itself, the paging state response + // returned to the user is always NoMorePages. It used to be so before + // the deserialization refactor, too. + .into_legacy_rows(PagingStateResponse::NoMorePages) + { + Ok(rows) => rows, + Err(err) => return Poll::Ready(Some(Err(err.into()))), + }; + s.current_page = rows; + s.current_row_idx = 0; + + if let Some(tracing_id) = received_page.tracing_id { + s.tracing_ids.push(tracing_id); } - // Send last_error to RowIterator - query failed fully - self.log_query_error(&last_error); - let (proof, _) = self.sender.send(Err(last_error)).await; - proof + Poll::Ready(Some(Ok(()))) } - // Given a working connection query as many pages as possible until the first error. - // - // Contract: this function must either: - // - Return an error - // - Return Ok but have attempted to send a page via self.sender - async fn query_pages( - &mut self, - connection: &Arc, - consistency: Consistency, - node: NodeRef<'_>, - ) -> Result { - loop { - let request_span = (self.span_creator)(); - match self - .query_one_page(connection, consistency, node, &request_span) - .instrument(request_span.span().clone()) - .await? - { - ControlFlow::Break(proof) => return Ok(proof), - ControlFlow::Continue(_) => {} - } + /// Converts this iterator into an iterator over rows parsed as given type + pub fn into_typed(self) -> LegacyTypedRowIterator { + LegacyTypedRowIterator { + row_iterator: self, + phantom_data: Default::default(), } } - async fn query_one_page( - &mut self, - connection: &Arc, - consistency: Consistency, - node: NodeRef<'_>, - request_span: &RequestSpan, - ) -> Result, QueryError> { - self.metrics.inc_total_paged_queries(); - let query_start = std::time::Instant::now(); - - trace!( - connection = %connection.get_connect_address(), - "Sending" - ); - self.log_attempt_start(connection.get_connect_address()); - - let query_response = - (self.page_query)(connection.clone(), consistency, self.paging_state.clone()) - .await - .and_then(QueryResponse::into_non_error_query_response); + pub(crate) async fn new_for_query( + query: Query, + execution_profile: Arc, + cluster_data: Arc, + metrics: Arc, + ) -> Result { + let (sender, receiver) = mpsc::channel(1); - let elapsed = query_start.elapsed(); + let consistency = query + .config + .consistency + .unwrap_or(execution_profile.consistency); + let serial_consistency = query + .config + .serial_consistency + .unwrap_or(execution_profile.serial_consistency); - request_span.record_shard_id(connection); + let page_size = query.get_validated_page_size(); - match query_response { - Ok(NonErrorQueryResponse { - response: - NonErrorResponse::Result(result::Result::Rows((rows, paging_state_response))), - tracing_id, - .. - }) => { - let _ = self.metrics.log_query_latency(elapsed.as_millis() as u64); - self.log_attempt_success(); - self.log_query_success(); - self.execution_profile - .load_balancing_policy - .on_query_success(&self.statement_info, elapsed, node); + let routing_info = RoutingInfo { + consistency, + serial_consistency, + ..Default::default() + }; - let received_page = ReceivedPage { rows, tracing_id }; + let retry_session = query + .get_retry_policy() + .map(|rp| &**rp) + .unwrap_or(&*execution_profile.retry_policy) + .new_session(); - // Send next page to RowIterator - let (proof, res) = self.sender.send(Ok(received_page)).await; - if res.is_err() { - // channel was closed, RowIterator was dropped - should shutdown - return Ok(ControlFlow::Break(proof)); - } + let parent_span = tracing::Span::current(); + let worker_task = async move { + let query_ref = &query; - match paging_state_response.into_paging_control_flow() { - ControlFlow::Continue(paging_state) => { - self.paging_state = paging_state; - } - ControlFlow::Break(()) => { - // Reached the last query, shutdown - return Ok(ControlFlow::Break(proof)); - } + let page_query = |connection: Arc, + consistency: Consistency, + paging_state: PagingState| { + async move { + connection + .query_raw_with_consistency( + query_ref, + consistency, + serial_consistency, + Some(page_size), + paging_state, + ) + .await } + }; - // Query succeeded, reset retry policy for future retries - self.retry_session.reset(); - self.log_query_start(); + let query_ref = &query; - Ok(ControlFlow::Continue(())) - } - Err(err) => { - let err = err.into(); - self.metrics.inc_failed_paged_queries(); - self.execution_profile - .load_balancing_policy - .on_query_failure(&self.statement_info, elapsed, node, &err); - Err(err) - } - Ok(NonErrorQueryResponse { - response: NonErrorResponse::Result(_), - tracing_id, - .. - }) => { - // We have most probably sent a modification statement (e.g. INSERT or UPDATE), - // so let's return an empty iterator as suggested in #631. + let span_creator = move || { + let span = RequestSpan::new_query(&query_ref.contents); + span.record_request_size(0); + span + }; - // We must attempt to send something because the iterator expects it. - let (proof, _) = self.sender.send_empty_page(tracing_id).await; - Ok(ControlFlow::Break(proof)) - } - Ok(response) => { - self.metrics.inc_failed_paged_queries(); - let err = - ProtocolError::UnexpectedResponse(response.response.to_response_kind()).into(); - self.execution_profile - .load_balancing_policy - .on_query_failure(&self.statement_info, elapsed, node, &err); - Err(err) - } - } - } + let worker = RowIteratorWorker { + sender: sender.into(), + page_query, + statement_info: routing_info, + query_is_idempotent: query.config.is_idempotent, + query_consistency: consistency, + retry_session, + execution_profile, + metrics, + paging_state: PagingState::start(), + history_listener: query.config.history_listener.clone(), + current_query_id: None, + current_attempt_id: None, + parent_span, + span_creator, + }; - fn log_query_start(&mut self) { - let history_listener: &dyn HistoryListener = match &self.history_listener { - Some(hl) => &**hl, - None => return, + worker.work(cluster_data).await }; - self.current_query_id = Some(history_listener.log_query_start()); + Self::new_from_worker_future(worker_task, receiver).await } - fn log_query_success(&mut self) { - let history_listener: &dyn HistoryListener = match &self.history_listener { - Some(hl) => &**hl, - None => return, - }; + pub(crate) async fn new_for_prepared_statement( + config: PreparedIteratorConfig, + ) -> Result { + let (sender, receiver) = mpsc::channel(1); + + let consistency = config + .prepared + .config + .consistency + .unwrap_or(config.execution_profile.consistency); + let serial_consistency = config + .prepared + .config + .serial_consistency + .unwrap_or(config.execution_profile.serial_consistency); + + let page_size = config.prepared.get_validated_page_size(); + + let retry_session = config + .prepared + .get_retry_policy() + .map(|rp| &**rp) + .unwrap_or(&*config.execution_profile.retry_policy) + .new_session(); + + let parent_span = tracing::Span::current(); + let worker_task = async move { + let prepared_ref = &config.prepared; + let values_ref = &config.values; + + let (partition_key, token) = match prepared_ref + .extract_partition_key_and_calculate_token( + prepared_ref.get_partitioner_name(), + values_ref, + ) { + Ok(res) => res.unzip(), + Err(err) => { + let (proof, _res) = ProvingSender::from(sender).send(Err(err)).await; + return proof; + } + }; + + let table_spec = config.prepared.get_table_spec(); + let statement_info = RoutingInfo { + consistency, + serial_consistency, + token, + table: table_spec, + is_confirmed_lwt: config.prepared.is_confirmed_lwt(), + }; + + let page_query = |connection: Arc, + consistency: Consistency, + paging_state: PagingState| async move { + connection + .execute_raw_with_consistency( + prepared_ref, + values_ref, + consistency, + serial_consistency, + Some(page_size), + paging_state, + ) + .await + }; + + let serialized_values_size = config.values.buffer_size(); - let query_id: history::QueryId = match &self.current_query_id { - Some(id) => *id, - None => return, - }; + let replicas: Option> = + if let (Some(table_spec), Some(token)) = + (statement_info.table, statement_info.token) + { + Some( + config + .cluster_data + .get_token_endpoints_iter(table_spec, token) + .map(|(node, shard)| (node.clone(), shard)) + .collect(), + ) + } else { + None + }; - history_listener.log_query_success(query_id); - } + let span_creator = move || { + let span = RequestSpan::new_prepared( + partition_key.as_ref().map(|pk| pk.iter()), + token, + serialized_values_size, + ); + if let Some(replicas) = replicas.as_ref() { + span.record_replicas(replicas); + } + span + }; - fn log_query_error(&mut self, error: &QueryError) { - let history_listener: &dyn HistoryListener = match &self.history_listener { - Some(hl) => &**hl, - None => return, - }; + let worker = RowIteratorWorker { + sender: sender.into(), + page_query, + statement_info, + query_is_idempotent: config.prepared.config.is_idempotent, + query_consistency: consistency, + retry_session, + execution_profile: config.execution_profile, + metrics: config.metrics, + paging_state: PagingState::start(), + history_listener: config.prepared.config.history_listener.clone(), + current_query_id: None, + current_attempt_id: None, + parent_span, + span_creator, + }; - let query_id: history::QueryId = match &self.current_query_id { - Some(id) => *id, - None => return, + worker.work(config.cluster_data).await }; - history_listener.log_query_error(query_id, error); + Self::new_from_worker_future(worker_task, receiver).await } - fn log_attempt_start(&mut self, node_addr: SocketAddr) { - let history_listener: &dyn HistoryListener = match &self.history_listener { - Some(hl) => &**hl, - None => return, - }; + pub(crate) async fn new_for_connection_query_iter( + query: Query, + connection: Arc, + consistency: Consistency, + serial_consistency: Option, + ) -> Result { + let (sender, receiver) = mpsc::channel::>(1); - let query_id: history::QueryId = match &self.current_query_id { - Some(id) => *id, - None => return, + let page_size = query.get_validated_page_size(); + + let worker_task = async move { + let worker = SingleConnectionRowIteratorWorker { + sender: sender.into(), + fetcher: |paging_state| { + connection.query_raw_with_consistency( + &query, + consistency, + serial_consistency, + Some(page_size), + paging_state, + ) + }, + }; + worker.work().await }; - self.current_attempt_id = - Some(history_listener.log_attempt_start(query_id, None, node_addr)); + Self::new_from_worker_future(worker_task, receiver).await } - fn log_attempt_success(&mut self) { - let history_listener: &dyn HistoryListener = match &self.history_listener { - Some(hl) => &**hl, - None => return, - }; + pub(crate) async fn new_for_connection_execute_iter( + prepared: PreparedStatement, + values: SerializedValues, + connection: Arc, + consistency: Consistency, + serial_consistency: Option, + ) -> Result { + let (sender, receiver) = mpsc::channel::>(1); - let attempt_id: history::AttemptId = match &self.current_attempt_id { - Some(id) => *id, - None => return, + let page_size = prepared.get_validated_page_size(); + + let worker_task = async move { + let worker = SingleConnectionRowIteratorWorker { + sender: sender.into(), + fetcher: |paging_state| { + connection.execute_raw_with_consistency( + &prepared, + &values, + consistency, + serial_consistency, + Some(page_size), + paging_state, + ) + }, + }; + worker.work().await }; - history_listener.log_attempt_success(attempt_id); + Self::new_from_worker_future(worker_task, receiver).await } - fn log_attempt_error(&mut self, error: &QueryError, retry_decision: &RetryDecision) { - let history_listener: &dyn HistoryListener = match &self.history_listener { - Some(hl) => &**hl, - None => return, - }; + async fn new_from_worker_future( + worker_task: impl Future + Send + 'static, + mut receiver: mpsc::Receiver>, + ) -> Result { + tokio::task::spawn(worker_task); - let attempt_id: history::AttemptId = match &self.current_attempt_id { - Some(id) => *id, - None => return, - }; + // This unwrap is safe because: + // - The future returned by worker.work sends at least one item + // to the channel (the PageSendAttemptedProof helps enforce this) + // - That future is polled in a tokio::task which isn't going to be + // cancelled + let pages_received = receiver.recv().await.unwrap()?; + let rows = pages_received + .rows + .into_legacy_rows(PagingStateResponse::NoMorePages)?; - history_listener.log_attempt_error(attempt_id, error, retry_decision); + Ok(Self { + current_row_idx: 0, + current_page: rows, + page_receiver: receiver, + tracing_ids: if let Some(tracing_id) = pages_received.tracing_id { + vec![tracing_id] + } else { + Vec::new() + }, + }) } -} - -/// A massively simplified version of the RowIteratorWorker. It does not have -/// any complicated logic related to retries, it just fetches pages from -/// a single connection. -struct SingleConnectionRowIteratorWorker { - sender: ProvingSender>, - fetcher: Fetcher, -} -impl SingleConnectionRowIteratorWorker -where - Fetcher: Fn(PagingState) -> FetchFut + Send + Sync, - FetchFut: Future> + Send, -{ - async fn work(mut self) -> PageSendAttemptedProof { - match self.do_work().await { - Ok(proof) => proof, - Err(err) => { - let (proof, _) = self.sender.send(Err(err)).await; - proof - } - } + /// If tracing was enabled returns tracing ids of all finished page queries + pub fn get_tracing_ids(&self) -> &[Uuid] { + &self.tracing_ids } - async fn do_work(&mut self) -> Result { - let mut paging_state = PagingState::start(); - loop { - let result = (self.fetcher)(paging_state).await?; - let response = result.into_non_error_query_response()?; - match response.response { - NonErrorResponse::Result(result::Result::Rows((rows, paging_state_response))) => { - let (proof, send_result) = self - .sender - .send(Ok(ReceivedPage { - rows, - tracing_id: response.tracing_id, - })) - .await; - - if send_result.is_err() { - // channel was closed, RowIterator was dropped - should shutdown - return Ok(proof); - } - - match paging_state_response.into_paging_control_flow() { - ControlFlow::Continue(new_paging_state) => { - paging_state = new_paging_state; - } - ControlFlow::Break(()) => { - // Reached the last query, shutdown - return Ok(proof); - } - } - } - NonErrorResponse::Result(_) => { - // We have most probably sent a modification statement (e.g. INSERT or UPDATE), - // so let's return an empty iterator as suggested in #631. + /// Returns specification of row columns + pub fn get_column_specs(&self) -> &[ColumnSpec<'static>] { + self.current_page.metadata.col_specs() + } - // We must attempt to send something because the iterator expects it. - let (proof, _) = self.sender.send_empty_page(response.tracing_id).await; - return Ok(proof); - } - _ => { - return Err(ProtocolError::UnexpectedResponse( - response.response.to_response_kind(), - ) - .into()); - } - } - } + fn is_current_page_exhausted(&self) -> bool { + self.current_row_idx >= self.current_page.rows.len() } } From ab79c39932bd5bf0e3fa599f57edc075a49fea64 Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Tue, 14 Mar 2023 15:08:27 +0100 Subject: [PATCH 16/42] iterator: adjust to the new deserialization framework MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This commit makes the RowIteratorWorker pass raw rows to the main tokio task, instead of the eagerly deserialized Rows. The equivalent of the old RowIterator is now RawIterator (notice a letter change). Despite the name, it cannot actually be conveniently iterated on, as it does not have any information about the column types. It exposes a `next()` method for deserializing consecutive `ColumnIterator`s. Users can manually perform deserialization using this method directly, but the preferred (typed) API will be added in the next commit. The legacy iterators are preserved by wrapping around RawIterator. Co-authored-by: Wojciech Przytuła --- scylla/src/transport/connection.rs | 15 +- scylla/src/transport/iterator.rs | 215 ++++++++++++++++++++--------- scylla/src/transport/session.rs | 15 +- 3 files changed, 168 insertions(+), 77 deletions(-) diff --git a/scylla/src/transport/connection.rs b/scylla/src/transport/connection.rs index 1a7fce16d..a2a3ef5f1 100644 --- a/scylla/src/transport/connection.rs +++ b/scylla/src/transport/connection.rs @@ -47,7 +47,7 @@ use std::{ }; use super::errors::{ProtocolError, UseKeyspaceProtocolError}; -use super::iterator::LegacyRowIterator; +use super::iterator::{LegacyRowIterator, RawIterator}; use super::locator::tablets::{RawTablet, TabletParsingError}; use super::query_result::QueryResult; use super::session::AddressTranslator; @@ -1188,13 +1188,9 @@ impl Connection { .determine_consistency(self.config.default_consistency); let serial_consistency = query.config.serial_consistency.flatten(); - LegacyRowIterator::new_for_connection_query_iter( - query, - self, - consistency, - serial_consistency, - ) - .await + RawIterator::new_for_connection_query_iter(query, self, consistency, serial_consistency) + .await + .map(RawIterator::into_legacy) } /// Executes a prepared statements and fetches its results over multiple pages, using @@ -1209,7 +1205,7 @@ impl Connection { .determine_consistency(self.config.default_consistency); let serial_consistency = prepared_statement.config.serial_consistency.flatten(); - LegacyRowIterator::new_for_connection_execute_iter( + RawIterator::new_for_connection_execute_iter( prepared_statement, values, self, @@ -1217,6 +1213,7 @@ impl Connection { serial_consistency, ) .await + .map(RawIterator::into_legacy) } #[allow(dead_code)] diff --git a/scylla/src/transport/iterator.rs b/scylla/src/transport/iterator.rs index 232e0a943..23ad2f54d 100644 --- a/scylla/src/transport/iterator.rs +++ b/scylla/src/transport/iterator.rs @@ -1,7 +1,6 @@ //! Iterators over rows returned by paged queries use std::future::Future; -use std::mem; use std::net::SocketAddr; use std::ops::ControlFlow; use std::pin::Pin; @@ -9,21 +8,24 @@ use std::sync::Arc; use std::task::{Context, Poll}; use futures::Stream; -use scylla_cql::frame::request::query::PagingStateResponse; +use scylla_cql::frame::frame_errors::RowsParseError; use scylla_cql::frame::response::result::RawMetadataAndRawRows; use scylla_cql::frame::response::NonErrorResponse; +use scylla_cql::types::deserialize::result::RawRowsLendingIterator; +use scylla_cql::types::deserialize::row::{ColumnIterator, DeserializeRow}; use scylla_cql::types::serialize::row::SerializedValues; use std::result::Result; use thiserror::Error; use tokio::sync::mpsc; use super::execution_profile::ExecutionProfileInner; +use super::query_result::ColumnSpecs; use super::session::RequestSpan; use crate::cql_to_rust::{FromRow, FromRowError}; use crate::frame::response::{ result, - result::{ColumnSpec, Row, Rows}, + result::{ColumnSpec, Row}, }; use crate::history::{self, HistoryListener}; use crate::statement::{prepared_statement::PreparedStatement, query::Query}; @@ -535,46 +537,97 @@ where } } -/// Iterator over rows returned by paged queries\ -/// Allows to easily access rows without worrying about handling multiple pages -pub struct LegacyRowIterator { - current_row_idx: usize, - current_page: Rows, +/// An intermediate object that allows to construct an iterator over a query +/// that is asynchronously paged in the background. +/// +/// TODO: implement and describe the new API +/// +/// A pre-0.15.0 interface is also available, although deprecated: +/// `into_legacy()` method converts RawIterator to LegacyRowIterator, +/// enabling Stream'ed operation on rows being eagerly deserialized +/// to a middle-man Row type. This is inefficient, especially if +/// the Row type is not the intended target type. +pub struct RawIterator { + current_page: RawRowsLendingIterator, page_receiver: mpsc::Receiver>, tracing_ids: Vec, } -/// Fetching pages is asynchronous so `RowIterator` does not implement the `Iterator` trait.\ -/// Instead it uses the asynchronous `Stream` trait -impl Stream for LegacyRowIterator { - type Item = Result; +/// RawIterator is not an iterator or a stream! However, it implements +/// a `next()` method that returns a ColumnIterator, which can be used +/// to manually deserialize a row. +/// The ColumnIterator borrows from the RawIterator, and the futures::Stream trait +/// does not allow for such a pattern. Lending streams are not a thing yet. +impl RawIterator { + /// Returns the next item (ColumnIterator) from the stream. + /// + /// This can be used with `type_check() for manual deserialization - see example below. + /// + /// This is not a part of the Stream interface because the returned iterator + /// borrows from self. + /// + /// This is cancel-safe. + /// + /// # Example + /// + /// // FIXME: change `text` to `rust` when Session API is migrated to the new deserialization framework. + /// ```text + /// # use scylla::Session; + /// # use std::error::Error; + /// # async fn check_only_compiles(session: &Session) -> Result<(), Box> { + /// use futures::stream::StreamExt; + /// use scylla::deserialize::DeserializeRow; + /// + /// let mut raw_iter = session + /// .query_iter("SELECT a, b FROM ks.t", &[]) + /// .await?; + /// + /// // Remember to type check! Failure to call type_check() can result + /// // in panics upon deserialization. + /// raw_iter.type_check::<(i32, i32)>()?; + /// + /// // Now that we type-checked, we can manually deserialize from RawIterator. + /// while let Some(column_iterator) = raw_iter.next().await.transpose()? { + /// let (a, b) = <(i32, i32) as DeserializeRow>::deserialize(column_iterator)?; + /// println!("a, b: {}, {}", a, b); + /// } + /// # Ok(()) + /// # } + /// ``` + pub async fn next(&mut self) -> Option> { + let res = std::future::poll_fn(|cx| Pin::new(&mut *self).poll_fill_page(cx)).await; + match res { + Some(Ok(())) => {} + Some(Err(err)) => return Some(Err(err)), + None => return None, + } - fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - self.poll_next_internal(cx) + // We are guaranteed here to have a non-empty page, so unwrap + Some( + self.current_page + .next() + .unwrap() + .map_err(|e| RowsParseError::from(e).into()), + ) } -} -impl LegacyRowIterator { - fn poll_next_internal( - mut self: Pin<&mut Self>, + /// Tries to acquire a non-empty page, if current page is exhausted. + fn poll_fill_page<'r>( + mut self: Pin<&'r mut Self>, cx: &mut Context<'_>, - ) -> Poll>> { - if self.as_ref().is_current_page_exhausted() { - ready_some_ok!(self.as_mut().poll_next_page(cx)); + ) -> Poll>> { + if !self.is_current_page_exhausted() { + return Poll::Ready(Some(Ok(()))); } - - let mut s = self.as_mut(); - - let idx = s.current_row_idx; - if idx < s.current_page.rows.len() { - let row = mem::take(&mut s.current_page.rows[idx]); - s.current_row_idx += 1; - return Poll::Ready(Some(Ok(row))); + ready_some_ok!(self.as_mut().poll_next_page(cx)); + if self.is_current_page_exhausted() { + // We most likely got a zero-sized page. + // Try again later. + cx.waker().wake_by_ref(); + Poll::Pending + } else { + Poll::Ready(Some(Ok(()))) } - // We probably got a zero-sized page - // Yield, but tell that we are ready - cx.waker().wake_by_ref(); - Poll::Pending } /// Makes an attempt to acquire the next page (which may be empty). @@ -589,18 +642,9 @@ impl LegacyRowIterator { let mut s = self.as_mut(); let received_page = ready_some_ok!(Pin::new(&mut s.page_receiver).poll_recv(cx)); - let rows = match received_page - .rows - // As RowIteratorWorker manages paging itself, the paging state response - // returned to the user is always NoMorePages. It used to be so before - // the deserialization refactor, too. - .into_legacy_rows(PagingStateResponse::NoMorePages) - { - Ok(rows) => rows, - Err(err) => return Poll::Ready(Some(Err(err.into()))), - }; - s.current_page = rows; - s.current_row_idx = 0; + let raw_rows_with_deserialized_metadata = + received_page.rows.deserialize_owned_metadata()?; + s.current_page = RawRowsLendingIterator::new(raw_rows_with_deserialized_metadata); if let Some(tracing_id) = received_page.tracing_id { s.tracing_ids.push(tracing_id); @@ -609,12 +653,11 @@ impl LegacyRowIterator { Poll::Ready(Some(Ok(()))) } - /// Converts this iterator into an iterator over rows parsed as given type - pub fn into_typed(self) -> LegacyTypedRowIterator { - LegacyTypedRowIterator { - row_iterator: self, - phantom_data: Default::default(), - } + /// Converts this iterator into an iterator over rows parsed as given type, + /// using the legacy deserialization framework. + #[inline] + pub fn into_legacy(self) -> LegacyRowIterator { + LegacyRowIterator { raw_iterator: self } } pub(crate) async fn new_for_query( @@ -888,16 +931,14 @@ impl LegacyRowIterator { // to the channel (the PageSendAttemptedProof helps enforce this) // - That future is polled in a tokio::task which isn't going to be // cancelled - let pages_received = receiver.recv().await.unwrap()?; - let rows = pages_received - .rows - .into_legacy_rows(PagingStateResponse::NoMorePages)?; + let page_received = receiver.recv().await.unwrap()?; + let raw_rows_with_deserialized_metadata = + page_received.rows.deserialize_owned_metadata()?; Ok(Self { - current_row_idx: 0, - current_page: rows, + current_page: RawRowsLendingIterator::new(raw_rows_with_deserialized_metadata), page_receiver: receiver, - tracing_ids: if let Some(tracing_id) = pages_received.tracing_id { + tracing_ids: if let Some(tracing_id) = page_received.tracing_id { vec![tracing_id] } else { Vec::new() @@ -906,17 +947,63 @@ impl LegacyRowIterator { } /// If tracing was enabled returns tracing ids of all finished page queries - pub fn get_tracing_ids(&self) -> &[Uuid] { + #[inline] + pub fn tracing_ids(&self) -> &[Uuid] { &self.tracing_ids } /// Returns specification of row columns - pub fn get_column_specs(&self) -> &[ColumnSpec<'static>] { - self.current_page.metadata.col_specs() + #[inline] + pub fn column_specs(&self) -> ColumnSpecs<'static, '_> { + ColumnSpecs::new(self.current_page.metadata().col_specs()) } fn is_current_page_exhausted(&self) -> bool { - self.current_row_idx >= self.current_page.rows.len() + self.current_page.rows_remaining() == 0 + } +} + +/// Iterator over rows returned by paged queries. +/// +/// Allows to easily access rows without worrying about handling multiple pages. +pub struct LegacyRowIterator { + raw_iterator: RawIterator, +} + +impl Stream for LegacyRowIterator { + type Item = Result; + + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + let mut s = self.as_mut(); + + let next_fut = s.raw_iterator.next(); + futures::pin_mut!(next_fut); + + let next_column_iter = ready_some_ok!(next_fut.poll(cx)); + + let next_ready_row = + Row::deserialize(next_column_iter).map_err(|e| RowsParseError::from(e).into()); + + Poll::Ready(Some(next_ready_row)) + } +} + +impl LegacyRowIterator { + /// If tracing was enabled returns tracing ids of all finished page queries + pub fn get_tracing_ids(&self) -> &[Uuid] { + self.raw_iterator.tracing_ids() + } + + /// Returns specification of row columns + pub fn get_column_specs(&self) -> &[ColumnSpec<'static>] { + self.raw_iterator.column_specs().inner() + } + + pub fn into_typed(self) -> LegacyTypedRowIterator { + LegacyTypedRowIterator { + row_iterator: self, + _phantom_data: Default::default(), + } } } @@ -925,16 +1012,18 @@ impl LegacyRowIterator { /// Returned by `RowIterator::into_typed` pub struct LegacyTypedRowIterator { row_iterator: LegacyRowIterator, - phantom_data: std::marker::PhantomData, + _phantom_data: std::marker::PhantomData, } impl LegacyTypedRowIterator { /// If tracing was enabled returns tracing ids of all finished page queries + #[inline] pub fn get_tracing_ids(&self) -> &[Uuid] { self.row_iterator.get_tracing_ids() } /// Returns specification of row columns + #[inline] pub fn get_column_specs(&self) -> &[ColumnSpec<'static>] { self.row_iterator.get_column_specs() } diff --git a/scylla/src/transport/session.rs b/scylla/src/transport/session.rs index d9b19617b..bf875c8a0 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; +use crate::LegacyQueryResult; use crate::history; use crate::history::HistoryListener; @@ -42,7 +43,8 @@ use super::connection::QueryResponse; use super::connection::SslConfig; use super::errors::TracingProtocolError; use super::execution_profile::{ExecutionProfile, ExecutionProfileHandle, ExecutionProfileInner}; -use super::legacy_query_result::{LegacyQueryResult, MaybeFirstRowTypedError}; +use super::iterator::RawIterator; +use super::legacy_query_result::MaybeFirstRowTypedError; #[cfg(feature = "cloud")] use super::node::CloudEndpoint; use super::node::{InternalKnownNode, KnownNode}; @@ -915,20 +917,21 @@ impl Session { .access(); if values.is_empty() { - LegacyRowIterator::new_for_query( + RawIterator::new_for_query( query, execution_profile, self.cluster.get_data(), self.metrics.clone(), ) .await + .map(RawIterator::into_legacy) } else { - // Making LegacyRowIterator::new_for_query work with values is too hard (if even possible) + // Making RawIterator::new_for_query work with values is too hard (if even possible) // so instead of sending one prepare to a specific connection on each iterator query, // we fully prepare a statement beforehand. let prepared = self.prepare(query).await?; let values = prepared.serialize_values(&values)?; - LegacyRowIterator::new_for_prepared_statement(PreparedIteratorConfig { + RawIterator::new_for_prepared_statement(PreparedIteratorConfig { prepared, values, execution_profile, @@ -936,6 +939,7 @@ impl Session { metrics: self.metrics.clone(), }) .await + .map(RawIterator::into_legacy) } } @@ -1318,7 +1322,7 @@ impl Session { .unwrap_or_else(|| self.get_default_execution_profile_handle()) .access(); - LegacyRowIterator::new_for_prepared_statement(PreparedIteratorConfig { + RawIterator::new_for_prepared_statement(PreparedIteratorConfig { prepared, values: serialized_values, execution_profile, @@ -1326,6 +1330,7 @@ impl Session { metrics: self.metrics.clone(), }) .await + .map(RawIterator::into_legacy) } /// Perform a batch request.\ From 5282c44e5d258ed8269b82fe17861fc63fc91495 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Wojciech=20Przytu=C5=82a?= Date: Tue, 22 Oct 2024 20:43:43 +0200 Subject: [PATCH 17/42] iterator: typed API for new deserialization framework This commit finishes the work related to adjusting the iterators module to the new deserialization framework. The previous commit brought RawIterator, which can deserialize ColumnIterators. This commit introduces new TypedRowIterator, which type-checks once and then deserializes from ColumnIterators into rows. RawIterator can be converted to TypedRowIterator by calling the `into_typed()` method. Unfortunately, due to the limitations of the Stream trait (no support for lending streams, analogous to lending iterators in case of RawRowsLendingIterator), a Stream cannot be used to deserialize borrowed types (i.e. those that borrow from the frame serialized contents). In order to give users both capabilities: 1) deserializing borrowed types (for efficiency), 2) deserializing using Stream (for convienience), two distinct types are used: TypedRowIterator and TypedRowStream. The first supports borrowed types and the second implements Stream. To sum up, instead of `RowIterator` (returning `Row`s) and `TypedRowIterator` (returning instances of the target type) both implementing `Stream`, now we have the following: - `RawIterator` - cannot implement `Stream`, because returns `ColumnIterator`s that borrow from it, - provide `type_check()` and `next()` methods that can be used for low-level, manual deserialization (not recommended for ordinary users) - supports deserializing manually borrowed types (such as `&str`). - `TypedRowIterator` - created by calling `into_typed::()` on `RawIterator`, - type checks upon creation, - supports deserializing borrowed types (such as `&str`), - does not implement `Stream` in order to support borrowed types, - provides basic Stream-like methods (`next()`, `try_next()`), - `TypedRowStream` - created by calling `into_stream()` on `TypedRowIterator`, - implements `Stream` and hence does not support borrowed types. Co-authored-by: Piotr Dulikowski --- scylla/src/transport/iterator.rs | 133 ++++++++++++++++++++++++++++++- 1 file changed, 132 insertions(+), 1 deletion(-) diff --git a/scylla/src/transport/iterator.rs b/scylla/src/transport/iterator.rs index 23ad2f54d..7897935f0 100644 --- a/scylla/src/transport/iterator.rs +++ b/scylla/src/transport/iterator.rs @@ -13,6 +13,7 @@ use scylla_cql::frame::response::result::RawMetadataAndRawRows; use scylla_cql::frame::response::NonErrorResponse; use scylla_cql::types::deserialize::result::RawRowsLendingIterator; use scylla_cql::types::deserialize::row::{ColumnIterator, DeserializeRow}; +use scylla_cql::types::deserialize::TypeCheckError; use scylla_cql::types::serialize::row::SerializedValues; use std::result::Result; use thiserror::Error; @@ -540,7 +541,10 @@ where /// An intermediate object that allows to construct an iterator over a query /// that is asynchronously paged in the background. /// -/// TODO: implement and describe the new API +/// Before the results can be processed in a convenient way, the RawIterator +/// needs to be cast into a typed iterator. This is done by use of `into_typed()` method. +/// As the method is generic over the target type, the turbofish syntax +/// can come in handy there, e.g. `raw_iter.into_typed::<(i32, &str, Uuid)>()`. /// /// A pre-0.15.0 interface is also available, although deprecated: /// `into_legacy()` method converts RawIterator to LegacyRowIterator, @@ -653,6 +657,30 @@ impl RawIterator { Poll::Ready(Some(Ok(()))) } + /// Type-checks the iterator against given type. + /// + /// This is automatically called upon transforming [RawIterator] into [TypedRowIterator]. + /// Can be used with `next()` for manual deserialization. See `next()` for an example. + #[inline] + pub fn type_check<'frame, 'metadata, RowT: DeserializeRow<'frame, 'metadata>>( + &self, + ) -> Result<(), TypeCheckError> { + RowT::type_check(self.column_specs().inner()) + } + + /// Casts the iterator to a given row type, enabling Stream'ed operations + /// on rows, which deserialize them in-fly to that given type. + /// Begins with performing type check. + #[inline] + pub fn into_typed<'frame, 'metadata, RowT: DeserializeRow<'frame, 'metadata>>( + self, + ) -> Result, TypeCheckError> + where + 'frame: 'metadata, + { + TypedRowIterator::::new(self) + } + /// Converts this iterator into an iterator over rows parsed as given type, /// using the legacy deserialization framework. #[inline] @@ -963,6 +991,109 @@ impl RawIterator { } } +/// Returned by [RawIterator::into_typed]. +/// +/// Does not implement [Stream], but permits deserialization of borrowed types. +/// To use [Stream] API (only accessible for owned types), use [TypedRowIterator::into_stream]. +pub struct TypedRowIterator { + raw_iterator: RawIterator, + _phantom: std::marker::PhantomData, +} + +impl Unpin for TypedRowIterator {} + +impl<'frame, 'metadata, RowT> TypedRowIterator +where + 'frame: 'metadata, + RowT: DeserializeRow<'frame, 'metadata>, +{ + fn new(raw_iterator: RawIterator) -> Result { + raw_iterator.type_check::()?; + + Ok(Self { + raw_iterator, + _phantom: Default::default(), + }) + } + + /// If tracing was enabled, returns tracing ids of all finished page queries. + #[inline] + pub fn tracing_ids(&self) -> &[Uuid] { + self.raw_iterator.tracing_ids() + } + + /// Returns specification of row columns + #[inline] + pub fn column_specs(&self) -> ColumnSpecs { + self.raw_iterator.column_specs() + } + + /// Stream-like next() implementation for TypedRowIterator. + /// + /// It also works with borrowed types! For example, &str is supported. + /// However, this is not a Stream. To create a Stream, use `into_stream()`. + #[inline] + pub async fn next(&'frame mut self) -> Option> { + self.raw_iterator.next().await.map(|res| { + res.and_then(|column_iterator| { + ::deserialize(column_iterator) + .map_err(|err| RowsParseError::from(err).into()) + }) + }) + } + + /// Stream-like try_next() implementation for TypedRowIterator. + /// + /// It also works with borrowed types! For example, &str is supported. + /// However, this is not a Stream. To create a Stream, use `into_stream()`. + #[inline] + pub async fn try_next(&'frame mut self) -> Result, QueryError> { + self.next().await.transpose() + } +} + +impl TypedRowIterator { + /// Transforms [TypedRowIterator] into [TypedRowStream]. + /// + /// If you deserialize to owned types only, use this method to unleash power of the `Stream` API. + /// This operation involves no runtime cost, but it limits the iterator to owned types only. + /// Therefore, if you want to work with borrowed types (e.g., to avoid heap allocations), + /// you can't use the `Stream` trait. + pub fn into_stream(self) -> TypedRowStream { + TypedRowStream { + typed_row_iterator: self, + } + } +} + +/// Returned by [TypedRowIterator::into_stream]. +/// +/// Implements [Stream], but only permits deserialization of owned types. +pub struct TypedRowStream { + typed_row_iterator: TypedRowIterator, +} + +impl Unpin for TypedRowStream {} + +/// Stream implementation for TypedRowStream. +/// +/// It only works with owned types! For example, &str is not supported. +impl Stream for TypedRowStream +where + RowT: for<'r> DeserializeRow<'r, 'r>, +{ + type Item = Result; + + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + let mut s = self.as_mut(); + + let next_fut = s.typed_row_iterator.next(); + futures::pin_mut!(next_fut); + let value = ready_some_ok!(next_fut.poll(cx)); + Poll::Ready(Some(Ok(value))) + } +} + /// Iterator over rows returned by paged queries. /// /// Allows to easily access rows without worrying about handling multiple pages. From 7ec3c8eac12e5aa7a9b3446c71f283f4a9f254e7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Wojciech=20Przytu=C5=82a?= Date: Fri, 11 Oct 2024 11:48:05 +0200 Subject: [PATCH 18/42] result: delete legacy Rows type It is no longer needed. For compatibility with LegacyQueryResult and LegacyRowIterator, higher-layer conversions suffice. --- scylla-cql/src/frame/response/result.rs | 33 ------------------------- 1 file changed, 33 deletions(-) diff --git a/scylla-cql/src/frame/response/result.rs b/scylla-cql/src/frame/response/result.rs index 44de5ea38..71b72cf6b 100644 --- a/scylla-cql/src/frame/response/result.rs +++ b/scylla-cql/src/frame/response/result.rs @@ -791,16 +791,6 @@ impl<'frame> DeserializedMetadataAndRawRows<'frame, RawRowsBorrowed<'frame>> { } } -#[derive(Debug)] -pub struct Rows { - pub metadata: ResultMetadataHolder<'static>, - pub paging_state_response: PagingStateResponse, - pub rows_count: usize, - pub rows: Vec, - /// Original size of the serialized rows. - pub serialized_size: usize, -} - #[derive(Debug)] pub enum Result { Void, @@ -1229,29 +1219,6 @@ impl RawMetadataAndRawRows { deser_col_specs_owned, ) } - - pub fn into_legacy_rows( - self, - paging_state_response: PagingStateResponse, - ) -> StdResult { - let raw_rows_with_deserialized_metadata = self.deserialize_owned_metadata()?; - - let rows_size = raw_rows_with_deserialized_metadata.rows_bytes_size(); - let rows_count = raw_rows_with_deserialized_metadata.rows_count(); - let rows = raw_rows_with_deserialized_metadata - .rows_iter::()? - .collect::>()?; - - let metadata = raw_rows_with_deserialized_metadata.into_metadata(); - - Ok(Rows { - metadata, - paging_state_response, - rows_count, - rows, - serialized_size: rows_size, - }) - } } fn deser_prepared_metadata( From 1692a672a4764fcf06c494600a44d71eb5931960 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Wojciech=20Przytu=C5=82a?= Date: Mon, 28 Oct 2024 08:52:27 +0100 Subject: [PATCH 19/42] session,iterator: record raw metadata&rows size Even though we can no longer record rows serialized size without accounting metadata, we can record their serialized size together. --- scylla-cql/src/frame/response/result.rs | 6 ++++++ scylla/src/transport/iterator.rs | 2 ++ scylla/src/transport/query_result.rs | 4 ++++ scylla/src/transport/session.rs | 20 +++++++++++++++++++- 4 files changed, 31 insertions(+), 1 deletion(-) diff --git a/scylla-cql/src/frame/response/result.rs b/scylla-cql/src/frame/response/result.rs index 71b72cf6b..848a016e7 100644 --- a/scylla-cql/src/frame/response/result.rs +++ b/scylla-cql/src/frame/response/result.rs @@ -641,6 +641,12 @@ impl RawMetadataAndRawRows { cached_metadata: None, } } + + /// Returns the serialized size of the raw metadata + raw rows. + #[inline] + pub fn metadata_and_rows_bytes_size(&self) -> usize { + self.raw_metadata_and_rows.len() + } } /// RESULT:Rows response, in partially serialized form. diff --git a/scylla/src/transport/iterator.rs b/scylla/src/transport/iterator.rs index 7897935f0..91b3e3bda 100644 --- a/scylla/src/transport/iterator.rs +++ b/scylla/src/transport/iterator.rs @@ -328,6 +328,8 @@ where .load_balancing_policy .on_query_success(&self.statement_info, elapsed, node); + request_span.record_raw_rows_fields(&rows); + let received_page = ReceivedPage { rows, tracing_id }; // Send next page to RowIterator diff --git a/scylla/src/transport/query_result.rs b/scylla/src/transport/query_result.rs index 3f7249b31..ea2a1d81b 100644 --- a/scylla/src/transport/query_result.rs +++ b/scylla/src/transport/query_result.rs @@ -172,6 +172,10 @@ impl QueryResult { } } + pub(crate) fn raw_metadata_and_rows(&self) -> Option<&RawMetadataAndRawRows> { + self.raw_metadata_and_rows.as_ref() + } + /// Warnings emitted by the database. #[inline] pub fn warnings(&self) -> impl Iterator { diff --git a/scylla/src/transport/session.rs b/scylla/src/transport/session.rs index bf875c8a0..8468183d2 100644 --- a/scylla/src/transport/session.rs +++ b/scylla/src/transport/session.rs @@ -18,6 +18,7 @@ use async_trait::async_trait; use futures::future::join_all; use futures::future::try_join_all; use itertools::{Either, Itertools}; +use scylla_cql::frame::response::result::RawMetadataAndRawRows; use scylla_cql::frame::response::result::{deser_cql_value, ColumnSpec}; use scylla_cql::frame::response::NonErrorResponse; use scylla_cql::types::serialize::batch::BatchValues; @@ -828,6 +829,7 @@ impl Session { self.handle_auto_await_schema_agreement(&response).await?; let (result, paging_state) = response.into_query_result_and_paging_state()?; + span.record_result_fields(&result); let result = result.into_legacy_result()?; Ok((result, paging_state)) } @@ -1265,6 +1267,7 @@ impl Session { self.handle_auto_await_schema_agreement(&response).await?; let (result, paging_state) = response.into_query_result_and_paging_state()?; + span.record_result_fields(&result); let result = result.into_legacy_result()?; Ok((result, paging_state)) } @@ -1460,8 +1463,12 @@ impl Session { let result = match run_query_result { RunQueryResult::IgnoredWriteError => LegacyQueryResult::mock_empty(), - RunQueryResult::Completed(response) => response.into_legacy_result()?, + RunQueryResult::Completed(result) => { + span.record_result_fields(&result); + result.into_legacy_result()? + } }; + Ok(result) } @@ -2180,6 +2187,17 @@ impl RequestSpan { } } + pub(crate) fn record_raw_rows_fields(&self, raw_rows: &RawMetadataAndRawRows) { + self.span + .record("raw_result_size", raw_rows.metadata_and_rows_bytes_size()); + } + + pub(crate) fn record_result_fields(&self, query_result: &QueryResult) { + if let Some(raw_metadata_and_rows) = query_result.raw_metadata_and_rows() { + self.record_raw_rows_fields(raw_metadata_and_rows); + } + } + pub(crate) fn record_replicas<'a>(&'a self, replicas: &'a [(impl Borrow>, Shard)]) { struct ReplicaIps<'a, N>(&'a [(N, Shard)]); impl<'a, N> Display for ReplicaIps<'a, N> From 778c39216e56aa392c36d20e3e066557280a8f69 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Wojciech=20Przytu=C5=82a?= Date: Wed, 30 Oct 2024 17:29:08 +0100 Subject: [PATCH 20/42] result: make ColumnSpec::borrowed `const` This is analogous to why TableSpec::borrowed is const. This simplifies tests, because those methods can be used in `static` and `const` contexts. --- scylla-cql/src/frame/response/result.rs | 2 +- scylla-cql/src/types/deserialize/mod.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/scylla-cql/src/frame/response/result.rs b/scylla-cql/src/frame/response/result.rs index 848a016e7..1a0b1cb2c 100644 --- a/scylla-cql/src/frame/response/result.rs +++ b/scylla-cql/src/frame/response/result.rs @@ -494,7 +494,7 @@ impl ColumnSpec<'static> { impl<'frame> ColumnSpec<'frame> { #[inline] - pub fn borrowed( + pub const fn borrowed( name: &'frame str, typ: ColumnType<'frame>, table_spec: TableSpec<'frame>, diff --git a/scylla-cql/src/types/deserialize/mod.rs b/scylla-cql/src/types/deserialize/mod.rs index 8d01b7352..affc2c0fc 100644 --- a/scylla-cql/src/types/deserialize/mod.rs +++ b/scylla-cql/src/types/deserialize/mod.rs @@ -342,7 +342,7 @@ mod tests { bytes.freeze() } - pub(super) fn spec<'a>(name: &'a str, typ: ColumnType<'a>) -> ColumnSpec<'a> { + pub(super) const fn spec<'a>(name: &'a str, typ: ColumnType<'a>) -> ColumnSpec<'a> { ColumnSpec::borrowed(name, typ, TableSpec::borrowed("ks", "tbl")) } } From d59d593a3ce3e94ec2b399b780c178cabbed2077 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Wojciech=20Przytu=C5=82a?= Date: Mon, 4 Nov 2024 07:48:35 +0100 Subject: [PATCH 21/42] WIP: yoked version --- scylla-cql/Cargo.toml | 2 + scylla-cql/src/frame/response/result.rs | 361 ++++++++++---------- scylla-cql/src/types/deserialize/result.rs | 14 +- scylla/src/transport/iterator.rs | 12 +- scylla/src/transport/legacy_query_result.rs | 17 +- scylla/src/transport/query_result.rs | 173 ++-------- 6 files changed, 228 insertions(+), 351 deletions(-) diff --git a/scylla-cql/Cargo.toml b/scylla-cql/Cargo.toml index 1b5ea06cc..fa0787f99 100644 --- a/scylla-cql/Cargo.toml +++ b/scylla-cql/Cargo.toml @@ -27,6 +27,8 @@ lz4_flex = { version = "0.11.1" } async-trait = "0.1.57" serde = { version = "1.0", features = ["derive"], optional = true } time-03 = { package = "time", version = "0.3", optional = true } +yoke = { version = "0.7", features = ["derive"] } +stable_deref_trait = "1.2" [dev-dependencies] assert_matches = "1.5.0" diff --git a/scylla-cql/src/frame/response/result.rs b/scylla-cql/src/frame/response/result.rs index 1a0b1cb2c..34edac89d 100644 --- a/scylla-cql/src/frame/response/result.rs +++ b/scylla-cql/src/frame/response/result.rs @@ -19,10 +19,10 @@ use crate::types::deserialize::{DeserializationError, FrameSlice, TypeCheckError use bytes::{Buf, Bytes}; use std::borrow::Cow; use std::fmt::Debug; -use std::ops::Deref; use std::sync::Arc; use std::{net::IpAddr, result::Result as StdResult, str}; use uuid::Uuid; +use yoke::Yokeable; #[derive(Debug)] pub struct SetKeyspace { @@ -522,7 +522,7 @@ impl<'frame> ColumnSpec<'frame> { } } -#[derive(Debug, Clone)] +#[derive(Debug, Clone, Yokeable)] pub struct ResultMetadata<'a> { col_count: usize, col_specs: Vec>, @@ -556,21 +556,17 @@ impl<'a> ResultMetadata<'a> { /// or the cached metadata in PreparedStatement; /// 2) owning it after deserializing from RESULT:Rows; /// 3) sharing ownership of metadata cached in PreparedStatement. -#[derive(Debug, Clone)] -pub enum ResultMetadataHolder<'frame> { - Owned(ResultMetadata<'frame>), - Borrowed(&'frame ResultMetadata<'frame>), +#[derive(Debug)] +pub enum ResultMetadataHolder { + SelfBorrowed(SelfBorrowedMetadataContainer), SharedCached(Arc>), } -impl<'frame> Deref for ResultMetadataHolder<'frame> { - type Target = ResultMetadata<'frame>; - - fn deref(&self) -> &Self::Target { +impl ResultMetadataHolder { + pub fn inner(&self) -> &ResultMetadata<'_> { match self { - ResultMetadataHolder::Owned(metadata) => metadata, - ResultMetadataHolder::Borrowed(metadata) => metadata, - ResultMetadataHolder::SharedCached(arc) => arc.deref(), + ResultMetadataHolder::SelfBorrowed(c) => c.metadata(), + ResultMetadataHolder::SharedCached(s) => s, } } } @@ -649,80 +645,109 @@ impl RawMetadataAndRawRows { } } -/// RESULT:Rows response, in partially serialized form. -/// -/// Paging state and metadata are deserialized, rows remain serialized. -/// -/// See [`RawRowsKind`] for explanation what it is and why it is needed. -#[derive(Debug)] -pub struct DeserializedMetadataAndRawRows<'frame, RawRowsRepr: RawRowsKind> { - metadata: ResultMetadataHolder<'frame>, - rows_count: usize, - raw_rows: RawRowsRepr, -} +mod metadata_container { + use std::ops::Deref; -mod sealed { - // This is a sealed trait - its whole purpose is to be unnameable. - // This means we need to disable the check. - #[allow(unknown_lints)] // Rust 1.70 (our MSRV) doesn't know this lint. - #[allow(unnameable_types)] - pub trait Sealed {} -} + use bytes::Bytes; + use yoke::Yoke; -/// This abstracts over two different ways of storing the frame: -/// - shared ownership (Bytes), -/// - borrowing (FrameSlice). -/// -/// Its whole purpose is to restrict the type parameter of `DeserializedMetadataAndRawRows` -/// to the two valid variants. -/// -/// ### Why is a trait used, and not an enum, as it's done for `ResultMetadataHolder`? -/// -/// The problem arises with the `rows_iter` method. -/// - in case of `DeserializedMetadataAndRawRows`, the struct itself -/// owns the frame. Therefore, the reference to `self` should have the `'frame` -/// lifetime (and this way bound the lifetime of deserialized items). -/// - in case of `DeserializedMetadataAndRawRows`, the struct -/// borrows the frame with some lifetime 'frame. Therefore, the reference to -/// `self` should only have the `'metadata` lifetime, as the frame is owned -/// independently of Self's lifetime. -/// -/// This discrepancy is not expressible by enums. Therefore, an entirely separate -/// `rows_iter` must be defined for both cases, and thus both cases must be separate -/// types - and this is guaranteed by having a different type parameter (because they -/// are distinct instantiations of a generic type). -pub trait RawRowsKind: sealed::Sealed + Debug { - fn as_slice(&self) -> &[u8]; -} -#[derive(Debug)] -pub struct RawRowsOwned(Bytes); -impl sealed::Sealed for RawRowsOwned {} -impl RawRowsKind for RawRowsOwned { - fn as_slice(&self) -> &[u8] { - &self.0 + use super::ResultMetadata; + + use crate::types::deserialize::FrameSlice; + + #[derive(Debug, Clone)] + struct BytesWrapper { + inner: Bytes, + } + + impl Deref for BytesWrapper { + type Target = [u8]; + + fn deref(&self) -> &Self::Target { + &self.inner + } + } + + unsafe impl stable_deref_trait::StableDeref for BytesWrapper {} + unsafe impl yoke::CloneableCart for BytesWrapper {} + + #[derive(Debug, Clone)] + pub struct SelfBorrowedMetadataContainer { + metadata_and_frame: Yoke, BytesWrapper>, + } + + impl SelfBorrowedMetadataContainer { + pub(crate) fn mock_empty() -> Self { + Self { + metadata_and_frame: Yoke::attach_to_cart( + BytesWrapper { + inner: Bytes::new(), + }, + |_| ResultMetadata::mock_empty(), + ), + } + } + + pub(crate) fn metadata(&self) -> &ResultMetadata<'_> { + self.metadata_and_frame.get() + } + + // Returns Self (deserialized metadata) and the rest of the bytes, + // which contain rows count and then rows themselves. + pub(crate) fn make_deserialized_metadata( + frame: Bytes, + deserializer: F, + ) -> Result<(Self, Bytes), ErrorT> + where + F: for<'frame> FnOnce( + &mut FrameSlice<'frame>, + ) -> Result, ErrorT>, + { + let metadata_with_slice: Yoke<(&'static [u8], ResultMetadata<'static>), BytesWrapper> = + Yoke::try_attach_to_cart(BytesWrapper { inner: frame }, |slice| { + let mut frame_slice = FrameSlice::new_borrowed(slice); + let metadata = deserializer(&mut frame_slice)?; + Ok((frame_slice.as_slice(), metadata)) + })?; + + let raw_rows_with_count = metadata_with_slice + .backing_cart() + .inner + .slice_ref(metadata_with_slice.get().0); + + Ok(( + Self { + metadata_and_frame: metadata_with_slice + .map_project(|(_, metadata), _| metadata), + }, + raw_rows_with_count, + )) + } } } +pub use metadata_container::SelfBorrowedMetadataContainer; + +/// RESULT:Rows response, in partially serialized form. +/// +/// Paging state and metadata are deserialized, rows remain serialized. #[derive(Debug)] -pub struct RawRowsBorrowed<'frame>(FrameSlice<'frame>); -impl<'frame> sealed::Sealed for RawRowsBorrowed<'frame> {} -impl<'frame> RawRowsKind for RawRowsBorrowed<'frame> { - fn as_slice(&self) -> &'frame [u8] { - self.0.as_slice() - } +pub struct DeserializedMetadataAndRawRows { + metadata: ResultMetadataHolder, + rows_count: usize, + raw_rows: Bytes, } -impl<'frame, RawRowsRepr: RawRowsKind> DeserializedMetadataAndRawRows<'frame, RawRowsRepr> { +impl DeserializedMetadataAndRawRows { /// Returns the metadata associated with this response /// (table and column specifications). #[inline] - pub fn metadata(&self) -> &ResultMetadata<'frame> { - self.metadata.deref() + pub fn metadata(&self) -> &ResultMetadata<'_> { + self.metadata.inner() } - /// Consumes the `DeserializedMetadataAndRawRows` and returns metadata /// associated with the response (or cached metadata, if used in its stead). #[inline] - pub fn into_metadata(self) -> ResultMetadataHolder<'frame> { + pub fn into_metadata(self) -> ResultMetadataHolder { self.metadata } @@ -735,24 +760,24 @@ impl<'frame, RawRowsRepr: RawRowsKind> DeserializedMetadataAndRawRows<'frame, Ra /// Returns the serialized size of the raw rows. #[inline] pub fn rows_bytes_size(&self) -> usize { - self.raw_rows.as_slice().len() + self.raw_rows.len() } -} -impl DeserializedMetadataAndRawRows<'static, RawRowsOwned> { // Preferred to implementing Default, because users shouldn't be encouraged to create // empty DeserializedMetadataAndRawRows. #[inline] pub fn mock_empty() -> Self { Self { - metadata: ResultMetadataHolder::Owned(ResultMetadata::mock_empty()), + metadata: ResultMetadataHolder::SelfBorrowed( + SelfBorrowedMetadataContainer::mock_empty(), + ), rows_count: 0, - raw_rows: RawRowsOwned(Bytes::new()), + raw_rows: Bytes::new(), } } - pub(crate) fn into_inner(self) -> (ResultMetadataHolder<'static>, usize, Bytes) { - (self.metadata, self.rows_count, self.raw_rows.0) + pub(crate) fn into_inner(self) -> (ResultMetadataHolder, usize, Bytes) { + (self.metadata, self.rows_count, self.raw_rows) } /// Creates a typed iterator over the rows that lazily deserializes @@ -766,33 +791,12 @@ impl DeserializedMetadataAndRawRows<'static, RawRowsOwned> { where 'frame: 'metadata, { - let frame_slice = FrameSlice::new(&self.raw_rows.0); - let raw = RowIterator::new(self.rows_count, self.metadata.col_specs(), frame_slice); - TypedRowIterator::new(raw) - } -} - -impl<'frame> DeserializedMetadataAndRawRows<'frame, RawRowsBorrowed<'frame>> { - // Preferred to implementing Default, because users shouldn't be encouraged to create - // empty DeserializedMetadataAndRawRows. - #[inline] - pub fn mock_empty() -> Self { - Self { - metadata: ResultMetadataHolder::Owned(ResultMetadata::mock_empty()), - rows_count: 0, - raw_rows: RawRowsBorrowed(FrameSlice::new_empty()), - } - } - - /// Creates a typed iterator over the rows that lazily deserializes - /// rows in the result. - /// - /// Returns Err if the schema of returned result doesn't match R. - #[inline] - pub fn rows_iter<'metadata, R: DeserializeRow<'frame, 'metadata>>( - &'metadata self, - ) -> StdResult, TypeCheckError> { - let raw = RowIterator::new(self.rows_count, self.metadata.col_specs(), self.raw_rows.0); + let frame_slice = FrameSlice::new(&self.raw_rows); + let raw = RowIterator::new( + self.rows_count, + self.metadata.inner().col_specs(), + frame_slice, + ); TypedRowIterator::new(raw) } } @@ -1123,40 +1127,30 @@ impl RawMetadataAndRawRows { } } -type DeserColSpecsFn<'this, 'result> = - fn( - &mut &'this [u8], - Option>, - usize, - ) -> StdResult>, ColumnSpecParseError>; - impl RawMetadataAndRawRows { /// Deserializes ResultMetadata and deserializes rows count. Keeps rows in the serialized form. /// /// If metadata is cached (in the PreparedStatement), it is reused (shared) from cache /// instead of deserializing. - fn deserialize_metadata_generic<'this, 'result, T: RawRowsKind>( - &'this self, - raw_rows_constructor: fn(FrameSlice<'this>) -> T, - use_cached_metadata: fn( - &'this Arc>, - ) -> ResultMetadataHolder<'result>, - deser_col_specs: DeserColSpecsFn<'this, 'result>, - ) -> StdResult, RowsParseError> { - let mut frame_slice = FrameSlice::new(&self.raw_metadata_and_rows); - - let metadata = match self.cached_metadata.as_ref() { + pub fn deserialize_metadata(self) -> StdResult { + let (metadata_deserialized, row_count_and_raw_rows) = match self.cached_metadata { Some(cached) if self.no_metadata => { // Server sent no metadata, but we have metadata cached. This means that we asked the server // not to send metadata in the response as an optimization. We use cached metadata instead. - use_cached_metadata(cached) + ( + ResultMetadataHolder::SharedCached(cached), + self.raw_metadata_and_rows, + ) } None if self.no_metadata => { // Server sent no metadata and we have no metadata cached. Having no metadata cached, // we wouldn't have asked the server for skipping metadata. Therefore, this is most probably // not a SELECT, because in such case the server would send empty metadata both in Prepared // and in Result responses. - ResultMetadataHolder::Owned(ResultMetadata::mock_empty()) + ( + ResultMetadataHolder::SharedCached(Arc::new(ResultMetadata::mock_empty())), + self.raw_metadata_and_rows, + ) } Some(_) | None => { // Two possibilities: @@ -1167,64 +1161,77 @@ impl RawMetadataAndRawRows { // too, because it's suspicious, so we had better use the new metadata just in case. // Also, we simply need to advance the buffer pointer past metadata, and this requires // parsing metadata. - let server_metadata = { - let global_table_spec = self - .global_tables_spec - .then(|| deser_table_spec(frame_slice.as_slice_mut())) - .transpose() - .map_err(ResultMetadataParseError::from)?; - - let col_specs = deser_col_specs( - frame_slice.as_slice_mut(), - global_table_spec, - self.col_count, - ) - .map_err(ResultMetadataParseError::from)?; - - ResultMetadata { - col_count: self.col_count, - col_specs, + + // This function is needed, because creating the deserializer closure + // directly in the enclosing function does not provide enough type hints + // for the compiler, so it demands a type annotation. We cannot, however, + // write a correct type annotation, because this way we would limit the lifetime + // to a concrete lifetime, and our closure needs to be `impl for<'frame> ...`. + // This is a proud trick by Wojciech Przytuła, which crowns the brilliant + // idea of Karol Baryła to use Yoke to enable borrowing ResultMetadata + // from itself. + fn make_deserializer( + col_count: usize, + global_tables_spec: bool, + ) -> impl for<'frame> FnOnce( + &mut FrameSlice<'frame>, + ) -> StdResult< + ResultMetadata<'frame>, + RowsParseError, + > { + move |frame_slice| { + let server_metadata = { + let global_table_spec = global_tables_spec + .then(|| deser_table_spec(frame_slice.as_slice_mut())) + .transpose() + .map_err(ResultMetadataParseError::from)?; + + let col_specs = deser_col_specs_borrowed( + frame_slice.as_slice_mut(), + global_table_spec, + col_count, + ) + .map_err(ResultMetadataParseError::from)?; + + ResultMetadata { + col_count, + col_specs, + } + }; + if server_metadata.col_count() != server_metadata.col_specs().len() { + return Err(RowsParseError::ColumnCountMismatch { + col_count: server_metadata.col_count(), + col_specs_count: server_metadata.col_specs().len(), + }); + } + Ok(server_metadata) } - }; - if server_metadata.col_count() != server_metadata.col_specs().len() { - return Err(RowsParseError::ColumnCountMismatch { - col_count: server_metadata.col_count(), - col_specs_count: server_metadata.col_specs().len(), - }); } - ResultMetadataHolder::Owned(server_metadata) + let deserializer = make_deserializer(self.col_count, self.global_tables_spec); + + let (metadata_container, raw_rows_with_count) = + metadata_container::SelfBorrowedMetadataContainer::make_deserialized_metadata( + self.raw_metadata_and_rows, + deserializer, + )?; + ( + ResultMetadataHolder::SelfBorrowed(metadata_container), + raw_rows_with_count, + ) } }; + let mut frame_slice = FrameSlice::new(&row_count_and_raw_rows); + let rows_count: usize = types::read_int_length(frame_slice.as_slice_mut()) .map_err(RowsParseError::RowsCountParseError)?; Ok(DeserializedMetadataAndRawRows { - metadata, + metadata: metadata_deserialized, rows_count, - raw_rows: raw_rows_constructor(frame_slice), + raw_rows: frame_slice.to_bytes(), }) } - - pub fn deserialize_borrowed_metadata( - &self, - ) -> StdResult>, RowsParseError> { - self.deserialize_metadata_generic( - RawRowsBorrowed, - |cached: &Arc| ResultMetadataHolder::Borrowed(cached), - deser_col_specs_borrowed, - ) - } - - pub fn deserialize_owned_metadata( - &self, - ) -> StdResult, RowsParseError> { - self.deserialize_metadata_generic( - |frame_slice| RawRowsOwned(frame_slice.to_bytes()), - |cached: &Arc| ResultMetadataHolder::SharedCached(Arc::clone(cached)), - deser_col_specs_owned, - ) - } } fn deser_prepared_metadata( @@ -1696,18 +1703,18 @@ mod test_utils { } } - impl<'frame> DeserializedMetadataAndRawRows<'frame, RawRowsOwned> { + impl DeserializedMetadataAndRawRows { #[inline] #[doc(hidden)] pub fn new_for_test( - metadata: ResultMetadata<'frame>, + metadata: ResultMetadata<'static>, rows_count: usize, raw_rows: Bytes, ) -> Self { Self { - metadata: ResultMetadataHolder::Owned(metadata), + metadata: ResultMetadataHolder::SharedCached(Arc::new(metadata)), rows_count, - raw_rows: RawRowsOwned(raw_rows), + raw_rows, } } } diff --git a/scylla-cql/src/types/deserialize/result.rs b/scylla-cql/src/types/deserialize/result.rs index 04b3af6c1..c31c2a2d3 100644 --- a/scylla-cql/src/types/deserialize/result.rs +++ b/scylla-cql/src/types/deserialize/result.rs @@ -1,7 +1,7 @@ use bytes::Bytes; use crate::frame::response::result::{ - ColumnSpec, DeserializedMetadataAndRawRows, RawRowsOwned, ResultMetadata, ResultMetadataHolder, + ColumnSpec, DeserializedMetadataAndRawRows, ResultMetadata, ResultMetadataHolder, }; use super::row::{mk_deser_err, BuiltinDeserializationErrorKind, ColumnIterator, DeserializeRow}; @@ -150,7 +150,7 @@ where /// library to represent this concept yet). #[derive(Debug)] pub struct RawRowsLendingIterator { - metadata: ResultMetadataHolder<'static>, + metadata: ResultMetadataHolder, remaining: usize, at: usize, raw_rows: Bytes, @@ -159,7 +159,7 @@ pub struct RawRowsLendingIterator { impl RawRowsLendingIterator { /// Creates a new `RawRowsLendingIterator`, consuming given `RawRows`. #[inline] - pub fn new(raw_rows: DeserializedMetadataAndRawRows<'static, RawRowsOwned>) -> Self { + pub fn new(raw_rows: DeserializedMetadataAndRawRows) -> Self { let (metadata, rows_count, raw_rows) = raw_rows.into_inner(); Self { metadata, @@ -186,10 +186,10 @@ impl RawRowsLendingIterator { // Ideally, we would prefer to preserve the FrameSlice between calls to `next()`, // but borrowing from oneself is impossible, so we have to recreate it this way. - let iter = ColumnIterator::new(self.metadata.col_specs(), remaining_frame); + let iter = ColumnIterator::new(self.metadata.inner().col_specs(), remaining_frame); // Skip the row here, manually - for (column_index, spec) in self.metadata.col_specs().iter().enumerate() { + for (column_index, spec) in self.metadata.inner().col_specs().iter().enumerate() { let remaining_frame_len_before_column_read = remaining_frame.as_slice().len(); if let Err(err) = remaining_frame.read_cql_bytes() { return Some(Err(mk_deser_err::( @@ -220,8 +220,8 @@ impl RawRowsLendingIterator { /// Returns the metadata associated with the response (paging state and /// column specifications). #[inline] - pub fn metadata(&self) -> &ResultMetadata<'static> { - &self.metadata + pub fn metadata(&self) -> &ResultMetadata<'_> { + self.metadata.inner() } /// Returns the remaining number of rows that this iterator is expected diff --git a/scylla/src/transport/iterator.rs b/scylla/src/transport/iterator.rs index 91b3e3bda..28b77f208 100644 --- a/scylla/src/transport/iterator.rs +++ b/scylla/src/transport/iterator.rs @@ -648,8 +648,7 @@ impl RawIterator { let mut s = self.as_mut(); let received_page = ready_some_ok!(Pin::new(&mut s.page_receiver).poll_recv(cx)); - let raw_rows_with_deserialized_metadata = - received_page.rows.deserialize_owned_metadata()?; + let raw_rows_with_deserialized_metadata = received_page.rows.deserialize_metadata()?; s.current_page = RawRowsLendingIterator::new(raw_rows_with_deserialized_metadata); if let Some(tracing_id) = received_page.tracing_id { @@ -962,8 +961,7 @@ impl RawIterator { // - That future is polled in a tokio::task which isn't going to be // cancelled let page_received = receiver.recv().await.unwrap()?; - let raw_rows_with_deserialized_metadata = - page_received.rows.deserialize_owned_metadata()?; + let raw_rows_with_deserialized_metadata = page_received.rows.deserialize_metadata()?; Ok(Self { current_page: RawRowsLendingIterator::new(raw_rows_with_deserialized_metadata), @@ -984,7 +982,7 @@ impl RawIterator { /// Returns specification of row columns #[inline] - pub fn column_specs(&self) -> ColumnSpecs<'static, '_> { + pub fn column_specs(&self) -> ColumnSpecs<'_> { ColumnSpecs::new(self.current_page.metadata().col_specs()) } @@ -1128,7 +1126,7 @@ impl LegacyRowIterator { } /// Returns specification of row columns - pub fn get_column_specs(&self) -> &[ColumnSpec<'static>] { + pub fn get_column_specs(&self) -> &[ColumnSpec<'_>] { self.raw_iterator.column_specs().inner() } @@ -1157,7 +1155,7 @@ impl LegacyTypedRowIterator { /// Returns specification of row columns #[inline] - pub fn get_column_specs(&self) -> &[ColumnSpec<'static>] { + pub fn get_column_specs(&self) -> &[ColumnSpec<'_>] { self.row_iterator.get_column_specs() } } diff --git a/scylla/src/transport/legacy_query_result.rs b/scylla/src/transport/legacy_query_result.rs index 833c95c23..b8c0986f5 100644 --- a/scylla/src/transport/legacy_query_result.rs +++ b/scylla/src/transport/legacy_query_result.rs @@ -19,7 +19,7 @@ pub struct LegacyQueryResult { /// CQL Tracing uuid - can only be Some if tracing is enabled for this query pub tracing_id: Option, /// Metadata returned along with this response. - pub(crate) metadata: Option>, + pub(crate) metadata: Option, /// The original size of the serialized rows in request pub serialized_size: usize, } @@ -136,16 +136,16 @@ impl LegacyQueryResult { /// Returns column specifications. #[inline] - pub fn col_specs(&self) -> &[ColumnSpec<'static>] { + pub fn col_specs(&self) -> &[ColumnSpec<'_>] { self.metadata .as_ref() - .map(|metadata| metadata.col_specs()) + .map(|metadata| metadata.inner().col_specs()) .unwrap_or_default() } /// Returns a column specification for a column with given name, or None if not found #[inline] - pub fn get_column_spec<'a>(&'a self, name: &str) -> Option<(usize, &'a ColumnSpec<'static>)> { + pub fn get_column_spec<'a>(&'a self, name: &str) -> Option<(usize, &'a ColumnSpec<'_>)> { self.col_specs() .iter() .enumerate() @@ -283,6 +283,7 @@ mod tests { test_utils::setup_tracing, }; use std::convert::TryInto; + use std::sync::Arc; use assert_matches::assert_matches; use scylla_cql::frame::response::result::{ColumnType, ResultMetadata, TableSpec}; @@ -333,14 +334,18 @@ mod tests { fn make_rows_query_result(rows_num: usize) -> LegacyQueryResult { let mut res = make_not_rows_query_result(); res.rows = Some(make_rows(rows_num)); - res.metadata = Some(ResultMetadataHolder::Owned(make_test_metadata())); + res.metadata = Some(ResultMetadataHolder::SharedCached(Arc::new( + make_test_metadata(), + ))); res } fn make_string_rows_query_result(rows_num: usize) -> LegacyQueryResult { let mut res = make_not_rows_query_result(); res.rows = Some(make_string_rows(rows_num)); - res.metadata = Some(ResultMetadataHolder::Owned(make_test_metadata())); + res.metadata = Some(ResultMetadataHolder::SharedCached(Arc::new( + make_test_metadata(), + ))); res } diff --git a/scylla/src/transport/query_result.rs b/scylla/src/transport/query_result.rs index ea2a1d81b..a70ae5d44 100644 --- a/scylla/src/transport/query_result.rs +++ b/scylla/src/transport/query_result.rs @@ -4,8 +4,7 @@ use uuid::Uuid; use scylla_cql::frame::frame_errors::RowsParseError; use scylla_cql::frame::response::result::{ - ColumnSpec, ColumnType, DeserializedMetadataAndRawRows, RawMetadataAndRawRows, RawRowsBorrowed, - RawRowsKind, RawRowsOwned, Row, TableSpec, + ColumnSpec, ColumnType, DeserializedMetadataAndRawRows, RawMetadataAndRawRows, Row, TableSpec, }; use scylla_cql::types::deserialize::result::TypedRowIterator; use scylla_cql::types::deserialize::row::DeserializeRow; @@ -234,66 +233,11 @@ impl QueryResult { /// # } /// /// ``` - pub fn rows_deserializer( - &self, - ) -> Result>>, RowsParseError> { + pub fn into_rows_result(self) -> Result, RowsParseError> { self.raw_metadata_and_rows - .as_ref() .map(|raw_rows| { - let raw_rows_with_metadata = raw_rows.deserialize_borrowed_metadata()?; - Ok(RowsDeserializer { - raw_rows_with_metadata, - }) - }) - .transpose() - } - - /// Creates an owned [`RowsDeserializer`] to enable deserializing rows contained - /// in this [`QueryResult`]'s frame. Deserializes result metadata and allocates it, - /// so this should be considered a moderately costly operation and performed only once. - /// - /// Returns `None` if the response is not of Rows kind. - /// - /// The created [`RowsDeserializer`] does not borrow from the [`QueryResult`], - /// so it does not not limit flexibility. However, the cost involves more string - /// heap allocations. - /// If you don't need that flexibility, use cheaper [`QueryResult::rows_deserializer`]. - /// - /// ```compile_fail - /// # use scylla::transport::QueryResult; - /// fn example(query: impl FnOnce() -> QueryResult) -> Result<(), Box> { - /// let deserializer = query().rows_deserializer()?.unwrap(); - /// - /// // Compiler complains: "Temporary value dropped when borrowed". - /// let col_specs = deserializer.column_specs(); - /// - /// Ok(()) - /// } - /// ``` - /// - /// ```rust - /// # use scylla::transport::query_result::{QueryResult, RowsDeserializerOwning}; - /// fn example( - /// query: impl FnOnce() -> QueryResult - /// ) -> Result> { - /// let deserializer = query().rows_deserializer_owned()?.unwrap(); - /// - /// // This compiles. - /// let col_specs = deserializer.column_specs(); - /// - /// // RowsDeserializer is fully owned and independent, but at cost - /// // of moderately more expensive metadata deserialization. - /// Ok(deserializer) - /// } - /// ``` - pub fn rows_deserializer_owned( - &self, - ) -> Result>, RowsParseError> { - self.raw_metadata_and_rows - .as_ref() - .map(|raw_rows| { - let raw_rows_with_metadata = raw_rows.deserialize_owned_metadata()?; - Ok(RowsDeserializer { + let raw_rows_with_metadata = raw_rows.deserialize_metadata()?; + Ok(QueryRowsResult { raw_rows_with_metadata, }) }) @@ -305,7 +249,7 @@ impl QueryResult { /// period to the new API. 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_owned_metadata()?; + let raw_rows_with_metadata = raw_rows.deserialize_metadata()?; let deserialized_rows = raw_rows_with_metadata .rows_iter::()? @@ -364,14 +308,11 @@ impl QueryResult { /// /// ``` #[derive(Debug)] -pub struct RowsDeserializer<'frame, Kind: RawRowsKind> { - raw_rows_with_metadata: DeserializedMetadataAndRawRows<'frame, Kind>, +pub struct QueryRowsResult { + raw_rows_with_metadata: DeserializedMetadataAndRawRows, } -pub type RowsDeserializerOwning = RowsDeserializer<'static, RawRowsOwned>; -pub type RowsDeserializerBorrowing<'frame> = RowsDeserializer<'frame, RawRowsBorrowed<'frame>>; - -impl<'frame, RawRows: RawRowsKind> RowsDeserializer<'frame, RawRows> { +impl QueryRowsResult { /// Returns the number of received rows. #[inline] pub fn rows_num(&self) -> usize { @@ -391,84 +332,7 @@ impl<'frame, RawRows: RawRowsKind> RowsDeserializer<'frame, RawRows> { } } -impl<'frame> RowsDeserializer<'frame, RawRowsBorrowed<'frame>> { - /// Returns the received rows when present. - /// - /// Returns an error if the rows in the response are of incorrect type. - #[inline] - pub fn rows<'metadata, R: DeserializeRow<'frame, 'metadata>>( - &'metadata self, - ) -> Result, RowsError> { - self.raw_rows_with_metadata - .rows_iter() - .map_err(RowsError::TypeCheckFailed) - } - - /// Returns `Option` containing the first of a result. - /// - /// Fails when the the rows in the response are of incorrect type, - /// or when the deserialization fails. - pub fn maybe_first_row<'metadata, R: DeserializeRow<'frame, 'metadata>>( - &'metadata self, - ) -> Result, MaybeFirstRowError> { - self.rows::() - .map_err(|err| match err { - RowsError::TypeCheckFailed(typck_err) => { - MaybeFirstRowError::TypeCheckFailed(typck_err) - } - })? - .next() - .transpose() - .map_err(MaybeFirstRowError::DeserializationFailed) - } - - /// Returns first row from the received rows. - /// - /// When the first row is not available, returns an error. - /// Fails when the the rows in the response are of incorrect type, - /// or when the deserialization fails. - pub fn first_row<'metadata, R: DeserializeRow<'frame, 'metadata>>( - &'metadata self, - ) -> Result { - match self.maybe_first_row::() { - Ok(Some(row)) => Ok(row), - Ok(None) => Err(FirstRowError::RowsEmpty), - Err(MaybeFirstRowError::TypeCheckFailed(err)) => { - Err(FirstRowError::TypeCheckFailed(err)) - } - Err(MaybeFirstRowError::DeserializationFailed(err)) => { - Err(FirstRowError::DeserializationFailed(err)) - } - } - } - - /// Returns the only received row. - /// - /// Fails if the result is anything else than a single row. - /// Fails when the the rows in the response are of incorrect type, - /// or when the deserialization fails. - pub fn single_row<'metadata, R: DeserializeRow<'frame, 'metadata>>( - &'metadata self, - ) -> Result { - match self.rows::() { - Ok(mut rows) => match rows.next() { - Some(Ok(row)) => { - if rows.rows_remaining() != 0 { - return Err(SingleRowError::UnexpectedRowCount( - rows.rows_remaining() + 1, - )); - } - Ok(row) - } - Some(Err(err)) => Err(SingleRowError::DeserializationFailed(err)), - None => Err(SingleRowError::UnexpectedRowCount(0)), - }, - Err(RowsError::TypeCheckFailed(err)) => Err(SingleRowError::TypeCheckFailed(err)), - } - } -} - -impl RowsDeserializer<'static, RawRowsOwned> { +impl QueryRowsResult { /// Returns the received rows when present. /// /// Returns an error if the rows in the response are of incorrect type. @@ -692,7 +556,7 @@ mod tests { // Not RESULT::Rows response -> no column specs { let rqr = QueryResult::new(None, None, Vec::new()); - let qr = rqr.rows_deserializer().unwrap(); + let qr = rqr.into_rows_result().unwrap(); assert_matches!(qr, None); } @@ -703,7 +567,7 @@ mod tests { let rr = RawMetadataAndRawRows::new_for_test(None, Some(metadata), false, 0, &[]) .unwrap(); let rqr = QueryResult::new(Some(rr), None, Vec::new()); - let qr = rqr.rows_deserializer().unwrap().unwrap(); + let qr = rqr.into_rows_result().unwrap().unwrap(); let column_specs = qr.column_specs(); assert_eq!(column_specs.len(), n); @@ -750,7 +614,7 @@ mod tests { // Not RESULT::Rows { let rqr = QueryResult::new(None, None, Vec::new()); - let qr = rqr.rows_deserializer().unwrap(); + let qr = rqr.into_rows_result().unwrap(); assert_matches!(qr, None); } @@ -758,10 +622,10 @@ mod tests { { let rr = sample_raw_rows(1, 0); let rqr = QueryResult::new(Some(rr), None, Vec::new()); - let qr = rqr.rows_deserializer().unwrap().unwrap(); - assert_matches!(rqr.result_not_rows(), Err(ResultNotRowsError)); + let qr = rqr.into_rows_result().unwrap().unwrap(); + // Type check error { assert_matches!(qr.rows::<(i32,)>(), Err(RowsError::TypeCheckFailed(_))); @@ -800,14 +664,15 @@ mod tests { let rr_good_data = sample_raw_rows(2, 1); let rr_bad_data = sample_raw_rows_invalid_bytes(2, 1); let rqr_good_data = QueryResult::new(Some(rr_good_data), None, Vec::new()); - let qr_good_data = rqr_good_data.rows_deserializer().unwrap().unwrap(); let rqr_bad_data = QueryResult::new(Some(rr_bad_data), None, Vec::new()); - let qr_bad_data = rqr_bad_data.rows_deserializer().unwrap().unwrap(); for rqr in [&rqr_good_data, &rqr_bad_data] { assert_matches!(rqr.result_not_rows(), Err(ResultNotRowsError)); } + let qr_good_data = rqr_good_data.into_rows_result().unwrap().unwrap(); + let qr_bad_data = rqr_bad_data.into_rows_result().unwrap().unwrap(); + for qr in [&qr_good_data, &qr_bad_data] { // Type check error { @@ -860,10 +725,10 @@ mod tests { { let rr = sample_raw_rows(2, 2); let rqr = QueryResult::new(Some(rr), None, Vec::new()); - let qr = rqr.rows_deserializer().unwrap().unwrap(); - assert_matches!(rqr.result_not_rows(), Err(ResultNotRowsError)); + let qr = rqr.into_rows_result().unwrap().unwrap(); + // Type check error { assert_matches!(qr.rows::<(i32, i32)>(), Err(RowsError::TypeCheckFailed(_))); From 10c7b989c6f5eec8d0d300ebae8aee67101c7b57 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Wojciech=20Przytu=C5=82a?= Date: Mon, 4 Nov 2024 08:47:23 +0100 Subject: [PATCH 22/42] FIX: QueryResult --- scylla/src/lib.rs | 2 +- scylla/src/transport/query_result.rs | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/scylla/src/lib.rs b/scylla/src/lib.rs index 289460ba9..e9cc262f8 100644 --- a/scylla/src/lib.rs +++ b/scylla/src/lib.rs @@ -259,7 +259,7 @@ pub use frame::response::cql_to_rust::FromRow; pub use transport::caching_session::CachingSession; pub use transport::execution_profile::ExecutionProfile; pub use transport::legacy_query_result::LegacyQueryResult; -pub use transport::query_result::QueryResult; +pub use transport::query_result::{QueryResult, QueryRowsResult}; pub use transport::session::{IntoTypedRows, Session, SessionConfig}; pub use transport::session_builder::SessionBuilder; diff --git a/scylla/src/transport/query_result.rs b/scylla/src/transport/query_result.rs index a70ae5d44..1c8579e15 100644 --- a/scylla/src/transport/query_result.rs +++ b/scylla/src/transport/query_result.rs @@ -195,7 +195,7 @@ impl QueryResult { /// Returns `Ok` for a request's result that shouldn't contain any rows.\ /// Will return `Ok` for `INSERT` result, but a `SELECT` result, even an empty one, will cause an error.\ - /// Opposite of [`rows_deserializer()`](QueryResult::rows_deserializer). + /// Opposite of [`into_rows_result()`](QueryResult::rows_deserializer). #[inline] pub fn result_not_rows(&self) -> Result<(), ResultNotRowsError> { match &self.raw_metadata_and_rows { @@ -219,7 +219,7 @@ impl QueryResult { /// ```rust /// # use scylla::transport::query_result::{QueryResult, RowsDeserializer}; /// # fn example(query_result: QueryResult) -> Result<(), Box> { - /// let rows_deserializer = query_result.rows_deserializer()?; + /// let rows_deserializer = query_result.into_rows_result()?; /// if let Some(rows_result) = rows_deserializer { /// let mut rows_iter = rows_result.rows::<(i32, &str)>()?; /// while let Some((num, text)) = rows_iter.next().transpose()? { @@ -293,7 +293,7 @@ impl QueryResult { /// ```rust /// # use scylla::transport::query_result::QueryResult; /// # fn example(query_result: QueryResult) -> Result<(), Box> { -/// let rows_deserializer = query_result.rows_deserializer()?; +/// let rows_deserializer = query_result.into_rows_result()?; /// if let Some(rows_result) = rows_deserializer { /// let mut rows_iter = rows_result.rows::<(i32, &str)>()?; /// while let Some((num, text)) = rows_iter.next().transpose()? { From 3001dce189b3884fda322fd6a9424def5bdb2af7 Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Wed, 15 Mar 2023 15:35:35 +0100 Subject: [PATCH 23/42] treewide: rename Session to LegacySession MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This is a preparation for the API change of the Session: current implementation is renamed to LegacySession, a new one will be introduced later and everything will be gradually switched to the new implementation. Co-authored-by: Wojciech Przytuła --- examples/allocations.rs | 6 +- examples/basic.rs | 4 +- examples/compare-tokens.rs | 4 +- examples/cql-time-types.rs | 4 +- examples/cqlsh-rs.rs | 4 +- examples/custom_deserialization.rs | 4 +- examples/custom_load_balancing_policy.rs | 4 +- examples/execution_profile.rs | 8 +- examples/get_by_name.rs | 4 +- examples/logging.rs | 4 +- examples/logging_log.rs | 4 +- examples/parallel-prepared.rs | 4 +- examples/parallel.rs | 4 +- examples/query_history.rs | 4 +- examples/schema_agreement.rs | 4 +- examples/select-paging.rs | 4 +- examples/speculative-execution.rs | 4 +- examples/tls.rs | 4 +- examples/tower.rs | 2 +- examples/tracing.rs | 6 +- examples/user-defined-type.rs | 4 +- examples/value_list.rs | 4 +- scylla/src/lib.rs | 14 +- scylla/src/transport/caching_session.rs | 14 +- scylla/src/transport/cql_collections_test.rs | 16 +- scylla/src/transport/cql_types_test.rs | 40 ++--- scylla/src/transport/cql_value_test.rs | 6 +- scylla/src/transport/execution_profile.rs | 8 +- .../transport/large_batch_statements_test.rs | 6 +- scylla/src/transport/session.rs | 58 +++---- scylla/src/transport/session_builder.rs | 156 +++++++++--------- scylla/src/transport/session_test.rs | 30 ++-- .../transport/silent_prepare_batch_test.rs | 4 +- scylla/src/utils/test_utils.rs | 6 +- scylla/tests/integration/consistency.rs | 16 +- scylla/tests/integration/lwt_optimisation.rs | 4 +- scylla/tests/integration/retries.rs | 8 +- scylla/tests/integration/self_identity.rs | 4 +- .../tests/integration/silent_prepare_query.rs | 6 +- .../integration/skip_metadata_optimization.rs | 6 +- scylla/tests/integration/tablets.rs | 12 +- 41 files changed, 253 insertions(+), 255 deletions(-) diff --git a/examples/allocations.rs b/examples/allocations.rs index a3ec2a5cb..039d21e01 100644 --- a/examples/allocations.rs +++ b/examples/allocations.rs @@ -1,5 +1,5 @@ use anyhow::Result; -use scylla::{statement::prepared_statement::PreparedStatement, Session, SessionBuilder}; +use scylla::{statement::prepared_statement::PreparedStatement, LegacySession, SessionBuilder}; use std::io::Write; use std::sync::atomic::{AtomicUsize, Ordering}; use std::sync::Arc; @@ -65,7 +65,7 @@ fn print_stats(stats: &stats_alloc::Stats, reqs: f64) { } async fn measure( - session: Arc, + session: Arc, prepared: Arc, reqs: usize, parallelism: usize, @@ -128,7 +128,7 @@ async fn main() -> Result<()> { println!("Connecting to {} ...", args.node); - let session: Session = SessionBuilder::new().known_node(args.node).build().await?; + let session: LegacySession = SessionBuilder::new().known_node(args.node).build().await?; let session = Arc::new(session); session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; diff --git a/examples/basic.rs b/examples/basic.rs index 72b6a5ce1..48d97b713 100644 --- a/examples/basic.rs +++ b/examples/basic.rs @@ -1,7 +1,7 @@ use anyhow::Result; use futures::TryStreamExt; use scylla::macros::FromRow; -use scylla::transport::session::Session; +use scylla::transport::session::LegacySession; use scylla::SessionBuilder; use std::env; @@ -11,7 +11,7 @@ async fn main() -> Result<()> { println!("Connecting to {} ...", uri); - let session: Session = SessionBuilder::new().known_node(uri).build().await?; + let session: LegacySession = SessionBuilder::new().known_node(uri).build().await?; session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; diff --git a/examples/compare-tokens.rs b/examples/compare-tokens.rs index 9e9431d86..e302b9f83 100644 --- a/examples/compare-tokens.rs +++ b/examples/compare-tokens.rs @@ -1,7 +1,7 @@ use anyhow::Result; use scylla::routing::Token; use scylla::transport::NodeAddr; -use scylla::{Session, SessionBuilder}; +use scylla::{LegacySession, SessionBuilder}; use std::env; #[tokio::main] @@ -10,7 +10,7 @@ async fn main() -> Result<()> { println!("Connecting to {} ...", uri); - let session: Session = SessionBuilder::new().known_node(uri).build().await?; + let session: LegacySession = SessionBuilder::new().known_node(uri).build().await?; session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; diff --git a/examples/cql-time-types.rs b/examples/cql-time-types.rs index 548ac6987..8a8cedb66 100644 --- a/examples/cql-time-types.rs +++ b/examples/cql-time-types.rs @@ -6,7 +6,7 @@ use chrono::{DateTime, NaiveDate, NaiveTime, Utc}; use futures::{StreamExt, TryStreamExt}; use scylla::frame::response::result::CqlValue; use scylla::frame::value::{CqlDate, CqlTime, CqlTimestamp}; -use scylla::transport::session::Session; +use scylla::transport::session::LegacySession; use scylla::SessionBuilder; use std::env; @@ -16,7 +16,7 @@ async fn main() -> Result<()> { println!("Connecting to {} ...", uri); - let session: Session = SessionBuilder::new().known_node(uri).build().await?; + let session: LegacySession = SessionBuilder::new().known_node(uri).build().await?; session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; diff --git a/examples/cqlsh-rs.rs b/examples/cqlsh-rs.rs index c12b17a76..0b9cd5a18 100644 --- a/examples/cqlsh-rs.rs +++ b/examples/cqlsh-rs.rs @@ -4,7 +4,7 @@ use rustyline::error::ReadlineError; use rustyline::{CompletionType, Config, Context, Editor}; use rustyline_derive::{Helper, Highlighter, Hinter, Validator}; use scylla::transport::Compression; -use scylla::{LegacyQueryResult, Session, SessionBuilder}; +use scylla::{LegacyQueryResult, LegacySession, SessionBuilder}; use std::env; #[derive(Helper, Highlighter, Validator, Hinter)] @@ -199,7 +199,7 @@ async fn main() -> Result<()> { println!("Connecting to {} ...", uri); - let session: Session = SessionBuilder::new() + let session: LegacySession = SessionBuilder::new() .known_node(uri) .compression(Some(Compression::Lz4)) .build() diff --git a/examples/custom_deserialization.rs b/examples/custom_deserialization.rs index 1d0173ca5..976afe468 100644 --- a/examples/custom_deserialization.rs +++ b/examples/custom_deserialization.rs @@ -2,7 +2,7 @@ use anyhow::Result; use scylla::cql_to_rust::{FromCqlVal, FromCqlValError}; use scylla::frame::response::result::CqlValue; use scylla::macros::impl_from_cql_value_from_method; -use scylla::{Session, SessionBuilder}; +use scylla::{LegacySession, SessionBuilder}; use std::env; #[tokio::main] @@ -11,7 +11,7 @@ async fn main() -> Result<()> { println!("Connecting to {} ...", uri); - let session: Session = SessionBuilder::new().known_node(uri).build().await?; + let session: LegacySession = SessionBuilder::new().known_node(uri).build().await?; session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; session diff --git a/examples/custom_load_balancing_policy.rs b/examples/custom_load_balancing_policy.rs index 5c279f233..9fa505384 100644 --- a/examples/custom_load_balancing_policy.rs +++ b/examples/custom_load_balancing_policy.rs @@ -6,7 +6,7 @@ use scylla::{ load_balancing::{LoadBalancingPolicy, RoutingInfo}, routing::Shard, transport::{ClusterData, ExecutionProfile}, - Session, SessionBuilder, + LegacySession, SessionBuilder, }; use std::{env, sync::Arc}; @@ -68,7 +68,7 @@ async fn main() -> Result<()> { .load_balancing_policy(Arc::new(custom_load_balancing)) .build(); - let _session: Session = SessionBuilder::new() + let _session: LegacySession = SessionBuilder::new() .known_node(uri) .default_execution_profile_handle(profile.into_handle()) .build() diff --git a/examples/execution_profile.rs b/examples/execution_profile.rs index 3562966ac..944245660 100644 --- a/examples/execution_profile.rs +++ b/examples/execution_profile.rs @@ -4,7 +4,7 @@ use scylla::query::Query; use scylla::retry_policy::{DefaultRetryPolicy, FallthroughRetryPolicy}; use scylla::speculative_execution::PercentileSpeculativeExecutionPolicy; use scylla::statement::{Consistency, SerialConsistency}; -use scylla::transport::session::Session; +use scylla::transport::session::LegacySession; use scylla::transport::ExecutionProfile; use scylla::{SessionBuilder, SessionConfig}; use std::env; @@ -42,13 +42,13 @@ async fn main() -> Result<()> { let mut handle2 = profile2.into_handle(); // It is even possible to use multiple sessions interleaved, having them configured with different profiles. - let session1: Session = SessionBuilder::new() + let session1: LegacySession = SessionBuilder::new() .known_node(&uri) .default_execution_profile_handle(handle1.clone()) .build() .await?; - let session2: Session = SessionBuilder::new() + let session2: LegacySession = SessionBuilder::new() .known_node(&uri) .default_execution_profile_handle(handle2.clone()) .build() @@ -57,7 +57,7 @@ async fn main() -> Result<()> { // As default execution profile is not provided explicitly, session 3 uses a predefined one. let mut session_3_config = SessionConfig::new(); session_3_config.add_known_node(uri); - let session3: Session = Session::connect(session_3_config).await?; + let session3: LegacySession = LegacySession::connect(session_3_config).await?; session1.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; diff --git a/examples/get_by_name.rs b/examples/get_by_name.rs index bb750de1b..2f3996e5e 100644 --- a/examples/get_by_name.rs +++ b/examples/get_by_name.rs @@ -1,5 +1,5 @@ use anyhow::{anyhow, Result}; -use scylla::transport::session::Session; +use scylla::transport::session::LegacySession; use scylla::SessionBuilder; use std::env; @@ -10,7 +10,7 @@ async fn main() -> Result<()> { println!("Connecting to {} ...", uri); - let session: Session = SessionBuilder::new().known_node(uri).build().await?; + let session: LegacySession = SessionBuilder::new().known_node(uri).build().await?; session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; diff --git a/examples/logging.rs b/examples/logging.rs index 6b090acbc..00071c4cd 100644 --- a/examples/logging.rs +++ b/examples/logging.rs @@ -1,5 +1,5 @@ use anyhow::Result; -use scylla::transport::session::Session; +use scylla::transport::session::LegacySession; use scylla::SessionBuilder; use std::env; use tracing::info; @@ -16,7 +16,7 @@ async fn main() -> Result<()> { let uri = env::var("SCYLLA_URI").unwrap_or_else(|_| "127.0.0.1:9042".to_string()); info!("Connecting to {}", uri); - let session: Session = SessionBuilder::new().known_node(uri).build().await?; + let session: LegacySession = SessionBuilder::new().known_node(uri).build().await?; session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; session.query_unpaged("USE examples_ks", &[]).await?; diff --git a/examples/logging_log.rs b/examples/logging_log.rs index da82f4224..9e8c81687 100644 --- a/examples/logging_log.rs +++ b/examples/logging_log.rs @@ -1,5 +1,5 @@ use anyhow::Result; -use scylla::transport::session::Session; +use scylla::transport::session::LegacySession; use scylla::SessionBuilder; use std::env; use tracing::info; @@ -18,7 +18,7 @@ async fn main() -> Result<()> { let uri = env::var("SCYLLA_URI").unwrap_or_else(|_| "127.0.0.1:9042".to_string()); info!("Connecting to {}", uri); - let session: Session = SessionBuilder::new().known_node(uri).build().await?; + let session: LegacySession = SessionBuilder::new().known_node(uri).build().await?; session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; session.query_unpaged("USE examples_ks", &[]).await?; diff --git a/examples/parallel-prepared.rs b/examples/parallel-prepared.rs index 167b58394..e848b305c 100644 --- a/examples/parallel-prepared.rs +++ b/examples/parallel-prepared.rs @@ -1,5 +1,5 @@ use anyhow::Result; -use scylla::{Session, SessionBuilder}; +use scylla::{LegacySession, SessionBuilder}; use std::env; use std::sync::Arc; @@ -11,7 +11,7 @@ async fn main() -> Result<()> { println!("Connecting to {} ...", uri); - let session: Session = SessionBuilder::new().known_node(uri).build().await?; + let session: LegacySession = SessionBuilder::new().known_node(uri).build().await?; let session = Arc::new(session); session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; diff --git a/examples/parallel.rs b/examples/parallel.rs index 716225fb7..3cf191661 100644 --- a/examples/parallel.rs +++ b/examples/parallel.rs @@ -1,5 +1,5 @@ use anyhow::Result; -use scylla::{Session, SessionBuilder}; +use scylla::{LegacySession, SessionBuilder}; use std::env; use std::sync::Arc; @@ -11,7 +11,7 @@ async fn main() -> Result<()> { println!("Connecting to {} ...", uri); - let session: Session = SessionBuilder::new().known_node(uri).build().await?; + let session: LegacySession = SessionBuilder::new().known_node(uri).build().await?; let session = Arc::new(session); session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; diff --git a/examples/query_history.rs b/examples/query_history.rs index d5e361f0e..61ea56723 100644 --- a/examples/query_history.rs +++ b/examples/query_history.rs @@ -4,7 +4,7 @@ use anyhow::Result; use futures::StreamExt; use scylla::history::{HistoryCollector, StructuredHistory}; use scylla::query::Query; -use scylla::transport::session::Session; +use scylla::transport::session::LegacySession; use scylla::SessionBuilder; use std::env; use std::sync::Arc; @@ -15,7 +15,7 @@ async fn main() -> Result<()> { println!("Connecting to {} ...", uri); - let session: Session = SessionBuilder::new().known_node(uri).build().await?; + let session: LegacySession = SessionBuilder::new().known_node(uri).build().await?; session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; diff --git a/examples/schema_agreement.rs b/examples/schema_agreement.rs index 4709873fc..bfc5ed01b 100644 --- a/examples/schema_agreement.rs +++ b/examples/schema_agreement.rs @@ -1,7 +1,7 @@ use anyhow::{bail, Result}; use futures::TryStreamExt; use scylla::transport::errors::QueryError; -use scylla::transport::session::Session; +use scylla::transport::session::LegacySession; use scylla::SessionBuilder; use std::env; use std::time::Duration; @@ -13,7 +13,7 @@ async fn main() -> Result<()> { println!("Connecting to {} ...", uri); - let session: Session = SessionBuilder::new() + let session: LegacySession = SessionBuilder::new() .known_node(uri) .schema_agreement_interval(Duration::from_secs(1)) // check every second for schema agreement if not agreed first check .build() diff --git a/examples/select-paging.rs b/examples/select-paging.rs index b3a19e324..36d40f62d 100644 --- a/examples/select-paging.rs +++ b/examples/select-paging.rs @@ -1,7 +1,7 @@ use anyhow::Result; use futures::stream::StreamExt; use scylla::statement::PagingState; -use scylla::{query::Query, Session, SessionBuilder}; +use scylla::{query::Query, LegacySession, SessionBuilder}; use std::env; use std::ops::ControlFlow; @@ -11,7 +11,7 @@ async fn main() -> Result<()> { println!("Connecting to {} ...", uri); - let session: Session = SessionBuilder::new().known_node(uri).build().await?; + let session: LegacySession = SessionBuilder::new().known_node(uri).build().await?; session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; diff --git a/examples/speculative-execution.rs b/examples/speculative-execution.rs index e6c64e3ad..13513c1d9 100644 --- a/examples/speculative-execution.rs +++ b/examples/speculative-execution.rs @@ -1,6 +1,6 @@ use scylla::{ speculative_execution::PercentileSpeculativeExecutionPolicy, - transport::execution_profile::ExecutionProfile, Session, SessionBuilder, + transport::execution_profile::ExecutionProfile, LegacySession, SessionBuilder, }; use anyhow::Result; @@ -20,7 +20,7 @@ async fn main() -> Result<()> { .speculative_execution_policy(Some(Arc::new(speculative))) .build(); - let session: Session = SessionBuilder::new() + let session: LegacySession = SessionBuilder::new() .known_node(uri) .default_execution_profile_handle(speculative_profile.into_handle()) .build() diff --git a/examples/tls.rs b/examples/tls.rs index 067135214..c41e5e7f9 100644 --- a/examples/tls.rs +++ b/examples/tls.rs @@ -1,6 +1,6 @@ use anyhow::Result; use futures::TryStreamExt; -use scylla::transport::session::Session; +use scylla::transport::session::LegacySession; use scylla::SessionBuilder; use std::env; use std::fs; @@ -44,7 +44,7 @@ async fn main() -> Result<()> { context_builder.set_ca_file(ca_dir.as_path())?; context_builder.set_verify(SslVerifyMode::PEER); - let session: Session = SessionBuilder::new() + let session: LegacySession = SessionBuilder::new() .known_node(uri) .ssl_context(Some(context_builder.build())) .build() diff --git a/examples/tower.rs b/examples/tower.rs index 0d28407da..5f89890fc 100644 --- a/examples/tower.rs +++ b/examples/tower.rs @@ -7,7 +7,7 @@ use std::task::Poll; use tower::Service; struct SessionService { - session: Arc, + session: Arc, } // A trivial service implementation for sending parameterless simple string requests to Scylla. diff --git a/examples/tracing.rs b/examples/tracing.rs index 12767de5b..2ce7b2e61 100644 --- a/examples/tracing.rs +++ b/examples/tracing.rs @@ -10,7 +10,7 @@ use scylla::statement::{ use scylla::tracing::TracingInfo; use scylla::transport::iterator::LegacyRowIterator; use scylla::LegacyQueryResult; -use scylla::{Session, SessionBuilder}; +use scylla::{LegacySession, SessionBuilder}; use std::env; use std::num::NonZeroU32; use std::time::Duration; @@ -21,7 +21,7 @@ async fn main() -> Result<()> { let uri = env::var("SCYLLA_URI").unwrap_or_else(|_| "127.0.0.1:9042".to_string()); println!("Connecting to {} ...", uri); - let session: Session = SessionBuilder::new() + let session: LegacySession = SessionBuilder::new() .known_node(uri.as_str()) .build() .await?; @@ -112,7 +112,7 @@ async fn main() -> Result<()> { // Session configuration allows specifying custom settings for querying tracing info. // Tracing info might not immediately be available on queried node // so the driver performs a few attempts with sleeps in between. - let session: Session = SessionBuilder::new() + let session: LegacySession = SessionBuilder::new() .known_node(uri) .tracing_info_fetch_attempts(NonZeroU32::new(8).unwrap()) .tracing_info_fetch_interval(Duration::from_millis(100)) diff --git a/examples/user-defined-type.rs b/examples/user-defined-type.rs index 6e2d65286..e8be4b2f9 100644 --- a/examples/user-defined-type.rs +++ b/examples/user-defined-type.rs @@ -1,7 +1,7 @@ use anyhow::Result; use futures::TryStreamExt; use scylla::macros::FromUserType; -use scylla::{SerializeValue, Session, SessionBuilder}; +use scylla::{LegacySession, SerializeValue, SessionBuilder}; use std::env; #[tokio::main] @@ -10,7 +10,7 @@ async fn main() -> Result<()> { println!("Connecting to {} ...", uri); - let session: Session = SessionBuilder::new().known_node(uri).build().await?; + let session: LegacySession = SessionBuilder::new().known_node(uri).build().await?; session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; diff --git a/examples/value_list.rs b/examples/value_list.rs index 81568baee..e72d488b4 100644 --- a/examples/value_list.rs +++ b/examples/value_list.rs @@ -1,5 +1,5 @@ use anyhow::Result; -use scylla::{Session, SessionBuilder}; +use scylla::{LegacySession, SessionBuilder}; use std::env; #[tokio::main] @@ -8,7 +8,7 @@ async fn main() -> Result<()> { println!("Connecting to {} ...", uri); - let session: Session = SessionBuilder::new().known_node(uri).build().await?; + let session: LegacySession = SessionBuilder::new().known_node(uri).build().await?; session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; diff --git a/scylla/src/lib.rs b/scylla/src/lib.rs index e9cc262f8..34405e05d 100644 --- a/scylla/src/lib.rs +++ b/scylla/src/lib.rs @@ -17,12 +17,12 @@ //! `Session` is created by specifying a few known nodes and connecting to them: //! //! ```rust,no_run -//! use scylla::{Session, SessionBuilder}; +//! use scylla::{LegacySession, SessionBuilder}; //! use std::error::Error; //! //! #[tokio::main] //! async fn main() -> Result<(), Box> { -//! let session: Session = SessionBuilder::new() +//! let session: LegacySession = SessionBuilder::new() //! .known_node("127.0.0.1:9042") //! .known_node("1.2.3.4:9876") //! .build() @@ -50,9 +50,9 @@ //! //! The easiest way to specify bound values in a query is using a tuple: //! ```rust -//! # use scylla::Session; +//! # use scylla::LegacySession; //! # use std::error::Error; -//! # async fn check_only_compiles(session: &Session) -> Result<(), Box> { +//! # async fn check_only_compiles(session: &LegacySession) -> Result<(), Box> { //! // Insert an int and text into the table //! session //! .query_unpaged( @@ -69,9 +69,9 @@ //! The easiest way to read rows returned by a query is to cast each row to a tuple of values: //! //! ```rust -//! # use scylla::Session; +//! # use scylla::LegacySession; //! # use std::error::Error; -//! # async fn check_only_compiles(session: &Session) -> Result<(), Box> { +//! # async fn check_only_compiles(session: &LegacySession) -> Result<(), Box> { //! use scylla::IntoTypedRows; //! //! // Read rows containing an int and text @@ -260,7 +260,7 @@ pub use transport::caching_session::CachingSession; pub use transport::execution_profile::ExecutionProfile; pub use transport::legacy_query_result::LegacyQueryResult; pub use transport::query_result::{QueryResult, QueryRowsResult}; -pub use transport::session::{IntoTypedRows, Session, SessionConfig}; +pub use transport::session::{IntoTypedRows, LegacySession, SessionConfig}; pub use transport::session_builder::SessionBuilder; #[cfg(feature = "cloud")] diff --git a/scylla/src/transport/caching_session.rs b/scylla/src/transport/caching_session.rs index cbf9d3c6d..2b0fcc05e 100644 --- a/scylla/src/transport/caching_session.rs +++ b/scylla/src/transport/caching_session.rs @@ -5,7 +5,7 @@ use crate::statement::{PagingState, PagingStateResponse}; use crate::transport::errors::QueryError; use crate::transport::iterator::LegacyRowIterator; use crate::transport::partitioner::PartitionerName; -use crate::{LegacyQueryResult, Session}; +use crate::{LegacyQueryResult, LegacySession}; use bytes::Bytes; use dashmap::DashMap; use futures::future::try_join_all; @@ -35,7 +35,7 @@ pub struct CachingSession where S: Clone + BuildHasher, { - session: Session, + session: LegacySession, /// The prepared statement cache size /// If a prepared statement is added while the limit is reached, the oldest prepared statement /// is removed from the cache @@ -47,7 +47,7 @@ impl CachingSession where S: Default + BuildHasher + Clone, { - pub fn from(session: Session, cache_size: usize) -> Self { + pub fn from(session: LegacySession, cache_size: usize) -> Self { Self { session, max_capacity: cache_size, @@ -62,7 +62,7 @@ where { /// Builds a [`CachingSession`] from a [`Session`], a cache size, and a [`BuildHasher`]., /// using a customer hasher. - pub fn with_hasher(session: Session, cache_size: usize, hasher: S) -> Self { + pub fn with_hasher(session: LegacySession, cache_size: usize, hasher: S) -> Self { Self { session, max_capacity: cache_size, @@ -212,7 +212,7 @@ where self.max_capacity } - pub fn get_session(&self) -> &Session { + pub fn get_session(&self) -> &LegacySession { &self.session } } @@ -227,12 +227,12 @@ mod tests { use crate::{ batch::{Batch, BatchStatement}, prepared_statement::PreparedStatement, - CachingSession, Session, + CachingSession, LegacySession, }; use futures::TryStreamExt; use std::collections::BTreeSet; - async fn new_for_test(with_tablet_support: bool) -> Session { + async fn new_for_test(with_tablet_support: bool) -> LegacySession { let session = create_new_session_builder() .build() .await diff --git a/scylla/src/transport/cql_collections_test.rs b/scylla/src/transport/cql_collections_test.rs index d9fb52150..fe2a8a8d2 100644 --- a/scylla/src/transport/cql_collections_test.rs +++ b/scylla/src/transport/cql_collections_test.rs @@ -1,11 +1,11 @@ use crate::cql_to_rust::FromCqlVal; use crate::test_utils::{create_new_session_builder, setup_tracing}; use crate::utils::test_utils::unique_keyspace_name; -use crate::{frame::response::result::CqlValue, Session}; +use crate::{frame::response::result::CqlValue, LegacySession}; use scylla_cql::types::serialize::value::SerializeValue; use std::collections::{BTreeMap, BTreeSet, HashMap, HashSet}; -async fn connect() -> Session { +async fn connect() -> LegacySession { let session = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); @@ -14,7 +14,7 @@ async fn connect() -> Session { session } -async fn create_table(session: &Session, table_name: &str, value_type: &str) { +async fn create_table(session: &LegacySession, table_name: &str, value_type: &str) { session .query_unpaged( format!( @@ -28,7 +28,7 @@ async fn create_table(session: &Session, table_name: &str, value_type: &str) { } async fn insert_and_select( - session: &Session, + session: &LegacySession, table_name: &str, to_insert: &InsertT, expected: &SelectT, @@ -58,7 +58,7 @@ async fn insert_and_select( #[tokio::test] async fn test_cql_list() { setup_tracing(); - let session: Session = connect().await; + let session: LegacySession = connect().await; let table_name: &str = "test_cql_list_tab"; create_table(&session, table_name, "list").await; @@ -91,7 +91,7 @@ async fn test_cql_list() { #[tokio::test] async fn test_cql_set() { setup_tracing(); - let session: Session = connect().await; + let session: LegacySession = connect().await; let table_name: &str = "test_cql_set_tab"; create_table(&session, table_name, "set").await; @@ -155,7 +155,7 @@ async fn test_cql_set() { #[tokio::test] async fn test_cql_map() { setup_tracing(); - let session: Session = connect().await; + let session: LegacySession = connect().await; let table_name: &str = "test_cql_map_tab"; create_table(&session, table_name, "map").await; @@ -206,7 +206,7 @@ async fn test_cql_map() { #[tokio::test] async fn test_cql_tuple() { setup_tracing(); - let session: Session = connect().await; + let session: LegacySession = connect().await; let table_name: &str = "test_cql_tuple_tab"; create_table(&session, table_name, "tuple").await; diff --git a/scylla/src/transport/cql_types_test.rs b/scylla/src/transport/cql_types_test.rs index 072e7b8fd..32dd11638 100644 --- a/scylla/src/transport/cql_types_test.rs +++ b/scylla/src/transport/cql_types_test.rs @@ -4,7 +4,7 @@ use crate::frame::response::result::CqlValue; use crate::frame::value::{Counter, CqlDate, CqlTime, CqlTimestamp}; use crate::macros::FromUserType; use crate::test_utils::{create_new_session_builder, scylla_supports_tablets, setup_tracing}; -use crate::transport::session::Session; +use crate::transport::session::LegacySession; use crate::utils::test_utils::unique_keyspace_name; use itertools::Itertools; use scylla_cql::frame::value::{CqlTimeuuid, CqlVarint}; @@ -22,8 +22,8 @@ async fn init_test_maybe_without_tablets( table_name: &str, type_name: &str, supports_tablets: bool, -) -> Session { - let session: Session = create_new_session_builder().build().await.unwrap(); +) -> LegacySession { + let session: LegacySession = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); let mut create_ks = format!( @@ -61,7 +61,7 @@ async fn init_test_maybe_without_tablets( // Used to prepare a table for test // Creates a new keyspace // Drops and creates table {table_name} (id int PRIMARY KEY, val {type_name}) -async fn init_test(table_name: &str, type_name: &str) -> Session { +async fn init_test(table_name: &str, type_name: &str) -> LegacySession { init_test_maybe_without_tablets(table_name, type_name, true).await } @@ -77,7 +77,7 @@ async fn run_tests(tests: &[&str], type_name: &str) where T: SerializeValue + FromCqlVal + FromStr + Debug + Clone + PartialEq, { - let session: Session = init_test(type_name, type_name).await; + let session: LegacySession = init_test(type_name, type_name).await; session.await_schema_agreement().await.unwrap(); for test in tests.iter() { @@ -168,7 +168,7 @@ async fn test_cql_varint() { ]; let table_name = "cql_varint_tests"; - let session: Session = create_new_session_builder().build().await.unwrap(); + let session: LegacySession = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); session @@ -278,7 +278,7 @@ async fn test_counter() { // Can't use run_tests, because counters are special and can't be inserted let type_name = "counter"; - let session: Session = init_test_maybe_without_tablets(type_name, type_name, false).await; + let session: LegacySession = init_test_maybe_without_tablets(type_name, type_name, false).await; for (i, test) in tests.iter().enumerate() { let update_bound_value = format!("UPDATE {} SET val = val + ? WHERE id = ?", type_name); @@ -311,7 +311,7 @@ async fn test_naive_date_04() { use chrono::Datelike; use chrono::NaiveDate; - let session: Session = init_test("chrono_naive_date_tests", "date").await; + let session: LegacySession = init_test("chrono_naive_date_tests", "date").await; let min_naive_date: NaiveDate = NaiveDate::MIN; let min_naive_date_string = min_naive_date.format("%Y-%m-%d").to_string(); @@ -404,7 +404,7 @@ async fn test_cql_date() { setup_tracing(); // Tests value::Date which allows to insert dates outside NaiveDate range - let session: Session = init_test("cql_date_tests", "date").await; + let session: LegacySession = init_test("cql_date_tests", "date").await; let tests = [ ("1970-01-01", CqlDate(2_u32.pow(31))), @@ -465,7 +465,7 @@ async fn test_date_03() { setup_tracing(); use time::{Date, Month::*}; - let session: Session = init_test("time_date_tests", "date").await; + let session: LegacySession = init_test("time_date_tests", "date").await; let tests = [ // Basic test values @@ -551,7 +551,7 @@ async fn test_cql_time() { // CqlTime is an i64 - nanoseconds since midnight // in range 0..=86399999999999 - let session: Session = init_test("cql_time_tests", "time").await; + let session: LegacySession = init_test("cql_time_tests", "time").await; let max_time: i64 = 24 * 60 * 60 * 1_000_000_000 - 1; assert_eq!(max_time, 86399999999999); @@ -784,7 +784,7 @@ async fn test_time_03() { #[tokio::test] async fn test_cql_timestamp() { setup_tracing(); - let session: Session = init_test("cql_timestamp_tests", "timestamp").await; + let session: LegacySession = init_test("cql_timestamp_tests", "timestamp").await; //let epoch_date = NaiveDate::from_ymd_opt(1970, 1, 1).unwrap(); @@ -1164,7 +1164,7 @@ async fn test_offset_date_time_03() { #[tokio::test] async fn test_timeuuid() { setup_tracing(); - let session: Session = init_test("timeuuid_tests", "timeuuid").await; + let session: LegacySession = init_test("timeuuid_tests", "timeuuid").await; // A few random timeuuids generated manually let tests = [ @@ -1234,7 +1234,7 @@ async fn test_timeuuid() { #[tokio::test] async fn test_timeuuid_ordering() { setup_tracing(); - let session: Session = create_new_session_builder().build().await.unwrap(); + let session: LegacySession = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); session @@ -1316,7 +1316,7 @@ async fn test_timeuuid_ordering() { #[tokio::test] async fn test_inet() { setup_tracing(); - let session: Session = init_test("inet_tests", "inet").await; + let session: LegacySession = init_test("inet_tests", "inet").await; let tests = [ ("0.0.0.0", IpAddr::V4(Ipv4Addr::new(0, 0, 0, 0))), @@ -1397,7 +1397,7 @@ async fn test_inet() { #[tokio::test] async fn test_blob() { setup_tracing(); - let session: Session = init_test("blob_tests", "blob").await; + let session: LegacySession = init_test("blob_tests", "blob").await; let long_blob: Vec = vec![0x11; 1234]; let mut long_blob_str: String = "0x".to_string(); @@ -1466,7 +1466,7 @@ async fn test_udt_after_schema_update() { let table_name = "udt_tests"; let type_name = "usertype1"; - let session: Session = create_new_session_builder().build().await.unwrap(); + let session: LegacySession = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); session @@ -1595,7 +1595,7 @@ async fn test_udt_after_schema_update() { #[tokio::test] async fn test_empty() { setup_tracing(); - let session: Session = init_test("empty_tests", "int").await; + let session: LegacySession = init_test("empty_tests", "int").await; session .query_unpaged( @@ -1638,7 +1638,7 @@ async fn test_udt_with_missing_field() { let table_name = "udt_tests"; let type_name = "usertype1"; - let session: Session = create_new_session_builder().build().await.unwrap(); + let session: LegacySession = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); session @@ -1689,7 +1689,7 @@ async fn test_udt_with_missing_field() { let mut id = 0; async fn verify_insert_select_identity( - session: &Session, + session: &LegacySession, table_name: &str, id: i32, element: TQ, diff --git a/scylla/src/transport/cql_value_test.rs b/scylla/src/transport/cql_value_test.rs index 781ab919b..be1047ede 100644 --- a/scylla/src/transport/cql_value_test.rs +++ b/scylla/src/transport/cql_value_test.rs @@ -2,12 +2,12 @@ use crate::frame::{response::result::CqlValue, value::CqlDuration}; use crate::test_utils::{create_new_session_builder, setup_tracing}; use crate::utils::test_utils::unique_keyspace_name; -use crate::Session; +use crate::LegacySession; #[tokio::test] async fn test_cqlvalue_udt() { setup_tracing(); - let session: Session = create_new_session_builder().build().await.unwrap(); + let session: LegacySession = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); session .query_unpaged( @@ -72,7 +72,7 @@ async fn test_cqlvalue_udt() { #[tokio::test] async fn test_cqlvalue_duration() { setup_tracing(); - let session: Session = create_new_session_builder().build().await.unwrap(); + let session: LegacySession = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); session diff --git a/scylla/src/transport/execution_profile.rs b/scylla/src/transport/execution_profile.rs index a94addec5..2854611b4 100644 --- a/scylla/src/transport/execution_profile.rs +++ b/scylla/src/transport/execution_profile.rs @@ -16,7 +16,7 @@ //! # extern crate scylla; //! # use std::error::Error; //! # async fn check_only_compiles() -> Result<(), Box> { -//! use scylla::{Session, SessionBuilder}; +//! use scylla::{LegacySession, SessionBuilder}; //! use scylla::statement::Consistency; //! use scylla::transport::ExecutionProfile; //! @@ -27,7 +27,7 @@ //! //! let handle = profile.into_handle(); //! -//! let session: Session = SessionBuilder::new() +//! let session: LegacySession = SessionBuilder::new() //! .known_node("127.0.0.1:9042") //! .default_execution_profile_handle(handle) //! .build() @@ -109,7 +109,7 @@ //! # extern crate scylla; //! # use std::error::Error; //! # async fn check_only_compiles() -> Result<(), Box> { -//! use scylla::{Session, SessionBuilder}; +//! use scylla::{LegacySession, SessionBuilder}; //! use scylla::query::Query; //! use scylla::statement::Consistency; //! use scylla::transport::ExecutionProfile; @@ -125,7 +125,7 @@ //! let mut handle1 = profile1.clone().into_handle(); //! let mut handle2 = profile2.clone().into_handle(); //! -//! let session: Session = SessionBuilder::new() +//! let session: LegacySession = SessionBuilder::new() //! .known_node("127.0.0.1:9042") //! .default_execution_profile_handle(handle1.clone()) //! .build() diff --git a/scylla/src/transport/large_batch_statements_test.rs b/scylla/src/transport/large_batch_statements_test.rs index 33628a49d..2b394ed32 100644 --- a/scylla/src/transport/large_batch_statements_test.rs +++ b/scylla/src/transport/large_batch_statements_test.rs @@ -7,7 +7,7 @@ use crate::transport::errors::{BadQuery, QueryError}; use crate::{ batch::Batch, test_utils::{create_new_session_builder, unique_keyspace_name}, - LegacyQueryResult, Session, + LegacyQueryResult, LegacySession, }; #[tokio::test] @@ -31,7 +31,7 @@ async fn test_large_batch_statements() { ) } -async fn create_test_session(session: Session, ks: &String) -> Session { +async fn create_test_session(session: LegacySession, ks: &String) -> LegacySession { session .query_unpaged( format!("CREATE KEYSPACE {} WITH REPLICATION = {{ 'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1 }}",ks), @@ -52,7 +52,7 @@ async fn create_test_session(session: Session, ks: &String) -> Session { } async fn write_batch( - session: &Session, + session: &LegacySession, n: usize, ks: &String, ) -> Result { diff --git a/scylla/src/transport/session.rs b/scylla/src/transport/session.rs index 8468183d2..6b42348b2 100644 --- a/scylla/src/transport/session.rs +++ b/scylla/src/transport/session.rs @@ -153,7 +153,7 @@ impl AddressTranslator for HashMap<&'static str, &'static str> { } /// `Session` manages connections to the cluster and allows to perform queries -pub struct Session { +pub struct LegacySession { cluster: Cluster, default_execution_profile_handle: ExecutionProfileHandle, schema_agreement_interval: Duration, @@ -169,7 +169,7 @@ pub struct Session { /// This implementation deliberately omits some details from Cluster in order /// to avoid cluttering the print with much information of little usability. -impl std::fmt::Debug for Session { +impl std::fmt::Debug for LegacySession { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { f.debug_struct("Session") .field("cluster", &ClusterNeatDebug(&self.cluster)) @@ -465,8 +465,8 @@ pub(crate) enum RunQueryResult { /// Represents a CQL session, which can be used to communicate /// with the database -impl Session { - /// Establishes a CQL session with the database +impl LegacySession { + /// Estabilishes a CQL session with the database /// /// Usually it's easier to use [SessionBuilder](crate::transport::session_builder::SessionBuilder) /// instead of calling `Session::connect` directly, because it's more convenient. @@ -478,17 +478,17 @@ impl Session { /// ```rust /// # use std::error::Error; /// # async fn check_only_compiles() -> Result<(), Box> { - /// use scylla::{Session, SessionConfig}; + /// use scylla::{LegacySession, SessionConfig}; /// use scylla::transport::KnownNode; /// /// let mut config = SessionConfig::new(); /// config.known_nodes.push(KnownNode::Hostname("127.0.0.1:9042".to_string())); /// - /// let session: Session = Session::connect(config).await?; + /// let session: LegacySession = LegacySession::connect(config).await?; /// # Ok(()) /// # } /// ``` - pub async fn connect(config: SessionConfig) -> Result { + pub async fn connect(config: SessionConfig) -> Result { let known_nodes = config.known_nodes; #[cfg(feature = "cloud")] @@ -562,7 +562,7 @@ impl Session { let default_execution_profile_handle = config.default_execution_profile_handle; - let session = Session { + let session = LegacySession { cluster, default_execution_profile_handle, schema_agreement_interval: config.schema_agreement_interval, @@ -608,9 +608,9 @@ impl Session { /// /// # Examples /// ```rust - /// # use scylla::Session; + /// # use scylla::LegacySession; /// # use std::error::Error; - /// # async fn check_only_compiles(session: &Session) -> Result<(), Box> { + /// # async fn check_only_compiles(session: &LegacySession) -> Result<(), Box> { /// // Insert an int and text into a table. /// session /// .query_unpaged( @@ -622,9 +622,9 @@ impl Session { /// # } /// ``` /// ```rust - /// # use scylla::Session; + /// # use scylla::LegacySession; /// # use std::error::Error; - /// # async fn check_only_compiles(session: &Session) -> Result<(), Box> { + /// # async fn check_only_compiles(session: &LegacySession) -> Result<(), Box> { /// use scylla::IntoTypedRows; /// /// // Read rows containing an int and text. @@ -676,9 +676,9 @@ impl Session { /// # Example /// /// ```rust - /// # use scylla::Session; + /// # use scylla::LegacySession; /// # use std::error::Error; - /// # async fn check_only_compiles(session: &Session) -> Result<(), Box> { + /// # async fn check_only_compiles(session: &LegacySession) -> Result<(), Box> { /// use std::ops::ControlFlow; /// use scylla::statement::PagingState; /// @@ -888,9 +888,9 @@ impl Session { /// # Example /// /// ```rust - /// # use scylla::Session; + /// # use scylla::LegacySession; /// # use std::error::Error; - /// # async fn check_only_compiles(session: &Session) -> Result<(), Box> { + /// # async fn check_only_compiles(session: &LegacySession) -> Result<(), Box> { /// use scylla::IntoTypedRows; /// use futures::stream::StreamExt; /// @@ -965,9 +965,9 @@ impl Session { /// /// # Example /// ```rust - /// # use scylla::Session; + /// # use scylla::LegacySession; /// # use std::error::Error; - /// # async fn check_only_compiles(session: &Session) -> Result<(), Box> { + /// # async fn check_only_compiles(session: &LegacySession) -> Result<(), Box> { /// use scylla::prepared_statement::PreparedStatement; /// /// // Prepare the query for later execution @@ -1064,9 +1064,9 @@ impl Session { /// /// # Example /// ```rust - /// # use scylla::Session; + /// # use scylla::LegacySession; /// # use std::error::Error; - /// # async fn check_only_compiles(session: &Session) -> Result<(), Box> { + /// # async fn check_only_compiles(session: &LegacySession) -> Result<(), Box> { /// use scylla::prepared_statement::PreparedStatement; /// /// // Prepare the query for later execution @@ -1108,9 +1108,9 @@ impl Session { /// # Example /// /// ```rust - /// # use scylla::Session; + /// # use scylla::LegacySession; /// # use std::error::Error; - /// # async fn check_only_compiles(session: &Session) -> Result<(), Box> { + /// # async fn check_only_compiles(session: &LegacySession) -> Result<(), Box> { /// use std::ops::ControlFlow; /// use scylla::query::Query; /// use scylla::statement::{PagingState, PagingStateResponse}; @@ -1287,9 +1287,9 @@ impl Session { /// # Example /// /// ```rust - /// # use scylla::Session; + /// # use scylla::LegacySession; /// # use std::error::Error; - /// # async fn check_only_compiles(session: &Session) -> Result<(), Box> { + /// # async fn check_only_compiles(session: &LegacySession) -> Result<(), Box> { /// use scylla::prepared_statement::PreparedStatement; /// use scylla::IntoTypedRows; /// use futures::stream::StreamExt; @@ -1355,9 +1355,9 @@ impl Session { /// /// # Example /// ```rust - /// # use scylla::Session; + /// # use scylla::LegacySession; /// # use std::error::Error; - /// # async fn check_only_compiles(session: &Session) -> Result<(), Box> { + /// # async fn check_only_compiles(session: &LegacySession) -> Result<(), Box> { /// use scylla::batch::Batch; /// /// let mut batch: Batch = Default::default(); @@ -1477,9 +1477,9 @@ impl Session { /// /// # Example /// ```rust /// # extern crate scylla; - /// # use scylla::Session; + /// # use scylla::LegacySession; /// # use std::error::Error; - /// # async fn check_only_compiles(session: &Session) -> Result<(), Box> { + /// # async fn check_only_compiles(session: &LegacySession) -> Result<(), Box> { /// use scylla::batch::Batch; /// /// // Create a batch statement with unprepared statements @@ -1538,7 +1538,7 @@ impl Session { /// * `case_sensitive` - if set to true the generated query will put keyspace name in quotes /// # Example /// ```rust - /// # use scylla::{Session, SessionBuilder}; + /// # use scylla::{LegacySession, SessionBuilder}; /// # use scylla::transport::Compression; /// # async fn example() -> Result<(), Box> { /// # let session = SessionBuilder::new().known_node("127.0.0.1:9042").build().await?; diff --git a/scylla/src/transport/session_builder.rs b/scylla/src/transport/session_builder.rs index 9a7a9cbf7..15e10e098 100644 --- a/scylla/src/transport/session_builder.rs +++ b/scylla/src/transport/session_builder.rs @@ -2,7 +2,7 @@ use super::connection::SelfIdentity; use super::execution_profile::ExecutionProfileHandle; -use super::session::{AddressTranslator, Session, SessionConfig}; +use super::session::{AddressTranslator, LegacySession, SessionConfig}; use super::Compression; #[cfg(feature = "cloud")] @@ -59,10 +59,10 @@ pub type CloudSessionBuilder = GenericSessionBuilder; /// # Example /// /// ``` -/// # use scylla::{Session, SessionBuilder}; +/// # use scylla::{LegacySession, SessionBuilder}; /// # use scylla::transport::Compression; /// # async fn example() -> Result<(), Box> { -/// let session: Session = SessionBuilder::new() +/// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .compression(Some(Compression::Snappy)) /// .build() @@ -94,17 +94,17 @@ impl GenericSessionBuilder { /// Add a known node with a hostname /// # Examples /// ``` - /// # use scylla::{Session, SessionBuilder}; + /// # use scylla::{LegacySession, SessionBuilder}; /// # async fn example() -> Result<(), Box> { - /// let session: Session = SessionBuilder::new().known_node("127.0.0.1:9042").build().await?; + /// let session: LegacySession = SessionBuilder::new().known_node("127.0.0.1:9042").build().await?; /// # Ok(()) /// # } /// ``` /// /// ``` - /// # use scylla::{Session, SessionBuilder}; + /// # use scylla::{LegacySession, SessionBuilder}; /// # async fn example() -> Result<(), Box> { - /// let session: Session = SessionBuilder::new().known_node("db1.example.com").build().await?; + /// let session: LegacySession = SessionBuilder::new().known_node("db1.example.com").build().await?; /// # Ok(()) /// # } /// ``` @@ -116,10 +116,10 @@ impl GenericSessionBuilder { /// Add a known node with an IP address /// # Example /// ``` - /// # use scylla::{Session, SessionBuilder}; + /// # use scylla::{LegacySession, SessionBuilder}; /// # use std::net::{SocketAddr, IpAddr, Ipv4Addr}; /// # async fn example() -> Result<(), Box> { - /// let session: Session = SessionBuilder::new() + /// let session: LegacySession = SessionBuilder::new() /// .known_node_addr(SocketAddr::new(IpAddr::V4(Ipv4Addr::new(127, 0, 0, 1)), 9042)) /// .build() /// .await?; @@ -134,9 +134,9 @@ impl GenericSessionBuilder { /// Add a list of known nodes with hostnames /// # Example /// ``` - /// # use scylla::{Session, SessionBuilder}; + /// # use scylla::{LegacySession, SessionBuilder}; /// # async fn example() -> Result<(), Box> { - /// let session: Session = SessionBuilder::new() + /// let session: LegacySession = SessionBuilder::new() /// .known_nodes(["127.0.0.1:9042", "db1.example.com"]) /// .build() /// .await?; @@ -151,13 +151,13 @@ impl GenericSessionBuilder { /// Add a list of known nodes with IP addresses /// # Example /// ``` - /// # use scylla::{Session, SessionBuilder}; + /// # use scylla::{LegacySession, SessionBuilder}; /// # use std::net::{SocketAddr, IpAddr, Ipv4Addr}; /// # async fn example() -> Result<(), Box> { /// let addr1 = SocketAddr::new(IpAddr::V4(Ipv4Addr::new(172, 17, 0, 3)), 9042); /// let addr2 = SocketAddr::new(IpAddr::V4(Ipv4Addr::new(172, 17, 0, 4)), 9042); /// - /// let session: Session = SessionBuilder::new() + /// let session: LegacySession = SessionBuilder::new() /// .known_nodes_addr([addr1, addr2]) /// .build() /// .await?; @@ -177,10 +177,10 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{Session, SessionBuilder}; + /// # use scylla::{LegacySession, SessionBuilder}; /// # use scylla::transport::Compression; /// # async fn example() -> Result<(), Box> { - /// let session: Session = SessionBuilder::new() + /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .use_keyspace("my_keyspace_name", false) /// .user("cassandra", "cassandra") @@ -203,7 +203,7 @@ impl GenericSessionBuilder { /// ``` /// # use std::sync::Arc; /// use bytes::Bytes; - /// use scylla::{Session, SessionBuilder}; + /// use scylla::{LegacySession, SessionBuilder}; /// use async_trait::async_trait; /// use scylla::authentication::{AuthenticatorProvider, AuthenticatorSession, AuthError}; /// # use scylla::transport::Compression; @@ -231,7 +231,7 @@ impl GenericSessionBuilder { /// } /// /// # async fn example() -> Result<(), Box> { - /// let session: Session = SessionBuilder::new() + /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .use_keyspace("my_keyspace_name", false) /// .user("cassandra", "cassandra") @@ -257,7 +257,7 @@ impl GenericSessionBuilder { /// # use async_trait::async_trait; /// # use std::net::SocketAddr; /// # use std::sync::Arc; - /// # use scylla::{Session, SessionBuilder}; + /// # use scylla::{LegacySession, SessionBuilder}; /// # use scylla::transport::session::{AddressTranslator, TranslationError}; /// # use scylla::transport::topology::UntranslatedPeer; /// struct IdentityTranslator; @@ -273,7 +273,7 @@ impl GenericSessionBuilder { /// } /// /// # async fn example() -> Result<(), Box> { - /// let session: Session = SessionBuilder::new() + /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .address_translator(Arc::new(IdentityTranslator)) /// .build() @@ -287,7 +287,7 @@ impl GenericSessionBuilder { /// # use std::sync::Arc; /// # use std::collections::HashMap; /// # use std::str::FromStr; - /// # use scylla::{Session, SessionBuilder}; + /// # use scylla::{LegacySession, SessionBuilder}; /// # use scylla::transport::session::{AddressTranslator, TranslationError}; /// # /// # async fn example() -> Result<(), Box> { @@ -295,7 +295,7 @@ impl GenericSessionBuilder { /// let addr_before_translation = SocketAddr::from_str("192.168.0.42:19042").unwrap(); /// let addr_after_translation = SocketAddr::from_str("157.123.12.42:23203").unwrap(); /// translation_rules.insert(addr_before_translation, addr_after_translation); - /// let session: Session = SessionBuilder::new() + /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .address_translator(Arc::new(translation_rules)) /// .build() @@ -318,7 +318,7 @@ impl GenericSessionBuilder { /// ``` /// # use std::fs; /// # use std::path::PathBuf; - /// # use scylla::{Session, SessionBuilder}; + /// # use scylla::{LegacySession, SessionBuilder}; /// # use openssl::ssl::{SslContextBuilder, SslVerifyMode, SslMethod, SslFiletype}; /// # async fn example() -> Result<(), Box> { /// let certdir = fs::canonicalize(PathBuf::from("./examples/certs/scylla.crt"))?; @@ -326,7 +326,7 @@ impl GenericSessionBuilder { /// context_builder.set_certificate_file(certdir.as_path(), SslFiletype::PEM)?; /// context_builder.set_verify(SslVerifyMode::NONE); /// - /// let session: Session = SessionBuilder::new() + /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .ssl_context(Some(context_builder.build())) /// .build() @@ -341,8 +341,8 @@ impl GenericSessionBuilder { } } -// NOTE: this `impl` block contains configuration options specific for **Cloud** [`Session`]. -// This means that if an option fits both non-Cloud and Cloud `Session`s, it should NOT be put +// NOTE: this `impl` block contains configuration options specific for **Cloud** [`LegacySession`]. +// This means that if an option fits both non-Cloud and Cloud `LegacySession`s, it should NOT be put // here, but rather in `impl GenericSessionBuilder` block. #[cfg(feature = "cloud")] impl CloudSessionBuilder { @@ -377,10 +377,10 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{Session, SessionBuilder}; + /// # use scylla::{LegacySession, SessionBuilder}; /// # use scylla::transport::Compression; /// # async fn example() -> Result<(), Box> { - /// let session: Session = SessionBuilder::new() + /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .compression(Some(Compression::Snappy)) /// .build() @@ -398,10 +398,10 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{Session, SessionBuilder}; + /// # use scylla::{LegacySession, SessionBuilder}; /// # use std::time::Duration; /// # async fn example() -> Result<(), Box> { - /// let session: Session = SessionBuilder::new() + /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .schema_agreement_interval(Duration::from_secs(5)) /// .build() @@ -418,14 +418,14 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{statement::Consistency, ExecutionProfile, Session, SessionBuilder}; + /// # use scylla::{statement::Consistency, ExecutionProfile, LegacySession, SessionBuilder}; /// # use std::time::Duration; /// # async fn example() -> Result<(), Box> { /// let execution_profile = ExecutionProfile::builder() /// .consistency(Consistency::All) /// .request_timeout(Some(Duration::from_secs(2))) /// .build(); - /// let session: Session = SessionBuilder::new() + /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .default_execution_profile_handle(execution_profile.into_handle()) /// .build() @@ -446,9 +446,9 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{Session, SessionBuilder}; + /// # use scylla::{LegacySession, SessionBuilder}; /// # async fn example() -> Result<(), Box> { - /// let session: Session = SessionBuilder::new() + /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .tcp_nodelay(true) /// .build() @@ -469,9 +469,9 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{Session, SessionBuilder}; + /// # use scylla::{LegacySession, SessionBuilder}; /// # async fn example() -> Result<(), Box> { - /// let session: Session = SessionBuilder::new() + /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .tcp_keepalive_interval(std::time::Duration::from_secs(42)) /// .build() @@ -497,10 +497,10 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{Session, SessionBuilder}; + /// # use scylla::{LegacySession, SessionBuilder}; /// # use scylla::transport::Compression; /// # async fn example() -> Result<(), Box> { - /// let session: Session = SessionBuilder::new() + /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .use_keyspace("my_keyspace_name", false) /// .build() @@ -518,10 +518,10 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{Session, SessionBuilder}; + /// # use scylla::{LegacySession, SessionBuilder}; /// # use scylla::transport::Compression; /// # async fn example() -> Result<(), Box> { - /// let session: Session = SessionBuilder::new() + /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .compression(Some(Compression::Snappy)) /// .build() // Turns SessionBuilder into Session @@ -529,8 +529,8 @@ impl GenericSessionBuilder { /// # Ok(()) /// # } /// ``` - pub async fn build(&self) -> Result { - Session::connect(self.config.clone()).await + pub async fn build(&self) -> Result { + LegacySession::connect(self.config.clone()).await } /// Changes connection timeout @@ -539,10 +539,10 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{Session, SessionBuilder}; + /// # use scylla::{LegacySession, SessionBuilder}; /// # use std::time::Duration; /// # async fn example() -> Result<(), Box> { - /// let session: Session = SessionBuilder::new() + /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .connection_timeout(Duration::from_secs(30)) /// .build() // Turns SessionBuilder into Session @@ -560,14 +560,14 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{Session, SessionBuilder}; + /// # use scylla::{LegacySession, SessionBuilder}; /// # async fn example() -> Result<(), Box> { /// use std::num::NonZeroUsize; /// use scylla::transport::session::PoolSize; /// /// // This session will establish 4 connections to each node. /// // For Scylla clusters, this number will be divided across shards - /// let session: Session = SessionBuilder::new() + /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .pool_size(PoolSize::PerHost(NonZeroUsize::new(4).unwrap())) /// .build() @@ -604,9 +604,9 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{Session, SessionBuilder}; + /// # use scylla::{LegacySession, SessionBuilder}; /// # async fn example() -> Result<(), Box> { - /// let session: Session = SessionBuilder::new() + /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .disallow_shard_aware_port(true) /// .build() @@ -624,9 +624,9 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{Session, SessionBuilder}; + /// # use scylla::{LegacySession, SessionBuilder}; /// # async fn example() -> Result<(), Box> { - /// let session: Session = SessionBuilder::new() + /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .keyspaces_to_fetch(["my_keyspace"]) /// .build() @@ -647,9 +647,9 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{Session, SessionBuilder}; + /// # use scylla::{LegacySession, SessionBuilder}; /// # async fn example() -> Result<(), Box> { - /// let session: Session = SessionBuilder::new() + /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .fetch_schema_metadata(true) /// .build() @@ -670,9 +670,9 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{Session, SessionBuilder}; + /// # use scylla::{LegacySession, SessionBuilder}; /// # async fn example() -> Result<(), Box> { - /// let session: Session = SessionBuilder::new() + /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .keepalive_interval(std::time::Duration::from_secs(42)) /// .build() @@ -700,9 +700,9 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{Session, SessionBuilder}; + /// # use scylla::{LegacySession, SessionBuilder}; /// # async fn example() -> Result<(), Box> { - /// let session: Session = SessionBuilder::new() + /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .keepalive_timeout(std::time::Duration::from_secs(42)) /// .build() @@ -727,9 +727,9 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{Session, SessionBuilder}; + /// # use scylla::{LegacySession, SessionBuilder}; /// # async fn example() -> Result<(), Box> { - /// let session: Session = SessionBuilder::new() + /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .schema_agreement_timeout(std::time::Duration::from_secs(120)) /// .build() @@ -747,9 +747,9 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{Session, SessionBuilder}; + /// # use scylla::{LegacySession, SessionBuilder}; /// # async fn example() -> Result<(), Box> { - /// let session: Session = SessionBuilder::new() + /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .auto_await_schema_agreement(false) /// .build() @@ -775,13 +775,13 @@ impl GenericSessionBuilder { /// # use async_trait::async_trait; /// # use std::net::SocketAddr; /// # use std::sync::Arc; - /// # use scylla::{Session, SessionBuilder}; + /// # use scylla::{LegacySession, SessionBuilder}; /// # use scylla::transport::session::{AddressTranslator, TranslationError}; /// # use scylla::transport::host_filter::DcHostFilter; /// /// # async fn example() -> Result<(), Box> { /// // The session will only connect to nodes from "my-local-dc" - /// let session: Session = SessionBuilder::new() + /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .host_filter(Arc::new(DcHostFilter::new("my-local-dc".to_string()))) /// .build() @@ -799,9 +799,9 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{Session, SessionBuilder}; + /// # use scylla::{LegacySession, SessionBuilder}; /// # async fn example() -> Result<(), Box> { - /// let session: Session = SessionBuilder::new() + /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .refresh_metadata_on_auto_schema_agreement(true) /// .build() @@ -815,7 +815,7 @@ impl GenericSessionBuilder { } /// Set the number of attempts to fetch [TracingInfo](crate::tracing::TracingInfo) - /// in [`Session::get_tracing_info`]. + /// in [`LegacySession::get_tracing_info`]. /// The default is 5 attempts. /// /// Tracing info might not be available immediately on queried node - that's why @@ -827,10 +827,10 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{Session, SessionBuilder}; + /// # use scylla::{LegacySession, SessionBuilder}; /// # use std::num::NonZeroU32; /// # async fn example() -> Result<(), Box> { - /// let session: Session = SessionBuilder::new() + /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .tracing_info_fetch_attempts(NonZeroU32::new(10).unwrap()) /// .build() @@ -844,7 +844,7 @@ impl GenericSessionBuilder { } /// Set the delay between attempts to fetch [TracingInfo](crate::tracing::TracingInfo) - /// in [`Session::get_tracing_info`]. + /// in [`LegacySession::get_tracing_info`]. /// The default is 3 milliseconds. /// /// Tracing info might not be available immediately on queried node - that's why @@ -856,10 +856,10 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{Session, SessionBuilder}; + /// # use scylla::{LegacySession, SessionBuilder}; /// # use std::time::Duration; /// # async fn example() -> Result<(), Box> { - /// let session: Session = SessionBuilder::new() + /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .tracing_info_fetch_interval(Duration::from_millis(50)) /// .build() @@ -873,14 +873,14 @@ impl GenericSessionBuilder { } /// Set the consistency level of fetching [TracingInfo](crate::tracing::TracingInfo) - /// in [`Session::get_tracing_info`]. + /// in [`LegacySession::get_tracing_info`]. /// The default is [`Consistency::One`]. /// /// # Example /// ``` - /// # use scylla::{Session, SessionBuilder, statement::Consistency}; + /// # use scylla::{LegacySession, SessionBuilder, statement::Consistency}; /// # async fn example() -> Result<(), Box> { - /// let session: Session = SessionBuilder::new() + /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .tracing_info_fetch_consistency(Consistency::One) /// .build() @@ -908,10 +908,10 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{Session, SessionBuilder}; + /// # use scylla::{LegacySession, SessionBuilder}; /// # use scylla::transport::Compression; /// # async fn example() -> Result<(), Box> { - /// let session: Session = SessionBuilder::new() + /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .write_coalescing(false) // Enabled by default /// .build() @@ -933,9 +933,9 @@ impl GenericSessionBuilder { /// means that the metadata is refreshed every 20 seconds. /// # Example /// ``` - /// # use scylla::{Session, SessionBuilder}; + /// # use scylla::{LegacySession, SessionBuilder}; /// # async fn example() -> Result<(), Box> { - /// let session: Session = SessionBuilder::new() + /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .cluster_metadata_refresh_interval(std::time::Duration::from_secs(20)) /// .build() @@ -956,13 +956,13 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{Session, SessionBuilder}; + /// # use scylla::{LegacySession, SessionBuilder}; /// # use scylla::transport::SelfIdentity; /// # async fn example() -> Result<(), Box> { /// let (app_major, app_minor, app_patch) = (2, 1, 3); /// let app_version = format!("{app_major}.{app_minor}.{app_patch}"); /// - /// let session: Session = SessionBuilder::new() + /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .custom_identity( /// SelfIdentity::new() diff --git a/scylla/src/transport/session_test.rs b/scylla/src/transport/session_test.rs index d4222d3b5..b596e05b2 100644 --- a/scylla/src/transport/session_test.rs +++ b/scylla/src/transport/session_test.rs @@ -22,7 +22,7 @@ use crate::utils::test_utils::{ use crate::CachingSession; use crate::ExecutionProfile; use crate::LegacyQueryResult; -use crate::{Session, SessionBuilder}; +use crate::{LegacySession, SessionBuilder}; use assert_matches::assert_matches; use futures::{FutureExt, StreamExt, TryStreamExt}; use itertools::Itertools; @@ -714,7 +714,7 @@ async fn test_use_keyspace() { )); // Make sure that use_keyspace on SessionBuiler works - let session2: Session = create_new_session_builder() + let session2: LegacySession = create_new_session_builder() .use_keyspace(ks.clone(), false) .build() .await @@ -956,7 +956,7 @@ async fn test_tracing() { test_tracing_batch(&session, ks.clone()).await; } -async fn test_tracing_query(session: &Session, ks: String) { +async fn test_tracing_query(session: &LegacySession, ks: String) { // A query without tracing enabled has no tracing uuid in result let untraced_query: Query = Query::new(format!("SELECT * FROM {}.tab", ks)); let untraced_query_result: LegacyQueryResult = @@ -976,7 +976,7 @@ async fn test_tracing_query(session: &Session, ks: String) { assert_in_tracing_table(session, traced_query_result.tracing_id.unwrap()).await; } -async fn test_tracing_execute(session: &Session, ks: String) { +async fn test_tracing_execute(session: &LegacySession, ks: String) { // Executing a prepared statement without tracing enabled has no tracing uuid in result let untraced_prepared = session .prepare(format!("SELECT * FROM {}.tab", ks)) @@ -1008,7 +1008,7 @@ async fn test_tracing_execute(session: &Session, ks: String) { assert_in_tracing_table(session, traced_prepared_result.tracing_id.unwrap()).await; } -async fn test_tracing_prepare(session: &Session, ks: String) { +async fn test_tracing_prepare(session: &LegacySession, ks: String) { // Preparing a statement without tracing enabled has no tracing uuids in result let untraced_prepared = session .prepare(format!("SELECT * FROM {}.tab", ks)) @@ -1030,7 +1030,7 @@ async fn test_tracing_prepare(session: &Session, ks: String) { } } -async fn test_get_tracing_info(session: &Session, ks: String) { +async fn test_get_tracing_info(session: &LegacySession, ks: String) { // A query with tracing enabled has a tracing uuid in result let mut traced_query: Query = Query::new(format!("SELECT * FROM {}.tab", ks)); traced_query.config.tracing = true; @@ -1045,7 +1045,7 @@ async fn test_get_tracing_info(session: &Session, ks: String) { assert!(!tracing_info.nodes().is_empty()); } -async fn test_tracing_query_iter(session: &Session, ks: String) { +async fn test_tracing_query_iter(session: &LegacySession, ks: String) { // A query without tracing enabled has no tracing ids let untraced_query: Query = Query::new(format!("SELECT * FROM {}.tab", ks)); @@ -1080,7 +1080,7 @@ async fn test_tracing_query_iter(session: &Session, ks: String) { } } -async fn test_tracing_execute_iter(session: &Session, ks: String) { +async fn test_tracing_execute_iter(session: &LegacySession, ks: String) { // A prepared statement without tracing enabled has no tracing ids let untraced_prepared = session .prepare(format!("SELECT * FROM {}.tab", ks)) @@ -1121,7 +1121,7 @@ async fn test_tracing_execute_iter(session: &Session, ks: String) { } } -async fn test_tracing_batch(session: &Session, ks: String) { +async fn test_tracing_batch(session: &LegacySession, ks: String) { // A batch without tracing enabled has no tracing id let mut untraced_batch: Batch = Default::default(); untraced_batch.append_statement(&format!("INSERT INTO {}.tab (a) VALUES('a')", ks)[..]); @@ -1141,7 +1141,7 @@ async fn test_tracing_batch(session: &Session, ks: String) { assert_in_tracing_table(session, traced_batch_result.tracing_id.unwrap()).await; } -async fn assert_in_tracing_table(session: &Session, tracing_uuid: Uuid) { +async fn assert_in_tracing_table(session: &LegacySession, tracing_uuid: Uuid) { let mut traces_query = Query::new("SELECT * FROM system_traces.sessions WHERE session_id = ?"); traces_query.config.consistency = Some(Consistency::One); @@ -2009,7 +2009,7 @@ async fn test_prepared_partitioner() { ); } -async fn rename(session: &Session, rename_str: &str) { +async fn rename(session: &LegacySession, rename_str: &str) { session .query_unpaged(format!("ALTER TABLE tab RENAME {}", rename_str), ()) .await @@ -2338,7 +2338,7 @@ async fn test_views_in_schema_info() { ) } -async fn assert_test_batch_table_rows_contain(sess: &Session, expected_rows: &[(i32, i32)]) { +async fn assert_test_batch_table_rows_contain(sess: &LegacySession, expected_rows: &[(i32, i32)]) { let selected_rows: BTreeSet<(i32, i32)> = sess .query_unpaged("SELECT a, b FROM test_batch_table", ()) .await @@ -2583,7 +2583,7 @@ async fn test_batch_lwts() { } async fn test_batch_lwts_for_scylla( - session: &Session, + session: &LegacySession, batch: &Batch, batch_res: LegacyQueryResult, ) { @@ -2627,7 +2627,7 @@ async fn test_batch_lwts_for_scylla( } async fn test_batch_lwts_for_cassandra( - session: &Session, + session: &LegacySession, batch: &Batch, batch_res: LegacyQueryResult, ) { @@ -2917,7 +2917,7 @@ async fn test_manual_primary_key_computation() { session.use_keyspace(&ks, true).await.unwrap(); async fn assert_tokens_equal( - session: &Session, + session: &LegacySession, prepared: &PreparedStatement, serialized_pk_values_in_pk_order: &SerializedValues, all_values_in_query_order: impl SerializeRow, diff --git a/scylla/src/transport/silent_prepare_batch_test.rs b/scylla/src/transport/silent_prepare_batch_test.rs index ece8d1d3f..f8c7fb328 100644 --- a/scylla/src/transport/silent_prepare_batch_test.rs +++ b/scylla/src/transport/silent_prepare_batch_test.rs @@ -2,7 +2,7 @@ use crate::{ batch::Batch, prepared_statement::PreparedStatement, test_utils::{create_new_session_builder, setup_tracing, unique_keyspace_name}, - Session, + LegacySession, }; use std::collections::BTreeSet; @@ -91,7 +91,7 @@ async fn test_quietly_prepare_batch() { } } -async fn assert_test_batch_table_rows_contain(sess: &Session, expected_rows: &[(i32, i32)]) { +async fn assert_test_batch_table_rows_contain(sess: &LegacySession, expected_rows: &[(i32, i32)]) { let selected_rows: BTreeSet<(i32, i32)> = sess .query_unpaged("SELECT a, b FROM test_batch_table", ()) .await diff --git a/scylla/src/utils/test_utils.rs b/scylla/src/utils/test_utils.rs index 6c52fde35..6f9f2a9ec 100644 --- a/scylla/src/utils/test_utils.rs +++ b/scylla/src/utils/test_utils.rs @@ -1,6 +1,6 @@ #[cfg(test)] use crate::transport::session_builder::{GenericSessionBuilder, SessionBuilderKind}; -use crate::Session; +use crate::LegacySession; #[cfg(test)] use std::{num::NonZeroU32, time::Duration}; use std::{ @@ -25,7 +25,7 @@ pub fn unique_keyspace_name() -> String { } #[cfg(test)] -pub(crate) async fn supports_feature(session: &Session, feature: &str) -> bool { +pub(crate) async fn supports_feature(session: &LegacySession, feature: &str) -> bool { // Cassandra doesn't have a concept of features, so first detect // if there is the `supported_features` column in system.local @@ -92,7 +92,7 @@ pub fn create_new_session_builder() -> GenericSessionBuilder bool { +pub async fn scylla_supports_tablets(session: &LegacySession) -> bool { let result = session .query_unpaged( "select column_name from system_schema.columns where diff --git a/scylla/tests/integration/consistency.rs b/scylla/tests/integration/consistency.rs index f12f2d867..4a3b1306f 100644 --- a/scylla/tests/integration/consistency.rs +++ b/scylla/tests/integration/consistency.rs @@ -6,7 +6,7 @@ use scylla::prepared_statement::PreparedStatement; use scylla::retry_policy::FallthroughRetryPolicy; use scylla::routing::{Shard, Token}; use scylla::test_utils::unique_keyspace_name; -use scylla::transport::session::Session; +use scylla::transport::session::LegacySession; use scylla::transport::NodeRef; use scylla_cql::frame::response::result::TableSpec; use tokio::sync::mpsc::{self, UnboundedReceiver, UnboundedSender}; @@ -59,7 +59,7 @@ fn pairs_of_all_consistencies() -> impl Iterator, @@ -81,7 +81,7 @@ async fn query_consistency_set_directly( } async fn execute_consistency_set_directly( - session: &Session, + session: &LegacySession, prepared: &PreparedStatement, c: Consistency, sc: Option, @@ -94,7 +94,7 @@ async fn execute_consistency_set_directly( } async fn batch_consistency_set_directly( - session: &Session, + session: &LegacySession, batch: &Batch, c: Consistency, sc: Option, @@ -107,7 +107,7 @@ async fn batch_consistency_set_directly( // The following functions perform a request with consistencies set on a per-statement execution profile. async fn query_consistency_set_on_exec_profile( - session: &Session, + session: &LegacySession, query: &Query, profile: ExecutionProfileHandle, ) { @@ -118,7 +118,7 @@ async fn query_consistency_set_on_exec_profile( } async fn execute_consistency_set_on_exec_profile( - session: &Session, + session: &LegacySession, prepared: &PreparedStatement, profile: ExecutionProfileHandle, ) { @@ -129,7 +129,7 @@ async fn execute_consistency_set_on_exec_profile( } async fn batch_consistency_set_on_exec_profile( - session: &Session, + session: &LegacySession, batch: &Batch, profile: ExecutionProfileHandle, ) { diff --git a/scylla/tests/integration/lwt_optimisation.rs b/scylla/tests/integration/lwt_optimisation.rs index ca56cff93..f0d59f1f0 100644 --- a/scylla/tests/integration/lwt_optimisation.rs +++ b/scylla/tests/integration/lwt_optimisation.rs @@ -2,7 +2,7 @@ use crate::utils::{setup_tracing, test_with_3_node_cluster}; use scylla::retry_policy::FallthroughRetryPolicy; use scylla::test_utils::scylla_supports_tablets; use scylla::test_utils::unique_keyspace_name; -use scylla::transport::session::Session; +use scylla::transport::session::LegacySession; use scylla::{ExecutionProfile, SessionBuilder}; use scylla_cql::frame::protocol_features::ProtocolFeatures; use scylla_cql::frame::types; @@ -52,7 +52,7 @@ async fn if_lwt_optimisation_mark_offered_then_negotiatied_and_lwt_routed_optima .into_handle(); // DB preparation phase - let session: Session = SessionBuilder::new() + let session: LegacySession = SessionBuilder::new() .known_node(proxy_uris[0].as_str()) .default_execution_profile_handle(handle) .address_translator(Arc::new(translation_map)) diff --git a/scylla/tests/integration/retries.rs b/scylla/tests/integration/retries.rs index 43cbf5807..f6e1711ac 100644 --- a/scylla/tests/integration/retries.rs +++ b/scylla/tests/integration/retries.rs @@ -1,7 +1,7 @@ use crate::utils::{setup_tracing, test_with_3_node_cluster}; use scylla::retry_policy::FallthroughRetryPolicy; use scylla::speculative_execution::SimpleSpeculativeExecutionPolicy; -use scylla::transport::session::Session; +use scylla::transport::session::LegacySession; use scylla::ExecutionProfile; use scylla::SessionBuilder; use scylla::{query::Query, test_utils::unique_keyspace_name}; @@ -27,7 +27,7 @@ async fn speculative_execution_is_fired() { max_retry_count: 2, retry_interval: Duration::from_millis(10), }))).retry_policy(Arc::new(FallthroughRetryPolicy)).build(); - let session: Session = SessionBuilder::new() + let session: LegacySession = SessionBuilder::new() .known_node(proxy_uris[0].as_str()) .default_execution_profile_handle(simple_speculative_no_retry_profile.into_handle()) .address_translator(Arc::new(translation_map)) @@ -104,7 +104,7 @@ async fn retries_occur() { let res = test_with_3_node_cluster(ShardAwareness::QueryNode, |proxy_uris, translation_map, mut running_proxy| async move { // DB preparation phase - let session: Session = SessionBuilder::new() + let session: LegacySession = SessionBuilder::new() .known_node(proxy_uris[0].as_str()) .address_translator(Arc::new(translation_map)) .build() @@ -183,7 +183,7 @@ async fn speculative_execution_panic_regression_test() { .retry_policy(Arc::new(FallthroughRetryPolicy)) .build(); // DB preparation phase - let session: Session = SessionBuilder::new() + let session: LegacySession = SessionBuilder::new() .known_node(proxy_uris[0].as_str()) .address_translator(Arc::new(translation_map)) .default_execution_profile_handle(profile.into_handle()) diff --git a/scylla/tests/integration/self_identity.rs b/scylla/tests/integration/self_identity.rs index cba46f717..d68bb0add 100644 --- a/scylla/tests/integration/self_identity.rs +++ b/scylla/tests/integration/self_identity.rs @@ -1,5 +1,5 @@ use crate::utils::{setup_tracing, test_with_3_node_cluster}; -use scylla::{Session, SessionBuilder}; +use scylla::{LegacySession, SessionBuilder}; use scylla_cql::frame::request::options; use scylla_cql::frame::types; use std::sync::Arc; @@ -50,7 +50,7 @@ async fn test_given_self_identity(self_identity: SelfIdentity<'static>) { )])); // DB preparation phase - let _session: Session = SessionBuilder::new() + let _session: LegacySession = SessionBuilder::new() .known_node(proxy_uris[0].as_str()) .address_translator(Arc::new(translation_map)) .custom_identity(self_identity.clone()) diff --git a/scylla/tests/integration/silent_prepare_query.rs b/scylla/tests/integration/silent_prepare_query.rs index d814f70a8..ffb200c7c 100644 --- a/scylla/tests/integration/silent_prepare_query.rs +++ b/scylla/tests/integration/silent_prepare_query.rs @@ -1,5 +1,5 @@ use crate::utils::{setup_tracing, test_with_3_node_cluster}; -use scylla::transport::session::Session; +use scylla::transport::session::LegacySession; use scylla::SessionBuilder; use scylla::{query::Query, test_utils::unique_keyspace_name}; use scylla_proxy::{ @@ -19,7 +19,7 @@ async fn test_prepare_query_with_values() { let res = test_with_3_node_cluster(ShardAwareness::QueryNode, |proxy_uris, translation_map, mut running_proxy| async move { // DB preparation phase - let session: Session = SessionBuilder::new() + let session: LegacySession = SessionBuilder::new() .known_node(proxy_uris[0].as_str()) .address_translator(Arc::new(translation_map)) .build() @@ -70,7 +70,7 @@ async fn test_query_with_no_values() { let res = test_with_3_node_cluster(ShardAwareness::QueryNode, |proxy_uris, translation_map, mut running_proxy| async move { // DB preparation phase - let session: Session = SessionBuilder::new() + let session: LegacySession = SessionBuilder::new() .known_node(proxy_uris[0].as_str()) .address_translator(Arc::new(translation_map)) .build() diff --git a/scylla/tests/integration/skip_metadata_optimization.rs b/scylla/tests/integration/skip_metadata_optimization.rs index 1c84569e7..eee25f908 100644 --- a/scylla/tests/integration/skip_metadata_optimization.rs +++ b/scylla/tests/integration/skip_metadata_optimization.rs @@ -1,5 +1,5 @@ use crate::utils::{setup_tracing, test_with_3_node_cluster}; -use scylla::transport::session::Session; +use scylla::transport::session::LegacySession; use scylla::SessionBuilder; use scylla::{prepared_statement::PreparedStatement, test_utils::unique_keyspace_name}; use scylla_cql::frame::request::query::{PagingState, PagingStateResponse}; @@ -20,7 +20,7 @@ async fn test_skip_result_metadata() { let res = test_with_3_node_cluster(ShardAwareness::QueryNode, |proxy_uris, translation_map, mut running_proxy| async move { // DB preparation phase - let session: Session = SessionBuilder::new() + let session: LegacySession = SessionBuilder::new() .known_node(proxy_uris[0].as_str()) .address_translator(Arc::new(translation_map)) .build() @@ -51,7 +51,7 @@ async fn test_skip_result_metadata() { } async fn test_with_flags_predicate( - session: &Session, + session: &LegacySession, prepared: &PreparedStatement, rx: &mut tokio::sync::mpsc::UnboundedReceiver<(ResponseFrame, Option)>, predicate: impl FnOnce(i32) -> bool diff --git a/scylla/tests/integration/tablets.rs b/scylla/tests/integration/tablets.rs index 2bdf96987..67fb2fd88 100644 --- a/scylla/tests/integration/tablets.rs +++ b/scylla/tests/integration/tablets.rs @@ -16,9 +16,7 @@ use scylla::test_utils::unique_keyspace_name; use scylla::transport::ClusterData; use scylla::transport::Node; use scylla::transport::NodeRef; -use scylla::ExecutionProfile; -use scylla::LegacyQueryResult; -use scylla::Session; +use scylla::{ExecutionProfile, LegacyQueryResult, LegacySession}; use scylla::transport::errors::QueryError; use scylla_proxy::{ @@ -42,7 +40,7 @@ struct Tablet { replicas: Vec<(Arc, i32)>, } -async fn get_tablets(session: &Session, ks: &str, table: &str) -> Vec { +async fn get_tablets(session: &LegacySession, ks: &str, table: &str) -> Vec { let cluster_data = session.get_cluster_data(); let endpoints = cluster_data.get_nodes_info(); for endpoint in endpoints.iter() { @@ -181,7 +179,7 @@ impl LoadBalancingPolicy for SingleTargetLBP { } async fn send_statement_everywhere( - session: &Session, + session: &LegacySession, cluster: &ClusterData, statement: &PreparedStatement, values: &dyn SerializeRow, @@ -207,7 +205,7 @@ async fn send_statement_everywhere( } async fn send_unprepared_query_everywhere( - session: &Session, + session: &LegacySession, cluster: &ClusterData, query: &Query, ) -> Result, QueryError> { @@ -249,7 +247,7 @@ fn count_tablet_feedbacks( .count() } -async fn prepare_schema(session: &Session, ks: &str, table: &str, tablet_count: usize) { +async fn prepare_schema(session: &LegacySession, ks: &str, table: &str, tablet_count: usize) { session .query_unpaged( format!( From 9e19180938c69d56dacdb7ca1d1abf87a757b42b Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Thu, 16 Mar 2023 18:19:06 +0100 Subject: [PATCH 24/42] session: make generic and introduce "session kind" parameter MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit The LegacySession and the upcoming Session will differ on a small number of methods, but otherwise will share remaining ones. In order to reduce boilerplate the (Legacy)Session is converted into a generic, with a type parameter indicating the kind of the API it supports (legacy or the current one). The common methods will be implemented for GenericSession for any K, and methods specific to the particular kind will only be implemented for GenericSession for that particular K. Co-authored-by: Wojciech Przytuła --- scylla/src/transport/session.rs | 33 +++++++++++++++++++++++++++++---- 1 file changed, 29 insertions(+), 4 deletions(-) diff --git a/scylla/src/transport/session.rs b/scylla/src/transport/session.rs index 6b42348b2..f512936d8 100644 --- a/scylla/src/transport/session.rs +++ b/scylla/src/transport/session.rs @@ -27,6 +27,7 @@ use std::borrow::Borrow; use std::collections::HashMap; use std::fmt::Display; use std::future::Future; +use std::marker::PhantomData; use std::net::SocketAddr; use std::num::NonZeroU32; use std::str::FromStr; @@ -83,6 +84,14 @@ use crate::authentication::AuthenticatorProvider; #[cfg(feature = "ssl")] use openssl::ssl::SslContext; +mod sealed { + // This is a sealed trait - its whole purpose is to be unnameable. + // This means we need to disable the check. + #[allow(unknown_lints)] // Rust 1.70 (our MSRV) doesn't know this lint + #[allow(unnameable_types)] + pub trait Sealed {} +} + pub(crate) const TABLET_CHANNEL_SIZE: usize = 8192; const TRACING_QUERY_PAGE_SIZE: i32 = 1024; @@ -152,8 +161,17 @@ impl AddressTranslator for HashMap<&'static str, &'static str> { } } +pub trait DeserializationApiKind: sealed::Sealed {} + +pub enum LegacyDeserializationApi {} +impl sealed::Sealed for LegacyDeserializationApi {} +impl DeserializationApiKind for LegacyDeserializationApi {} + /// `Session` manages connections to the cluster and allows to perform queries -pub struct LegacySession { +pub struct GenericSession +where + DeserializationApi: DeserializationApiKind, +{ cluster: Cluster, default_execution_profile_handle: ExecutionProfileHandle, schema_agreement_interval: Duration, @@ -165,11 +183,17 @@ pub struct LegacySession { tracing_info_fetch_attempts: NonZeroU32, tracing_info_fetch_interval: Duration, tracing_info_fetch_consistency: Consistency, + _phantom_deser_api: PhantomData, } +pub type LegacySession = GenericSession; + /// This implementation deliberately omits some details from Cluster in order /// to avoid cluttering the print with much information of little usability. -impl std::fmt::Debug for LegacySession { +impl std::fmt::Debug for GenericSession +where + DeserApi: DeserializationApiKind, +{ fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { f.debug_struct("Session") .field("cluster", &ClusterNeatDebug(&self.cluster)) @@ -488,7 +512,7 @@ impl LegacySession { /// # Ok(()) /// # } /// ``` - pub async fn connect(config: SessionConfig) -> Result { + pub async fn connect(config: SessionConfig) -> Result { let known_nodes = config.known_nodes; #[cfg(feature = "cloud")] @@ -562,7 +586,7 @@ impl LegacySession { let default_execution_profile_handle = config.default_execution_profile_handle; - let session = LegacySession { + let session = Self { cluster, default_execution_profile_handle, schema_agreement_interval: config.schema_agreement_interval, @@ -575,6 +599,7 @@ impl LegacySession { 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, + _phantom_deser_api: PhantomData, }; if let Some(keyspace_name) = config.used_keyspace { From 83419ae69e7e4d6856507174ca885a5fe94c86fe Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Thu, 16 Mar 2023 18:20:10 +0100 Subject: [PATCH 25/42] session: move query-related methods to a separate block MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Both Session and LegacySession will support methods that allow sending queries/prepared statements/batches and will share most of the implementation - it's just that return types will be slightly different. This commit moves the core of those methods to private methods `do_xyz` for every `xyz` method from the API. This will allow to implement the public methods for both API kinds with minimal boilerplate. Co-authored-by: Wojciech Przytuła --- scylla/src/transport/session.rs | 785 +++++++++++++++++--------------- 1 file changed, 424 insertions(+), 361 deletions(-) diff --git a/scylla/src/transport/session.rs b/scylla/src/transport/session.rs index f512936d8..6d1d5faa3 100644 --- a/scylla/src/transport/session.rs +++ b/scylla/src/transport/session.rs @@ -487,130 +487,7 @@ pub(crate) enum RunQueryResult { Completed(ResT), } -/// Represents a CQL session, which can be used to communicate -/// with the database -impl LegacySession { - /// Estabilishes a CQL session with the database - /// - /// Usually it's easier to use [SessionBuilder](crate::transport::session_builder::SessionBuilder) - /// instead of calling `Session::connect` directly, because it's more convenient. - /// # Arguments - /// * `config` - Connection configuration - known nodes, Compression, etc. - /// Must contain at least one known node. - /// - /// # Example - /// ```rust - /// # use std::error::Error; - /// # async fn check_only_compiles() -> Result<(), Box> { - /// use scylla::{LegacySession, SessionConfig}; - /// use scylla::transport::KnownNode; - /// - /// let mut config = SessionConfig::new(); - /// config.known_nodes.push(KnownNode::Hostname("127.0.0.1:9042".to_string())); - /// - /// let session: LegacySession = LegacySession::connect(config).await?; - /// # Ok(()) - /// # } - /// ``` - pub async fn connect(config: SessionConfig) -> Result { - let known_nodes = config.known_nodes; - - #[cfg(feature = "cloud")] - let cloud_known_nodes: Option> = - if let Some(ref cloud_config) = config.cloud_config { - let cloud_servers = cloud_config - .get_datacenters() - .iter() - .map(|(dc_name, dc_data)| { - InternalKnownNode::CloudEndpoint(CloudEndpoint { - hostname: dc_data.get_server().to_owned(), - datacenter: dc_name.clone(), - }) - }) - .collect(); - Some(cloud_servers) - } else { - None - }; - - #[cfg(not(feature = "cloud"))] - let cloud_known_nodes: Option> = None; - - let known_nodes = cloud_known_nodes - .unwrap_or_else(|| known_nodes.into_iter().map(|node| node.into()).collect()); - - // Ensure there is at least one known node - if known_nodes.is_empty() { - return Err(NewSessionError::EmptyKnownNodesList); - } - - let (tablet_sender, tablet_receiver) = tokio::sync::mpsc::channel(TABLET_CHANNEL_SIZE); - - let connection_config = ConnectionConfig { - compression: config.compression, - tcp_nodelay: config.tcp_nodelay, - tcp_keepalive_interval: config.tcp_keepalive_interval, - #[cfg(feature = "ssl")] - ssl_config: config.ssl_context.map(SslConfig::new_with_global_context), - authenticator: config.authenticator.clone(), - connect_timeout: config.connect_timeout, - event_sender: None, - default_consistency: Default::default(), - address_translator: config.address_translator, - #[cfg(feature = "cloud")] - cloud_config: config.cloud_config, - enable_write_coalescing: config.enable_write_coalescing, - keepalive_interval: config.keepalive_interval, - keepalive_timeout: config.keepalive_timeout, - tablet_sender: Some(tablet_sender), - identity: config.identity, - }; - - let pool_config = PoolConfig { - connection_config, - pool_size: config.connection_pool_size, - can_use_shard_aware_port: !config.disallow_shard_aware_port, - keepalive_interval: config.keepalive_interval, - }; - - let cluster = Cluster::new( - known_nodes, - pool_config, - config.keyspaces_to_fetch, - config.fetch_schema_metadata, - config.host_filter, - config.cluster_metadata_refresh_interval, - tablet_receiver, - ) - .await?; - - let default_execution_profile_handle = config.default_execution_profile_handle; - - let session = Self { - cluster, - default_execution_profile_handle, - schema_agreement_interval: config.schema_agreement_interval, - metrics: Arc::new(Metrics::new()), - schema_agreement_timeout: config.schema_agreement_timeout, - 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 - 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, - _phantom_deser_api: PhantomData, - }; - - if let Some(keyspace_name) = config.used_keyspace { - session - .use_keyspace(keyspace_name, config.keyspace_case_sensitive) - .await?; - } - - Ok(session) - } - +impl GenericSession { /// Sends a request to the database and receives a response.\ /// Performs an unpaged query, i.e. all results are received in a single response. /// @@ -675,15 +552,7 @@ impl LegacySession { query: impl Into, values: impl SerializeRow, ) -> Result { - let query = query.into(); - let (result, paging_state_response) = self - .query(&query, values, None, PagingState::start()) - .await?; - if !paging_state_response.finished() { - error!("Unpaged unprepared query returned a non-empty paging state! This is a driver-side or server-side bug."); - return Err(ProtocolError::NonfinishedPagingState.into()); - } - Ok(result) + self.do_query_unpaged(&query.into(), values).await } /// Queries a single page from the database, optionally continuing from a saved point. @@ -736,13 +605,425 @@ impl LegacySession { /// ``` pub async fn query_single_page( &self, - query: impl Into, + query: impl Into, + values: impl SerializeRow, + paging_state: PagingState, + ) -> Result<(LegacyQueryResult, PagingStateResponse), QueryError> { + self.do_query_single_page(&query.into(), values, paging_state) + .await + } + + /// Run an unprepared query with paging\ + /// This method will query all pages of the result\ + /// + /// Returns an async iterator (stream) over all received rows\ + /// Page size can be specified in the [Query] passed to the function + /// + /// It is discouraged to use this method with non-empty values argument (`is_empty()` method from `SerializeRow` + /// trait returns false). In such case, query first needs to be prepared (on a single connection), so + /// driver will initially perform 2 round trips instead of 1. Please use [`Session::execute_iter()`] instead. + /// + /// See [the book](https://rust-driver.docs.scylladb.com/stable/queries/paged.html) for more information. + /// + /// # Arguments + /// * `query` - statement to be executed, can be just a `&str` or the [Query] struct. + /// * `values` - values bound to the query, the easiest way is to use a tuple of bound values. + /// + /// # Example + /// + /// ```rust + /// # use scylla::LegacySession; + /// # use std::error::Error; + /// # async fn check_only_compiles(session: &LegacySession) -> Result<(), Box> { + /// use scylla::IntoTypedRows; + /// use futures::stream::StreamExt; + /// + /// let mut rows_stream = session + /// .query_iter("SELECT a, b FROM ks.t", &[]) + /// .await? + /// .into_typed::<(i32, i32)>(); + /// + /// while let Some(next_row_res) = rows_stream.next().await { + /// let (a, b): (i32, i32) = next_row_res?; + /// println!("a, b: {}, {}", a, b); + /// } + /// # Ok(()) + /// # } + /// ``` + pub async fn query_iter( + &self, + query: impl Into, + values: impl SerializeRow, + ) -> Result { + self.do_query_iter(query, values).await + } + + /// Execute a prepared statement. Requires a [PreparedStatement] + /// generated using [`Session::prepare`](Session::prepare).\ + /// Performs an unpaged query, i.e. all results are received in a single response. + /// + /// As all results come in one response (no paging is done!), the memory footprint and latency may be huge + /// for statements returning rows (i.e. SELECTs)! Prefer this method for non-SELECTs, and for SELECTs + /// it is best to use paged queries: + /// - to receive multiple pages and transparently iterate through them, use [execute_iter](Session::execute_iter). + /// - to manually receive multiple pages and iterate through them, use [execute_single_page](Session::execute_single_page). + /// + /// Prepared queries are much faster than simple queries: + /// * Database doesn't need to parse the query + /// * They are properly load balanced using token aware routing + /// + /// > ***Warning***\ + /// > For token/shard aware load balancing to work properly, all partition key values + /// > must be sent as bound values + /// > (see [performance section](https://rust-driver.docs.scylladb.com/stable/queries/prepared.html#performance)). + /// + /// See [the book](https://rust-driver.docs.scylladb.com/stable/queries/prepared.html) for more information. + /// + /// # Arguments + /// * `prepared` - the prepared statement to execute, generated using [`Session::prepare`](Session::prepare) + /// * `values` - values bound to the query, the easiest way is to use a tuple of bound values + /// + /// # Example + /// ```rust + /// # use scylla::LegacySession; + /// # use std::error::Error; + /// # async fn check_only_compiles(session: &LegacySession) -> Result<(), Box> { + /// use scylla::prepared_statement::PreparedStatement; + /// + /// // Prepare the query for later execution + /// let prepared: PreparedStatement = session + /// .prepare("INSERT INTO ks.tab (a) VALUES(?)") + /// .await?; + /// + /// // Run the prepared query with some values, just like a simple query. + /// let to_insert: i32 = 12345; + /// session.execute_unpaged(&prepared, (to_insert,)).await?; + /// # Ok(()) + /// # } + /// ``` + pub async fn execute_unpaged( + &self, + prepared: &PreparedStatement, + values: impl SerializeRow, + ) -> Result { + self.do_execute_unpaged(prepared, values).await + } + + /// Executes a prepared statement, restricting results to single page. + /// Optionally continues fetching results from a saved point. + /// + /// # Arguments + /// + /// * `prepared` - a statement prepared with [prepare](crate::Session::prepare) + /// * `values` - values bound to the query + /// * `paging_state` - continuation based on a paging state received from a previous paged query or None + /// + /// # Example + /// + /// ```rust + /// # use scylla::LegacySession; + /// # use std::error::Error; + /// # async fn check_only_compiles(session: &LegacySession) -> Result<(), Box> { + /// use std::ops::ControlFlow; + /// use scylla::query::Query; + /// use scylla::statement::{PagingState, PagingStateResponse}; + /// + /// let paged_prepared = session + /// .prepare( + /// Query::new("SELECT a, b FROM ks.tbl") + /// .with_page_size(100.try_into().unwrap()), + /// ) + /// .await?; + /// + /// // Manual paging in a loop, prepared statement. + /// let mut paging_state = PagingState::start(); + /// loop { + /// let (res, paging_state_response) = session + /// .execute_single_page(&paged_prepared, &[], paging_state) + /// .await?; + /// + /// // Do something with a single page of results. + /// for row in res.rows_typed::<(i32, String)>()? { + /// let (a, b) = row?; + /// } + /// + /// match paging_state_response.into_paging_control_flow() { + /// ControlFlow::Break(()) => { + /// // No more pages to be fetched. + /// break; + /// } + /// ControlFlow::Continue(new_paging_state) => { + /// // Update paging continuation from the paging state, so that query + /// // will be resumed from where it ended the last time. + /// paging_state = new_paging_state; + /// } + /// } + /// } + /// # Ok(()) + /// # } + /// ``` + pub async fn execute_single_page( + &self, + prepared: &PreparedStatement, + values: impl SerializeRow, + paging_state: PagingState, + ) -> Result<(LegacyQueryResult, PagingStateResponse), QueryError> { + self.do_execute_single_page(prepared, values, paging_state) + .await + } + + /// Run a prepared query with paging.\ + /// This method will query all pages of the result.\ + /// + /// Returns an async iterator (stream) over all received rows.\ + /// Page size can be specified in the [PreparedStatement] passed to the function. + /// + /// See [the book](https://rust-driver.docs.scylladb.com/stable/queries/paged.html) for more information. + /// + /// # Arguments + /// * `prepared` - the prepared statement to execute, generated using [`Session::prepare`](Session::prepare) + /// * `values` - values bound to the query, the easiest way is to use a tuple of bound values + /// + /// # Example + /// + /// ```rust + /// # use scylla::LegacySession; + /// # use std::error::Error; + /// # async fn check_only_compiles(session: &LegacySession) -> Result<(), Box> { + /// use scylla::prepared_statement::PreparedStatement; + /// use scylla::IntoTypedRows; + /// use futures::stream::StreamExt; + /// + /// // Prepare the query for later execution + /// let prepared: PreparedStatement = session + /// .prepare("SELECT a, b FROM ks.t") + /// .await?; + /// + /// // Execute the query and receive all pages + /// let mut rows_stream = session + /// .execute_iter(prepared, &[]) + /// .await? + /// .into_typed::<(i32, i32)>(); + /// + /// while let Some(next_row_res) = rows_stream.next().await { + /// let (a, b): (i32, i32) = next_row_res?; + /// println!("a, b: {}, {}", a, b); + /// } + /// # Ok(()) + /// # } + /// ``` + pub async fn execute_iter( + &self, + prepared: impl Into, + values: impl SerializeRow, + ) -> Result { + self.do_execute_iter(prepared, values).await + } + + /// Perform a batch query\ + /// Batch contains many `simple` or `prepared` queries which are executed at once\ + /// Batch doesn't return any rows + /// + /// Batch values must contain values for each of the queries + /// + /// Avoid using non-empty values (`SerializeRow::is_empty()` return false) for simple queries + /// inside the batch. Such queries will first need to be prepared, so the driver will need to + /// send (numer_of_unprepared_queries_with_values + 1) requests instead of 1 request, severly + /// affecting performance. + /// + /// See [the book](https://rust-driver.docs.scylladb.com/stable/queries/batch.html) for more information + /// + /// # Arguments + /// * `batch` - [Batch] to be performed + /// * `values` - List of values for each query, it's the easiest to use a tuple of tuples + /// + /// # Example + /// ```rust + /// # use scylla::LegacySession; + /// # use std::error::Error; + /// # async fn check_only_compiles(session: &LegacySession) -> Result<(), Box> { + /// use scylla::batch::Batch; + /// + /// let mut batch: Batch = Default::default(); + /// + /// // A query with two bound values + /// batch.append_statement("INSERT INTO ks.tab(a, b) VALUES(?, ?)"); + /// + /// // A query with one bound value + /// batch.append_statement("INSERT INTO ks.tab(a, b) VALUES(3, ?)"); + /// + /// // A query with no bound values + /// batch.append_statement("INSERT INTO ks.tab(a, b) VALUES(5, 6)"); + /// + /// // Batch values is a tuple of 3 tuples containing values for each query + /// let batch_values = ((1_i32, 2_i32), // Tuple with two values for the first query + /// (4_i32,), // Tuple with one value for the second query + /// ()); // Empty tuple/unit for the third query + /// + /// // Run the batch + /// session.batch(&batch, batch_values).await?; + /// # Ok(()) + /// # } + /// ``` + pub async fn batch( + &self, + batch: &Batch, + values: impl BatchValues, + ) -> Result { + self.do_batch(batch, values).await + } +} + +/// Represents a CQL session, which can be used to communicate +/// with the database +impl GenericSession +where + DeserApi: DeserializationApiKind, +{ + /// Estabilishes a CQL session with the database + /// + /// Usually it's easier to use [SessionBuilder](crate::transport::session_builder::SessionBuilder) + /// instead of calling `Session::connect` directly, because it's more convenient. + /// # Arguments + /// * `config` - Connection configuration - known nodes, Compression, etc. + /// Must contain at least one known node. + /// + /// # Example + /// ```rust + /// # use std::error::Error; + /// # async fn check_only_compiles() -> Result<(), Box> { + /// use scylla::{LegacySession, SessionConfig}; + /// use scylla::transport::KnownNode; + /// + /// let mut config = SessionConfig::new(); + /// config.known_nodes.push(KnownNode::Hostname("127.0.0.1:9042".to_string())); + /// + /// let session: LegacySession = LegacySession::connect(config).await?; + /// # Ok(()) + /// # } + /// ``` + pub async fn connect(config: SessionConfig) -> Result { + let known_nodes = config.known_nodes; + + #[cfg(feature = "cloud")] + let cloud_known_nodes: Option> = + if let Some(ref cloud_config) = config.cloud_config { + let cloud_servers = cloud_config + .get_datacenters() + .iter() + .map(|(dc_name, dc_data)| { + InternalKnownNode::CloudEndpoint(CloudEndpoint { + hostname: dc_data.get_server().to_owned(), + datacenter: dc_name.clone(), + }) + }) + .collect(); + Some(cloud_servers) + } else { + None + }; + + #[cfg(not(feature = "cloud"))] + let cloud_known_nodes: Option> = None; + + let known_nodes = cloud_known_nodes + .unwrap_or_else(|| known_nodes.into_iter().map(|node| node.into()).collect()); + + // Ensure there is at least one known node + if known_nodes.is_empty() { + return Err(NewSessionError::EmptyKnownNodesList); + } + + let (tablet_sender, tablet_receiver) = tokio::sync::mpsc::channel(TABLET_CHANNEL_SIZE); + + let connection_config = ConnectionConfig { + compression: config.compression, + tcp_nodelay: config.tcp_nodelay, + tcp_keepalive_interval: config.tcp_keepalive_interval, + #[cfg(feature = "ssl")] + ssl_config: config.ssl_context.map(SslConfig::new_with_global_context), + authenticator: config.authenticator.clone(), + connect_timeout: config.connect_timeout, + event_sender: None, + default_consistency: Default::default(), + address_translator: config.address_translator, + #[cfg(feature = "cloud")] + cloud_config: config.cloud_config, + enable_write_coalescing: config.enable_write_coalescing, + keepalive_interval: config.keepalive_interval, + keepalive_timeout: config.keepalive_timeout, + tablet_sender: Some(tablet_sender), + identity: config.identity, + }; + + let pool_config = PoolConfig { + connection_config, + pool_size: config.connection_pool_size, + can_use_shard_aware_port: !config.disallow_shard_aware_port, + keepalive_interval: config.keepalive_interval, + }; + + let cluster = Cluster::new( + known_nodes, + pool_config, + config.keyspaces_to_fetch, + config.fetch_schema_metadata, + config.host_filter, + config.cluster_metadata_refresh_interval, + tablet_receiver, + ) + .await?; + + let default_execution_profile_handle = config.default_execution_profile_handle; + + let session = Self { + cluster, + default_execution_profile_handle, + schema_agreement_interval: config.schema_agreement_interval, + metrics: Arc::new(Metrics::new()), + schema_agreement_timeout: config.schema_agreement_timeout, + 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 + 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, + _phantom_deser_api: PhantomData, + }; + + if let Some(keyspace_name) = config.used_keyspace { + session + .use_keyspace(keyspace_name, config.keyspace_case_sensitive) + .await?; + } + + Ok(session) + } + + async fn do_query_unpaged( + &self, + query: &Query, + values: impl SerializeRow, + ) -> Result { + let (result, paging_state_response) = self + .query(query, values, None, PagingState::start()) + .await?; + if !paging_state_response.finished() { + error!("Unpaged unprepared query returned a non-empty paging state! This is a driver-side or server-side bug."); + return Err(ProtocolError::NonfinishedPagingState.into()); + } + Ok(result) + } + + async fn do_query_single_page( + &self, + query: &Query, values: impl SerializeRow, paging_state: PagingState, ) -> Result<(LegacyQueryResult, PagingStateResponse), QueryError> { - let query = query.into(); self.query( - &query, + query, values, Some(query.get_validated_page_size()), paging_state, @@ -894,44 +1175,7 @@ impl LegacySession { Ok(()) } - /// Run an unprepared query with paging\ - /// This method will query all pages of the result\ - /// - /// Returns an async iterator (stream) over all received rows\ - /// Page size can be specified in the [Query] passed to the function - /// - /// It is discouraged to use this method with non-empty values argument (`is_empty()` method from `SerializeRow` - /// trait returns false). In such case, query first needs to be prepared (on a single connection), so - /// driver will initially perform 2 round trips instead of 1. Please use [`Session::execute_iter()`] instead. - /// - /// See [the book](https://rust-driver.docs.scylladb.com/stable/queries/paged.html) for more information. - /// - /// # Arguments - /// * `query` - statement to be executed, can be just a `&str` or the [Query] struct. - /// * `values` - values bound to the query, the easiest way is to use a tuple of bound values. - /// - /// # Example - /// - /// ```rust - /// # use scylla::LegacySession; - /// # use std::error::Error; - /// # async fn check_only_compiles(session: &LegacySession) -> Result<(), Box> { - /// use scylla::IntoTypedRows; - /// use futures::stream::StreamExt; - /// - /// let mut rows_stream = session - /// .query_iter("SELECT a, b FROM ks.t", &[]) - /// .await? - /// .into_typed::<(i32, i32)>(); - /// - /// while let Some(next_row_res) = rows_stream.next().await { - /// let (a, b): (i32, i32) = next_row_res?; - /// println!("a, b: {}, {}", a, b); - /// } - /// # Ok(()) - /// # } - /// ``` - pub async fn query_iter( + async fn do_query_iter( &self, query: impl Into, values: impl SerializeRow, @@ -1062,50 +1306,7 @@ impl LegacySession { .as_deref() } - /// Execute a prepared statement. Requires a [PreparedStatement] - /// generated using [`Session::prepare`](Session::prepare).\ - /// Performs an unpaged query, i.e. all results are received in a single response. - /// - /// As all results come in one response (no paging is done!), the memory footprint and latency may be huge - /// for statements returning rows (i.e. SELECTs)! Prefer this method for non-SELECTs, and for SELECTs - /// it is best to use paged queries: - /// - to receive multiple pages and transparently iterate through them, use [execute_iter](Session::execute_iter). - /// - to manually receive multiple pages and iterate through them, use [execute_single_page](Session::execute_single_page). - /// - /// Prepared queries are much faster than simple queries: - /// * Database doesn't need to parse the query - /// * They are properly load balanced using token aware routing - /// - /// > ***Warning***\ - /// > For token/shard aware load balancing to work properly, all partition key values - /// > must be sent as bound values - /// > (see [performance section](https://rust-driver.docs.scylladb.com/stable/queries/prepared.html#performance)). - /// - /// See [the book](https://rust-driver.docs.scylladb.com/stable/queries/prepared.html) for more information. - /// - /// # Arguments - /// * `prepared` - the prepared statement to execute, generated using [`Session::prepare`](Session::prepare) - /// * `values` - values bound to the query, the easiest way is to use a tuple of bound values - /// - /// # Example - /// ```rust - /// # use scylla::LegacySession; - /// # use std::error::Error; - /// # async fn check_only_compiles(session: &LegacySession) -> Result<(), Box> { - /// use scylla::prepared_statement::PreparedStatement; - /// - /// // Prepare the query for later execution - /// let prepared: PreparedStatement = session - /// .prepare("INSERT INTO ks.tab (a) VALUES(?)") - /// .await?; - /// - /// // Run the prepared query with some values, just like a simple query. - /// let to_insert: i32 = 12345; - /// session.execute_unpaged(&prepared, (to_insert,)).await?; - /// # Ok(()) - /// # } - /// ``` - pub async fn execute_unpaged( + async fn do_execute_unpaged( &self, prepared: &PreparedStatement, values: impl SerializeRow, @@ -1121,60 +1322,7 @@ impl LegacySession { Ok(result) } - /// Executes a prepared statement, restricting results to single page. - /// Optionally continues fetching results from a saved point. - /// - /// # Arguments - /// - /// * `prepared` - a statement prepared with [prepare](crate::Session::prepare) - /// * `values` - values bound to the query - /// * `paging_state` - continuation based on a paging state received from a previous paged query or None - /// - /// # Example - /// - /// ```rust - /// # use scylla::LegacySession; - /// # use std::error::Error; - /// # async fn check_only_compiles(session: &LegacySession) -> Result<(), Box> { - /// use std::ops::ControlFlow; - /// use scylla::query::Query; - /// use scylla::statement::{PagingState, PagingStateResponse}; - /// - /// let paged_prepared = session - /// .prepare( - /// Query::new("SELECT a, b FROM ks.tbl") - /// .with_page_size(100.try_into().unwrap()), - /// ) - /// .await?; - /// - /// // Manual paging in a loop, prepared statement. - /// let mut paging_state = PagingState::start(); - /// loop { - /// let (res, paging_state_response) = session - /// .execute_single_page(&paged_prepared, &[], paging_state) - /// .await?; - /// - /// // Do something with a single page of results. - /// for row in res.rows_typed::<(i32, String)>()? { - /// let (a, b) = row?; - /// } - /// - /// match paging_state_response.into_paging_control_flow() { - /// ControlFlow::Break(()) => { - /// // No more pages to be fetched. - /// break; - /// } - /// ControlFlow::Continue(new_paging_state) => { - /// // Update paging continuation from the paging state, so that query - /// // will be resumed from where it ended the last time. - /// paging_state = new_paging_state; - /// } - /// } - /// } - /// # Ok(()) - /// # } - /// ``` - pub async fn execute_single_page( + async fn do_execute_single_page( &self, prepared: &PreparedStatement, values: impl SerializeRow, @@ -1297,47 +1445,7 @@ impl LegacySession { Ok((result, paging_state)) } - /// Run a prepared query with paging.\ - /// This method will query all pages of the result.\ - /// - /// Returns an async iterator (stream) over all received rows.\ - /// Page size can be specified in the [PreparedStatement] passed to the function. - /// - /// See [the book](https://rust-driver.docs.scylladb.com/stable/queries/paged.html) for more information. - /// - /// # Arguments - /// * `prepared` - the prepared statement to execute, generated using [`Session::prepare`](Session::prepare) - /// * `values` - values bound to the query, the easiest way is to use a tuple of bound values - /// - /// # Example - /// - /// ```rust - /// # use scylla::LegacySession; - /// # use std::error::Error; - /// # async fn check_only_compiles(session: &LegacySession) -> Result<(), Box> { - /// use scylla::prepared_statement::PreparedStatement; - /// use scylla::IntoTypedRows; - /// use futures::stream::StreamExt; - /// - /// // Prepare the query for later execution - /// let prepared: PreparedStatement = session - /// .prepare("SELECT a, b FROM ks.t") - /// .await?; - /// - /// // Execute the query and receive all pages - /// let mut rows_stream = session - /// .execute_iter(prepared, &[]) - /// .await? - /// .into_typed::<(i32, i32)>(); - /// - /// while let Some(next_row_res) = rows_stream.next().await { - /// let (a, b): (i32, i32) = next_row_res?; - /// println!("a, b: {}, {}", a, b); - /// } - /// # Ok(()) - /// # } - /// ``` - pub async fn execute_iter( + async fn do_execute_iter( &self, prepared: impl Into, values: impl SerializeRow, @@ -1361,52 +1469,7 @@ impl LegacySession { .map(RawIterator::into_legacy) } - /// Perform a batch request.\ - /// Batch contains many `simple` or `prepared` queries which are executed at once.\ - /// Batch doesn't return any rows. - /// - /// Batch values must contain values for each of the queries. - /// - /// Avoid using non-empty values (`SerializeRow::is_empty()` return false) for unprepared statements - /// inside the batch. Such statements will first need to be prepared, so the driver will need to - /// send (numer_of_unprepared_statements_with_values + 1) requests instead of 1 request, severly - /// affecting performance. - /// - /// See [the book](https://rust-driver.docs.scylladb.com/stable/queries/batch.html) for more information. - /// - /// # Arguments - /// * `batch` - [Batch] to be performed - /// * `values` - List of values for each query, it's the easiest to use a tuple of tuples - /// - /// # Example - /// ```rust - /// # use scylla::LegacySession; - /// # use std::error::Error; - /// # async fn check_only_compiles(session: &LegacySession) -> Result<(), Box> { - /// use scylla::batch::Batch; - /// - /// let mut batch: Batch = Default::default(); - /// - /// // A query with two bound values - /// batch.append_statement("INSERT INTO ks.tab(a, b) VALUES(?, ?)"); - /// - /// // A query with one bound value - /// batch.append_statement("INSERT INTO ks.tab(a, b) VALUES(3, ?)"); - /// - /// // A query with no bound values - /// batch.append_statement("INSERT INTO ks.tab(a, b) VALUES(5, 6)"); - /// - /// // Batch values is a tuple of 3 tuples containing values for each query - /// let batch_values = ((1_i32, 2_i32), // Tuple with two values for the first query - /// (4_i32,), // Tuple with one value for the second query - /// ()); // Empty tuple/unit for the third query - /// - /// // Run the batch - /// session.batch(&batch, batch_values).await?; - /// # Ok(()) - /// # } - /// ``` - pub async fn batch( + async fn do_batch( &self, batch: &Batch, values: impl BatchValues, @@ -1676,8 +1739,8 @@ impl LegacySession { traces_events_query.set_page_size(TRACING_QUERY_PAGE_SIZE); let (traces_session_res, traces_events_res) = tokio::try_join!( - self.query_unpaged(traces_session_query, (tracing_id,)), - self.query_unpaged(traces_events_query, (tracing_id,)) + self.do_query_unpaged(&traces_session_query, (tracing_id,)), + self.do_query_unpaged(&traces_events_query, (tracing_id,)) )?; // Get tracing info From f6f29d95fe477ae0709e3f43b8ade016f42a148a Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Thu, 16 Mar 2023 18:20:29 +0100 Subject: [PATCH 26/42] session: re-introduce the Session type as an alias MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Adds Session as an alias over GenericSession. No methods (apart from the common ones) are added to it yet. Co-authored-by: Wojciech Przytuła --- scylla/src/lib.rs | 2 +- scylla/src/transport/session.rs | 5 +++++ scylla/src/transport/session_builder.rs | 8 ++++---- 3 files changed, 10 insertions(+), 5 deletions(-) diff --git a/scylla/src/lib.rs b/scylla/src/lib.rs index 34405e05d..6a3566825 100644 --- a/scylla/src/lib.rs +++ b/scylla/src/lib.rs @@ -260,7 +260,7 @@ pub use transport::caching_session::CachingSession; pub use transport::execution_profile::ExecutionProfile; pub use transport::legacy_query_result::LegacyQueryResult; pub use transport::query_result::{QueryResult, QueryRowsResult}; -pub use transport::session::{IntoTypedRows, LegacySession, SessionConfig}; +pub use transport::session::{IntoTypedRows, LegacySession, Session, SessionConfig}; pub use transport::session_builder::SessionBuilder; #[cfg(feature = "cloud")] diff --git a/scylla/src/transport/session.rs b/scylla/src/transport/session.rs index 6d1d5faa3..63910e11c 100644 --- a/scylla/src/transport/session.rs +++ b/scylla/src/transport/session.rs @@ -163,6 +163,10 @@ impl AddressTranslator for HashMap<&'static str, &'static str> { pub trait DeserializationApiKind: sealed::Sealed {} +pub enum CurrentDeserializationApi {} +impl sealed::Sealed for CurrentDeserializationApi {} +impl DeserializationApiKind for CurrentDeserializationApi {} + pub enum LegacyDeserializationApi {} impl sealed::Sealed for LegacyDeserializationApi {} impl DeserializationApiKind for LegacyDeserializationApi {} @@ -186,6 +190,7 @@ where _phantom_deser_api: PhantomData, } +pub type Session = GenericSession; pub type LegacySession = GenericSession; /// This implementation deliberately omits some details from Cluster in order diff --git a/scylla/src/transport/session_builder.rs b/scylla/src/transport/session_builder.rs index 15e10e098..da3de24fc 100644 --- a/scylla/src/transport/session_builder.rs +++ b/scylla/src/transport/session_builder.rs @@ -493,7 +493,7 @@ impl GenericSessionBuilder { /// Set keyspace to be used on all connections.\ /// Each connection will send `"USE "` before sending any requests.\ - /// This can be later changed with [`Session::use_keyspace`] + /// This can be later changed with [`crate::Session::use_keyspace`] /// /// # Example /// ``` @@ -815,7 +815,7 @@ impl GenericSessionBuilder { } /// Set the number of attempts to fetch [TracingInfo](crate::tracing::TracingInfo) - /// in [`LegacySession::get_tracing_info`]. + /// in [`Session::get_tracing_info`](crate::Session::get_tracing_info). /// The default is 5 attempts. /// /// Tracing info might not be available immediately on queried node - that's why @@ -844,7 +844,7 @@ impl GenericSessionBuilder { } /// Set the delay between attempts to fetch [TracingInfo](crate::tracing::TracingInfo) - /// in [`LegacySession::get_tracing_info`]. + /// in [`Session::get_tracing_info`](crate::Session::get_tracing_info). /// The default is 3 milliseconds. /// /// Tracing info might not be available immediately on queried node - that's why @@ -873,7 +873,7 @@ impl GenericSessionBuilder { } /// Set the consistency level of fetching [TracingInfo](crate::tracing::TracingInfo) - /// in [`LegacySession::get_tracing_info`]. + /// in [`Session::get_tracing_info`](crate::Session::get_tracing_info). /// The default is [`Consistency::One`]. /// /// # Example From 01811c1153668052102b312ecde5fe81b8ba6f0e Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Thu, 16 Mar 2023 11:54:43 +0100 Subject: [PATCH 27/42] session_builder: rename build->build_legacy and then reintroduce MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This commit renames the SessionBuilder::build method to build_legacy, and then reintroduces the build method so that it returns the new Session (not LegacySession). All the examples, tests, documentation will gradually be migrated to use SessionBuilder::build again in following commits. Co-authored-by: Wojciech Przytuła --- examples/allocations.rs | 5 +- examples/auth.rs | 2 +- examples/basic.rs | 2 +- examples/cloud.rs | 2 +- examples/compare-tokens.rs | 2 +- examples/cql-time-types.rs | 2 +- examples/cqlsh-rs.rs | 2 +- examples/custom_deserialization.rs | 2 +- examples/custom_load_balancing_policy.rs | 2 +- examples/execution_profile.rs | 4 +- examples/get_by_name.rs | 2 +- examples/logging.rs | 2 +- examples/logging_log.rs | 2 +- examples/parallel-prepared.rs | 2 +- examples/parallel.rs | 2 +- examples/query_history.rs | 2 +- examples/schema_agreement.rs | 2 +- examples/select-paging.rs | 2 +- examples/speculative-execution.rs | 2 +- examples/tls.rs | 2 +- examples/tower.rs | 2 +- examples/tracing.rs | 4 +- examples/user-defined-type.rs | 2 +- examples/value_list.rs | 2 +- scylla/src/history.rs | 6 +- scylla/src/lib.rs | 2 +- scylla/src/transport/authenticate_test.rs | 4 +- scylla/src/transport/caching_session.rs | 2 +- scylla/src/transport/connection.rs | 4 +- scylla/src/transport/cql_collections_test.rs | 2 +- scylla/src/transport/cql_types_test.rs | 10 +- scylla/src/transport/cql_value_test.rs | 4 +- scylla/src/transport/execution_profile.rs | 4 +- .../transport/large_batch_statements_test.rs | 2 +- .../src/transport/load_balancing/default.rs | 2 +- scylla/src/transport/session.rs | 2 +- scylla/src/transport/session_builder.rs | 114 ++++++++++++------ scylla/src/transport/session_test.rs | 98 ++++++++------- .../transport/silent_prepare_batch_test.rs | 2 +- scylla/tests/integration/consistency.rs | 6 +- .../tests/integration/execution_profiles.rs | 2 +- scylla/tests/integration/lwt_optimisation.rs | 2 +- scylla/tests/integration/new_session.rs | 2 +- scylla/tests/integration/retries.rs | 6 +- scylla/tests/integration/self_identity.rs | 2 +- scylla/tests/integration/shards.rs | 2 +- .../tests/integration/silent_prepare_query.rs | 4 +- .../integration/skip_metadata_optimization.rs | 2 +- scylla/tests/integration/tablets.rs | 6 +- 49 files changed, 192 insertions(+), 155 deletions(-) diff --git a/examples/allocations.rs b/examples/allocations.rs index 039d21e01..f87f7641f 100644 --- a/examples/allocations.rs +++ b/examples/allocations.rs @@ -128,7 +128,10 @@ async fn main() -> Result<()> { println!("Connecting to {} ...", args.node); - let session: LegacySession = SessionBuilder::new().known_node(args.node).build().await?; + let session: LegacySession = SessionBuilder::new() + .known_node(args.node) + .build_legacy() + .await?; let session = Arc::new(session); session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; diff --git a/examples/auth.rs b/examples/auth.rs index 22fbee007..ded1115f3 100644 --- a/examples/auth.rs +++ b/examples/auth.rs @@ -10,7 +10,7 @@ async fn main() -> Result<()> { let session = SessionBuilder::new() .known_node(uri) .user("cassandra", "cassandra") - .build() + .build_legacy() .await .unwrap(); diff --git a/examples/basic.rs b/examples/basic.rs index 48d97b713..ad8570db5 100644 --- a/examples/basic.rs +++ b/examples/basic.rs @@ -11,7 +11,7 @@ async fn main() -> Result<()> { println!("Connecting to {} ...", uri); - let session: LegacySession = SessionBuilder::new().known_node(uri).build().await?; + let session: LegacySession = SessionBuilder::new().known_node(uri).build_legacy().await?; session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; diff --git a/examples/cloud.rs b/examples/cloud.rs index 63265e41f..5859ef12e 100644 --- a/examples/cloud.rs +++ b/examples/cloud.rs @@ -12,7 +12,7 @@ async fn main() -> Result<()> { .unwrap_or("examples/config_data.yaml".to_owned()); let session = CloudSessionBuilder::new(Path::new(&config_path)) .unwrap() - .build() + .build_legacy() .await .unwrap(); diff --git a/examples/compare-tokens.rs b/examples/compare-tokens.rs index e302b9f83..4863608ff 100644 --- a/examples/compare-tokens.rs +++ b/examples/compare-tokens.rs @@ -10,7 +10,7 @@ async fn main() -> Result<()> { println!("Connecting to {} ...", uri); - let session: LegacySession = SessionBuilder::new().known_node(uri).build().await?; + let session: LegacySession = SessionBuilder::new().known_node(uri).build_legacy().await?; session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; diff --git a/examples/cql-time-types.rs b/examples/cql-time-types.rs index 8a8cedb66..1b9e475d4 100644 --- a/examples/cql-time-types.rs +++ b/examples/cql-time-types.rs @@ -16,7 +16,7 @@ async fn main() -> Result<()> { println!("Connecting to {} ...", uri); - let session: LegacySession = SessionBuilder::new().known_node(uri).build().await?; + let session: LegacySession = SessionBuilder::new().known_node(uri).build_legacy().await?; session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; diff --git a/examples/cqlsh-rs.rs b/examples/cqlsh-rs.rs index 0b9cd5a18..a4371909a 100644 --- a/examples/cqlsh-rs.rs +++ b/examples/cqlsh-rs.rs @@ -202,7 +202,7 @@ async fn main() -> Result<()> { let session: LegacySession = SessionBuilder::new() .known_node(uri) .compression(Some(Compression::Lz4)) - .build() + .build_legacy() .await?; let config = Config::builder() diff --git a/examples/custom_deserialization.rs b/examples/custom_deserialization.rs index 976afe468..7bd694c81 100644 --- a/examples/custom_deserialization.rs +++ b/examples/custom_deserialization.rs @@ -11,7 +11,7 @@ async fn main() -> Result<()> { println!("Connecting to {} ...", uri); - let session: LegacySession = SessionBuilder::new().known_node(uri).build().await?; + let session: LegacySession = SessionBuilder::new().known_node(uri).build_legacy().await?; session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; session diff --git a/examples/custom_load_balancing_policy.rs b/examples/custom_load_balancing_policy.rs index 9fa505384..e70ed0213 100644 --- a/examples/custom_load_balancing_policy.rs +++ b/examples/custom_load_balancing_policy.rs @@ -71,7 +71,7 @@ async fn main() -> Result<()> { let _session: LegacySession = SessionBuilder::new() .known_node(uri) .default_execution_profile_handle(profile.into_handle()) - .build() + .build_legacy() .await?; Ok(()) diff --git a/examples/execution_profile.rs b/examples/execution_profile.rs index 944245660..46ae8e03f 100644 --- a/examples/execution_profile.rs +++ b/examples/execution_profile.rs @@ -45,13 +45,13 @@ async fn main() -> Result<()> { let session1: LegacySession = SessionBuilder::new() .known_node(&uri) .default_execution_profile_handle(handle1.clone()) - .build() + .build_legacy() .await?; let session2: LegacySession = SessionBuilder::new() .known_node(&uri) .default_execution_profile_handle(handle2.clone()) - .build() + .build_legacy() .await?; // As default execution profile is not provided explicitly, session 3 uses a predefined one. diff --git a/examples/get_by_name.rs b/examples/get_by_name.rs index 2f3996e5e..a0a21b855 100644 --- a/examples/get_by_name.rs +++ b/examples/get_by_name.rs @@ -10,7 +10,7 @@ async fn main() -> Result<()> { println!("Connecting to {} ...", uri); - let session: LegacySession = SessionBuilder::new().known_node(uri).build().await?; + let session: LegacySession = SessionBuilder::new().known_node(uri).build_legacy().await?; session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; diff --git a/examples/logging.rs b/examples/logging.rs index 00071c4cd..37e534b8c 100644 --- a/examples/logging.rs +++ b/examples/logging.rs @@ -16,7 +16,7 @@ async fn main() -> Result<()> { let uri = env::var("SCYLLA_URI").unwrap_or_else(|_| "127.0.0.1:9042".to_string()); info!("Connecting to {}", uri); - let session: LegacySession = SessionBuilder::new().known_node(uri).build().await?; + let session: LegacySession = SessionBuilder::new().known_node(uri).build_legacy().await?; session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; session.query_unpaged("USE examples_ks", &[]).await?; diff --git a/examples/logging_log.rs b/examples/logging_log.rs index 9e8c81687..a1f962419 100644 --- a/examples/logging_log.rs +++ b/examples/logging_log.rs @@ -18,7 +18,7 @@ async fn main() -> Result<()> { let uri = env::var("SCYLLA_URI").unwrap_or_else(|_| "127.0.0.1:9042".to_string()); info!("Connecting to {}", uri); - let session: LegacySession = SessionBuilder::new().known_node(uri).build().await?; + let session: LegacySession = SessionBuilder::new().known_node(uri).build_legacy().await?; session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; session.query_unpaged("USE examples_ks", &[]).await?; diff --git a/examples/parallel-prepared.rs b/examples/parallel-prepared.rs index e848b305c..531f6d7b4 100644 --- a/examples/parallel-prepared.rs +++ b/examples/parallel-prepared.rs @@ -11,7 +11,7 @@ async fn main() -> Result<()> { println!("Connecting to {} ...", uri); - let session: LegacySession = SessionBuilder::new().known_node(uri).build().await?; + let session: LegacySession = SessionBuilder::new().known_node(uri).build_legacy().await?; let session = Arc::new(session); session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; diff --git a/examples/parallel.rs b/examples/parallel.rs index 3cf191661..5e3f119fb 100644 --- a/examples/parallel.rs +++ b/examples/parallel.rs @@ -11,7 +11,7 @@ async fn main() -> Result<()> { println!("Connecting to {} ...", uri); - let session: LegacySession = SessionBuilder::new().known_node(uri).build().await?; + let session: LegacySession = SessionBuilder::new().known_node(uri).build_legacy().await?; let session = Arc::new(session); session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; diff --git a/examples/query_history.rs b/examples/query_history.rs index 61ea56723..710f9616d 100644 --- a/examples/query_history.rs +++ b/examples/query_history.rs @@ -15,7 +15,7 @@ async fn main() -> Result<()> { println!("Connecting to {} ...", uri); - let session: LegacySession = SessionBuilder::new().known_node(uri).build().await?; + let session: LegacySession = SessionBuilder::new().known_node(uri).build_legacy().await?; session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; diff --git a/examples/schema_agreement.rs b/examples/schema_agreement.rs index bfc5ed01b..9b9369ac9 100644 --- a/examples/schema_agreement.rs +++ b/examples/schema_agreement.rs @@ -16,7 +16,7 @@ async fn main() -> Result<()> { let session: LegacySession = SessionBuilder::new() .known_node(uri) .schema_agreement_interval(Duration::from_secs(1)) // check every second for schema agreement if not agreed first check - .build() + .build_legacy() .await?; let schema_version = session.await_schema_agreement().await?; diff --git a/examples/select-paging.rs b/examples/select-paging.rs index 36d40f62d..f9027675a 100644 --- a/examples/select-paging.rs +++ b/examples/select-paging.rs @@ -11,7 +11,7 @@ async fn main() -> Result<()> { println!("Connecting to {} ...", uri); - let session: LegacySession = SessionBuilder::new().known_node(uri).build().await?; + let session: LegacySession = SessionBuilder::new().known_node(uri).build_legacy().await?; session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; diff --git a/examples/speculative-execution.rs b/examples/speculative-execution.rs index 13513c1d9..c53285cac 100644 --- a/examples/speculative-execution.rs +++ b/examples/speculative-execution.rs @@ -23,7 +23,7 @@ async fn main() -> Result<()> { let session: LegacySession = SessionBuilder::new() .known_node(uri) .default_execution_profile_handle(speculative_profile.into_handle()) - .build() + .build_legacy() .await?; session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; diff --git a/examples/tls.rs b/examples/tls.rs index c41e5e7f9..1bb354e56 100644 --- a/examples/tls.rs +++ b/examples/tls.rs @@ -47,7 +47,7 @@ async fn main() -> Result<()> { let session: LegacySession = SessionBuilder::new() .known_node(uri) .ssl_context(Some(context_builder.build())) - .build() + .build_legacy() .await?; session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; diff --git a/examples/tower.rs b/examples/tower.rs index 5f89890fc..0b6085e00 100644 --- a/examples/tower.rs +++ b/examples/tower.rs @@ -35,7 +35,7 @@ async fn main() -> anyhow::Result<()> { session: Arc::new( scylla::SessionBuilder::new() .known_node(uri) - .build() + .build_legacy() .await?, ), }; diff --git a/examples/tracing.rs b/examples/tracing.rs index 2ce7b2e61..435e356c7 100644 --- a/examples/tracing.rs +++ b/examples/tracing.rs @@ -23,7 +23,7 @@ async fn main() -> Result<()> { println!("Connecting to {} ...", uri); let session: LegacySession = SessionBuilder::new() .known_node(uri.as_str()) - .build() + .build_legacy() .await?; session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; @@ -117,7 +117,7 @@ async fn main() -> Result<()> { .tracing_info_fetch_attempts(NonZeroU32::new(8).unwrap()) .tracing_info_fetch_interval(Duration::from_millis(100)) .tracing_info_fetch_consistency(Consistency::One) - .build() + .build_legacy() .await?; let _custom_info: TracingInfo = session.get_tracing_info(&query_tracing_id).await?; diff --git a/examples/user-defined-type.rs b/examples/user-defined-type.rs index e8be4b2f9..9e01586a7 100644 --- a/examples/user-defined-type.rs +++ b/examples/user-defined-type.rs @@ -10,7 +10,7 @@ async fn main() -> Result<()> { println!("Connecting to {} ...", uri); - let session: LegacySession = SessionBuilder::new().known_node(uri).build().await?; + let session: LegacySession = SessionBuilder::new().known_node(uri).build_legacy().await?; session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; diff --git a/examples/value_list.rs b/examples/value_list.rs index e72d488b4..ce997b70e 100644 --- a/examples/value_list.rs +++ b/examples/value_list.rs @@ -8,7 +8,7 @@ async fn main() -> Result<()> { println!("Connecting to {} ...", uri); - let session: LegacySession = SessionBuilder::new().known_node(uri).build().await?; + let session: LegacySession = SessionBuilder::new().known_node(uri).build_legacy().await?; session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; diff --git a/scylla/src/history.rs b/scylla/src/history.rs index d4693163e..62ee1ad4d 100644 --- a/scylla/src/history.rs +++ b/scylla/src/history.rs @@ -917,7 +917,7 @@ mod tests { #[tokio::test] async fn successful_query_history() { setup_tracing(); - let session = create_new_session_builder().build().await.unwrap(); + let session = create_new_session_builder().build_legacy().await.unwrap(); let mut query = Query::new("SELECT * FROM system.local"); let history_collector = Arc::new(HistoryCollector::new()); @@ -984,7 +984,7 @@ mod tests { #[tokio::test] async fn failed_query_history() { setup_tracing(); - let session = create_new_session_builder().build().await.unwrap(); + let session = create_new_session_builder().build_legacy().await.unwrap(); let mut query = Query::new("This isnt even CQL"); let history_collector = Arc::new(HistoryCollector::new()); @@ -1021,7 +1021,7 @@ mod tests { #[tokio::test] async fn iterator_query_history() { setup_tracing(); - let session = create_new_session_builder().build().await.unwrap(); + let session = create_new_session_builder().build_legacy().await.unwrap(); let ks = unique_keyspace_name(); session .query_unpaged(format!("CREATE KEYSPACE {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]) diff --git a/scylla/src/lib.rs b/scylla/src/lib.rs index 6a3566825..1f16faba5 100644 --- a/scylla/src/lib.rs +++ b/scylla/src/lib.rs @@ -25,7 +25,7 @@ //! let session: LegacySession = SessionBuilder::new() //! .known_node("127.0.0.1:9042") //! .known_node("1.2.3.4:9876") -//! .build() +//! .build_legacy() //! .await?; //! //! Ok(()) diff --git a/scylla/src/transport/authenticate_test.rs b/scylla/src/transport/authenticate_test.rs index 78e72dea4..75d628ce4 100644 --- a/scylla/src/transport/authenticate_test.rs +++ b/scylla/src/transport/authenticate_test.rs @@ -16,7 +16,7 @@ async fn authenticate_superuser() { let session = crate::SessionBuilder::new() .known_node(uri) .user("cassandra", "cassandra") - .build() + .build_legacy() .await .unwrap(); let ks = unique_keyspace_name(); @@ -75,7 +75,7 @@ async fn custom_authentication() { let session = crate::SessionBuilder::new() .known_node(uri) .authenticator_provider(Arc::new(CustomAuthenticatorProvider)) - .build() + .build_legacy() .await .unwrap(); let ks = unique_keyspace_name(); diff --git a/scylla/src/transport/caching_session.rs b/scylla/src/transport/caching_session.rs index 2b0fcc05e..5b0f920f0 100644 --- a/scylla/src/transport/caching_session.rs +++ b/scylla/src/transport/caching_session.rs @@ -234,7 +234,7 @@ mod tests { async fn new_for_test(with_tablet_support: bool) -> LegacySession { let session = create_new_session_builder() - .build() + .build_legacy() .await .expect("Could not create session"); let ks = unique_keyspace_name(); diff --git a/scylla/src/transport/connection.rs b/scylla/src/transport/connection.rs index a2a3ef5f1..79551845d 100644 --- a/scylla/src/transport/connection.rs +++ b/scylla/src/transport/connection.rs @@ -2443,7 +2443,7 @@ mod tests { // Preparation phase let session = SessionBuilder::new() .known_node_addr(addr) - .build() + .build_legacy() .await .unwrap(); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks.clone()), &[]).await.unwrap(); @@ -2534,7 +2534,7 @@ mod tests { // Preparation phase let session = SessionBuilder::new() .known_node_addr(addr) - .build() + .build_legacy() .await .unwrap(); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks.clone()), &[]).await.unwrap(); diff --git a/scylla/src/transport/cql_collections_test.rs b/scylla/src/transport/cql_collections_test.rs index fe2a8a8d2..6322e6e92 100644 --- a/scylla/src/transport/cql_collections_test.rs +++ b/scylla/src/transport/cql_collections_test.rs @@ -6,7 +6,7 @@ use scylla_cql::types::serialize::value::SerializeValue; use std::collections::{BTreeMap, BTreeSet, HashMap, HashSet}; async fn connect() -> LegacySession { - let session = create_new_session_builder().build().await.unwrap(); + let session = create_new_session_builder().build_legacy().await.unwrap(); let ks = unique_keyspace_name(); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); session.use_keyspace(ks, false).await.unwrap(); diff --git a/scylla/src/transport/cql_types_test.rs b/scylla/src/transport/cql_types_test.rs index 32dd11638..1429b498b 100644 --- a/scylla/src/transport/cql_types_test.rs +++ b/scylla/src/transport/cql_types_test.rs @@ -23,7 +23,7 @@ async fn init_test_maybe_without_tablets( type_name: &str, supports_tablets: bool, ) -> LegacySession { - let session: LegacySession = create_new_session_builder().build().await.unwrap(); + let session: LegacySession = create_new_session_builder().build_legacy().await.unwrap(); let ks = unique_keyspace_name(); let mut create_ks = format!( @@ -168,7 +168,7 @@ async fn test_cql_varint() { ]; let table_name = "cql_varint_tests"; - let session: LegacySession = create_new_session_builder().build().await.unwrap(); + let session: LegacySession = create_new_session_builder().build_legacy().await.unwrap(); let ks = unique_keyspace_name(); session @@ -1234,7 +1234,7 @@ async fn test_timeuuid() { #[tokio::test] async fn test_timeuuid_ordering() { setup_tracing(); - let session: LegacySession = create_new_session_builder().build().await.unwrap(); + let session: LegacySession = create_new_session_builder().build_legacy().await.unwrap(); let ks = unique_keyspace_name(); session @@ -1466,7 +1466,7 @@ async fn test_udt_after_schema_update() { let table_name = "udt_tests"; let type_name = "usertype1"; - let session: LegacySession = create_new_session_builder().build().await.unwrap(); + let session: LegacySession = create_new_session_builder().build_legacy().await.unwrap(); let ks = unique_keyspace_name(); session @@ -1638,7 +1638,7 @@ async fn test_udt_with_missing_field() { let table_name = "udt_tests"; let type_name = "usertype1"; - let session: LegacySession = create_new_session_builder().build().await.unwrap(); + let session: LegacySession = create_new_session_builder().build_legacy().await.unwrap(); let ks = unique_keyspace_name(); session diff --git a/scylla/src/transport/cql_value_test.rs b/scylla/src/transport/cql_value_test.rs index be1047ede..d3d2d18e4 100644 --- a/scylla/src/transport/cql_value_test.rs +++ b/scylla/src/transport/cql_value_test.rs @@ -7,7 +7,7 @@ use crate::LegacySession; #[tokio::test] async fn test_cqlvalue_udt() { setup_tracing(); - let session: LegacySession = create_new_session_builder().build().await.unwrap(); + let session: LegacySession = create_new_session_builder().build_legacy().await.unwrap(); let ks = unique_keyspace_name(); session .query_unpaged( @@ -72,7 +72,7 @@ async fn test_cqlvalue_udt() { #[tokio::test] async fn test_cqlvalue_duration() { setup_tracing(); - let session: LegacySession = create_new_session_builder().build().await.unwrap(); + let session: LegacySession = create_new_session_builder().build_legacy().await.unwrap(); let ks = unique_keyspace_name(); session diff --git a/scylla/src/transport/execution_profile.rs b/scylla/src/transport/execution_profile.rs index 2854611b4..421a7da7e 100644 --- a/scylla/src/transport/execution_profile.rs +++ b/scylla/src/transport/execution_profile.rs @@ -30,7 +30,7 @@ //! let session: LegacySession = SessionBuilder::new() //! .known_node("127.0.0.1:9042") //! .default_execution_profile_handle(handle) -//! .build() +//! .build_legacy() //! .await?; //! # Ok(()) //! # } @@ -128,7 +128,7 @@ //! let session: LegacySession = SessionBuilder::new() //! .known_node("127.0.0.1:9042") //! .default_execution_profile_handle(handle1.clone()) -//! .build() +//! .build_legacy() //! .await?; //! //! let mut query1 = Query::from("SELECT * FROM ks.table"); diff --git a/scylla/src/transport/large_batch_statements_test.rs b/scylla/src/transport/large_batch_statements_test.rs index 2b394ed32..0e250fc7d 100644 --- a/scylla/src/transport/large_batch_statements_test.rs +++ b/scylla/src/transport/large_batch_statements_test.rs @@ -13,7 +13,7 @@ use crate::{ #[tokio::test] async fn test_large_batch_statements() { setup_tracing(); - let mut session = create_new_session_builder().build().await.unwrap(); + let mut session = create_new_session_builder().build_legacy().await.unwrap(); let ks = unique_keyspace_name(); session = create_test_session(session, &ks).await; diff --git a/scylla/src/transport/load_balancing/default.rs b/scylla/src/transport/load_balancing/default.rs index 51db7f97f..beffebb1f 100644 --- a/scylla/src/transport/load_balancing/default.rs +++ b/scylla/src/transport/load_balancing/default.rs @@ -3860,7 +3860,7 @@ mod latency_awareness { let session = create_new_session_builder() .default_execution_profile_handle(handle) - .build() + .build_legacy() .await .unwrap(); diff --git a/scylla/src/transport/session.rs b/scylla/src/transport/session.rs index 63910e11c..f1d2314b4 100644 --- a/scylla/src/transport/session.rs +++ b/scylla/src/transport/session.rs @@ -1634,7 +1634,7 @@ where /// # use scylla::{LegacySession, SessionBuilder}; /// # use scylla::transport::Compression; /// # async fn example() -> Result<(), Box> { - /// # let session = SessionBuilder::new().known_node("127.0.0.1:9042").build().await?; + /// # let session = SessionBuilder::new().known_node("127.0.0.1:9042").build_legacy().await?; /// session /// .query_unpaged("INSERT INTO my_keyspace.tab (a) VALUES ('test1')", &[]) /// .await?; diff --git a/scylla/src/transport/session_builder.rs b/scylla/src/transport/session_builder.rs index da3de24fc..73725d83e 100644 --- a/scylla/src/transport/session_builder.rs +++ b/scylla/src/transport/session_builder.rs @@ -2,7 +2,10 @@ use super::connection::SelfIdentity; use super::execution_profile::ExecutionProfileHandle; -use super::session::{AddressTranslator, LegacySession, SessionConfig}; +use super::session::{ + AddressTranslator, CurrentDeserializationApi, GenericSession, LegacyDeserializationApi, + SessionConfig, +}; use super::Compression; #[cfg(feature = "cloud")] @@ -65,7 +68,7 @@ pub type CloudSessionBuilder = GenericSessionBuilder; /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .compression(Some(Compression::Snappy)) -/// .build() +/// .build_legacy() /// .await?; /// # Ok(()) /// # } @@ -96,7 +99,10 @@ impl GenericSessionBuilder { /// ``` /// # use scylla::{LegacySession, SessionBuilder}; /// # async fn example() -> Result<(), Box> { - /// let session: LegacySession = SessionBuilder::new().known_node("127.0.0.1:9042").build().await?; + /// let session: LegacySession = SessionBuilder::new() + /// .known_node("127.0.0.1:9042") + /// .build_legacy() + /// .await?; /// # Ok(()) /// # } /// ``` @@ -104,7 +110,10 @@ impl GenericSessionBuilder { /// ``` /// # use scylla::{LegacySession, SessionBuilder}; /// # async fn example() -> Result<(), Box> { - /// let session: LegacySession = SessionBuilder::new().known_node("db1.example.com").build().await?; + /// let session: LegacySession = SessionBuilder::new() + /// .known_node("db1.example.com") + /// .build_legacy() + /// .await?; /// # Ok(()) /// # } /// ``` @@ -121,7 +130,7 @@ impl GenericSessionBuilder { /// # async fn example() -> Result<(), Box> { /// let session: LegacySession = SessionBuilder::new() /// .known_node_addr(SocketAddr::new(IpAddr::V4(Ipv4Addr::new(127, 0, 0, 1)), 9042)) - /// .build() + /// .build_legacy() /// .await?; /// # Ok(()) /// # } @@ -138,7 +147,7 @@ impl GenericSessionBuilder { /// # async fn example() -> Result<(), Box> { /// let session: LegacySession = SessionBuilder::new() /// .known_nodes(["127.0.0.1:9042", "db1.example.com"]) - /// .build() + /// .build_legacy() /// .await?; /// # Ok(()) /// # } @@ -159,7 +168,7 @@ impl GenericSessionBuilder { /// /// let session: LegacySession = SessionBuilder::new() /// .known_nodes_addr([addr1, addr2]) - /// .build() + /// .build_legacy() /// .await?; /// # Ok(()) /// # } @@ -184,7 +193,7 @@ impl GenericSessionBuilder { /// .known_node("127.0.0.1:9042") /// .use_keyspace("my_keyspace_name", false) /// .user("cassandra", "cassandra") - /// .build() + /// .build_legacy() /// .await?; /// # Ok(()) /// # } @@ -236,7 +245,7 @@ impl GenericSessionBuilder { /// .use_keyspace("my_keyspace_name", false) /// .user("cassandra", "cassandra") /// .authenticator_provider(Arc::new(CustomAuthenticatorProvider)) - /// .build() + /// .build_legacy() /// .await?; /// # Ok(()) /// # } @@ -276,7 +285,7 @@ impl GenericSessionBuilder { /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .address_translator(Arc::new(IdentityTranslator)) - /// .build() + /// .build_legacy() /// .await?; /// # Ok(()) /// # } @@ -298,7 +307,7 @@ impl GenericSessionBuilder { /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .address_translator(Arc::new(translation_rules)) - /// .build() + /// .build_legacy() /// .await?; /// # Ok(()) /// # } @@ -329,7 +338,7 @@ impl GenericSessionBuilder { /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .ssl_context(Some(context_builder.build())) - /// .build() + /// .build_legacy() /// .await?; /// # Ok(()) /// # } @@ -383,7 +392,7 @@ impl GenericSessionBuilder { /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .compression(Some(Compression::Snappy)) - /// .build() + /// .build_legacy() /// .await?; /// # Ok(()) /// # } @@ -404,7 +413,7 @@ impl GenericSessionBuilder { /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .schema_agreement_interval(Duration::from_secs(5)) - /// .build() + /// .build_legacy() /// .await?; /// # Ok(()) /// # } @@ -428,7 +437,7 @@ impl GenericSessionBuilder { /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .default_execution_profile_handle(execution_profile.into_handle()) - /// .build() + /// .build_legacy() /// .await?; /// # Ok(()) /// # } @@ -451,7 +460,7 @@ impl GenericSessionBuilder { /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .tcp_nodelay(true) - /// .build() + /// .build_legacy() /// .await?; /// # Ok(()) /// # } @@ -474,7 +483,7 @@ impl GenericSessionBuilder { /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .tcp_keepalive_interval(std::time::Duration::from_secs(42)) - /// .build() + /// .build_legacy() /// .await?; /// # Ok(()) /// # } @@ -503,7 +512,7 @@ impl GenericSessionBuilder { /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .use_keyspace("my_keyspace_name", false) - /// .build() + /// .build_legacy() /// .await?; /// # Ok(()) /// # } @@ -514,7 +523,10 @@ impl GenericSessionBuilder { self } - /// Builds the Session after setting all the options + /// Builds the Session after setting all the options. + /// + /// The new session object uses the legacy deserialization API. If you wish + /// to use the new API, use [`SessionBuilder::build`]. /// /// # Example /// ``` @@ -524,13 +536,39 @@ impl GenericSessionBuilder { /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .compression(Some(Compression::Snappy)) + /// .build_legacy() // Turns SessionBuilder into Session + /// .await?; + /// # Ok(()) + /// # } + /// ``` + pub async fn build_legacy( + &self, + ) -> Result, NewSessionError> { + GenericSession::connect(self.config.clone()).await + } + + /// Builds the Session after setting all the options. + /// + /// The new session object uses the new deserialization API. If you wish + /// to use the old API, use [`SessionBuilder::build_legacy`]. + /// + /// # Example + /// ``` + /// # use scylla::{Session, SessionBuilder}; + /// # use scylla::transport::Compression; + /// # async fn example() -> Result<(), Box> { + /// let session: Session = SessionBuilder::new() + /// .known_node("127.0.0.1:9042") + /// .compression(Some(Compression::Snappy)) /// .build() // Turns SessionBuilder into Session /// .await?; /// # Ok(()) /// # } /// ``` - pub async fn build(&self) -> Result { - LegacySession::connect(self.config.clone()).await + pub async fn build( + &self, + ) -> Result, NewSessionError> { + GenericSession::connect(self.config.clone()).await } /// Changes connection timeout @@ -545,7 +583,7 @@ impl GenericSessionBuilder { /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .connection_timeout(Duration::from_secs(30)) - /// .build() // Turns SessionBuilder into Session + /// .build_legacy() // Turns SessionBuilder into Session /// .await?; /// # Ok(()) /// # } @@ -570,7 +608,7 @@ impl GenericSessionBuilder { /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .pool_size(PoolSize::PerHost(NonZeroUsize::new(4).unwrap())) - /// .build() + /// .build_legacy() /// .await?; /// # Ok(()) /// # } @@ -609,7 +647,7 @@ impl GenericSessionBuilder { /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .disallow_shard_aware_port(true) - /// .build() + /// .build_legacy() /// .await?; /// # Ok(()) /// # } @@ -629,7 +667,7 @@ impl GenericSessionBuilder { /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .keyspaces_to_fetch(["my_keyspace"]) - /// .build() + /// .build_legacy() /// .await?; /// # Ok(()) /// # } @@ -652,7 +690,7 @@ impl GenericSessionBuilder { /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .fetch_schema_metadata(true) - /// .build() + /// .build_legacy() /// .await?; /// # Ok(()) /// # } @@ -675,7 +713,7 @@ impl GenericSessionBuilder { /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .keepalive_interval(std::time::Duration::from_secs(42)) - /// .build() + /// .build_legacy() /// .await?; /// # Ok(()) /// # } @@ -705,7 +743,7 @@ impl GenericSessionBuilder { /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .keepalive_timeout(std::time::Duration::from_secs(42)) - /// .build() + /// .build_legacy() /// .await?; /// # Ok(()) /// # } @@ -732,7 +770,7 @@ impl GenericSessionBuilder { /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .schema_agreement_timeout(std::time::Duration::from_secs(120)) - /// .build() + /// .build_legacy() /// .await?; /// # Ok(()) /// # } @@ -752,7 +790,7 @@ impl GenericSessionBuilder { /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .auto_await_schema_agreement(false) - /// .build() + /// .build_legacy() /// .await?; /// # Ok(()) /// # } @@ -784,7 +822,7 @@ impl GenericSessionBuilder { /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .host_filter(Arc::new(DcHostFilter::new("my-local-dc".to_string()))) - /// .build() + /// .build_legacy() /// .await?; /// # Ok(()) /// # } @@ -804,7 +842,7 @@ impl GenericSessionBuilder { /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .refresh_metadata_on_auto_schema_agreement(true) - /// .build() + /// .build_legacy() /// .await?; /// # Ok(()) /// # } @@ -833,7 +871,7 @@ impl GenericSessionBuilder { /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .tracing_info_fetch_attempts(NonZeroU32::new(10).unwrap()) - /// .build() + /// .build_legacy() /// .await?; /// # Ok(()) /// # } @@ -862,7 +900,7 @@ impl GenericSessionBuilder { /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .tracing_info_fetch_interval(Duration::from_millis(50)) - /// .build() + /// .build_legacy() /// .await?; /// # Ok(()) /// # } @@ -883,7 +921,7 @@ impl GenericSessionBuilder { /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .tracing_info_fetch_consistency(Consistency::One) - /// .build() + /// .build_legacy() /// .await?; /// # Ok(()) /// # } @@ -914,7 +952,7 @@ impl GenericSessionBuilder { /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .write_coalescing(false) // Enabled by default - /// .build() + /// .build_legacy() /// .await?; /// # Ok(()) /// # } @@ -938,7 +976,7 @@ impl GenericSessionBuilder { /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .cluster_metadata_refresh_interval(std::time::Duration::from_secs(20)) - /// .build() + /// .build_legacy() /// .await?; /// # Ok(()) /// # } @@ -970,7 +1008,7 @@ impl GenericSessionBuilder { /// .with_application_name("my-app") /// .with_application_version(app_version) /// ) - /// .build() + /// .build_legacy() /// .await?; /// # Ok(()) /// # } diff --git a/scylla/src/transport/session_test.rs b/scylla/src/transport/session_test.rs index b596e05b2..c3eb0de7f 100644 --- a/scylla/src/transport/session_test.rs +++ b/scylla/src/transport/session_test.rs @@ -55,7 +55,10 @@ async fn test_connection_failure() { .remote_handle(); tokio::spawn(fut); - let res = SessionBuilder::new().known_node_addr(addr).build().await; + let res = SessionBuilder::new() + .known_node_addr(addr) + .build_legacy() + .await; match res { Ok(_) => panic!("Unexpected success"), Err(err) => println!("Connection error (it was expected): {:?}", err), @@ -65,7 +68,7 @@ async fn test_connection_failure() { #[tokio::test] async fn test_unprepared_statement() { setup_tracing(); - let session = create_new_session_builder().build().await.unwrap(); + let session = create_new_session_builder().build_legacy().await.unwrap(); let ks = unique_keyspace_name(); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); @@ -167,7 +170,7 @@ async fn test_unprepared_statement() { #[tokio::test] async fn test_prepared_statement() { setup_tracing(); - let session = create_new_session_builder().build().await.unwrap(); + let session = create_new_session_builder().build_legacy().await.unwrap(); let ks = unique_keyspace_name(); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); @@ -368,7 +371,7 @@ async fn test_counter_batch() { use scylla_cql::frame::request::batch::BatchType; setup_tracing(); - let session = Arc::new(create_new_session_builder().build().await.unwrap()); + let session = Arc::new(create_new_session_builder().build_legacy().await.unwrap()); let ks = unique_keyspace_name(); // Need to disable tablets in this test because they don't support counters yet. @@ -423,7 +426,7 @@ async fn test_counter_batch() { #[tokio::test] async fn test_batch() { setup_tracing(); - let session = Arc::new(create_new_session_builder().build().await.unwrap()); + let session = Arc::new(create_new_session_builder().build_legacy().await.unwrap()); let ks = unique_keyspace_name(); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); @@ -525,7 +528,7 @@ async fn test_batch() { #[tokio::test] async fn test_token_calculation() { setup_tracing(); - let session = create_new_session_builder().build().await.unwrap(); + let session = create_new_session_builder().build_legacy().await.unwrap(); let ks = unique_keyspace_name(); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); @@ -584,7 +587,7 @@ async fn test_token_calculation() { #[tokio::test] async fn test_token_awareness() { setup_tracing(); - let session = create_new_session_builder().build().await.unwrap(); + let session = create_new_session_builder().build_legacy().await.unwrap(); let ks = unique_keyspace_name(); // Need to disable tablets in this test because they make token routing @@ -646,7 +649,7 @@ async fn test_token_awareness() { #[tokio::test] async fn test_use_keyspace() { setup_tracing(); - let session = create_new_session_builder().build().await.unwrap(); + let session = create_new_session_builder().build_legacy().await.unwrap(); let ks = unique_keyspace_name(); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); @@ -716,7 +719,7 @@ async fn test_use_keyspace() { // Make sure that use_keyspace on SessionBuiler works let session2: LegacySession = create_new_session_builder() .use_keyspace(ks.clone(), false) - .build() + .build_legacy() .await .unwrap(); @@ -737,7 +740,7 @@ async fn test_use_keyspace() { #[tokio::test] async fn test_use_keyspace_case_sensitivity() { setup_tracing(); - let session = create_new_session_builder().build().await.unwrap(); + let session = create_new_session_builder().build_legacy().await.unwrap(); let ks_lower = unique_keyspace_name().to_lowercase(); let ks_upper = ks_lower.to_uppercase(); @@ -810,7 +813,7 @@ async fn test_use_keyspace_case_sensitivity() { #[tokio::test] async fn test_raw_use_keyspace() { setup_tracing(); - let session = create_new_session_builder().build().await.unwrap(); + let session = create_new_session_builder().build_legacy().await.unwrap(); let ks = unique_keyspace_name(); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); @@ -862,7 +865,7 @@ async fn test_raw_use_keyspace() { #[tokio::test] async fn test_fetch_system_keyspace() { setup_tracing(); - let session = create_new_session_builder().build().await.unwrap(); + let session = create_new_session_builder().build_legacy().await.unwrap(); let prepared_statement = session .prepare("SELECT * FROM system_schema.keyspaces") @@ -879,7 +882,7 @@ async fn test_fetch_system_keyspace() { #[tokio::test] async fn test_db_errors() { setup_tracing(); - let session = create_new_session_builder().build().await.unwrap(); + let session = create_new_session_builder().build_legacy().await.unwrap(); let ks = unique_keyspace_name(); // SyntaxError on bad query @@ -934,7 +937,7 @@ async fn test_db_errors() { #[tokio::test] async fn test_tracing() { setup_tracing(); - let session = create_new_session_builder().build().await.unwrap(); + let session = create_new_session_builder().build_legacy().await.unwrap(); let ks = unique_keyspace_name(); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); @@ -1176,21 +1179,14 @@ async fn assert_in_tracing_table(session: &LegacySession, tracing_uuid: Uuid) { #[tokio::test] async fn test_await_schema_agreement() { setup_tracing(); - let session = create_new_session_builder().build().await.unwrap(); + let session = create_new_session_builder().build_legacy().await.unwrap(); let _schema_version = session.await_schema_agreement().await.unwrap(); } -#[tokio::test] -async fn test_await_timed_schema_agreement() { - setup_tracing(); - let session = create_new_session_builder().build().await.unwrap(); - session.await_schema_agreement().await.unwrap(); -} - #[tokio::test] async fn test_timestamp() { setup_tracing(); - let session = create_new_session_builder().build().await.unwrap(); + let session = create_new_session_builder().build_legacy().await.unwrap(); let ks = unique_keyspace_name(); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); @@ -1309,7 +1305,7 @@ async fn test_request_timeout() { .into_handle(); { - let session = create_new_session_builder().build().await.unwrap(); + let session = create_new_session_builder().build_legacy().await.unwrap(); let mut query: Query = Query::new("SELECT * FROM system_schema.tables"); query.set_request_timeout(Some(Duration::from_millis(1))); @@ -1332,7 +1328,7 @@ async fn test_request_timeout() { { let timeouting_session = create_new_session_builder() .default_execution_profile_handle(fast_timeouting_profile_handle) - .build() + .build_legacy() .await .unwrap(); @@ -1368,7 +1364,7 @@ async fn test_request_timeout() { #[tokio::test] async fn test_prepared_config() { setup_tracing(); - let session = create_new_session_builder().build().await.unwrap(); + let session = create_new_session_builder().build_legacy().await.unwrap(); let mut query = Query::new("SELECT * FROM system_schema.tables"); query.set_is_idempotent(true); @@ -1455,7 +1451,7 @@ fn udt_type_c_def(ks: &str) -> Arc { #[tokio::test] async fn test_schema_types_in_metadata() { setup_tracing(); - let session = create_new_session_builder().build().await.unwrap(); + let session = create_new_session_builder().build_legacy().await.unwrap(); let ks = unique_keyspace_name(); session @@ -1614,7 +1610,7 @@ async fn test_schema_types_in_metadata() { #[tokio::test] async fn test_user_defined_types_in_metadata() { setup_tracing(); - let session = create_new_session_builder().build().await.unwrap(); + let session = create_new_session_builder().build_legacy().await.unwrap(); let ks = unique_keyspace_name(); session @@ -1678,7 +1674,7 @@ async fn test_user_defined_types_in_metadata() { #[tokio::test] async fn test_column_kinds_in_metadata() { setup_tracing(); - let session = create_new_session_builder().build().await.unwrap(); + let session = create_new_session_builder().build_legacy().await.unwrap(); let ks = unique_keyspace_name(); session @@ -1724,7 +1720,7 @@ async fn test_column_kinds_in_metadata() { #[tokio::test] async fn test_primary_key_ordering_in_metadata() { setup_tracing(); - let session = create_new_session_builder().build().await.unwrap(); + let session = create_new_session_builder().build_legacy().await.unwrap(); let ks = unique_keyspace_name(); session @@ -1773,7 +1769,7 @@ async fn test_table_partitioner_in_metadata() { return; } - let session = create_new_session_builder().build().await.unwrap(); + let session = create_new_session_builder().build_legacy().await.unwrap(); let ks = unique_keyspace_name(); // This test uses CDC which is not yet compatible with Scylla's tablets. @@ -1819,7 +1815,7 @@ async fn test_turning_off_schema_fetching() { setup_tracing(); let session = create_new_session_builder() .fetch_schema_metadata(false) - .build() + .build_legacy() .await .unwrap(); let ks = unique_keyspace_name(); @@ -1895,7 +1891,7 @@ async fn test_turning_off_schema_fetching() { #[tokio::test] async fn test_named_bind_markers() { - let session = create_new_session_builder().build().await.unwrap(); + let session = create_new_session_builder().build_legacy().await.unwrap(); let ks = unique_keyspace_name(); session @@ -1952,7 +1948,7 @@ async fn test_named_bind_markers() { #[tokio::test] async fn test_prepared_partitioner() { - let session = create_new_session_builder().build().await.unwrap(); + let session = create_new_session_builder().build_legacy().await.unwrap(); let ks = unique_keyspace_name(); // This test uses CDC which is not yet compatible with Scylla's tablets. @@ -2034,7 +2030,7 @@ async fn rename_caching(session: &CachingSession, rename_str: &str) { async fn test_unprepared_reprepare_in_execute() { let _ = tracing_subscriber::fmt::try_init(); - let session = create_new_session_builder().build().await.unwrap(); + let session = create_new_session_builder().build_legacy().await.unwrap(); let ks = unique_keyspace_name(); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); @@ -2095,7 +2091,7 @@ async fn test_unprepared_reprepare_in_execute() { async fn test_unusual_valuelists() { let _ = tracing_subscriber::fmt::try_init(); - let session = create_new_session_builder().build().await.unwrap(); + let session = create_new_session_builder().build_legacy().await.unwrap(); let ks = unique_keyspace_name(); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); @@ -2163,7 +2159,7 @@ async fn test_unusual_valuelists() { async fn test_unprepared_reprepare_in_batch() { let _ = tracing_subscriber::fmt::try_init(); - let session = create_new_session_builder().build().await.unwrap(); + let session = create_new_session_builder().build_legacy().await.unwrap(); let ks = unique_keyspace_name(); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); @@ -2228,7 +2224,7 @@ async fn test_unprepared_reprepare_in_batch() { async fn test_unprepared_reprepare_in_caching_session_execute() { let _ = tracing_subscriber::fmt::try_init(); - let session = create_new_session_builder().build().await.unwrap(); + let session = create_new_session_builder().build_legacy().await.unwrap(); let ks = unique_keyspace_name(); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); @@ -2288,7 +2284,7 @@ async fn test_unprepared_reprepare_in_caching_session_execute() { async fn test_views_in_schema_info() { let _ = tracing_subscriber::fmt::try_init(); - let session = create_new_session_builder().build().await.unwrap(); + let session = create_new_session_builder().build_legacy().await.unwrap(); let ks = unique_keyspace_name(); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); @@ -2359,7 +2355,7 @@ async fn assert_test_batch_table_rows_contain(sess: &LegacySession, expected_row #[tokio::test] async fn test_prepare_batch() { - let session = create_new_session_builder().build().await.unwrap(); + let session = create_new_session_builder().build_legacy().await.unwrap(); let ks = unique_keyspace_name(); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); @@ -2456,7 +2452,7 @@ async fn test_prepare_batch() { #[tokio::test] async fn test_refresh_metadata_after_schema_agreement() { - let session = create_new_session_builder().build().await.unwrap(); + let session = create_new_session_builder().build_legacy().await.unwrap(); let ks = unique_keyspace_name(); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); @@ -2494,7 +2490,7 @@ async fn test_refresh_metadata_after_schema_agreement() { #[tokio::test] async fn test_rate_limit_exceeded_exception() { - let session = create_new_session_builder().build().await.unwrap(); + let session = create_new_session_builder().build_legacy().await.unwrap(); // Typed errors in RPC were introduced along with per-partition rate limiting. // There is no dedicated feature for per-partition rate limiting, so we are @@ -2542,7 +2538,7 @@ async fn test_rate_limit_exceeded_exception() { // Batches containing LWT queries (IF col = som) return rows with information whether the queries were applied. #[tokio::test] async fn test_batch_lwts() { - let session = create_new_session_builder().build().await.unwrap(); + let session = create_new_session_builder().build_legacy().await.unwrap(); let ks = unique_keyspace_name(); let mut create_ks = format!("CREATE KEYSPACE {} WITH REPLICATION = {{'class': 'NetworkTopologyStrategy', 'replication_factor': 1}}", ks); @@ -2669,7 +2665,7 @@ async fn test_keyspaces_to_fetch() { let ks1 = unique_keyspace_name(); let ks2 = unique_keyspace_name(); - let session_default = create_new_session_builder().build().await.unwrap(); + let session_default = create_new_session_builder().build_legacy().await.unwrap(); for ks in [&ks1, &ks2] { session_default .query_unpaged(format!("CREATE KEYSPACE {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]) @@ -2688,7 +2684,7 @@ async fn test_keyspaces_to_fetch() { let session1 = create_new_session_builder() .keyspaces_to_fetch([&ks1]) - .build() + .build_legacy() .await .unwrap(); assert!(session1.get_cluster_data().keyspaces.contains_key(&ks1)); @@ -2696,7 +2692,7 @@ async fn test_keyspaces_to_fetch() { let session_all = create_new_session_builder() .keyspaces_to_fetch([] as [String; 0]) - .build() + .build_legacy() .await .unwrap(); assert!(session_all.get_cluster_data().keyspaces.contains_key(&ks1)); @@ -2738,7 +2734,7 @@ async fn test_iter_works_when_retry_policy_returns_ignore_write_error() { let session = create_new_session_builder() .default_execution_profile_handle(handle) - .build() + .build_legacy() .await .unwrap(); @@ -2781,7 +2777,7 @@ async fn test_iter_works_when_retry_policy_returns_ignore_write_error() { #[tokio::test] async fn test_iter_methods_with_modification_statements() { - let session = create_new_session_builder().build().await.unwrap(); + let session = create_new_session_builder().build_legacy().await.unwrap(); let ks = unique_keyspace_name(); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); @@ -2822,7 +2818,7 @@ async fn test_get_keyspace_name() { // Create the keyspace // No keyspace is set in config, so get_keyspace() should return None. - let session = create_new_session_builder().build().await.unwrap(); + let session = create_new_session_builder().build_legacy().await.unwrap(); assert_eq!(session.get_keyspace(), None); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); assert_eq!(session.get_keyspace(), None); @@ -2847,7 +2843,7 @@ async fn test_get_keyspace_name() { #[tokio::test] async fn simple_strategy_test() { let ks = unique_keyspace_name(); - let session = create_new_session_builder().build().await.unwrap(); + let session = create_new_session_builder().build_legacy().await.unwrap(); session .query_unpaged( @@ -2912,7 +2908,7 @@ async fn simple_strategy_test() { async fn test_manual_primary_key_computation() { // Setup session let ks = unique_keyspace_name(); - let session = create_new_session_builder().build().await.unwrap(); + let session = create_new_session_builder().build_legacy().await.unwrap(); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); session.use_keyspace(&ks, true).await.unwrap(); diff --git a/scylla/src/transport/silent_prepare_batch_test.rs b/scylla/src/transport/silent_prepare_batch_test.rs index f8c7fb328..c7ae8e83d 100644 --- a/scylla/src/transport/silent_prepare_batch_test.rs +++ b/scylla/src/transport/silent_prepare_batch_test.rs @@ -9,7 +9,7 @@ use std::collections::BTreeSet; #[tokio::test] async fn test_quietly_prepare_batch() { setup_tracing(); - let session = create_new_session_builder().build().await.unwrap(); + let session = create_new_session_builder().build_legacy().await.unwrap(); let ks = unique_keyspace_name(); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); diff --git a/scylla/tests/integration/consistency.rs b/scylla/tests/integration/consistency.rs index 4a3b1306f..5531973ec 100644 --- a/scylla/tests/integration/consistency.rs +++ b/scylla/tests/integration/consistency.rs @@ -159,7 +159,7 @@ async fn check_for_all_consistencies_and_setting_options< let session = session_builder .clone() .default_execution_profile_handle(base_for_every_profile.clone().build().into_handle()) - .build() + .build_legacy() .await .unwrap(); create_schema(&session, ks).await; @@ -212,7 +212,7 @@ async fn check_for_all_consistencies_and_setting_options< let session_with_consistencies = session_builder .clone() .default_execution_profile_handle(handle) - .build() + .build_legacy() .await .unwrap(); session_with_consistencies @@ -473,7 +473,7 @@ async fn consistency_allows_for_paxos_selects() { let session = SessionBuilder::new() .known_node(uri.as_str()) - .build() + .build_legacy() .await .unwrap(); diff --git a/scylla/tests/integration/execution_profiles.rs b/scylla/tests/integration/execution_profiles.rs index 0a49bae78..46ca7c7a2 100644 --- a/scylla/tests/integration/execution_profiles.rs +++ b/scylla/tests/integration/execution_profiles.rs @@ -159,7 +159,7 @@ async fn test_execution_profiles() { .known_node(proxy_uris[0].as_str()) .address_translator(Arc::new(translation_map)) .default_execution_profile_handle(profile1.into_handle()) - .build() + .build_legacy() .await .unwrap(); let ks = unique_keyspace_name(); diff --git a/scylla/tests/integration/lwt_optimisation.rs b/scylla/tests/integration/lwt_optimisation.rs index f0d59f1f0..dd1e855d8 100644 --- a/scylla/tests/integration/lwt_optimisation.rs +++ b/scylla/tests/integration/lwt_optimisation.rs @@ -56,7 +56,7 @@ async fn if_lwt_optimisation_mark_offered_then_negotiatied_and_lwt_routed_optima .known_node(proxy_uris[0].as_str()) .default_execution_profile_handle(handle) .address_translator(Arc::new(translation_map)) - .build() + .build_legacy() .await .unwrap(); diff --git a/scylla/tests/integration/new_session.rs b/scylla/tests/integration/new_session.rs index 6f734f0da..d28fa1d7a 100644 --- a/scylla/tests/integration/new_session.rs +++ b/scylla/tests/integration/new_session.rs @@ -16,7 +16,7 @@ async fn proceed_if_only_some_hostnames_are_invalid() { let session = SessionBuilder::new() .known_nodes([uri1, uri2, uri3]) - .build() + .build_legacy() .await .unwrap(); session diff --git a/scylla/tests/integration/retries.rs b/scylla/tests/integration/retries.rs index f6e1711ac..e8bd5477c 100644 --- a/scylla/tests/integration/retries.rs +++ b/scylla/tests/integration/retries.rs @@ -31,7 +31,7 @@ async fn speculative_execution_is_fired() { .known_node(proxy_uris[0].as_str()) .default_execution_profile_handle(simple_speculative_no_retry_profile.into_handle()) .address_translator(Arc::new(translation_map)) - .build() + .build_legacy() .await .unwrap(); @@ -107,7 +107,7 @@ async fn retries_occur() { let session: LegacySession = SessionBuilder::new() .known_node(proxy_uris[0].as_str()) .address_translator(Arc::new(translation_map)) - .build() + .build_legacy() .await .unwrap(); @@ -187,7 +187,7 @@ async fn speculative_execution_panic_regression_test() { .known_node(proxy_uris[0].as_str()) .address_translator(Arc::new(translation_map)) .default_execution_profile_handle(profile.into_handle()) - .build() + .build_legacy() .await .unwrap(); diff --git a/scylla/tests/integration/self_identity.rs b/scylla/tests/integration/self_identity.rs index d68bb0add..e7378c6e2 100644 --- a/scylla/tests/integration/self_identity.rs +++ b/scylla/tests/integration/self_identity.rs @@ -54,7 +54,7 @@ async fn test_given_self_identity(self_identity: SelfIdentity<'static>) { .known_node(proxy_uris[0].as_str()) .address_translator(Arc::new(translation_map)) .custom_identity(self_identity.clone()) - .build() + .build_legacy() .await .unwrap(); diff --git a/scylla/tests/integration/shards.rs b/scylla/tests/integration/shards.rs index b22cfc397..3a4e0ecb1 100644 --- a/scylla/tests/integration/shards.rs +++ b/scylla/tests/integration/shards.rs @@ -32,7 +32,7 @@ async fn test_consistent_shard_awareness() { let session = SessionBuilder::new() .known_node(proxy_uris[0].as_str()) .address_translator(Arc::new(translation_map)) - .build() + .build_legacy() .await .unwrap(); let ks = unique_keyspace_name(); diff --git a/scylla/tests/integration/silent_prepare_query.rs b/scylla/tests/integration/silent_prepare_query.rs index ffb200c7c..64da40406 100644 --- a/scylla/tests/integration/silent_prepare_query.rs +++ b/scylla/tests/integration/silent_prepare_query.rs @@ -22,7 +22,7 @@ async fn test_prepare_query_with_values() { let session: LegacySession = SessionBuilder::new() .known_node(proxy_uris[0].as_str()) .address_translator(Arc::new(translation_map)) - .build() + .build_legacy() .await .unwrap(); @@ -73,7 +73,7 @@ async fn test_query_with_no_values() { let session: LegacySession = SessionBuilder::new() .known_node(proxy_uris[0].as_str()) .address_translator(Arc::new(translation_map)) - .build() + .build_legacy() .await .unwrap(); diff --git a/scylla/tests/integration/skip_metadata_optimization.rs b/scylla/tests/integration/skip_metadata_optimization.rs index eee25f908..a50e33b8b 100644 --- a/scylla/tests/integration/skip_metadata_optimization.rs +++ b/scylla/tests/integration/skip_metadata_optimization.rs @@ -23,7 +23,7 @@ async fn test_skip_result_metadata() { let session: LegacySession = SessionBuilder::new() .known_node(proxy_uris[0].as_str()) .address_translator(Arc::new(translation_map)) - .build() + .build_legacy() .await .unwrap(); diff --git a/scylla/tests/integration/tablets.rs b/scylla/tests/integration/tablets.rs index 67fb2fd88..a51d009c1 100644 --- a/scylla/tests/integration/tablets.rs +++ b/scylla/tests/integration/tablets.rs @@ -294,7 +294,7 @@ async fn test_default_policy_is_tablet_aware() { let session = scylla::SessionBuilder::new() .known_node(proxy_uris[0].as_str()) .address_translator(Arc::new(translation_map)) - .build() + .build_legacy() .await .unwrap(); @@ -425,7 +425,7 @@ async fn test_tablet_feedback_not_sent_for_unprepared_queries() { let session = scylla::SessionBuilder::new() .known_node(proxy_uris[0].as_str()) .address_translator(Arc::new(translation_map)) - .build() + .build_legacy() .await .unwrap(); @@ -497,7 +497,7 @@ async fn test_lwt_optimization_works_with_tablets() { let session = scylla::SessionBuilder::new() .known_node(proxy_uris[0].as_str()) .address_translator(Arc::new(translation_map)) - .build() + .build_legacy() .await .unwrap(); From 1cc4a9a1379891ed761e5c8bca4d613d5c662cba Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Wojciech=20Przytu=C5=82a?= Date: Thu, 8 Aug 2024 12:43:07 +0200 Subject: [PATCH 28/42] tests: scylla_supports_tablets[_legacy] suffix This is a temporary measure. The tests are going to be modernised in parts, which is why for some time we are going to need both functions: one for LegacySession and another for modern Session. --- scylla/src/transport/caching_session.rs | 6 ++++-- scylla/src/transport/cql_types_test.rs | 6 ++++-- scylla/src/transport/session_test.rs | 14 +++++++------- scylla/src/utils/test_utils.rs | 2 +- scylla/tests/integration/lwt_optimisation.rs | 4 ++-- scylla/tests/integration/shards.rs | 4 ++-- scylla/tests/integration/tablets.rs | 6 +++--- 7 files changed, 23 insertions(+), 19 deletions(-) diff --git a/scylla/src/transport/caching_session.rs b/scylla/src/transport/caching_session.rs index 5b0f920f0..5e77c48df 100644 --- a/scylla/src/transport/caching_session.rs +++ b/scylla/src/transport/caching_session.rs @@ -221,7 +221,9 @@ where mod tests { use crate::query::Query; use crate::statement::PagingState; - use crate::test_utils::{create_new_session_builder, scylla_supports_tablets, setup_tracing}; + use crate::test_utils::{ + create_new_session_builder, scylla_supports_tablets_legacy, setup_tracing, + }; use crate::transport::partitioner::PartitionerName; use crate::utils::test_utils::unique_keyspace_name; use crate::{ @@ -243,7 +245,7 @@ mod tests { "CREATE KEYSPACE IF NOT EXISTS {ks} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}" ); - if !with_tablet_support && scylla_supports_tablets(&session).await { + if !with_tablet_support && scylla_supports_tablets_legacy(&session).await { create_ks += " AND TABLETS = {'enabled': false}"; } diff --git a/scylla/src/transport/cql_types_test.rs b/scylla/src/transport/cql_types_test.rs index 1429b498b..4be1244eb 100644 --- a/scylla/src/transport/cql_types_test.rs +++ b/scylla/src/transport/cql_types_test.rs @@ -3,7 +3,9 @@ use crate::cql_to_rust::FromCqlVal; use crate::frame::response::result::CqlValue; use crate::frame::value::{Counter, CqlDate, CqlTime, CqlTimestamp}; use crate::macros::FromUserType; -use crate::test_utils::{create_new_session_builder, scylla_supports_tablets, setup_tracing}; +use crate::test_utils::{ + create_new_session_builder, scylla_supports_tablets_legacy, setup_tracing, +}; use crate::transport::session::LegacySession; use crate::utils::test_utils::unique_keyspace_name; use itertools::Itertools; @@ -32,7 +34,7 @@ async fn init_test_maybe_without_tablets( ks ); - if !supports_tablets && scylla_supports_tablets(&session).await { + if !supports_tablets && scylla_supports_tablets_legacy(&session).await { create_ks += " AND TABLETS = {'enabled': false}" } diff --git a/scylla/src/transport/session_test.rs b/scylla/src/transport/session_test.rs index c3eb0de7f..8923c85fb 100644 --- a/scylla/src/transport/session_test.rs +++ b/scylla/src/transport/session_test.rs @@ -6,7 +6,7 @@ use crate::query::Query; use crate::retry_policy::{QueryInfo, RetryDecision, RetryPolicy, RetrySession}; use crate::routing::Token; use crate::statement::Consistency; -use crate::test_utils::{scylla_supports_tablets, setup_tracing}; +use crate::test_utils::{scylla_supports_tablets_legacy, setup_tracing}; use crate::tracing::TracingInfo; use crate::transport::errors::{BadKeyspaceName, BadQuery, DbError, QueryError}; use crate::transport::partitioner::{ @@ -377,7 +377,7 @@ async fn test_counter_batch() { // Need to disable tablets in this test because they don't support counters yet. // (https://github.com/scylladb/scylladb/commit/c70f321c6f581357afdf3fd8b4fe8e5c5bb9736e). let mut create_ks = format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks); - if scylla_supports_tablets(&session).await { + if scylla_supports_tablets_legacy(&session).await { create_ks += " AND TABLETS = {'enabled': false}" } @@ -596,7 +596,7 @@ async fn test_token_awareness() { let mut create_ks = format!( "CREATE KEYSPACE IF NOT EXISTS {ks} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}" ); - if scylla_supports_tablets(&session).await { + if scylla_supports_tablets_legacy(&session).await { create_ks += " AND TABLETS = {'enabled': false}" } @@ -1776,7 +1776,7 @@ async fn test_table_partitioner_in_metadata() { let mut create_ks = format!( "CREATE KEYSPACE {ks} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}" ); - if scylla_supports_tablets(&session).await { + if scylla_supports_tablets_legacy(&session).await { create_ks += " AND TABLETS = {'enabled': false}"; } @@ -1954,7 +1954,7 @@ async fn test_prepared_partitioner() { // This test uses CDC which is not yet compatible with Scylla's tablets. let mut create_ks = format!( "CREATE KEYSPACE IF NOT EXISTS {ks} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}"); - if scylla_supports_tablets(&session).await { + if scylla_supports_tablets_legacy(&session).await { create_ks += " AND TABLETS = {'enabled': false}" } @@ -2542,7 +2542,7 @@ async fn test_batch_lwts() { let ks = unique_keyspace_name(); let mut create_ks = format!("CREATE KEYSPACE {} WITH REPLICATION = {{'class': 'NetworkTopologyStrategy', 'replication_factor': 1}}", ks); - if scylla_supports_tablets(&session).await { + if scylla_supports_tablets_legacy(&session).await { create_ks += " and TABLETS = { 'enabled': false}"; } session.query_unpaged(create_ks, &[]).await.unwrap(); @@ -2742,7 +2742,7 @@ async fn test_iter_works_when_retry_policy_returns_ignore_write_error() { let cluster_size = session.get_cluster_data().get_nodes_info().len(); let ks = unique_keyspace_name(); let mut create_ks = format!("CREATE KEYSPACE {} WITH REPLICATION = {{'class': 'NetworkTopologyStrategy', 'replication_factor': {}}}", ks, cluster_size + 1); - if scylla_supports_tablets(&session).await { + if scylla_supports_tablets_legacy(&session).await { create_ks += " and TABLETS = { 'enabled': false}"; } session.query_unpaged(create_ks, ()).await.unwrap(); diff --git a/scylla/src/utils/test_utils.rs b/scylla/src/utils/test_utils.rs index 6f9f2a9ec..a5c0dadab 100644 --- a/scylla/src/utils/test_utils.rs +++ b/scylla/src/utils/test_utils.rs @@ -92,7 +92,7 @@ pub fn create_new_session_builder() -> GenericSessionBuilder bool { +pub async fn scylla_supports_tablets_legacy(session: &LegacySession) -> bool { let result = session .query_unpaged( "select column_name from system_schema.columns where diff --git a/scylla/tests/integration/lwt_optimisation.rs b/scylla/tests/integration/lwt_optimisation.rs index dd1e855d8..508c953bd 100644 --- a/scylla/tests/integration/lwt_optimisation.rs +++ b/scylla/tests/integration/lwt_optimisation.rs @@ -1,6 +1,6 @@ use crate::utils::{setup_tracing, test_with_3_node_cluster}; use scylla::retry_policy::FallthroughRetryPolicy; -use scylla::test_utils::scylla_supports_tablets; +use scylla::test_utils::scylla_supports_tablets_legacy; use scylla::test_utils::unique_keyspace_name; use scylla::transport::session::LegacySession; use scylla::{ExecutionProfile, SessionBuilder}; @@ -70,7 +70,7 @@ async fn if_lwt_optimisation_mark_offered_then_negotiatied_and_lwt_routed_optima // Create schema let ks = unique_keyspace_name(); let mut create_ks = format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 3}}", ks); - if scylla_supports_tablets(&session).await { + if scylla_supports_tablets_legacy(&session).await { create_ks += " and TABLETS = { 'enabled': false}"; } session.query_unpaged(create_ks, &[]).await.unwrap(); diff --git a/scylla/tests/integration/shards.rs b/scylla/tests/integration/shards.rs index 3a4e0ecb1..c3db91ca2 100644 --- a/scylla/tests/integration/shards.rs +++ b/scylla/tests/integration/shards.rs @@ -1,7 +1,7 @@ use std::sync::Arc; use crate::utils::{setup_tracing, test_with_3_node_cluster}; -use scylla::test_utils::scylla_supports_tablets; +use scylla::test_utils::scylla_supports_tablets_legacy; use scylla::{test_utils::unique_keyspace_name, SessionBuilder}; use tokio::sync::mpsc; @@ -39,7 +39,7 @@ async fn test_consistent_shard_awareness() { /* Prepare schema */ let mut create_ks = format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 3}}", ks); - if scylla_supports_tablets(&session).await { + if scylla_supports_tablets_legacy(&session).await { create_ks += " and TABLETS = { 'enabled': false}"; } session.query_unpaged(create_ks, &[]).await.unwrap(); diff --git a/scylla/tests/integration/tablets.rs b/scylla/tests/integration/tablets.rs index a51d009c1..b570f0548 100644 --- a/scylla/tests/integration/tablets.rs +++ b/scylla/tests/integration/tablets.rs @@ -298,7 +298,7 @@ async fn test_default_policy_is_tablet_aware() { .await .unwrap(); - if !scylla::test_utils::scylla_supports_tablets(&session).await { + if !scylla::test_utils::scylla_supports_tablets_legacy(&session).await { tracing::warn!("Skipping test because this Scylla version doesn't support tablets"); return running_proxy; } @@ -429,7 +429,7 @@ async fn test_tablet_feedback_not_sent_for_unprepared_queries() { .await .unwrap(); - if !scylla::test_utils::scylla_supports_tablets(&session).await { + if !scylla::test_utils::scylla_supports_tablets_legacy(&session).await { tracing::warn!("Skipping test because this Scylla version doesn't support tablets"); return running_proxy; } @@ -501,7 +501,7 @@ async fn test_lwt_optimization_works_with_tablets() { .await .unwrap(); - if !scylla::test_utils::scylla_supports_tablets(&session).await { + if !scylla::test_utils::scylla_supports_tablets_legacy(&session).await { tracing::warn!("Skipping test because this Scylla version doesn't support tablets"); return running_proxy; } From 6d7b08084f26afe56b300195bd68840a1d2aaa6e Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Mon, 13 Mar 2023 14:04:10 +0100 Subject: [PATCH 29/42] session: de-genericise internal query/exec functions MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit The query/execute/batch statements are generic over the statement. They started by converting the statement to corresponding type (query/execute/batch) and then continued without the need for generics. However, those functions used to be non-trivial and would have to be monomorphised for every type of the arguments passed to the method, increasing compilation time more than necessary. Now that most of the implementation was moved to do_query etc. methods, we can restrict the generic part to the public query/execute/batch methods which convert the input statement to required type and then call the non-generic do_query etc. methods. This commit does just that - de-genericises do_query and friends, while leaving query and friends generic as they used to. Co-authored-by: Wojciech Przytuła --- scylla/src/transport/session.rs | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/scylla/src/transport/session.rs b/scylla/src/transport/session.rs index f1d2314b4..237ef256f 100644 --- a/scylla/src/transport/session.rs +++ b/scylla/src/transport/session.rs @@ -660,7 +660,7 @@ impl GenericSession { query: impl Into, values: impl SerializeRow, ) -> Result { - self.do_query_iter(query, values).await + self.do_query_iter(query.into(), values).await } /// Execute a prepared statement. Requires a [PreparedStatement] @@ -822,7 +822,7 @@ impl GenericSession { prepared: impl Into, values: impl SerializeRow, ) -> Result { - self.do_execute_iter(prepared, values).await + self.do_execute_iter(prepared.into(), values).await } /// Perform a batch query\ @@ -1182,11 +1182,9 @@ where async fn do_query_iter( &self, - query: impl Into, + query: Query, values: impl SerializeRow, ) -> Result { - let query: Query = query.into(); - let execution_profile = query .get_execution_profile_handle() .unwrap_or_else(|| self.get_default_execution_profile_handle()) @@ -1452,10 +1450,9 @@ where async fn do_execute_iter( &self, - prepared: impl Into, + prepared: PreparedStatement, values: impl SerializeRow, ) -> Result { - let prepared = prepared.into(); let serialized_values = prepared.serialize_values(&values)?; let execution_profile = prepared From ee34db74e56ee2248e5048df221470638d95e6d9 Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Mon, 13 Mar 2023 14:09:43 +0100 Subject: [PATCH 30/42] session: return new QueryResult from internal methods MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit QueryResult can be converted to LegacyQueryResult, but not the other way around. In order to support both APIs, internal methods (do_query, do_execute, etc.) need to be changed so that they return the new QueryResult. Co-authored-by: Wojciech Przytuła --- scylla/src/transport/legacy_query_result.rs | 10 --- scylla/src/transport/session.rs | 79 ++++++++++++--------- 2 files changed, 47 insertions(+), 42 deletions(-) diff --git a/scylla/src/transport/legacy_query_result.rs b/scylla/src/transport/legacy_query_result.rs index b8c0986f5..f1c63c719 100644 --- a/scylla/src/transport/legacy_query_result.rs +++ b/scylla/src/transport/legacy_query_result.rs @@ -25,16 +25,6 @@ pub struct LegacyQueryResult { } impl LegacyQueryResult { - pub(crate) fn mock_empty() -> Self { - Self { - rows: None, - warnings: Vec::new(), - tracing_id: None, - metadata: None, - serialized_size: 0, - } - } - /// Returns the number of received rows.\ /// Fails when the query isn't of a type that could return rows, same as [`rows()`](LegacyQueryResult::rows). pub fn rows_num(&self) -> Result { diff --git a/scylla/src/transport/session.rs b/scylla/src/transport/session.rs index 237ef256f..aebdfc672 100644 --- a/scylla/src/transport/session.rs +++ b/scylla/src/transport/session.rs @@ -557,7 +557,10 @@ impl GenericSession { query: impl Into, values: impl SerializeRow, ) -> Result { - self.do_query_unpaged(&query.into(), values).await + Ok(self + .do_query_unpaged(&query.into(), values) + .await? + .into_legacy_result()?) } /// Queries a single page from the database, optionally continuing from a saved point. @@ -614,8 +617,10 @@ impl GenericSession { values: impl SerializeRow, paging_state: PagingState, ) -> Result<(LegacyQueryResult, PagingStateResponse), QueryError> { - self.do_query_single_page(&query.into(), values, paging_state) - .await + let (result, paging_state_response) = self + .do_query_single_page(&query.into(), values, paging_state) + .await?; + Ok((result.into_legacy_result()?, paging_state_response)) } /// Run an unprepared query with paging\ @@ -660,7 +665,9 @@ impl GenericSession { query: impl Into, values: impl SerializeRow, ) -> Result { - self.do_query_iter(query.into(), values).await + self.do_query_iter(query.into(), values) + .await + .map(RawIterator::into_legacy) } /// Execute a prepared statement. Requires a [PreparedStatement] @@ -711,7 +718,10 @@ impl GenericSession { prepared: &PreparedStatement, values: impl SerializeRow, ) -> Result { - self.do_execute_unpaged(prepared, values).await + Ok(self + .do_execute_unpaged(prepared, values) + .await? + .into_legacy_result()?) } /// Executes a prepared statement, restricting results to single page. @@ -773,8 +783,10 @@ impl GenericSession { values: impl SerializeRow, paging_state: PagingState, ) -> Result<(LegacyQueryResult, PagingStateResponse), QueryError> { - self.do_execute_single_page(prepared, values, paging_state) - .await + let (result, paging_state_response) = self + .do_execute_single_page(prepared, values, paging_state) + .await?; + Ok((result.into_legacy_result()?, paging_state_response)) } /// Run a prepared query with paging.\ @@ -822,7 +834,9 @@ impl GenericSession { prepared: impl Into, values: impl SerializeRow, ) -> Result { - self.do_execute_iter(prepared.into(), values).await + self.do_execute_iter(prepared.into(), values) + .await + .map(RawIterator::into_legacy) } /// Perform a batch query\ @@ -875,7 +889,7 @@ impl GenericSession { batch: &Batch, values: impl BatchValues, ) -> Result { - self.do_batch(batch, values).await + Ok(self.do_batch(batch, values).await?.into_legacy_result()?) } } @@ -1010,7 +1024,7 @@ where &self, query: &Query, values: impl SerializeRow, - ) -> Result { + ) -> Result { let (result, paging_state_response) = self .query(query, values, None, PagingState::start()) .await?; @@ -1026,7 +1040,7 @@ where query: &Query, values: impl SerializeRow, paging_state: PagingState, - ) -> Result<(LegacyQueryResult, PagingStateResponse), QueryError> { + ) -> Result<(QueryResult, PagingStateResponse), QueryError> { self.query( query, values, @@ -1053,7 +1067,7 @@ where values: impl SerializeRow, page_size: Option, paging_state: PagingState, - ) -> Result<(LegacyQueryResult, PagingStateResponse), QueryError> { + ) -> Result<(QueryResult, PagingStateResponse), QueryError> { let execution_profile = query .get_execution_profile_handle() .unwrap_or_else(|| self.get_default_execution_profile_handle()) @@ -1139,10 +1153,10 @@ where self.handle_set_keyspace_response(&response).await?; self.handle_auto_await_schema_agreement(&response).await?; - let (result, paging_state) = response.into_query_result_and_paging_state()?; + let (result, paging_state_response) = response.into_query_result_and_paging_state()?; span.record_result_fields(&result); - let result = result.into_legacy_result()?; - Ok((result, paging_state)) + + Ok((result, paging_state_response)) } async fn handle_set_keyspace_response( @@ -1184,7 +1198,7 @@ where &self, query: Query, values: impl SerializeRow, - ) -> Result { + ) -> Result { let execution_profile = query .get_execution_profile_handle() .unwrap_or_else(|| self.get_default_execution_profile_handle()) @@ -1198,7 +1212,6 @@ where self.metrics.clone(), ) .await - .map(RawIterator::into_legacy) } else { // Making RawIterator::new_for_query work with values is too hard (if even possible) // so instead of sending one prepare to a specific connection on each iterator query, @@ -1213,7 +1226,6 @@ where metrics: self.metrics.clone(), }) .await - .map(RawIterator::into_legacy) } } @@ -1313,7 +1325,7 @@ where &self, prepared: &PreparedStatement, values: impl SerializeRow, - ) -> Result { + ) -> Result { let serialized_values = prepared.serialize_values(&values)?; let (result, paging_state) = self .execute(prepared, &serialized_values, None, PagingState::start()) @@ -1330,7 +1342,7 @@ where prepared: &PreparedStatement, values: impl SerializeRow, paging_state: PagingState, - ) -> Result<(LegacyQueryResult, PagingStateResponse), QueryError> { + ) -> Result<(QueryResult, PagingStateResponse), QueryError> { let serialized_values = prepared.serialize_values(&values)?; let page_size = prepared.get_validated_page_size(); self.execute(prepared, &serialized_values, Some(page_size), paging_state) @@ -1353,7 +1365,7 @@ where serialized_values: &SerializedValues, page_size: Option, paging_state: PagingState, - ) -> Result<(LegacyQueryResult, PagingStateResponse), QueryError> { + ) -> Result<(QueryResult, PagingStateResponse), QueryError> { let values_ref = &serialized_values; let paging_state_ref = &paging_state; @@ -1442,17 +1454,17 @@ where self.handle_set_keyspace_response(&response).await?; self.handle_auto_await_schema_agreement(&response).await?; - let (result, paging_state) = response.into_query_result_and_paging_state()?; + let (result, paging_state_response) = response.into_query_result_and_paging_state()?; span.record_result_fields(&result); - let result = result.into_legacy_result()?; - Ok((result, paging_state)) + + Ok((result, paging_state_response)) } async fn do_execute_iter( &self, prepared: PreparedStatement, values: impl SerializeRow, - ) -> Result { + ) -> Result { let serialized_values = prepared.serialize_values(&values)?; let execution_profile = prepared @@ -1468,14 +1480,13 @@ where metrics: self.metrics.clone(), }) .await - .map(RawIterator::into_legacy) } async fn do_batch( &self, batch: &Batch, values: impl BatchValues, - ) -> Result { + ) -> Result { // Shard-awareness behavior for batch will be to pick shard based on first batch statement's shard // If users batch statements by shard, they will be rewarded with full shard awareness @@ -1552,10 +1563,10 @@ where .await?; let result = match run_query_result { - RunQueryResult::IgnoredWriteError => LegacyQueryResult::mock_empty(), + RunQueryResult::IgnoredWriteError => QueryResult::mock_empty(), RunQueryResult::Completed(result) => { span.record_result_fields(&result); - result.into_legacy_result()? + result } }; @@ -1747,6 +1758,7 @@ where // Get tracing info let maybe_tracing_info: Option = traces_session_res + .into_legacy_result()? .maybe_first_row_typed() .map_err(|err| match err { MaybeFirstRowTypedError::RowsExpected(e) => { @@ -1763,9 +1775,12 @@ where }; // Get tracing events - let tracing_event_rows = traces_events_res.rows_typed().map_err(|err| { - ProtocolError::Tracing(TracingProtocolError::TracesEventsNotRows(err)) - })?; + let tracing_event_rows = traces_events_res + .into_legacy_result()? + .rows_typed() + .map_err(|err| { + ProtocolError::Tracing(TracingProtocolError::TracesEventsNotRows(err)) + })?; for event in tracing_event_rows { let tracing_event: TracingEvent = event.map_err(|err| { From 02f5237b2dc009d5c51d5c6babbef14be930ac69 Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Mon, 13 Mar 2023 17:34:26 +0100 Subject: [PATCH 31/42] session: add interface methods for the new deser API MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Implements methods related to sending queries for the new Session. Co-authored-by: Wojciech Przytuła --- scylla/src/transport/session.rs | 112 +++++++++++++++++++++++++------- 1 file changed, 89 insertions(+), 23 deletions(-) diff --git a/scylla/src/transport/session.rs b/scylla/src/transport/session.rs index aebdfc672..2563f376b 100644 --- a/scylla/src/transport/session.rs +++ b/scylla/src/transport/session.rs @@ -492,7 +492,7 @@ pub(crate) enum RunQueryResult { Completed(ResT), } -impl GenericSession { +impl GenericSession { /// Sends a request to the database and receives a response.\ /// Performs an unpaged query, i.e. all results are received in a single response. /// @@ -556,11 +556,8 @@ impl GenericSession { &self, query: impl Into, values: impl SerializeRow, - ) -> Result { - Ok(self - .do_query_unpaged(&query.into(), values) - .await? - .into_legacy_result()?) + ) -> Result { + self.do_query_unpaged(&query.into(), values).await } /// Queries a single page from the database, optionally continuing from a saved point. @@ -616,11 +613,11 @@ impl GenericSession { query: impl Into, values: impl SerializeRow, paging_state: PagingState, - ) -> Result<(LegacyQueryResult, PagingStateResponse), QueryError> { + ) -> Result<(QueryResult, PagingStateResponse), QueryError> { let (result, paging_state_response) = self .do_query_single_page(&query.into(), values, paging_state) .await?; - Ok((result.into_legacy_result()?, paging_state_response)) + Ok((result, paging_state_response)) } /// Run an unprepared query with paging\ @@ -664,10 +661,8 @@ impl GenericSession { &self, query: impl Into, values: impl SerializeRow, - ) -> Result { - self.do_query_iter(query.into(), values) - .await - .map(RawIterator::into_legacy) + ) -> Result { + self.do_query_iter(query.into(), values).await } /// Execute a prepared statement. Requires a [PreparedStatement] @@ -717,11 +712,8 @@ impl GenericSession { &self, prepared: &PreparedStatement, values: impl SerializeRow, - ) -> Result { - Ok(self - .do_execute_unpaged(prepared, values) - .await? - .into_legacy_result()?) + ) -> Result { + self.do_execute_unpaged(prepared, values).await } /// Executes a prepared statement, restricting results to single page. @@ -782,11 +774,11 @@ impl GenericSession { prepared: &PreparedStatement, values: impl SerializeRow, paging_state: PagingState, - ) -> Result<(LegacyQueryResult, PagingStateResponse), QueryError> { + ) -> Result<(QueryResult, PagingStateResponse), QueryError> { let (result, paging_state_response) = self .do_execute_single_page(prepared, values, paging_state) .await?; - Ok((result.into_legacy_result()?, paging_state_response)) + Ok((result, paging_state_response)) } /// Run a prepared query with paging.\ @@ -833,10 +825,8 @@ impl GenericSession { &self, prepared: impl Into, values: impl SerializeRow, - ) -> Result { - self.do_execute_iter(prepared.into(), values) - .await - .map(RawIterator::into_legacy) + ) -> Result { + self.do_execute_iter(prepared.into(), values).await } /// Perform a batch query\ @@ -884,6 +874,82 @@ impl GenericSession { /// # Ok(()) /// # } /// ``` + pub async fn batch( + &self, + batch: &Batch, + values: impl BatchValues, + ) -> Result { + self.do_batch(batch, values).await + } +} + +impl GenericSession { + pub async fn query_unpaged( + &self, + query: impl Into, + values: impl SerializeRow, + ) -> Result { + Ok(self + .do_query_unpaged(&query.into(), values) + .await? + .into_legacy_result()?) + } + + pub async fn query_single_page( + &self, + query: impl Into, + values: impl SerializeRow, + paging_state: PagingState, + ) -> Result<(LegacyQueryResult, PagingStateResponse), QueryError> { + let (result, paging_state_response) = self + .do_query_single_page(&query.into(), values, paging_state) + .await?; + Ok((result.into_legacy_result()?, paging_state_response)) + } + + pub async fn query_iter( + &self, + query: impl Into, + values: impl SerializeRow, + ) -> Result { + self.do_query_iter(query.into(), values) + .await + .map(RawIterator::into_legacy) + } + + pub async fn execute_unpaged( + &self, + prepared: &PreparedStatement, + values: impl SerializeRow, + ) -> Result { + Ok(self + .do_execute_unpaged(prepared, values) + .await? + .into_legacy_result()?) + } + + pub async fn execute_single_page( + &self, + prepared: &PreparedStatement, + values: impl SerializeRow, + paging_state: PagingState, + ) -> Result<(LegacyQueryResult, PagingStateResponse), QueryError> { + let (result, paging_state_response) = self + .do_execute_single_page(prepared, values, paging_state) + .await?; + Ok((result.into_legacy_result()?, paging_state_response)) + } + + pub async fn execute_iter( + &self, + prepared: impl Into, + values: impl SerializeRow, + ) -> Result { + self.do_execute_iter(prepared.into(), values) + .await + .map(RawIterator::into_legacy) + } + pub async fn batch( &self, batch: &Batch, From 35497837c48701017651d15013295f090773ec50 Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Tue, 14 Mar 2023 08:47:39 +0100 Subject: [PATCH 32/42] connection: switch to the new deserialization framework MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Adjusts the methods of Connection, apart from query_iter, to use the new deserialization API. Connection is meant to be an internal API, so we don't introduce a LegacyConnection for this. Co-authored-by: Wojciech Przytuła --- scylla/src/transport/connection.rs | 14 ++++++++++---- scylla/src/transport/errors.rs | 14 ++++++++++++-- 2 files changed, 22 insertions(+), 6 deletions(-) diff --git a/scylla/src/transport/connection.rs b/scylla/src/transport/connection.rs index 79551845d..ae72cf62b 100644 --- a/scylla/src/transport/connection.rs +++ b/scylla/src/transport/connection.rs @@ -46,7 +46,7 @@ use std::{ net::{Ipv4Addr, Ipv6Addr}, }; -use super::errors::{ProtocolError, UseKeyspaceProtocolError}; +use super::errors::{ProtocolError, SchemaVersionFetchError, UseKeyspaceProtocolError}; use super::iterator::{LegacyRowIterator, RawIterator}; use super::locator::tablets::{RawTablet, TabletParsingError}; use super::query_result::QueryResult; @@ -1436,9 +1436,15 @@ impl Connection { let (version_id,) = self .query_unpaged(LOCAL_VERSION) .await? - .into_legacy_result()? - .single_row_typed() - .map_err(ProtocolError::SchemaVersionFetch)?; + .into_rows_result()? + .ok_or(QueryError::ProtocolError( + ProtocolError::SchemaVersionFetch(SchemaVersionFetchError::ResultNotRows), + ))? + .single_row::<(Uuid,)>() + .map_err(|err| { + ProtocolError::SchemaVersionFetch(SchemaVersionFetchError::SingleRowError(err)) + })?; + Ok(version_id) } diff --git a/scylla/src/transport/errors.rs b/scylla/src/transport/errors.rs index d95383054..64f9989bb 100644 --- a/scylla/src/transport/errors.rs +++ b/scylla/src/transport/errors.rs @@ -32,7 +32,7 @@ use thiserror::Error; use crate::{authentication::AuthError, frame::response}; -use super::legacy_query_result::{RowsExpectedError, SingleRowTypedError}; +use super::{legacy_query_result::RowsExpectedError, query_result::SingleRowError}; /// Error that occurred during query execution #[derive(Error, Debug, Clone)] @@ -304,7 +304,7 @@ pub enum ProtocolError { /// A protocol error appeared during schema version fetch. #[error("Schema version fetch protocol error: {0}")] - SchemaVersionFetch(SingleRowTypedError), + SchemaVersionFetch(#[from] SchemaVersionFetchError), /// A result with nonfinished paging state received for unpaged query. #[error("Unpaged query returned a non-empty paging state! This is a driver-side or server-side bug.")] @@ -345,6 +345,16 @@ pub enum UseKeyspaceProtocolError { UnexpectedResponse(CqlResponseKind), } +/// A protocol error that occurred during schema version fetch. +#[derive(Error, Debug, Clone)] +#[non_exhaustive] +pub enum SchemaVersionFetchError { + #[error("Schema version query returned non-rows result")] + ResultNotRows, + #[error(transparent)] + SingleRowError(SingleRowError), +} + /// A protocol error that occurred during tracing info fetch. #[derive(Error, Debug, Clone)] #[non_exhaustive] From dd813defe9c180d28646621f38206ad421513f10 Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Tue, 14 Mar 2023 14:52:58 +0100 Subject: [PATCH 33/42] caching_session: make generic over session APIs MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit In a similar fashion to Session, CachingSession was also made generic over the session kind. Co-authored-by: Wojciech Przytuła --- scylla/src/lib.rs | 2 +- scylla/src/transport/caching_session.rs | 128 ++++++++++++++++++++---- scylla/src/transport/session_test.rs | 6 +- 3 files changed, 111 insertions(+), 25 deletions(-) diff --git a/scylla/src/lib.rs b/scylla/src/lib.rs index 1f16faba5..1dd70e90b 100644 --- a/scylla/src/lib.rs +++ b/scylla/src/lib.rs @@ -256,7 +256,7 @@ pub use statement::query; pub use frame::response::cql_to_rust; pub use frame::response::cql_to_rust::FromRow; -pub use transport::caching_session::CachingSession; +pub use transport::caching_session::{CachingSession, GenericCachingSession, LegacyCachingSession}; pub use transport::execution_profile::ExecutionProfile; pub use transport::legacy_query_result::LegacyQueryResult; pub use transport::query_result::{QueryResult, QueryRowsResult}; diff --git a/scylla/src/transport/caching_session.rs b/scylla/src/transport/caching_session.rs index 5e77c48df..e61a2dfe2 100644 --- a/scylla/src/transport/caching_session.rs +++ b/scylla/src/transport/caching_session.rs @@ -5,7 +5,7 @@ use crate::statement::{PagingState, PagingStateResponse}; use crate::transport::errors::QueryError; use crate::transport::iterator::LegacyRowIterator; use crate::transport::partitioner::PartitionerName; -use crate::{LegacyQueryResult, LegacySession}; +use crate::{LegacyQueryResult, QueryResult}; use bytes::Bytes; use dashmap::DashMap; use futures::future::try_join_all; @@ -16,6 +16,11 @@ use std::collections::hash_map::RandomState; use std::hash::BuildHasher; use std::sync::Arc; +use super::iterator::RawIterator; +use super::session::{ + CurrentDeserializationApi, DeserializationApiKind, GenericSession, LegacyDeserializationApi, +}; + /// Contains just the parts of a prepared statement that were returned /// from the database. All remaining parts (query string, page size, /// consistency, etc.) are taken from the Query passed @@ -31,11 +36,12 @@ struct RawPreparedStatementData { /// Provides auto caching while executing queries #[derive(Debug)] -pub struct CachingSession +pub struct GenericCachingSession where S: Clone + BuildHasher, + DeserializationApi: DeserializationApiKind, { - session: LegacySession, + session: GenericSession, /// The prepared statement cache size /// If a prepared statement is added while the limit is reached, the oldest prepared statement /// is removed from the cache @@ -43,11 +49,15 @@ where cache: DashMap, } -impl CachingSession +pub type CachingSession = GenericCachingSession; +pub type LegacyCachingSession = GenericCachingSession; + +impl GenericCachingSession where S: Default + BuildHasher + Clone, + DeserApi: DeserializationApiKind, { - pub fn from(session: LegacySession, cache_size: usize) -> Self { + pub fn from(session: GenericSession, cache_size: usize) -> Self { Self { session, max_capacity: cache_size, @@ -56,20 +66,88 @@ where } } -impl CachingSession +impl GenericCachingSession where S: BuildHasher + Clone, + DeserApi: DeserializationApiKind, { /// Builds a [`CachingSession`] from a [`Session`], a cache size, and a [`BuildHasher`]., /// using a customer hasher. - pub fn with_hasher(session: LegacySession, cache_size: usize, hasher: S) -> Self { + pub fn with_hasher(session: GenericSession, cache_size: usize, hasher: S) -> Self { Self { session, max_capacity: cache_size, cache: DashMap::with_hasher(hasher), } } +} +impl GenericCachingSession +where + S: BuildHasher + Clone, +{ + /// Does the same thing as [`Session::execute_unpaged`] but uses the prepared statement cache + pub async fn execute_unpaged( + &self, + query: impl Into, + values: impl SerializeRow, + ) -> Result { + let query = query.into(); + let prepared = self.add_prepared_statement_owned(query).await?; + self.session.execute_unpaged(&prepared, values).await + } + + /// Does the same thing as [`Session::execute_iter`] but uses the prepared statement cache + pub async fn execute_iter( + &self, + query: impl Into, + values: impl SerializeRow, + ) -> Result { + let query = query.into(); + let prepared = self.add_prepared_statement_owned(query).await?; + self.session.execute_iter(prepared, values).await + } + + /// Does the same thing as [`Session::execute_single_page`] but uses the prepared statement cache + pub async fn execute_single_page( + &self, + query: impl Into, + values: impl SerializeRow, + paging_state: PagingState, + ) -> Result<(QueryResult, PagingStateResponse), QueryError> { + let query = query.into(); + let prepared = self.add_prepared_statement_owned(query).await?; + self.session + .execute_single_page(&prepared, values, paging_state) + .await + } + + /// Does the same thing as [`Session::batch`] but uses the prepared statement cache\ + /// Prepares batch using CachingSession::prepare_batch if needed and then executes it + pub async fn batch( + &self, + batch: &Batch, + values: impl BatchValues, + ) -> Result { + let all_prepared: bool = batch + .statements + .iter() + .all(|stmt| matches!(stmt, BatchStatement::PreparedStatement(_))); + + if all_prepared { + self.session.batch(batch, &values).await + } else { + let prepared_batch: Batch = self.prepare_batch(batch).await?; + + self.session.batch(&prepared_batch, &values).await + } + } +} + +impl GenericCachingSession +where + S: BuildHasher + Clone, +{ /// Does the same thing as [`Session::execute_unpaged`] but uses the prepared statement cache pub async fn execute_unpaged( &self, @@ -126,7 +204,13 @@ where self.session.batch(&prepared_batch, &values).await } } +} +impl GenericCachingSession +where + S: BuildHasher + Clone, + DeserApi: DeserializationApiKind, +{ /// Prepares all statements within the batch and returns a new batch where every /// statement is prepared. /// Uses the prepared statements cache. @@ -212,7 +296,7 @@ where self.max_capacity } - pub fn get_session(&self) -> &LegacySession { + pub fn get_session(&self) -> &GenericSession { &self.session } } @@ -229,7 +313,7 @@ mod tests { use crate::{ batch::{Batch, BatchStatement}, prepared_statement::PreparedStatement, - CachingSession, LegacySession, + LegacyCachingSession, LegacySession, }; use futures::TryStreamExt; use std::collections::BTreeSet; @@ -273,8 +357,8 @@ mod tests { session } - async fn create_caching_session() -> CachingSession { - let session = CachingSession::from(new_for_test(true).await, 2); + async fn create_caching_session() -> LegacyCachingSession { + let session = LegacyCachingSession::from(new_for_test(true).await, 2); // Add a row, this makes it easier to check if the caching works combined with the regular execute fn on Session session @@ -385,7 +469,7 @@ mod tests { } async fn assert_test_batch_table_rows_contain( - sess: &CachingSession, + sess: &LegacyCachingSession, expected_rows: &[(i32, i32)], ) { let selected_rows: BTreeSet<(i32, i32)> = sess @@ -431,18 +515,18 @@ mod tests { } } - let _session: CachingSession = - CachingSession::from(new_for_test(true).await, 2); - let _session: CachingSession = - CachingSession::from(new_for_test(true).await, 2); - let _session: CachingSession = - CachingSession::with_hasher(new_for_test(true).await, 2, Default::default()); + let _session: LegacyCachingSession = + LegacyCachingSession::from(new_for_test(true).await, 2); + let _session: LegacyCachingSession = + LegacyCachingSession::from(new_for_test(true).await, 2); + let _session: LegacyCachingSession = + LegacyCachingSession::with_hasher(new_for_test(true).await, 2, Default::default()); } #[tokio::test] async fn test_batch() { setup_tracing(); - let session: CachingSession = create_caching_session().await; + let session: LegacyCachingSession = create_caching_session().await; session .execute_unpaged( @@ -565,7 +649,8 @@ mod tests { #[tokio::test] async fn test_parameters_caching() { setup_tracing(); - let session: CachingSession = CachingSession::from(new_for_test(true).await, 100); + let session: LegacyCachingSession = + LegacyCachingSession::from(new_for_test(true).await, 100); session .execute_unpaged("CREATE TABLE tbl (a int PRIMARY KEY, b int)", ()) @@ -618,7 +703,8 @@ mod tests { } // This test uses CDC which is not yet compatible with Scylla's tablets. - let session: CachingSession = CachingSession::from(new_for_test(false).await, 100); + let session: LegacyCachingSession = + LegacyCachingSession::from(new_for_test(false).await, 100); session .execute_unpaged( diff --git a/scylla/src/transport/session_test.rs b/scylla/src/transport/session_test.rs index 8923c85fb..1dddb7d40 100644 --- a/scylla/src/transport/session_test.rs +++ b/scylla/src/transport/session_test.rs @@ -19,8 +19,8 @@ use crate::transport::topology::{ use crate::utils::test_utils::{ create_new_session_builder, supports_feature, unique_keyspace_name, }; -use crate::CachingSession; use crate::ExecutionProfile; +use crate::LegacyCachingSession; use crate::LegacyQueryResult; use crate::{LegacySession, SessionBuilder}; use assert_matches::assert_matches; @@ -2012,7 +2012,7 @@ async fn rename(session: &LegacySession, rename_str: &str) { .unwrap(); } -async fn rename_caching(session: &CachingSession, rename_str: &str) { +async fn rename_caching(session: &LegacyCachingSession, rename_str: &str) { session .execute_unpaged(format!("ALTER TABLE tab RENAME {}", rename_str), &()) .await @@ -2230,7 +2230,7 @@ async fn test_unprepared_reprepare_in_caching_session_execute() { session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); session.use_keyspace(ks, false).await.unwrap(); - let caching_session: CachingSession = CachingSession::from(session, 64); + let caching_session: LegacyCachingSession = LegacyCachingSession::from(session, 64); caching_session .execute_unpaged( From d3686d0c6420c1b19a4ba8f5524260e7a4682ff4 Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Tue, 14 Mar 2023 14:53:08 +0100 Subject: [PATCH 34/42] caching_session: modernize tests MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Adjusts the CachingSession tests to use the new deserialization interface. Co-authored-by: Wojciech Przytuła --- scylla/src/transport/caching_session.rs | 65 +++++++++++++++---------- scylla/src/utils/test_utils.rs | 22 ++++++++- 2 files changed, 59 insertions(+), 28 deletions(-) diff --git a/scylla/src/transport/caching_session.rs b/scylla/src/transport/caching_session.rs index e61a2dfe2..8232c6486 100644 --- a/scylla/src/transport/caching_session.rs +++ b/scylla/src/transport/caching_session.rs @@ -305,22 +305,22 @@ where mod tests { use crate::query::Query; use crate::statement::PagingState; - use crate::test_utils::{ - create_new_session_builder, scylla_supports_tablets_legacy, setup_tracing, - }; + use crate::test_utils::{create_new_session_builder, scylla_supports_tablets, setup_tracing}; use crate::transport::partitioner::PartitionerName; + use crate::transport::session::Session; use crate::utils::test_utils::unique_keyspace_name; use crate::{ batch::{Batch, BatchStatement}, prepared_statement::PreparedStatement, - LegacyCachingSession, LegacySession, + CachingSession, }; use futures::TryStreamExt; + use scylla_cql::frame::response::result::Row; use std::collections::BTreeSet; - async fn new_for_test(with_tablet_support: bool) -> LegacySession { + async fn new_for_test(with_tablet_support: bool) -> Session { let session = create_new_session_builder() - .build_legacy() + .build() .await .expect("Could not create session"); let ks = unique_keyspace_name(); @@ -329,7 +329,7 @@ mod tests { "CREATE KEYSPACE IF NOT EXISTS {ks} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}" ); - if !with_tablet_support && scylla_supports_tablets_legacy(&session).await { + if !with_tablet_support && scylla_supports_tablets(&session).await { create_ks += " AND TABLETS = {'enabled': false}"; } @@ -357,8 +357,8 @@ mod tests { session } - async fn create_caching_session() -> LegacyCachingSession { - let session = LegacyCachingSession::from(new_for_test(true).await, 2); + async fn create_caching_session() -> CachingSession { + let session = CachingSession::from(new_for_test(true).await, 2); // Add a row, this makes it easier to check if the caching works combined with the regular execute fn on Session session @@ -419,17 +419,20 @@ mod tests { .execute_unpaged("select * from test_table", &[]) .await .unwrap(); + let result_rows = result.into_rows_result().unwrap().unwrap(); assert_eq!(1, session.cache.len()); - assert_eq!(1, result.rows_num().unwrap()); + assert_eq!(1, result_rows.rows_num()); let result = session .execute_unpaged("select * from test_table", &[]) .await .unwrap(); + let result_rows = result.into_rows_result().unwrap().unwrap(); + assert_eq!(1, session.cache.len()); - assert_eq!(1, result.rows_num().unwrap()); + assert_eq!(1, result_rows.rows_num()); } /// Checks that caching works with execute_iter @@ -443,7 +446,10 @@ mod tests { let iter = session .execute_iter("select * from test_table", &[]) .await - .unwrap(); + .unwrap() + .into_typed::() + .unwrap() + .into_stream(); let rows = iter.try_collect::>().await.unwrap().len(); @@ -465,18 +471,21 @@ mod tests { .unwrap(); assert_eq!(1, session.cache.len()); - assert_eq!(1, result.rows_num().unwrap()); + assert_eq!(1, result.into_rows_result().unwrap().unwrap().rows_num()); } async fn assert_test_batch_table_rows_contain( - sess: &LegacyCachingSession, + sess: &CachingSession, expected_rows: &[(i32, i32)], ) { let selected_rows: BTreeSet<(i32, i32)> = sess .execute_unpaged("SELECT a, b FROM test_batch_table", ()) .await .unwrap() - .rows_typed::<(i32, i32)>() + .into_rows_result() + .unwrap() + .unwrap() + .rows::<(i32, i32)>() .unwrap() .map(|r| r.unwrap()) .collect(); @@ -515,18 +524,18 @@ mod tests { } } - let _session: LegacyCachingSession = - LegacyCachingSession::from(new_for_test(true).await, 2); - let _session: LegacyCachingSession = - LegacyCachingSession::from(new_for_test(true).await, 2); - let _session: LegacyCachingSession = - LegacyCachingSession::with_hasher(new_for_test(true).await, 2, Default::default()); + let _session: CachingSession = + CachingSession::from(new_for_test(true).await, 2); + let _session: CachingSession = + CachingSession::from(new_for_test(true).await, 2); + let _session: CachingSession = + CachingSession::with_hasher(new_for_test(true).await, 2, Default::default()); } #[tokio::test] async fn test_batch() { setup_tracing(); - let session: LegacyCachingSession = create_caching_session().await; + let session: CachingSession = create_caching_session().await; session .execute_unpaged( @@ -649,8 +658,7 @@ mod tests { #[tokio::test] async fn test_parameters_caching() { setup_tracing(); - let session: LegacyCachingSession = - LegacyCachingSession::from(new_for_test(true).await, 100); + let session: CachingSession = CachingSession::from(new_for_test(true).await, 100); session .execute_unpaged("CREATE TABLE tbl (a int PRIMARY KEY, b int)", ()) @@ -686,7 +694,11 @@ mod tests { .execute_unpaged("SELECT b, WRITETIME(b) FROM tbl", ()) .await .unwrap() - .rows_typed_or_empty::<(i32, i64)>() + .into_rows_result() + .unwrap() + .unwrap() + .rows::<(i32, i64)>() + .unwrap() .collect::, _>>() .unwrap(); @@ -703,8 +715,7 @@ mod tests { } // This test uses CDC which is not yet compatible with Scylla's tablets. - let session: LegacyCachingSession = - LegacyCachingSession::from(new_for_test(false).await, 100); + let session: CachingSession = CachingSession::from(new_for_test(false).await, 100); session .execute_unpaged( diff --git a/scylla/src/utils/test_utils.rs b/scylla/src/utils/test_utils.rs index a5c0dadab..37390291d 100644 --- a/scylla/src/utils/test_utils.rs +++ b/scylla/src/utils/test_utils.rs @@ -1,6 +1,8 @@ +use scylla_cql::frame::response::result::Row; + #[cfg(test)] use crate::transport::session_builder::{GenericSessionBuilder, SessionBuilderKind}; -use crate::LegacySession; +use crate::{LegacySession, Session}; #[cfg(test)] use std::{num::NonZeroU32, time::Duration}; use std::{ @@ -106,6 +108,24 @@ pub async fn scylla_supports_tablets_legacy(session: &LegacySession) -> bool { result.single_row().is_ok() } +pub async fn scylla_supports_tablets(session: &Session) -> bool { + let result = session + .query_unpaged( + "select column_name from system_schema.columns where + keyspace_name = 'system_schema' + and table_name = 'scylla_keyspaces' + and column_name = 'initial_tablets'", + &[], + ) + .await + .unwrap(); + result.rows_deserializer().map_or(false, |opt| { + opt.map_or(false, |deserializer| { + deserializer.single_row::().is_ok() + }) + }) +} + #[cfg(test)] pub(crate) fn setup_tracing() { let _ = tracing_subscriber::fmt::fmt() From f5237ad0ec5685ca14497518c4ef78708e0c0b4d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Wojciech=20Przytu=C5=82a?= Date: Tue, 12 Mar 2024 15:53:56 +0100 Subject: [PATCH 35/42] connection: migrate query_iter to new deserialization framework The Connection::query_iter method is changed to use the new deserialization framework. All the internal uses of it in topology.rs are adjusted. Co-authored-by: Piotr Dulikowski --- scylla/src/transport/connection.rs | 17 ++-- scylla/src/transport/errors.rs | 12 +-- scylla/src/transport/topology.rs | 129 +++++++++++++++++------------ 3 files changed, 92 insertions(+), 66 deletions(-) diff --git a/scylla/src/transport/connection.rs b/scylla/src/transport/connection.rs index ae72cf62b..1e1f4b962 100644 --- a/scylla/src/transport/connection.rs +++ b/scylla/src/transport/connection.rs @@ -47,7 +47,7 @@ use std::{ }; use super::errors::{ProtocolError, SchemaVersionFetchError, UseKeyspaceProtocolError}; -use super::iterator::{LegacyRowIterator, RawIterator}; +use super::iterator::RawIterator; use super::locator::tablets::{RawTablet, TabletParsingError}; use super::query_result::QueryResult; use super::session::AddressTranslator; @@ -1182,7 +1182,7 @@ impl Connection { pub(crate) async fn query_iter( self: Arc, query: Query, - ) -> Result { + ) -> Result { let consistency = query .config .determine_consistency(self.config.default_consistency); @@ -1190,7 +1190,6 @@ impl Connection { RawIterator::new_for_connection_query_iter(query, self, consistency, serial_consistency) .await - .map(RawIterator::into_legacy) } /// Executes a prepared statements and fetches its results over multiple pages, using @@ -1199,7 +1198,7 @@ impl Connection { self: Arc, prepared_statement: PreparedStatement, values: SerializedValues, - ) -> Result { + ) -> Result { let consistency = prepared_statement .config .determine_consistency(self.config.default_consistency); @@ -1213,7 +1212,6 @@ impl Connection { serial_consistency, ) .await - .map(RawIterator::into_legacy) } #[allow(dead_code)] @@ -2382,6 +2380,7 @@ mod tests { use scylla_cql::frame::protocol_features::{ LWT_OPTIMIZATION_META_BIT_MASK_KEY, SCYLLA_LWT_ADD_METADATA_MARK_EXTENSION, }; + use scylla_cql::frame::response::result::Row; use scylla_cql::frame::types; use scylla_proxy::{ Condition, Node, Proxy, Reaction, RequestFrame, RequestOpcode, RequestReaction, @@ -2479,6 +2478,9 @@ mod tests { .query_iter(select_query.clone()) .await .unwrap() + .into_typed::() + .unwrap() + .into_stream() .try_collect::>() .await .unwrap(); @@ -2504,6 +2506,8 @@ mod tests { .await .unwrap() .into_typed::<(i32,)>() + .unwrap() + .into_stream() .map(|ret| ret.unwrap().0) .collect::>() .await; @@ -2517,6 +2521,9 @@ mod tests { )) .await .unwrap() + .into_typed::() + .unwrap() + .into_stream() .try_collect::>() .await .unwrap(); diff --git a/scylla/src/transport/errors.rs b/scylla/src/transport/errors.rs index 64f9989bb..478d03f09 100644 --- a/scylla/src/transport/errors.rs +++ b/scylla/src/transport/errors.rs @@ -25,7 +25,7 @@ use scylla_cql::{ response::CqlResponseKind, value::SerializeValuesError, }, - types::serialize::SerializationError, + types::{deserialize::TypeCheckError, serialize::SerializationError}, }; use thiserror::Error; @@ -436,7 +436,7 @@ pub enum PeersMetadataError { pub enum KeyspacesMetadataError { /// system_schema.keyspaces has invalid column type. #[error("system_schema.keyspaces has invalid column type: {0}")] - SchemaKeyspacesInvalidColumnType(FromRowError), + SchemaKeyspacesInvalidColumnType(TypeCheckError), /// Bad keyspace replication strategy. #[error("Bad keyspace <{keyspace}> replication strategy: {error}")] @@ -474,7 +474,7 @@ pub enum KeyspaceStrategyError { pub enum UdtMetadataError { /// system_schema.types has invalid column type. #[error("system_schema.types has invalid column type: {0}")] - SchemaTypesInvalidColumnType(FromRowError), + SchemaTypesInvalidColumnType(TypeCheckError), /// Circular UDT dependency detected. #[error("Detected circular dependency between user defined types - toposort is impossible!")] @@ -487,11 +487,11 @@ pub enum UdtMetadataError { pub enum TablesMetadataError { /// system_schema.tables has invalid column type. #[error("system_schema.tables has invalid column type: {0}")] - SchemaTablesInvalidColumnType(FromRowError), + SchemaTablesInvalidColumnType(TypeCheckError), /// system_schema.columns has invalid column type. #[error("system_schema.columns has invalid column type: {0}")] - SchemaColumnsInvalidColumnType(FromRowError), + SchemaColumnsInvalidColumnType(TypeCheckError), /// Unknown column kind. #[error("Unknown column kind '{column_kind}' for {keyspace_name}.{table_name}.{column_name}")] @@ -509,7 +509,7 @@ pub enum TablesMetadataError { pub enum ViewsMetadataError { /// system_schema.views has invalid column type. #[error("system_schema.views has invalid column type: {0}")] - SchemaViewsInvalidColumnType(FromRowError), + SchemaViewsInvalidColumnType(TypeCheckError), } /// Error caused by caller creating an invalid query diff --git a/scylla/src/transport/topology.rs b/scylla/src/transport/topology.rs index 93a80d2fa..ef972c119 100644 --- a/scylla/src/transport/topology.rs +++ b/scylla/src/transport/topology.rs @@ -13,8 +13,10 @@ use futures::stream::{self, StreamExt, TryStreamExt}; use futures::Stream; use rand::seq::SliceRandom; use rand::{thread_rng, Rng}; -use scylla_cql::frame::response::result::Row; -use scylla_macros::FromRow; +use scylla_cql::frame::frame_errors::RowsParseError; +use scylla_cql::types::deserialize::row::DeserializeRow; +use scylla_cql::types::deserialize::TypeCheckError; +use scylla_macros::DeserializeRow; use std::borrow::BorrowMut; use std::cell::Cell; use std::collections::HashMap; @@ -765,11 +767,13 @@ async fn query_metadata( Ok(Metadata { peers, keyspaces }) } -#[derive(FromRow)] -#[scylla_crate = "scylla_cql"] +#[derive(DeserializeRow)] +#[scylla(crate = "scylla_cql")] struct NodeInfoRow { host_id: Option, + #[scylla(rename = "rpc_address")] untranslated_ip_addr: IpAddr, + #[scylla(rename = "data_center")] datacenter: Option, rack: Option, tokens: Option>, @@ -799,6 +803,13 @@ async fn query_peers(conn: &Arc, connect_port: u16) -> Result() + .map_err(|err| RowsParseError::from(err).into()) + .map(|iter| iter.into_stream()) + }) + }) .into_stream() .try_flatten() .and_then(|row_result| future::ok((NodeInfoSource::Peer, row_result))); @@ -809,6 +820,13 @@ async fn query_peers(conn: &Arc, connect_port: u16) -> Result() + .map_err(|err| RowsParseError::from(err).into()) + .map(|iter| iter.into_stream()) + }) + }) .into_stream() .try_flatten() .and_then(|row_result| future::ok((NodeInfoSource::Local, row_result))); @@ -819,9 +837,8 @@ async fn query_peers(conn: &Arc, connect_port: u16) -> Result create_peer_from_row(source, row, local_address).await, + match row_result { + Ok((source, row)) => create_peer_from_row(source, row, local_address).await, Err(err) => { warn!( "system.peers or system.local has an invalid row, skipping it: {}", @@ -905,11 +922,15 @@ async fn create_peer_from_row( })) } -fn query_filter_keyspace_name<'a>( +fn query_filter_keyspace_name<'a, R>( conn: &Arc, query_str: &'a str, keyspaces_to_fetch: &'a [String], -) -> impl Stream> + 'a { + convert_typecheck_error: impl FnOnce(TypeCheckError) -> MetadataError + 'a, +) -> impl Stream> + 'a +where + R: for<'r> DeserializeRow<'r, 'r> + 'static, +{ let conn = conn.clone(); let fut = async move { @@ -929,6 +950,12 @@ fn query_filter_keyspace_name<'a>( let serialized_values = prepared.serialize_values(&keyspaces)?; conn.execute_iter(prepared, serialized_values).await } + .and_then(|it: super::iterator::RawIterator| { + it.into_typed::() + .map_err(convert_typecheck_error) + .map_err(Into::into) + }) + .map(|row_iterator| row_iterator.into_stream()) }; fut.into_stream().try_flatten() } @@ -938,10 +965,15 @@ async fn query_keyspaces( keyspaces_to_fetch: &[String], fetch_schema: bool, ) -> Result, QueryError> { - let rows = query_filter_keyspace_name( + let rows = query_filter_keyspace_name::<(String, HashMap)>( conn, "select keyspace_name, replication from system_schema.keyspaces", keyspaces_to_fetch, + |err| { + MetadataError::Keyspaces(KeyspacesMetadataError::SchemaKeyspacesInvalidColumnType( + err, + )) + }, ); let (mut all_tables, mut all_views, mut all_user_defined_types) = if fetch_schema { @@ -956,12 +988,7 @@ async fn query_keyspaces( }; rows.map(|row_result| { - let row = row_result?; - let (keyspace_name, strategy_map) = row.into_typed::<(String, _)>().map_err(|err| { - MetadataError::Keyspaces(KeyspacesMetadataError::SchemaKeyspacesInvalidColumnType( - err, - )) - })?; + let (keyspace_name, strategy_map) = row_result?; let strategy: Strategy = strategy_from_string_map(strategy_map).map_err(|error| { MetadataError::Keyspaces(KeyspacesMetadataError::Strategy { @@ -988,8 +1015,8 @@ async fn query_keyspaces( .await } -#[derive(FromRow, Debug)] -#[scylla_crate = "crate"] +#[derive(DeserializeRow, Debug)] +#[scylla(crate = "crate")] struct UdtRow { keyspace_name: String, type_name: String, @@ -1031,21 +1058,16 @@ async fn query_user_defined_types( conn: &Arc, keyspaces_to_fetch: &[String], ) -> Result>>, QueryError> { - let rows = query_filter_keyspace_name( + let rows = query_filter_keyspace_name::( conn, "select keyspace_name, type_name, field_names, field_types from system_schema.types", keyspaces_to_fetch, + |err| MetadataError::Udts(UdtMetadataError::SchemaTypesInvalidColumnType(err)), ); let mut udt_rows: Vec = rows .map(|row_result| { - let row = row_result?; - let udt_row = row - .into_typed::() - .map_err(|err| { - MetadataError::Udts(UdtMetadataError::SchemaTypesInvalidColumnType(err)) - })? - .try_into()?; + let udt_row = row_result?.try_into()?; Ok::<_, QueryError>(udt_row) }) @@ -1355,21 +1377,17 @@ async fn query_tables( keyspaces_to_fetch: &[String], udts: &HashMap>>, ) -> Result>, QueryError> { - let rows = query_filter_keyspace_name( + let rows = query_filter_keyspace_name::<(String, String)>( conn, "SELECT keyspace_name, table_name FROM system_schema.tables", keyspaces_to_fetch, + |err| MetadataError::Tables(TablesMetadataError::SchemaTablesInvalidColumnType(err)), ); let mut result = HashMap::new(); let mut tables = query_tables_schema(conn, keyspaces_to_fetch, udts).await?; rows.map(|row_result| { - let row = row_result?; - let (keyspace_name, table_name) = row.into_typed().map_err(|err| { - MetadataError::Tables(TablesMetadataError::SchemaTablesInvalidColumnType(err)) - })?; - - let keyspace_and_table_name = (keyspace_name, table_name); + let keyspace_and_table_name = row_result?; let table = tables.remove(&keyspace_and_table_name).unwrap_or(Table { columns: HashMap::new(), @@ -1396,20 +1414,18 @@ async fn query_views( keyspaces_to_fetch: &[String], udts: &HashMap>>, ) -> Result>, QueryError> { - let rows = query_filter_keyspace_name( + let rows = query_filter_keyspace_name::<(String, String, String)>( conn, "SELECT keyspace_name, view_name, base_table_name FROM system_schema.views", keyspaces_to_fetch, + |err| MetadataError::Views(ViewsMetadataError::SchemaViewsInvalidColumnType(err)), ); let mut result = HashMap::new(); let mut tables = query_tables_schema(conn, keyspaces_to_fetch, udts).await?; rows.map(|row_result| { - let row = row_result?; - let (keyspace_name, view_name, base_table_name) = row.into_typed().map_err(|err| { - MetadataError::Views(ViewsMetadataError::SchemaViewsInvalidColumnType(err)) - })?; + let (keyspace_name, view_name, base_table_name) = row_result?; let keyspace_and_view_name = (keyspace_name, view_name); @@ -1447,24 +1463,18 @@ async fn query_tables_schema( // This column shouldn't be exposed to the user but is currently exposed in system tables. const THRIFT_EMPTY_TYPE: &str = "empty"; - let rows = query_filter_keyspace_name(conn, - "select keyspace_name, table_name, column_name, kind, position, type from system_schema.columns", keyspaces_to_fetch + type RowType = (String, String, String, String, i32, String); + + let rows = query_filter_keyspace_name::(conn, + "select keyspace_name, table_name, column_name, kind, position, type from system_schema.columns", keyspaces_to_fetch, |err| { + MetadataError::Tables(TablesMetadataError::SchemaColumnsInvalidColumnType(err)) + } ); let mut tables_schema = HashMap::new(); rows.map(|row_result| { - let row = row_result?; - let (keyspace_name, table_name, column_name, kind, position, type_): ( - String, - String, - String, - String, - i32, - String, - ) = row.into_typed().map_err(|err| { - MetadataError::Tables(TablesMetadataError::SchemaColumnsInvalidColumnType(err)) - })?; + let (keyspace_name, table_name, column_name, kind, position, type_) = row_result?; if type_ == THRIFT_EMPTY_TYPE { return Ok::<_, QueryError>(()); @@ -1674,15 +1684,24 @@ async fn query_table_partitioners( let rows = conn .clone() .query_iter(partitioner_query) + .map(|it| { + it.and_then(|it| { + it.into_typed::<(String, String, Option)>() + .map_err(|err| { + MetadataError::Tables(TablesMetadataError::SchemaTablesInvalidColumnType( + err, + )) + .into() + }) + .map(|iter| iter.into_stream()) + }) + }) .into_stream() .try_flatten(); let result = rows .map(|row_result| { - let (keyspace_name, table_name, partitioner) = - row_result?.into_typed().map_err(|err| { - MetadataError::Tables(TablesMetadataError::SchemaTablesInvalidColumnType(err)) - })?; + let (keyspace_name, table_name, partitioner) = row_result?; Ok::<_, QueryError>(((keyspace_name, table_name), partitioner)) }) .try_collect::>() From eb8b720f57a08f5bad4815ca32da89f07c8caf2d Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Fri, 17 Mar 2023 13:41:49 +0100 Subject: [PATCH 36/42] {session,tracing}: switch to the new deser framework for tracing info MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Adjusts the Session::try_getting_tracing_info method to use the new deserialization framework. Co-authored-by: Wojciech Przytuła --- scylla/src/tracing.rs | 61 ++++----------------------------- scylla/src/transport/errors.rs | 28 ++++++++++----- scylla/src/transport/session.rs | 47 ++++++++++++++----------- 3 files changed, 53 insertions(+), 83 deletions(-) diff --git a/scylla/src/tracing.rs b/scylla/src/tracing.rs index 53019e786..459eb81e4 100644 --- a/scylla/src/tracing.rs +++ b/scylla/src/tracing.rs @@ -1,15 +1,14 @@ +use crate::frame::value::CqlTimestamp; use itertools::Itertools; use scylla_cql::frame::value::CqlTimeuuid; +use scylla_macros::DeserializeRow; use std::collections::HashMap; use std::net::IpAddr; -use crate::cql_to_rust::{FromRow, FromRowError}; -use crate::frame::response::result::Row; -use crate::frame::value::CqlTimestamp; - /// Tracing info retrieved from `system_traces.sessions` /// with all events from `system_traces.events` -#[derive(Debug, Clone, PartialEq, Eq)] +#[derive(Debug, DeserializeRow, Clone, PartialEq, Eq)] +#[scylla(crate = "crate")] pub struct TracingInfo { pub client: Option, pub command: Option, @@ -20,11 +19,13 @@ pub struct TracingInfo { /// started_at is a timestamp - time since unix epoch pub started_at: Option, + #[scylla(skip)] pub events: Vec, } /// A single event happening during a traced query -#[derive(Debug, Clone, PartialEq, Eq)] +#[derive(Debug, DeserializeRow, Clone, PartialEq, Eq)] +#[scylla(crate = "crate")] pub struct TracingEvent { pub event_id: CqlTimeuuid, pub activity: Option, @@ -53,51 +54,3 @@ pub(crate) const TRACES_SESSION_QUERY_STR: &str = pub(crate) const TRACES_EVENTS_QUERY_STR: &str = "SELECT event_id, activity, source, source_elapsed, thread \ FROM system_traces.events WHERE session_id = ?"; - -// Converts a row received by performing TRACES_SESSION_QUERY_STR to TracingInfo -impl FromRow for TracingInfo { - fn from_row(row: Row) -> Result { - let (client, command, coordinator, duration, parameters, request, started_at) = - <( - Option, - Option, - Option, - Option, - Option>, - Option, - Option, - )>::from_row(row)?; - - Ok(TracingInfo { - client, - command, - coordinator, - duration, - parameters, - request, - started_at, - events: Vec::new(), - }) - } -} - -// Converts a row received by performing TRACES_SESSION_QUERY_STR to TracingInfo -impl FromRow for TracingEvent { - fn from_row(row: Row) -> Result { - let (event_id, activity, source, source_elapsed, thread) = <( - CqlTimeuuid, - Option, - Option, - Option, - Option, - )>::from_row(row)?; - - Ok(TracingEvent { - event_id, - activity, - source, - source_elapsed, - thread, - }) - } -} diff --git a/scylla/src/transport/errors.rs b/scylla/src/transport/errors.rs index 478d03f09..93198a4ea 100644 --- a/scylla/src/transport/errors.rs +++ b/scylla/src/transport/errors.rs @@ -13,7 +13,6 @@ use std::{ }; use scylla_cql::{ - cql_to_rust::FromRowError, frame::{ frame_errors::{ CqlAuthChallengeParseError, CqlAuthSuccessParseError, CqlAuthenticateParseError, @@ -25,14 +24,17 @@ use scylla_cql::{ response::CqlResponseKind, value::SerializeValuesError, }, - types::{deserialize::TypeCheckError, serialize::SerializationError}, + types::{ + deserialize::{DeserializationError, TypeCheckError}, + serialize::SerializationError, + }, }; use thiserror::Error; use crate::{authentication::AuthError, frame::response}; -use super::{legacy_query_result::RowsExpectedError, query_result::SingleRowError}; +use super::query_result::SingleRowError; /// Error that occurred during query execution #[derive(Error, Debug, Clone)] @@ -360,20 +362,28 @@ pub enum SchemaVersionFetchError { #[non_exhaustive] pub enum TracingProtocolError { /// Response to system_traces.session is not RESULT:Rows. - #[error("Response to system_traces.session is not RESULT:Rows: {0}")] - TracesSessionNotRows(RowsExpectedError), + #[error("Response to system_traces.session is not RESULT:Rows")] + TracesSessionNotRows, /// system_traces.session has invalid column type. #[error("system_traces.session has invalid column type: {0}")] - TracesSessionInvalidColumnType(FromRowError), + TracesSessionInvalidColumnType(TypeCheckError), + + /// Response to system_traces.session failed to deserialize. + #[error("Response system_traces.session failed to deserialize: {0}")] + TracesSessionDeserializationFailed(DeserializationError), /// Response to system_traces.events is not RESULT:Rows. - #[error("Response to system_traces.events is not RESULT:Rows: {0}")] - TracesEventsNotRows(RowsExpectedError), + #[error("Response to system_traces.events is not RESULT:Rows")] + TracesEventsNotRows, /// system_traces.events has invalid column type. #[error("system_traces.events has invalid column type: {0}")] - TracesEventsInvalidColumnType(FromRowError), + TracesEventsInvalidColumnType(TypeCheckError), + + /// Response to system_traces.events failed to deserialize. + #[error("Response system_traces.events failed to deserialize: {0}")] + TracesEventsDeserializationFailed(DeserializationError), /// All tracing queries returned an empty result. #[error( diff --git a/scylla/src/transport/session.rs b/scylla/src/transport/session.rs index 2563f376b..74e63367a 100644 --- a/scylla/src/transport/session.rs +++ b/scylla/src/transport/session.rs @@ -46,11 +46,12 @@ use super::connection::SslConfig; use super::errors::TracingProtocolError; use super::execution_profile::{ExecutionProfile, ExecutionProfileHandle, ExecutionProfileInner}; use super::iterator::RawIterator; -use super::legacy_query_result::MaybeFirstRowTypedError; #[cfg(feature = "cloud")] use super::node::CloudEndpoint; use super::node::{InternalKnownNode, KnownNode}; use super::partitioner::PartitionerName; +use super::query_result::MaybeFirstRowError; +use super::query_result::RowsError; use super::topology::UntranslatedPeer; use super::{NodeRef, SelfIdentity}; use crate::cql_to_rust::FromRow; @@ -60,7 +61,7 @@ use crate::prepared_statement::PreparedStatement; use crate::query::Query; use crate::routing::{Shard, Token}; use crate::statement::{Consistency, PageSize, PagingState, PagingStateResponse}; -use crate::tracing::{TracingEvent, TracingInfo}; +use crate::tracing::TracingInfo; use crate::transport::cluster::{Cluster, ClusterData, ClusterNeatDebug}; use crate::transport::connection::{Connection, ConnectionConfig, VerifiedKeyspaceName}; use crate::transport::connection_pool::PoolConfig; @@ -1824,15 +1825,18 @@ where // Get tracing info let maybe_tracing_info: Option = traces_session_res - .into_legacy_result()? - .maybe_first_row_typed() + .into_rows_result()? + .ok_or(ProtocolError::Tracing( + TracingProtocolError::TracesSessionNotRows, + ))? + .maybe_first_row() .map_err(|err| match err { - MaybeFirstRowTypedError::RowsExpected(e) => { - ProtocolError::Tracing(TracingProtocolError::TracesSessionNotRows(e)) - } - MaybeFirstRowTypedError::FromRowError(e) => { + MaybeFirstRowError::TypeCheckFailed(e) => { ProtocolError::Tracing(TracingProtocolError::TracesSessionInvalidColumnType(e)) } + MaybeFirstRowError::DeserializationFailed(e) => ProtocolError::Tracing( + TracingProtocolError::TracesSessionDeserializationFailed(e), + ), })?; let mut tracing_info = match maybe_tracing_info { @@ -1841,20 +1845,23 @@ where }; // Get tracing events - let tracing_event_rows = traces_events_res - .into_legacy_result()? - .rows_typed() - .map_err(|err| { - ProtocolError::Tracing(TracingProtocolError::TracesEventsNotRows(err)) - })?; - - for event in tracing_event_rows { - let tracing_event: TracingEvent = event.map_err(|err| { + let tracing_event_deserializer = + traces_events_res + .into_rows_result()? + .ok_or(ProtocolError::Tracing( + TracingProtocolError::TracesEventsNotRows, + ))?; + let tracing_event_rows = tracing_event_deserializer.rows().map_err(|err| match err { + RowsError::TypeCheckFailed(err) => { ProtocolError::Tracing(TracingProtocolError::TracesEventsInvalidColumnType(err)) - })?; + } + })?; - tracing_info.events.push(tracing_event); - } + tracing_info.events = tracing_event_rows + .collect::>() + .map_err(|err| { + ProtocolError::Tracing(TracingProtocolError::TracesEventsDeserializationFailed(err)) + })?; if tracing_info.events.is_empty() { return Ok(None); From 8877d37acdea8a42123a0874551789fd46fcfe97 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Wojciech=20Przytu=C5=82a?= Date: Tue, 12 Mar 2024 14:28:51 +0100 Subject: [PATCH 37/42] treewide: switch tests to use the new framework This is a large commit which goes over all existing tests that haven't been migrated in previous commits and adjusts them to use the new deserialization framework. There were lots of changes to be made, but they are mostly independent from each other and very simple. --- scylla/src/history.rs | 7 +- scylla/src/transport/authenticate_test.rs | 4 +- scylla/src/transport/caching_session.rs | 7 +- scylla/src/transport/connection.rs | 9 +- scylla/src/transport/cql_collections_test.rs | 29 +- scylla/src/transport/cql_types_test.rs | 252 +++++++--- scylla/src/transport/cql_value_test.rs | 23 +- .../transport/large_batch_statements_test.rs | 12 +- .../src/transport/load_balancing/default.rs | 2 +- scylla/src/transport/session_test.rs | 468 +++++++++++------- .../transport/silent_prepare_batch_test.rs | 11 +- scylla/src/utils/test_utils.rs | 25 +- scylla/tests/integration/consistency.rs | 22 +- .../tests/integration/execution_profiles.rs | 2 +- scylla/tests/integration/lwt_optimisation.rs | 10 +- scylla/tests/integration/new_session.rs | 2 +- scylla/tests/integration/retries.rs | 14 +- scylla/tests/integration/self_identity.rs | 6 +- scylla/tests/integration/shards.rs | 6 +- .../tests/integration/silent_prepare_query.rs | 10 +- .../integration/skip_metadata_optimization.rs | 24 +- scylla/tests/integration/tablets.rs | 36 +- 22 files changed, 599 insertions(+), 382 deletions(-) diff --git a/scylla/src/history.rs b/scylla/src/history.rs index 62ee1ad4d..c790aab4b 100644 --- a/scylla/src/history.rs +++ b/scylla/src/history.rs @@ -469,7 +469,6 @@ mod tests { use crate::test_utils::create_new_session_builder; use assert_matches::assert_matches; use chrono::{DateTime, NaiveDate, NaiveDateTime, NaiveTime, Utc}; - use futures::StreamExt; use scylla_cql::Consistency; // Set a single time for all timestamps within StructuredHistory. @@ -917,7 +916,7 @@ mod tests { #[tokio::test] async fn successful_query_history() { setup_tracing(); - let session = create_new_session_builder().build_legacy().await.unwrap(); + let session = create_new_session_builder().build().await.unwrap(); let mut query = Query::new("SELECT * FROM system.local"); let history_collector = Arc::new(HistoryCollector::new()); @@ -984,7 +983,7 @@ mod tests { #[tokio::test] async fn failed_query_history() { setup_tracing(); - let session = create_new_session_builder().build_legacy().await.unwrap(); + let session = create_new_session_builder().build().await.unwrap(); let mut query = Query::new("This isnt even CQL"); let history_collector = Arc::new(HistoryCollector::new()); @@ -1021,7 +1020,7 @@ mod tests { #[tokio::test] async fn iterator_query_history() { setup_tracing(); - let session = create_new_session_builder().build_legacy().await.unwrap(); + let session = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); session .query_unpaged(format!("CREATE KEYSPACE {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]) diff --git a/scylla/src/transport/authenticate_test.rs b/scylla/src/transport/authenticate_test.rs index 75d628ce4..78e72dea4 100644 --- a/scylla/src/transport/authenticate_test.rs +++ b/scylla/src/transport/authenticate_test.rs @@ -16,7 +16,7 @@ async fn authenticate_superuser() { let session = crate::SessionBuilder::new() .known_node(uri) .user("cassandra", "cassandra") - .build_legacy() + .build() .await .unwrap(); let ks = unique_keyspace_name(); @@ -75,7 +75,7 @@ async fn custom_authentication() { let session = crate::SessionBuilder::new() .known_node(uri) .authenticator_provider(Arc::new(CustomAuthenticatorProvider)) - .build_legacy() + .build() .await .unwrap(); let ks = unique_keyspace_name(); diff --git a/scylla/src/transport/caching_session.rs b/scylla/src/transport/caching_session.rs index 8232c6486..c813a6e2d 100644 --- a/scylla/src/transport/caching_session.rs +++ b/scylla/src/transport/caching_session.rs @@ -451,7 +451,12 @@ mod tests { .unwrap() .into_stream(); - let rows = iter.try_collect::>().await.unwrap().len(); + let rows = iter + .into_stream() + .try_collect::>() + .await + .unwrap() + .len(); assert_eq!(1, rows); assert_eq!(1, session.cache.len()); diff --git a/scylla/src/transport/connection.rs b/scylla/src/transport/connection.rs index 1e1f4b962..47d3c7c54 100644 --- a/scylla/src/transport/connection.rs +++ b/scylla/src/transport/connection.rs @@ -2448,7 +2448,7 @@ mod tests { // Preparation phase let session = SessionBuilder::new() .known_node_addr(addr) - .build_legacy() + .build() .await .unwrap(); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks.clone()), &[]).await.unwrap(); @@ -2547,7 +2547,7 @@ mod tests { // Preparation phase let session = SessionBuilder::new() .known_node_addr(addr) - .build_legacy() + .build() .await .unwrap(); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks.clone()), &[]).await.unwrap(); @@ -2622,9 +2622,10 @@ mod tests { .query_unpaged("SELECT p, v FROM t") .await .unwrap() - .into_legacy_result() + .into_rows_result() .unwrap() - .rows_typed::<(i32, Vec)>() + .unwrap() + .rows::<(i32, Vec)>() .unwrap() .collect::, _>>() .unwrap(); diff --git a/scylla/src/transport/cql_collections_test.rs b/scylla/src/transport/cql_collections_test.rs index 6322e6e92..f37d28a8f 100644 --- a/scylla/src/transport/cql_collections_test.rs +++ b/scylla/src/transport/cql_collections_test.rs @@ -1,12 +1,14 @@ -use crate::cql_to_rust::FromCqlVal; +use crate::transport::session::Session; +use scylla_cql::types::deserialize::value::DeserializeValue; + +use crate::frame::response::result::CqlValue; use crate::test_utils::{create_new_session_builder, setup_tracing}; use crate::utils::test_utils::unique_keyspace_name; -use crate::{frame::response::result::CqlValue, LegacySession}; use scylla_cql::types::serialize::value::SerializeValue; use std::collections::{BTreeMap, BTreeSet, HashMap, HashSet}; -async fn connect() -> LegacySession { - let session = create_new_session_builder().build_legacy().await.unwrap(); +async fn connect() -> Session { + let session = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); session.use_keyspace(ks, false).await.unwrap(); @@ -14,7 +16,7 @@ async fn connect() -> LegacySession { session } -async fn create_table(session: &LegacySession, table_name: &str, value_type: &str) { +async fn create_table(session: &Session, table_name: &str, value_type: &str) { session .query_unpaged( format!( @@ -28,13 +30,13 @@ async fn create_table(session: &LegacySession, table_name: &str, value_type: &st } async fn insert_and_select( - session: &LegacySession, + session: &Session, table_name: &str, to_insert: &InsertT, expected: &SelectT, ) where InsertT: SerializeValue, - SelectT: FromCqlVal> + PartialEq + std::fmt::Debug, + SelectT: for<'r> DeserializeValue<'r, 'r> + PartialEq + std::fmt::Debug, { session .query_unpaged( @@ -48,7 +50,10 @@ async fn insert_and_select( .query_unpaged(format!("SELECT val FROM {} WHERE p = 0", table_name), ()) .await .unwrap() - .single_row_typed::<(SelectT,)>() + .into_rows_result() + .unwrap() + .unwrap() + .single_row::<(SelectT,)>() .unwrap() .0; @@ -58,7 +63,7 @@ async fn insert_and_select( #[tokio::test] async fn test_cql_list() { setup_tracing(); - let session: LegacySession = connect().await; + let session: Session = connect().await; let table_name: &str = "test_cql_list_tab"; create_table(&session, table_name, "list").await; @@ -91,7 +96,7 @@ async fn test_cql_list() { #[tokio::test] async fn test_cql_set() { setup_tracing(); - let session: LegacySession = connect().await; + let session: Session = connect().await; let table_name: &str = "test_cql_set_tab"; create_table(&session, table_name, "set").await; @@ -155,7 +160,7 @@ async fn test_cql_set() { #[tokio::test] async fn test_cql_map() { setup_tracing(); - let session: LegacySession = connect().await; + let session: Session = connect().await; let table_name: &str = "test_cql_map_tab"; create_table(&session, table_name, "map").await; @@ -206,7 +211,7 @@ async fn test_cql_map() { #[tokio::test] async fn test_cql_tuple() { setup_tracing(); - let session: LegacySession = connect().await; + let session: Session = connect().await; let table_name: &str = "test_cql_tuple_tab"; create_table(&session, table_name, "tuple").await; diff --git a/scylla/src/transport/cql_types_test.rs b/scylla/src/transport/cql_types_test.rs index 4be1244eb..0a1833fd7 100644 --- a/scylla/src/transport/cql_types_test.rs +++ b/scylla/src/transport/cql_types_test.rs @@ -1,17 +1,14 @@ use crate as scylla; -use crate::cql_to_rust::FromCqlVal; use crate::frame::response::result::CqlValue; use crate::frame::value::{Counter, CqlDate, CqlTime, CqlTimestamp}; -use crate::macros::FromUserType; -use crate::test_utils::{ - create_new_session_builder, scylla_supports_tablets_legacy, setup_tracing, -}; -use crate::transport::session::LegacySession; +use crate::test_utils::{create_new_session_builder, scylla_supports_tablets, setup_tracing}; +use crate::transport::session::Session; use crate::utils::test_utils::unique_keyspace_name; use itertools::Itertools; use scylla_cql::frame::value::{CqlTimeuuid, CqlVarint}; +use scylla_cql::types::deserialize::value::DeserializeValue; use scylla_cql::types::serialize::value::SerializeValue; -use scylla_macros::SerializeValue; +use scylla_macros::{DeserializeValue, SerializeValue}; use std::cmp::PartialEq; use std::fmt::Debug; use std::net::{IpAddr, Ipv4Addr, Ipv6Addr}; @@ -24,8 +21,8 @@ async fn init_test_maybe_without_tablets( table_name: &str, type_name: &str, supports_tablets: bool, -) -> LegacySession { - let session: LegacySession = create_new_session_builder().build_legacy().await.unwrap(); +) -> Session { + let session: Session = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); let mut create_ks = format!( @@ -34,7 +31,7 @@ async fn init_test_maybe_without_tablets( ks ); - if !supports_tablets && scylla_supports_tablets_legacy(&session).await { + if !supports_tablets && scylla_supports_tablets(&session).await { create_ks += " AND TABLETS = {'enabled': false}" } @@ -63,7 +60,7 @@ async fn init_test_maybe_without_tablets( // Used to prepare a table for test // Creates a new keyspace // Drops and creates table {table_name} (id int PRIMARY KEY, val {type_name}) -async fn init_test(table_name: &str, type_name: &str) -> LegacySession { +async fn init_test(table_name: &str, type_name: &str) -> Session { init_test_maybe_without_tablets(table_name, type_name, true).await } @@ -77,9 +74,9 @@ async fn init_test(table_name: &str, type_name: &str) -> LegacySession { // Expected values and bound values are computed using T::from_str async fn run_tests(tests: &[&str], type_name: &str) where - T: SerializeValue + FromCqlVal + FromStr + Debug + Clone + PartialEq, + T: SerializeValue + for<'r> DeserializeValue<'r, 'r> + FromStr + Debug + Clone + PartialEq, { - let session: LegacySession = init_test(type_name, type_name).await; + let session: Session = init_test(type_name, type_name).await; session.await_schema_agreement().await.unwrap(); for test in tests.iter() { @@ -102,7 +99,10 @@ where .query_unpaged(select_values, &[]) .await .unwrap() - .rows_typed::<(T,)>() + .into_rows_result() + .unwrap() + .unwrap() + .rows::<(T,)>() .unwrap() .map(Result::unwrap) .map(|row| row.0) @@ -170,7 +170,7 @@ async fn test_cql_varint() { ]; let table_name = "cql_varint_tests"; - let session: LegacySession = create_new_session_builder().build_legacy().await.unwrap(); + let session: Session = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); session @@ -220,7 +220,10 @@ async fn test_cql_varint() { .execute_unpaged(&prepared_select, &[]) .await .unwrap() - .rows_typed::<(CqlVarint,)>() + .into_rows_result() + .unwrap() + .unwrap() + .rows::<(CqlVarint,)>() .unwrap() .map(Result::unwrap) .map(|row| row.0) @@ -280,7 +283,7 @@ async fn test_counter() { // Can't use run_tests, because counters are special and can't be inserted let type_name = "counter"; - let session: LegacySession = init_test_maybe_without_tablets(type_name, type_name, false).await; + let session: Session = init_test_maybe_without_tablets(type_name, type_name, false).await; for (i, test) in tests.iter().enumerate() { let update_bound_value = format!("UPDATE {} SET val = val + ? WHERE id = ?", type_name); @@ -295,7 +298,10 @@ async fn test_counter() { .query_unpaged(select_values, (i as i32,)) .await .unwrap() - .rows_typed::<(Counter,)>() + .into_rows_result() + .unwrap() + .unwrap() + .rows::<(Counter,)>() .unwrap() .map(Result::unwrap) .map(|row| row.0) @@ -313,7 +319,7 @@ async fn test_naive_date_04() { use chrono::Datelike; use chrono::NaiveDate; - let session: LegacySession = init_test("chrono_naive_date_tests", "date").await; + let session: Session = init_test("chrono_naive_date_tests", "date").await; let min_naive_date: NaiveDate = NaiveDate::MIN; let min_naive_date_string = min_naive_date.format("%Y-%m-%d").to_string(); @@ -371,7 +377,10 @@ async fn test_naive_date_04() { .query_unpaged("SELECT val from chrono_naive_date_tests", &[]) .await .unwrap() - .rows_typed::<(NaiveDate,)>() + .into_rows_result() + .unwrap() + .unwrap() + .rows::<(NaiveDate,)>() .unwrap() .next() .unwrap() @@ -394,7 +403,10 @@ async fn test_naive_date_04() { .query_unpaged("SELECT val from chrono_naive_date_tests", &[]) .await .unwrap() - .single_row_typed::<(NaiveDate,)>() + .into_rows_result() + .unwrap() + .unwrap() + .single_row::<(NaiveDate,)>() .unwrap(); assert_eq!(read_date, *naive_date); } @@ -406,7 +418,7 @@ async fn test_cql_date() { setup_tracing(); // Tests value::Date which allows to insert dates outside NaiveDate range - let session: LegacySession = init_test("cql_date_tests", "date").await; + let session: Session = init_test("cql_date_tests", "date").await; let tests = [ ("1970-01-01", CqlDate(2_u32.pow(31))), @@ -429,15 +441,14 @@ async fn test_cql_date() { .await .unwrap(); - let read_date: CqlDate = session + let (read_date,): (CqlDate,) = session .query_unpaged("SELECT val from cql_date_tests", &[]) .await .unwrap() - .rows - .unwrap()[0] - .columns[0] - .as_ref() - .map(|cql_val| cql_val.as_cql_date().unwrap()) + .into_rows_result() + .unwrap() + .unwrap() + .single_row::<(CqlDate,)>() .unwrap(); assert_eq!(read_date, *date); @@ -467,7 +478,7 @@ async fn test_date_03() { setup_tracing(); use time::{Date, Month::*}; - let session: LegacySession = init_test("time_date_tests", "date").await; + let session: Session = init_test("time_date_tests", "date").await; let tests = [ // Basic test values @@ -520,7 +531,10 @@ async fn test_date_03() { .query_unpaged("SELECT val from time_date_tests", &[]) .await .unwrap() - .first_row_typed::<(Date,)>() + .into_rows_result() + .unwrap() + .unwrap() + .first_row::<(Date,)>() .ok() .map(|val| val.0); @@ -540,7 +554,10 @@ async fn test_date_03() { .query_unpaged("SELECT val from time_date_tests", &[]) .await .unwrap() - .first_row_typed::<(Date,)>() + .into_rows_result() + .unwrap() + .unwrap() + .first_row::<(Date,)>() .unwrap(); assert_eq!(read_date, *date); } @@ -553,7 +570,7 @@ async fn test_cql_time() { // CqlTime is an i64 - nanoseconds since midnight // in range 0..=86399999999999 - let session: LegacySession = init_test("cql_time_tests", "time").await; + let session: Session = init_test("cql_time_tests", "time").await; let max_time: i64 = 24 * 60 * 60 * 1_000_000_000 - 1; assert_eq!(max_time, 86399999999999); @@ -583,7 +600,10 @@ async fn test_cql_time() { .query_unpaged("SELECT val from cql_time_tests", &[]) .await .unwrap() - .single_row_typed::<(CqlTime,)>() + .into_rows_result() + .unwrap() + .unwrap() + .single_row::<(CqlTime,)>() .unwrap(); assert_eq!(read_time, *time_duration); @@ -601,7 +621,10 @@ async fn test_cql_time() { .query_unpaged("SELECT val from cql_time_tests", &[]) .await .unwrap() - .single_row_typed::<(CqlTime,)>() + .into_rows_result() + .unwrap() + .unwrap() + .single_row::<(CqlTime,)>() .unwrap(); assert_eq!(read_time, *time_duration); @@ -679,7 +702,10 @@ async fn test_naive_time_04() { .query_unpaged("SELECT val from chrono_time_tests", &[]) .await .unwrap() - .first_row_typed::<(NaiveTime,)>() + .into_rows_result() + .unwrap() + .unwrap() + .first_row::<(NaiveTime,)>() .unwrap(); assert_eq!(read_time, *time); @@ -697,7 +723,10 @@ async fn test_naive_time_04() { .query_unpaged("SELECT val from chrono_time_tests", &[]) .await .unwrap() - .first_row_typed::<(NaiveTime,)>() + .into_rows_result() + .unwrap() + .unwrap() + .first_row::<(NaiveTime,)>() .unwrap(); assert_eq!(read_time, *time); } @@ -759,7 +788,10 @@ async fn test_time_03() { .query_unpaged("SELECT val from time_time_tests", &[]) .await .unwrap() - .first_row_typed::<(Time,)>() + .into_rows_result() + .unwrap() + .unwrap() + .first_row::<(Time,)>() .unwrap(); assert_eq!(read_time, *time); @@ -777,7 +809,10 @@ async fn test_time_03() { .query_unpaged("SELECT val from time_time_tests", &[]) .await .unwrap() - .first_row_typed::<(Time,)>() + .into_rows_result() + .unwrap() + .unwrap() + .first_row::<(Time,)>() .unwrap(); assert_eq!(read_time, *time); } @@ -786,7 +821,7 @@ async fn test_time_03() { #[tokio::test] async fn test_cql_timestamp() { setup_tracing(); - let session: LegacySession = init_test("cql_timestamp_tests", "timestamp").await; + let session: Session = init_test("cql_timestamp_tests", "timestamp").await; //let epoch_date = NaiveDate::from_ymd_opt(1970, 1, 1).unwrap(); @@ -830,7 +865,10 @@ async fn test_cql_timestamp() { .query_unpaged("SELECT val from cql_timestamp_tests", &[]) .await .unwrap() - .single_row_typed::<(CqlTimestamp,)>() + .into_rows_result() + .unwrap() + .unwrap() + .single_row::<(CqlTimestamp,)>() .unwrap(); assert_eq!(read_timestamp, *timestamp_duration); @@ -848,7 +886,10 @@ async fn test_cql_timestamp() { .query_unpaged("SELECT val from cql_timestamp_tests", &[]) .await .unwrap() - .single_row_typed::<(CqlTimestamp,)>() + .into_rows_result() + .unwrap() + .unwrap() + .single_row::<(CqlTimestamp,)>() .unwrap(); assert_eq!(read_timestamp, *timestamp_duration); @@ -925,7 +966,10 @@ async fn test_date_time_04() { .query_unpaged("SELECT val from chrono_datetime_tests", &[]) .await .unwrap() - .first_row_typed::<(DateTime,)>() + .into_rows_result() + .unwrap() + .unwrap() + .first_row::<(DateTime,)>() .unwrap(); assert_eq!(read_datetime, *datetime); @@ -943,7 +987,10 @@ async fn test_date_time_04() { .query_unpaged("SELECT val from chrono_datetime_tests", &[]) .await .unwrap() - .first_row_typed::<(DateTime,)>() + .into_rows_result() + .unwrap() + .unwrap() + .first_row::<(DateTime,)>() .unwrap(); assert_eq!(read_datetime, *datetime); } @@ -971,7 +1018,10 @@ async fn test_date_time_04() { .query_unpaged("SELECT val from chrono_datetime_tests", &[]) .await .unwrap() - .first_row_typed::<(DateTime,)>() + .into_rows_result() + .unwrap() + .unwrap() + .first_row::<(DateTime,)>() .unwrap(); assert_eq!(read_datetime, nanosecond_precision_1st_half_rounded); @@ -997,7 +1047,10 @@ async fn test_date_time_04() { .query_unpaged("SELECT val from chrono_datetime_tests", &[]) .await .unwrap() - .first_row_typed::<(DateTime,)>() + .into_rows_result() + .unwrap() + .unwrap() + .first_row::<(DateTime,)>() .unwrap(); assert_eq!(read_datetime, nanosecond_precision_2nd_half_rounded); @@ -1086,7 +1139,10 @@ async fn test_offset_date_time_03() { .query_unpaged("SELECT val from time_datetime_tests", &[]) .await .unwrap() - .first_row_typed::<(OffsetDateTime,)>() + .into_rows_result() + .unwrap() + .unwrap() + .first_row::<(OffsetDateTime,)>() .unwrap(); assert_eq!(read_datetime, *datetime); @@ -1104,7 +1160,10 @@ async fn test_offset_date_time_03() { .query_unpaged("SELECT val from time_datetime_tests", &[]) .await .unwrap() - .first_row_typed::<(OffsetDateTime,)>() + .into_rows_result() + .unwrap() + .unwrap() + .first_row::<(OffsetDateTime,)>() .unwrap(); assert_eq!(read_datetime, *datetime); } @@ -1132,7 +1191,10 @@ async fn test_offset_date_time_03() { .query_unpaged("SELECT val from time_datetime_tests", &[]) .await .unwrap() - .first_row_typed::<(OffsetDateTime,)>() + .into_rows_result() + .unwrap() + .unwrap() + .first_row::<(OffsetDateTime,)>() .unwrap(); assert_eq!(read_datetime, nanosecond_precision_1st_half_rounded); @@ -1158,7 +1220,10 @@ async fn test_offset_date_time_03() { .query_unpaged("SELECT val from time_datetime_tests", &[]) .await .unwrap() - .first_row_typed::<(OffsetDateTime,)>() + .into_rows_result() + .unwrap() + .unwrap() + .first_row::<(OffsetDateTime,)>() .unwrap(); assert_eq!(read_datetime, nanosecond_precision_2nd_half_rounded); } @@ -1166,7 +1231,7 @@ async fn test_offset_date_time_03() { #[tokio::test] async fn test_timeuuid() { setup_tracing(); - let session: LegacySession = init_test("timeuuid_tests", "timeuuid").await; + let session: Session = init_test("timeuuid_tests", "timeuuid").await; // A few random timeuuids generated manually let tests = [ @@ -1207,7 +1272,10 @@ async fn test_timeuuid() { .query_unpaged("SELECT val from timeuuid_tests", &[]) .await .unwrap() - .single_row_typed::<(CqlTimeuuid,)>() + .into_rows_result() + .unwrap() + .unwrap() + .single_row::<(CqlTimeuuid,)>() .unwrap(); assert_eq!(read_timeuuid.as_bytes(), timeuuid_bytes); @@ -1226,7 +1294,10 @@ async fn test_timeuuid() { .query_unpaged("SELECT val from timeuuid_tests", &[]) .await .unwrap() - .single_row_typed::<(CqlTimeuuid,)>() + .into_rows_result() + .unwrap() + .unwrap() + .single_row::<(CqlTimeuuid,)>() .unwrap(); assert_eq!(read_timeuuid.as_bytes(), timeuuid_bytes); @@ -1236,7 +1307,7 @@ async fn test_timeuuid() { #[tokio::test] async fn test_timeuuid_ordering() { setup_tracing(); - let session: LegacySession = create_new_session_builder().build_legacy().await.unwrap(); + let session: Session = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); session @@ -1295,7 +1366,10 @@ async fn test_timeuuid_ordering() { .query_unpaged("SELECT t FROM tab WHERE p = 0", ()) .await .unwrap() - .rows_typed::<(CqlTimeuuid,)>() + .into_rows_result() + .unwrap() + .unwrap() + .rows::<(CqlTimeuuid,)>() .unwrap() .map(|r| r.unwrap().0) .collect(); @@ -1318,7 +1392,7 @@ async fn test_timeuuid_ordering() { #[tokio::test] async fn test_inet() { setup_tracing(); - let session: LegacySession = init_test("inet_tests", "inet").await; + let session: Session = init_test("inet_tests", "inet").await; let tests = [ ("0.0.0.0", IpAddr::V4(Ipv4Addr::new(0, 0, 0, 0))), @@ -1374,7 +1448,10 @@ async fn test_inet() { .query_unpaged("SELECT val from inet_tests WHERE id = 0", &[]) .await .unwrap() - .single_row_typed::<(IpAddr,)>() + .into_rows_result() + .unwrap() + .unwrap() + .single_row::<(IpAddr,)>() .unwrap(); assert_eq!(read_inet, *inet); @@ -1389,7 +1466,10 @@ async fn test_inet() { .query_unpaged("SELECT val from inet_tests WHERE id = 0", &[]) .await .unwrap() - .single_row_typed::<(IpAddr,)>() + .into_rows_result() + .unwrap() + .unwrap() + .single_row::<(IpAddr,)>() .unwrap(); assert_eq!(read_inet, *inet); @@ -1399,7 +1479,7 @@ async fn test_inet() { #[tokio::test] async fn test_blob() { setup_tracing(); - let session: LegacySession = init_test("blob_tests", "blob").await; + let session: Session = init_test("blob_tests", "blob").await; let long_blob: Vec = vec![0x11; 1234]; let mut long_blob_str: String = "0x".to_string(); @@ -1440,7 +1520,10 @@ async fn test_blob() { .query_unpaged("SELECT val from blob_tests WHERE id = 0", &[]) .await .unwrap() - .single_row_typed::<(Vec,)>() + .into_rows_result() + .unwrap() + .unwrap() + .single_row::<(Vec,)>() .unwrap(); assert_eq!(read_blob, *blob); @@ -1455,7 +1538,10 @@ async fn test_blob() { .query_unpaged("SELECT val from blob_tests WHERE id = 0", &[]) .await .unwrap() - .single_row_typed::<(Vec,)>() + .into_rows_result() + .unwrap() + .unwrap() + .single_row::<(Vec,)>() .unwrap(); assert_eq!(read_blob, *blob); @@ -1468,7 +1554,7 @@ async fn test_udt_after_schema_update() { let table_name = "udt_tests"; let type_name = "usertype1"; - let session: LegacySession = create_new_session_builder().build_legacy().await.unwrap(); + let session: Session = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); session @@ -1516,7 +1602,7 @@ async fn test_udt_after_schema_update() { .await .unwrap(); - #[derive(SerializeValue, FromUserType, Debug, PartialEq)] + #[derive(SerializeValue, DeserializeValue, Debug, PartialEq)] #[scylla(crate = crate)] struct UdtV1 { first: i32, @@ -1543,7 +1629,10 @@ async fn test_udt_after_schema_update() { .query_unpaged(format!("SELECT val from {} WHERE id = 0", table_name), &[]) .await .unwrap() - .single_row_typed::<(UdtV1,)>() + .into_rows_result() + .unwrap() + .unwrap() + .single_row::<(UdtV1,)>() .unwrap(); assert_eq!(read_udt, v1); @@ -1560,7 +1649,10 @@ async fn test_udt_after_schema_update() { .query_unpaged(format!("SELECT val from {} WHERE id = 0", table_name), &[]) .await .unwrap() - .single_row_typed::<(UdtV1,)>() + .into_rows_result() + .unwrap() + .unwrap() + .single_row::<(UdtV1,)>() .unwrap(); assert_eq!(read_udt, v1); @@ -1570,7 +1662,7 @@ async fn test_udt_after_schema_update() { .await .unwrap(); - #[derive(FromUserType, Debug, PartialEq)] + #[derive(DeserializeValue, Debug, PartialEq)] struct UdtV2 { first: i32, second: bool, @@ -1581,7 +1673,10 @@ async fn test_udt_after_schema_update() { .query_unpaged(format!("SELECT val from {} WHERE id = 0", table_name), &[]) .await .unwrap() - .single_row_typed::<(UdtV2,)>() + .into_rows_result() + .unwrap() + .unwrap() + .single_row::<(UdtV2,)>() .unwrap(); assert_eq!( @@ -1597,7 +1692,7 @@ async fn test_udt_after_schema_update() { #[tokio::test] async fn test_empty() { setup_tracing(); - let session: LegacySession = init_test("empty_tests", "int").await; + let session: Session = init_test("empty_tests", "int").await; session .query_unpaged( @@ -1611,7 +1706,10 @@ async fn test_empty() { .query_unpaged("SELECT val FROM empty_tests WHERE id = 0", ()) .await .unwrap() - .first_row_typed::<(CqlValue,)>() + .into_rows_result() + .unwrap() + .unwrap() + .first_row::<(CqlValue,)>() .unwrap(); assert_eq!(empty, CqlValue::Empty); @@ -1628,7 +1726,10 @@ async fn test_empty() { .query_unpaged("SELECT val FROM empty_tests WHERE id = 1", ()) .await .unwrap() - .first_row_typed::<(CqlValue,)>() + .into_rows_result() + .unwrap() + .unwrap() + .first_row::<(CqlValue,)>() .unwrap(); assert_eq!(empty, CqlValue::Empty); @@ -1640,7 +1741,7 @@ async fn test_udt_with_missing_field() { let table_name = "udt_tests"; let type_name = "usertype1"; - let session: LegacySession = create_new_session_builder().build_legacy().await.unwrap(); + let session: Session = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); session @@ -1691,14 +1792,14 @@ async fn test_udt_with_missing_field() { let mut id = 0; async fn verify_insert_select_identity( - session: &LegacySession, + session: &Session, table_name: &str, id: i32, element: TQ, expected: TR, ) where TQ: SerializeValue, - TR: FromCqlVal + PartialEq + Debug, + TR: for<'r> DeserializeValue<'r, 'r> + PartialEq + Debug, { session .query_unpaged( @@ -1714,13 +1815,16 @@ async fn test_udt_with_missing_field() { ) .await .unwrap() - .single_row_typed::<(TR,)>() + .into_rows_result() + .unwrap() + .unwrap() + .single_row::<(TR,)>() .unwrap() .0; assert_eq!(expected, result); } - #[derive(FromUserType, Debug, PartialEq)] + #[derive(DeserializeValue, Debug, PartialEq)] struct UdtFull { first: i32, second: bool, diff --git a/scylla/src/transport/cql_value_test.rs b/scylla/src/transport/cql_value_test.rs index d3d2d18e4..53560ed94 100644 --- a/scylla/src/transport/cql_value_test.rs +++ b/scylla/src/transport/cql_value_test.rs @@ -1,13 +1,14 @@ -use crate::frame::{response::result::CqlValue, value::CqlDuration}; +use crate::frame::response::result::{CqlValue, Row}; +use crate::frame::value::CqlDuration; use crate::test_utils::{create_new_session_builder, setup_tracing}; use crate::utils::test_utils::unique_keyspace_name; -use crate::LegacySession; +use crate::Session; #[tokio::test] async fn test_cqlvalue_udt() { setup_tracing(); - let session: LegacySession = create_new_session_builder().build_legacy().await.unwrap(); + let session: Session = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); session .query_unpaged( @@ -58,7 +59,12 @@ async fn test_cqlvalue_udt() { .query_unpaged("SELECT my FROM cqlvalue_udt_test", &[]) .await .unwrap() - .rows + .into_rows_result() + .unwrap() + .unwrap() + .rows::() + .unwrap() + .collect::, _>>() .unwrap(); assert_eq!(rows.len(), 1); @@ -72,7 +78,7 @@ async fn test_cqlvalue_udt() { #[tokio::test] async fn test_cqlvalue_duration() { setup_tracing(); - let session: LegacySession = create_new_session_builder().build_legacy().await.unwrap(); + let session: Session = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); session @@ -113,7 +119,12 @@ async fn test_cqlvalue_duration() { ) .await .unwrap() - .rows + .into_rows_result() + .unwrap() + .unwrap() + .rows::() + .unwrap() + .collect::, _>>() .unwrap(); assert_eq!(rows.len(), 4); diff --git a/scylla/src/transport/large_batch_statements_test.rs b/scylla/src/transport/large_batch_statements_test.rs index 0e250fc7d..7e8fc482c 100644 --- a/scylla/src/transport/large_batch_statements_test.rs +++ b/scylla/src/transport/large_batch_statements_test.rs @@ -7,13 +7,13 @@ use crate::transport::errors::{BadQuery, QueryError}; use crate::{ batch::Batch, test_utils::{create_new_session_builder, unique_keyspace_name}, - LegacyQueryResult, LegacySession, + QueryResult, Session, }; #[tokio::test] async fn test_large_batch_statements() { setup_tracing(); - let mut session = create_new_session_builder().build_legacy().await.unwrap(); + let mut session = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); session = create_test_session(session, &ks).await; @@ -31,7 +31,7 @@ async fn test_large_batch_statements() { ) } -async fn create_test_session(session: LegacySession, ks: &String) -> LegacySession { +async fn create_test_session(session: Session, ks: &String) -> Session { session .query_unpaged( format!("CREATE KEYSPACE {} WITH REPLICATION = {{ 'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1 }}",ks), @@ -51,11 +51,7 @@ async fn create_test_session(session: LegacySession, ks: &String) -> LegacySessi session } -async fn write_batch( - session: &LegacySession, - n: usize, - ks: &String, -) -> Result { +async fn write_batch(session: &Session, n: usize, ks: &String) -> Result { let mut batch_query = Batch::new(BatchType::Unlogged); let mut batch_values = Vec::new(); let query = format!("INSERT INTO {}.pairs (dummy, k, v) VALUES (0, ?, ?)", ks); diff --git a/scylla/src/transport/load_balancing/default.rs b/scylla/src/transport/load_balancing/default.rs index beffebb1f..51db7f97f 100644 --- a/scylla/src/transport/load_balancing/default.rs +++ b/scylla/src/transport/load_balancing/default.rs @@ -3860,7 +3860,7 @@ mod latency_awareness { let session = create_new_session_builder() .default_execution_profile_handle(handle) - .build_legacy() + .build() .await .unwrap(); diff --git a/scylla/src/transport/session_test.rs b/scylla/src/transport/session_test.rs index 1dddb7d40..9bb4fc5a5 100644 --- a/scylla/src/transport/session_test.rs +++ b/scylla/src/transport/session_test.rs @@ -1,17 +1,16 @@ -use crate as scylla; use crate::batch::{Batch, BatchStatement}; -use crate::frame::response::result::Row; use crate::prepared_statement::PreparedStatement; use crate::query::Query; use crate::retry_policy::{QueryInfo, RetryDecision, RetryPolicy, RetrySession}; use crate::routing::Token; use crate::statement::Consistency; -use crate::test_utils::{scylla_supports_tablets_legacy, setup_tracing}; +use crate::test_utils::{scylla_supports_tablets, setup_tracing}; use crate::tracing::TracingInfo; use crate::transport::errors::{BadKeyspaceName, BadQuery, DbError, QueryError}; use crate::transport::partitioner::{ calculate_token_for_partition_key, Murmur3Partitioner, Partitioner, PartitionerName, }; +use crate::transport::session::Session; use crate::transport::topology::Strategy::NetworkTopologyStrategy; use crate::transport::topology::{ CollectionType, ColumnKind, CqlType, NativeType, UserDefinedType, @@ -20,14 +19,14 @@ use crate::utils::test_utils::{ create_new_session_builder, supports_feature, unique_keyspace_name, }; use crate::ExecutionProfile; -use crate::LegacyCachingSession; -use crate::LegacyQueryResult; -use crate::{LegacySession, SessionBuilder}; +use crate::{self as scylla, QueryResult}; +use crate::{CachingSession, SessionBuilder}; use assert_matches::assert_matches; -use futures::{FutureExt, StreamExt, TryStreamExt}; +use futures::{FutureExt, TryStreamExt}; use itertools::Itertools; use scylla_cql::frame::request::query::{PagingState, PagingStateResponse}; use scylla_cql::frame::response::result::ColumnType; +use scylla_cql::frame::response::result::Row; use scylla_cql::types::serialize::row::{SerializeRow, SerializedValues}; use scylla_cql::types::serialize::value::SerializeValue; use std::collections::BTreeSet; @@ -37,6 +36,8 @@ use std::sync::Arc; use tokio::net::TcpListener; use uuid::Uuid; +use super::query_result::QueryRowsResult; + #[tokio::test] async fn test_connection_failure() { setup_tracing(); @@ -55,10 +56,7 @@ async fn test_connection_failure() { .remote_handle(); tokio::spawn(fut); - let res = SessionBuilder::new() - .known_node_addr(addr) - .build_legacy() - .await; + let res = SessionBuilder::new().known_node_addr(addr).build().await; match res { Ok(_) => panic!("Unexpected success"), Err(err) => println!("Connection error (it was expected): {:?}", err), @@ -68,7 +66,7 @@ async fn test_connection_failure() { #[tokio::test] async fn test_unprepared_statement() { setup_tracing(); - let session = create_new_session_builder().build_legacy().await.unwrap(); + let session = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); @@ -110,42 +108,40 @@ async fn test_unprepared_statement() { .await .unwrap(); - let (a_idx, _) = query_result.get_column_spec("a").unwrap(); - let (b_idx, _) = query_result.get_column_spec("b").unwrap(); - let (c_idx, _) = query_result.get_column_spec("c").unwrap(); - assert!(query_result.get_column_spec("d").is_none()); + let rows = query_result.into_rows_result().unwrap().unwrap(); - let rs = query_result.rows.unwrap(); + let col_specs = rows.column_specs(); + assert_eq!(col_specs.get_by_name("a").unwrap().0, 0); + assert_eq!(col_specs.get_by_name("b").unwrap().0, 1); + assert_eq!(col_specs.get_by_name("c").unwrap().0, 2); + assert!(col_specs.get_by_name("d").is_none()); + + let mut results = rows + .rows::<(i32, i32, String)>() + .unwrap() + .collect::, _>>() + .unwrap(); - let mut results: Vec<(i32, i32, &String)> = rs - .iter() - .map(|r| { - let a = r.columns[a_idx].as_ref().unwrap().as_int().unwrap(); - let b = r.columns[b_idx].as_ref().unwrap().as_int().unwrap(); - let c = r.columns[c_idx].as_ref().unwrap().as_text().unwrap(); - (a, b, c) - }) - .collect(); results.sort(); assert_eq!( results, vec![ - (1, 2, &String::from("abc")), - (1, 4, &String::from("hello")), - (7, 11, &String::from("")) + (1, 2, String::from("abc")), + (1, 4, String::from("hello")), + (7, 11, String::from("")) ] ); let query_result = session .query_iter(format!("SELECT a, b, c FROM {}.t", ks), &[]) .await .unwrap(); - let specs = query_result.get_column_specs(); + let specs = query_result.column_specs(); assert_eq!(specs.len(), 3); for (spec, name) in specs.iter().zip(["a", "b", "c"]) { assert_eq!(spec.name(), name); // Check column name. assert_eq!(spec.table_spec().ks_name(), ks); } - let mut results_from_manual_paging: Vec = vec![]; + let mut results_from_manual_paging = vec![]; let query = Query::new(format!("SELECT a, b, c FROM {}.t", ks)).with_page_size(1); let mut paging_state = PagingState::start(); let mut watchdog = 0; @@ -154,7 +150,15 @@ async fn test_unprepared_statement() { .query_single_page(query.clone(), &[], paging_state) .await .unwrap(); - results_from_manual_paging.append(&mut rs_manual.rows.unwrap()); + let mut page_results = rs_manual + .into_rows_result() + .unwrap() + .unwrap() + .rows::<(i32, i32, String)>() + .unwrap() + .collect::, _>>() + .unwrap(); + results_from_manual_paging.append(&mut page_results); match paging_state_response { PagingStateResponse::HasMorePages { state } => { paging_state = state; @@ -164,13 +168,13 @@ async fn test_unprepared_statement() { } watchdog += 1; } - assert_eq!(results_from_manual_paging, rs); + assert_eq!(results_from_manual_paging, results); } #[tokio::test] async fn test_prepared_statement() { setup_tracing(); - let session = create_new_session_builder().build_legacy().await.unwrap(); + let session = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); @@ -198,7 +202,7 @@ async fn test_prepared_statement() { .await .unwrap(); let query_result = session.execute_iter(prepared_statement, &[]).await.unwrap(); - let specs = query_result.get_column_specs(); + let specs = query_result.column_specs(); assert_eq!(specs.len(), 3); for (spec, name) in specs.iter().zip(["a", "b", "c"]) { assert_eq!(spec.name(), name); // Check column name. @@ -237,7 +241,10 @@ async fn test_prepared_statement() { .query_unpaged(format!("SELECT token(a) FROM {}.t2", ks), &[]) .await .unwrap() - .single_row_typed() + .into_rows_result() + .unwrap() + .unwrap() + .single_row::<(i64,)>() .unwrap(); let token = Token::new(value); let prepared_token = Murmur3Partitioner @@ -256,7 +263,10 @@ async fn test_prepared_statement() { .query_unpaged(format!("SELECT token(a,b,c) FROM {}.complex_pk", ks), &[]) .await .unwrap() - .single_row_typed() + .into_rows_result() + .unwrap() + .unwrap() + .single_row::<(i64,)>() .unwrap(); let token = Token::new(value); let prepared_token = Murmur3Partitioner.hash_one( @@ -278,15 +288,17 @@ async fn test_prepared_statement() { .query_unpaged(format!("SELECT a,b,c FROM {}.t2", ks), &[]) .await .unwrap() - .rows + .into_rows_result() + .unwrap() + .unwrap() + .rows::<(i32, i32, String)>() + .unwrap() + .collect::, _>>() .unwrap(); - let r = rs.first().unwrap(); - let a = r.columns[0].as_ref().unwrap().as_int().unwrap(); - let b = r.columns[1].as_ref().unwrap().as_int().unwrap(); - let c = r.columns[2].as_ref().unwrap().as_text().unwrap(); - assert_eq!((a, b, c), (17, 16, &String::from("I'm prepared!!!"))); + let r = &rs[0]; + assert_eq!(r, &(17, 16, String::from("I'm prepared!!!"))); - let mut results_from_manual_paging: Vec = vec![]; + let mut results_from_manual_paging = vec![]; let query = Query::new(format!("SELECT a, b, c FROM {}.t2", ks)).with_page_size(1); let prepared_paged = session.prepare(query).await.unwrap(); let mut paging_state = PagingState::start(); @@ -296,7 +308,15 @@ async fn test_prepared_statement() { .execute_single_page(&prepared_paged, &[], paging_state) .await .unwrap(); - results_from_manual_paging.append(&mut rs_manual.rows.unwrap()); + let mut page_results = rs_manual + .into_rows_result() + .unwrap() + .unwrap() + .rows::<(i32, i32, String)>() + .unwrap() + .collect::, _>>() + .unwrap(); + results_from_manual_paging.append(&mut page_results); match paging_state_response { PagingStateResponse::HasMorePages { state } => { paging_state = state; @@ -313,7 +333,10 @@ async fn test_prepared_statement() { .query_unpaged(format!("SELECT a,b,c,d,e FROM {}.complex_pk", ks), &[]) .await .unwrap() - .single_row_typed() + .into_rows_result() + .unwrap() + .unwrap() + .single_row::<(i32, i32, String, i32, Option)>() .unwrap(); assert!(e.is_none()); assert_eq!( @@ -321,9 +344,9 @@ async fn test_prepared_statement() { (17, 16, "I'm prepared!!!", 7, None) ); } - // Check that SerializeRow macro works + // Check that SerializeRow and DeserializeRow macros work { - #[derive(scylla::SerializeRow, scylla::FromRow, PartialEq, Debug, Clone)] + #[derive(scylla::SerializeRow, scylla::DeserializeRow, PartialEq, Debug, Clone)] #[scylla(crate = crate)] struct ComplexPk { a: i32, @@ -359,7 +382,10 @@ async fn test_prepared_statement() { ) .await .unwrap() - .single_row_typed() + .into_rows_result() + .unwrap() + .unwrap() + .single_row() .unwrap(); assert_eq!(input, output) } @@ -371,13 +397,13 @@ async fn test_counter_batch() { use scylla_cql::frame::request::batch::BatchType; setup_tracing(); - let session = Arc::new(create_new_session_builder().build_legacy().await.unwrap()); + let session = Arc::new(create_new_session_builder().build().await.unwrap()); let ks = unique_keyspace_name(); // Need to disable tablets in this test because they don't support counters yet. // (https://github.com/scylladb/scylladb/commit/c70f321c6f581357afdf3fd8b4fe8e5c5bb9736e). let mut create_ks = format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks); - if scylla_supports_tablets_legacy(&session).await { + if scylla_supports_tablets(&session).await { create_ks += " AND TABLETS = {'enabled': false}" } @@ -426,7 +452,7 @@ async fn test_counter_batch() { #[tokio::test] async fn test_batch() { setup_tracing(); - let session = Arc::new(create_new_session_builder().build_legacy().await.unwrap()); + let session = Arc::new(create_new_session_builder().build().await.unwrap()); let ks = unique_keyspace_name(); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); @@ -480,7 +506,10 @@ async fn test_batch() { .query_unpaged(format!("SELECT a, b, c FROM {}.t_batch", ks), &[]) .await .unwrap() - .rows_typed() + .into_rows_result() + .unwrap() + .unwrap() + .rows::<(i32, i32, String)>() .unwrap() .collect::>() .unwrap(); @@ -517,7 +546,10 @@ async fn test_batch() { ) .await .unwrap() - .rows_typed() + .into_rows_result() + .unwrap() + .unwrap() + .rows::<(i32, i32, String)>() .unwrap() .collect::>() .unwrap(); @@ -528,7 +560,7 @@ async fn test_batch() { #[tokio::test] async fn test_token_calculation() { setup_tracing(); - let session = create_new_session_builder().build_legacy().await.unwrap(); + let session = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); @@ -570,7 +602,10 @@ async fn test_token_calculation() { ) .await .unwrap() - .single_row_typed() + .into_rows_result() + .unwrap() + .unwrap() + .single_row::<(i64,)>() .unwrap(); let token = Token::new(value); let prepared_token = Murmur3Partitioner @@ -587,7 +622,7 @@ async fn test_token_calculation() { #[tokio::test] async fn test_token_awareness() { setup_tracing(); - let session = create_new_session_builder().build_legacy().await.unwrap(); + let session = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); // Need to disable tablets in this test because they make token routing @@ -596,7 +631,7 @@ async fn test_token_awareness() { let mut create_ks = format!( "CREATE KEYSPACE IF NOT EXISTS {ks} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}" ); - if scylla_supports_tablets_legacy(&session).await { + if scylla_supports_tablets(&session).await { create_ks += " AND TABLETS = {'enabled': false}" } @@ -626,7 +661,7 @@ async fn test_token_awareness() { .await .unwrap(); let tracing_info = session - .get_tracing_info(res.tracing_id.as_ref().unwrap()) + .get_tracing_info(res.tracing_id().as_ref().unwrap()) .await .unwrap(); @@ -638,7 +673,7 @@ async fn test_token_awareness() { .execute_iter(prepared_statement.clone(), values) .await .unwrap(); - let tracing_id = iter.get_tracing_ids()[0]; + let tracing_id = iter.tracing_ids()[0]; let tracing_info = session.get_tracing_info(&tracing_id).await.unwrap(); // Again, verify that only one node was involved @@ -649,7 +684,7 @@ async fn test_token_awareness() { #[tokio::test] async fn test_use_keyspace() { setup_tracing(); - let session = create_new_session_builder().build_legacy().await.unwrap(); + let session = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); @@ -678,7 +713,10 @@ async fn test_use_keyspace() { .query_unpaged("SELECT * FROM tab", &[]) .await .unwrap() - .rows_typed::<(String,)>() + .into_rows_result() + .unwrap() + .unwrap() + .rows::<(String,)>() .unwrap() .map(|res| res.unwrap().0) .collect(); @@ -717,9 +755,9 @@ async fn test_use_keyspace() { )); // Make sure that use_keyspace on SessionBuiler works - let session2: LegacySession = create_new_session_builder() + let session2: Session = create_new_session_builder() .use_keyspace(ks.clone(), false) - .build_legacy() + .build() .await .unwrap(); @@ -727,7 +765,10 @@ async fn test_use_keyspace() { .query_unpaged("SELECT * FROM tab", &[]) .await .unwrap() - .rows_typed::<(String,)>() + .into_rows_result() + .unwrap() + .unwrap() + .rows::<(String,)>() .unwrap() .map(|res| res.unwrap().0) .collect(); @@ -740,7 +781,7 @@ async fn test_use_keyspace() { #[tokio::test] async fn test_use_keyspace_case_sensitivity() { setup_tracing(); - let session = create_new_session_builder().build_legacy().await.unwrap(); + let session = create_new_session_builder().build().await.unwrap(); let ks_lower = unique_keyspace_name().to_lowercase(); let ks_upper = ks_lower.to_uppercase(); @@ -787,7 +828,10 @@ async fn test_use_keyspace_case_sensitivity() { .query_unpaged("SELECT * from tab", &[]) .await .unwrap() - .rows_typed::<(String,)>() + .into_rows_result() + .unwrap() + .unwrap() + .rows::<(String,)>() .unwrap() .map(|row| row.unwrap().0) .collect(); @@ -802,7 +846,10 @@ async fn test_use_keyspace_case_sensitivity() { .query_unpaged("SELECT * from tab", &[]) .await .unwrap() - .rows_typed::<(String,)>() + .into_rows_result() + .unwrap() + .unwrap() + .rows::<(String,)>() .unwrap() .map(|row| row.unwrap().0) .collect(); @@ -813,7 +860,7 @@ async fn test_use_keyspace_case_sensitivity() { #[tokio::test] async fn test_raw_use_keyspace() { setup_tracing(); - let session = create_new_session_builder().build_legacy().await.unwrap(); + let session = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); @@ -843,7 +890,10 @@ async fn test_raw_use_keyspace() { .query_unpaged("SELECT * FROM tab", &[]) .await .unwrap() - .rows_typed::<(String,)>() + .into_rows_result() + .unwrap() + .unwrap() + .rows::<(String,)>() .unwrap() .map(|res| res.unwrap().0) .collect(); @@ -865,7 +915,7 @@ async fn test_raw_use_keyspace() { #[tokio::test] async fn test_fetch_system_keyspace() { setup_tracing(); - let session = create_new_session_builder().build_legacy().await.unwrap(); + let session = create_new_session_builder().build().await.unwrap(); let prepared_statement = session .prepare("SELECT * FROM system_schema.keyspaces") @@ -882,7 +932,7 @@ async fn test_fetch_system_keyspace() { #[tokio::test] async fn test_db_errors() { setup_tracing(); - let session = create_new_session_builder().build_legacy().await.unwrap(); + let session = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); // SyntaxError on bad query @@ -937,7 +987,7 @@ async fn test_db_errors() { #[tokio::test] async fn test_tracing() { setup_tracing(); - let session = create_new_session_builder().build_legacy().await.unwrap(); + let session = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); @@ -959,39 +1009,38 @@ async fn test_tracing() { test_tracing_batch(&session, ks.clone()).await; } -async fn test_tracing_query(session: &LegacySession, ks: String) { +async fn test_tracing_query(session: &Session, ks: String) { // A query without tracing enabled has no tracing uuid in result let untraced_query: Query = Query::new(format!("SELECT * FROM {}.tab", ks)); - let untraced_query_result: LegacyQueryResult = + let untraced_query_result: QueryResult = session.query_unpaged(untraced_query, &[]).await.unwrap(); - assert!(untraced_query_result.tracing_id.is_none()); + assert!(untraced_query_result.tracing_id().is_none()); // A query with tracing enabled has a tracing uuid in result let mut traced_query: Query = Query::new(format!("SELECT * FROM {}.tab", ks)); traced_query.config.tracing = true; - let traced_query_result: LegacyQueryResult = - session.query_unpaged(traced_query, &[]).await.unwrap(); - assert!(traced_query_result.tracing_id.is_some()); + let traced_query_result: QueryResult = session.query_unpaged(traced_query, &[]).await.unwrap(); + assert!(traced_query_result.tracing_id().is_some()); // Querying this uuid from tracing table gives some results - assert_in_tracing_table(session, traced_query_result.tracing_id.unwrap()).await; + assert_in_tracing_table(session, traced_query_result.tracing_id().unwrap()).await; } -async fn test_tracing_execute(session: &LegacySession, ks: String) { +async fn test_tracing_execute(session: &Session, ks: String) { // Executing a prepared statement without tracing enabled has no tracing uuid in result let untraced_prepared = session .prepare(format!("SELECT * FROM {}.tab", ks)) .await .unwrap(); - let untraced_prepared_result: LegacyQueryResult = session + let untraced_prepared_result: QueryResult = session .execute_unpaged(&untraced_prepared, &[]) .await .unwrap(); - assert!(untraced_prepared_result.tracing_id.is_none()); + assert!(untraced_prepared_result.tracing_id().is_none()); // Executing a prepared statement with tracing enabled has a tracing uuid in result let mut traced_prepared = session @@ -1001,17 +1050,17 @@ async fn test_tracing_execute(session: &LegacySession, ks: String) { traced_prepared.config.tracing = true; - let traced_prepared_result: LegacyQueryResult = session + let traced_prepared_result: QueryResult = session .execute_unpaged(&traced_prepared, &[]) .await .unwrap(); - assert!(traced_prepared_result.tracing_id.is_some()); + assert!(traced_prepared_result.tracing_id().is_some()); // Querying this uuid from tracing table gives some results - assert_in_tracing_table(session, traced_prepared_result.tracing_id.unwrap()).await; + assert_in_tracing_table(session, traced_prepared_result.tracing_id().unwrap()).await; } -async fn test_tracing_prepare(session: &LegacySession, ks: String) { +async fn test_tracing_prepare(session: &Session, ks: String) { // Preparing a statement without tracing enabled has no tracing uuids in result let untraced_prepared = session .prepare(format!("SELECT * FROM {}.tab", ks)) @@ -1033,14 +1082,13 @@ async fn test_tracing_prepare(session: &LegacySession, ks: String) { } } -async fn test_get_tracing_info(session: &LegacySession, ks: String) { +async fn test_get_tracing_info(session: &Session, ks: String) { // A query with tracing enabled has a tracing uuid in result let mut traced_query: Query = Query::new(format!("SELECT * FROM {}.tab", ks)); traced_query.config.tracing = true; - let traced_query_result: LegacyQueryResult = - session.query_unpaged(traced_query, &[]).await.unwrap(); - let tracing_id: Uuid = traced_query_result.tracing_id.unwrap(); + let traced_query_result: QueryResult = session.query_unpaged(traced_query, &[]).await.unwrap(); + let tracing_id: Uuid = traced_query_result.tracing_id().unwrap(); // Getting tracing info from session using this uuid works let tracing_info: TracingInfo = session.get_tracing_info(&tracing_id).await.unwrap(); @@ -1048,7 +1096,7 @@ async fn test_get_tracing_info(session: &LegacySession, ks: String) { assert!(!tracing_info.nodes().is_empty()); } -async fn test_tracing_query_iter(session: &LegacySession, ks: String) { +async fn test_tracing_query_iter(session: &Session, ks: String) { // A query without tracing enabled has no tracing ids let untraced_query: Query = Query::new(format!("SELECT * FROM {}.tab", ks)); @@ -1057,11 +1105,11 @@ async fn test_tracing_query_iter(session: &LegacySession, ks: String) { // Receive rows } - assert!(untraced_row_iter.get_tracing_ids().is_empty()); + assert!(untraced_row_iter.tracing_ids().is_empty()); // The same is true for TypedRowIter - let untraced_typed_row_iter = untraced_row_iter.into_typed::<(String,)>(); - assert!(untraced_typed_row_iter.get_tracing_ids().is_empty()); + let untraced_typed_row_iter = untraced_row_iter.into_typed::<(&str,)>().unwrap(); + assert!(untraced_typed_row_iter.tracing_ids().is_empty()); // A query with tracing enabled has a tracing ids in result let mut traced_query: Query = Query::new(format!("SELECT * FROM {}.tab", ks)); @@ -1072,18 +1120,18 @@ async fn test_tracing_query_iter(session: &LegacySession, ks: String) { // Receive rows } - assert!(!traced_row_iter.get_tracing_ids().is_empty()); + assert!(!traced_row_iter.tracing_ids().is_empty()); // The same is true for TypedRowIter - let traced_typed_row_iter = traced_row_iter.into_typed::<(String,)>(); - assert!(!traced_typed_row_iter.get_tracing_ids().is_empty()); + let traced_typed_row_iter = traced_row_iter.into_typed::<(&str,)>().unwrap(); + assert!(!traced_typed_row_iter.tracing_ids().is_empty()); - for tracing_id in traced_typed_row_iter.get_tracing_ids() { + for tracing_id in traced_typed_row_iter.tracing_ids() { assert_in_tracing_table(session, *tracing_id).await; } } -async fn test_tracing_execute_iter(session: &LegacySession, ks: String) { +async fn test_tracing_execute_iter(session: &Session, ks: String) { // A prepared statement without tracing enabled has no tracing ids let untraced_prepared = session .prepare(format!("SELECT * FROM {}.tab", ks)) @@ -1095,11 +1143,11 @@ async fn test_tracing_execute_iter(session: &LegacySession, ks: String) { // Receive rows } - assert!(untraced_row_iter.get_tracing_ids().is_empty()); + assert!(untraced_row_iter.tracing_ids().is_empty()); // The same is true for TypedRowIter - let untraced_typed_row_iter = untraced_row_iter.into_typed::<(String,)>(); - assert!(untraced_typed_row_iter.get_tracing_ids().is_empty()); + let untraced_typed_row_iter = untraced_row_iter.into_typed::<(&str,)>().unwrap(); + assert!(untraced_typed_row_iter.tracing_ids().is_empty()); // A prepared statement with tracing enabled has a tracing ids in result let mut traced_prepared = session @@ -1113,38 +1161,37 @@ async fn test_tracing_execute_iter(session: &LegacySession, ks: String) { // Receive rows } - assert!(!traced_row_iter.get_tracing_ids().is_empty()); + assert!(!traced_row_iter.tracing_ids().is_empty()); // The same is true for TypedRowIter - let traced_typed_row_iter = traced_row_iter.into_typed::<(String,)>(); - assert!(!traced_typed_row_iter.get_tracing_ids().is_empty()); + let traced_typed_row_iter = traced_row_iter.into_typed::<(&str,)>().unwrap(); + assert!(!traced_typed_row_iter.tracing_ids().is_empty()); - for tracing_id in traced_typed_row_iter.get_tracing_ids() { + for tracing_id in traced_typed_row_iter.tracing_ids() { assert_in_tracing_table(session, *tracing_id).await; } } -async fn test_tracing_batch(session: &LegacySession, ks: String) { +async fn test_tracing_batch(session: &Session, ks: String) { // A batch without tracing enabled has no tracing id let mut untraced_batch: Batch = Default::default(); untraced_batch.append_statement(&format!("INSERT INTO {}.tab (a) VALUES('a')", ks)[..]); - let untraced_batch_result: LegacyQueryResult = - session.batch(&untraced_batch, ((),)).await.unwrap(); - assert!(untraced_batch_result.tracing_id.is_none()); + let untraced_batch_result: QueryResult = session.batch(&untraced_batch, ((),)).await.unwrap(); + assert!(untraced_batch_result.tracing_id().is_none()); // Batch with tracing enabled has a tracing uuid in result let mut traced_batch: Batch = Default::default(); traced_batch.append_statement(&format!("INSERT INTO {}.tab (a) VALUES('a')", ks)[..]); traced_batch.config.tracing = true; - let traced_batch_result: LegacyQueryResult = session.batch(&traced_batch, ((),)).await.unwrap(); - assert!(traced_batch_result.tracing_id.is_some()); + let traced_batch_result: QueryResult = session.batch(&traced_batch, ((),)).await.unwrap(); + assert!(traced_batch_result.tracing_id().is_some()); - assert_in_tracing_table(session, traced_batch_result.tracing_id.unwrap()).await; + assert_in_tracing_table(session, traced_batch_result.tracing_id().unwrap()).await; } -async fn assert_in_tracing_table(session: &LegacySession, tracing_uuid: Uuid) { +async fn assert_in_tracing_table(session: &Session, tracing_uuid: Uuid) { let mut traces_query = Query::new("SELECT * FROM system_traces.sessions WHERE session_id = ?"); traces_query.config.consistency = Some(Consistency::One); @@ -1160,9 +1207,10 @@ async fn assert_in_tracing_table(session: &LegacySession, tracing_uuid: Uuid) { .query_unpaged(traces_query.clone(), (tracing_uuid,)) .await .unwrap() - .rows_num() - .unwrap(); - + .into_rows_result() + .unwrap() + .unwrap() + .rows_num(); if rows_num > 0 { // Ok there was some row for this tracing_uuid return; @@ -1179,14 +1227,14 @@ async fn assert_in_tracing_table(session: &LegacySession, tracing_uuid: Uuid) { #[tokio::test] async fn test_await_schema_agreement() { setup_tracing(); - let session = create_new_session_builder().build_legacy().await.unwrap(); + let session = create_new_session_builder().build().await.unwrap(); let _schema_version = session.await_schema_agreement().await.unwrap(); } #[tokio::test] async fn test_timestamp() { setup_tracing(); - let session = create_new_session_builder().build_legacy().await.unwrap(); + let session = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); @@ -1267,14 +1315,19 @@ async fn test_timestamp() { .await .unwrap(); - let mut results = session + let query_rows_result = session .query_unpaged( format!("SELECT a, b, WRITETIME(b) FROM {}.t_timestamp", ks), &[], ) .await .unwrap() - .rows_typed::<(String, String, i64)>() + .into_rows_result() + .unwrap() + .unwrap(); + + let mut results = query_rows_result + .rows::<(&str, &str, i64)>() .unwrap() .map(Result::unwrap) .collect::>(); @@ -1286,8 +1339,8 @@ async fn test_timestamp() { ("regular query", "higher timestamp", 420), ("second query in batch", "higher timestamp", 420), ] - .iter() - .map(|(x, y, t)| (x.to_string(), y.to_string(), *t)) + .into_iter() + .map(|(x, y, t)| (x, y, t)) .collect::>(); assert_eq!(results, expected_results); @@ -1305,7 +1358,7 @@ async fn test_request_timeout() { .into_handle(); { - let session = create_new_session_builder().build_legacy().await.unwrap(); + let session = create_new_session_builder().build().await.unwrap(); let mut query: Query = Query::new("SELECT * FROM system_schema.tables"); query.set_request_timeout(Some(Duration::from_millis(1))); @@ -1328,7 +1381,7 @@ async fn test_request_timeout() { { let timeouting_session = create_new_session_builder() .default_execution_profile_handle(fast_timeouting_profile_handle) - .build_legacy() + .build() .await .unwrap(); @@ -1364,7 +1417,7 @@ async fn test_request_timeout() { #[tokio::test] async fn test_prepared_config() { setup_tracing(); - let session = create_new_session_builder().build_legacy().await.unwrap(); + let session = create_new_session_builder().build().await.unwrap(); let mut query = Query::new("SELECT * FROM system_schema.tables"); query.set_is_idempotent(true); @@ -1451,7 +1504,7 @@ fn udt_type_c_def(ks: &str) -> Arc { #[tokio::test] async fn test_schema_types_in_metadata() { setup_tracing(); - let session = create_new_session_builder().build_legacy().await.unwrap(); + let session = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); session @@ -1610,7 +1663,7 @@ async fn test_schema_types_in_metadata() { #[tokio::test] async fn test_user_defined_types_in_metadata() { setup_tracing(); - let session = create_new_session_builder().build_legacy().await.unwrap(); + let session = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); session @@ -1674,7 +1727,7 @@ async fn test_user_defined_types_in_metadata() { #[tokio::test] async fn test_column_kinds_in_metadata() { setup_tracing(); - let session = create_new_session_builder().build_legacy().await.unwrap(); + let session = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); session @@ -1720,7 +1773,7 @@ async fn test_column_kinds_in_metadata() { #[tokio::test] async fn test_primary_key_ordering_in_metadata() { setup_tracing(); - let session = create_new_session_builder().build_legacy().await.unwrap(); + let session = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); session @@ -1769,14 +1822,14 @@ async fn test_table_partitioner_in_metadata() { return; } - let session = create_new_session_builder().build_legacy().await.unwrap(); + let session = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); // This test uses CDC which is not yet compatible with Scylla's tablets. let mut create_ks = format!( "CREATE KEYSPACE {ks} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}" ); - if scylla_supports_tablets_legacy(&session).await { + if scylla_supports_tablets(&session).await { create_ks += " AND TABLETS = {'enabled': false}"; } @@ -1815,7 +1868,7 @@ async fn test_turning_off_schema_fetching() { setup_tracing(); let session = create_new_session_builder() .fetch_schema_metadata(false) - .build_legacy() + .build() .await .unwrap(); let ks = unique_keyspace_name(); @@ -1891,7 +1944,7 @@ async fn test_turning_off_schema_fetching() { #[tokio::test] async fn test_named_bind_markers() { - let session = create_new_session_builder().build_legacy().await.unwrap(); + let session = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); session @@ -1928,7 +1981,10 @@ async fn test_named_bind_markers() { .query_unpaged("SELECT pk, ck, v FROM t", &[]) .await .unwrap() - .rows_typed::<(i32, i32, i32)>() + .into_rows_result() + .unwrap() + .unwrap() + .rows::<(i32, i32, i32)>() .unwrap() .map(|res| res.unwrap()) .collect(); @@ -1948,13 +2004,13 @@ async fn test_named_bind_markers() { #[tokio::test] async fn test_prepared_partitioner() { - let session = create_new_session_builder().build_legacy().await.unwrap(); + let session = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); // This test uses CDC which is not yet compatible with Scylla's tablets. let mut create_ks = format!( "CREATE KEYSPACE IF NOT EXISTS {ks} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}"); - if scylla_supports_tablets_legacy(&session).await { + if scylla_supports_tablets(&session).await { create_ks += " AND TABLETS = {'enabled': false}" } @@ -2005,14 +2061,14 @@ async fn test_prepared_partitioner() { ); } -async fn rename(session: &LegacySession, rename_str: &str) { +async fn rename(session: &Session, rename_str: &str) { session .query_unpaged(format!("ALTER TABLE tab RENAME {}", rename_str), ()) .await .unwrap(); } -async fn rename_caching(session: &LegacyCachingSession, rename_str: &str) { +async fn rename_caching(session: &CachingSession, rename_str: &str) { session .execute_unpaged(format!("ALTER TABLE tab RENAME {}", rename_str), &()) .await @@ -2030,7 +2086,7 @@ async fn rename_caching(session: &LegacyCachingSession, rename_str: &str) { async fn test_unprepared_reprepare_in_execute() { let _ = tracing_subscriber::fmt::try_init(); - let session = create_new_session_builder().build_legacy().await.unwrap(); + let session = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); @@ -2079,7 +2135,10 @@ async fn test_unprepared_reprepare_in_execute() { .query_unpaged("SELECT a, b, c FROM tab", ()) .await .unwrap() - .rows_typed::<(i32, i32, i32)>() + .into_rows_result() + .unwrap() + .unwrap() + .rows::<(i32, i32, i32)>() .unwrap() .map(|r| r.unwrap()) .collect(); @@ -2091,7 +2150,7 @@ async fn test_unprepared_reprepare_in_execute() { async fn test_unusual_valuelists() { let _ = tracing_subscriber::fmt::try_init(); - let session = create_new_session_builder().build_legacy().await.unwrap(); + let session = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); @@ -2134,7 +2193,10 @@ async fn test_unusual_valuelists() { .query_unpaged("SELECT a, b, c FROM tab", ()) .await .unwrap() - .rows_typed::<(i32, i32, String)>() + .into_rows_result() + .unwrap() + .unwrap() + .rows::<(i32, i32, String)>() .unwrap() .map(|r| r.unwrap()) .collect(); @@ -2159,7 +2221,7 @@ async fn test_unusual_valuelists() { async fn test_unprepared_reprepare_in_batch() { let _ = tracing_subscriber::fmt::try_init(); - let session = create_new_session_builder().build_legacy().await.unwrap(); + let session = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); @@ -2205,7 +2267,10 @@ async fn test_unprepared_reprepare_in_batch() { .query_unpaged("SELECT a, b, c FROM tab", ()) .await .unwrap() - .rows_typed::<(i32, i32, i32)>() + .into_rows_result() + .unwrap() + .unwrap() + .rows::<(i32, i32, i32)>() .unwrap() .map(|r| r.unwrap()) .collect(); @@ -2224,13 +2289,13 @@ async fn test_unprepared_reprepare_in_batch() { async fn test_unprepared_reprepare_in_caching_session_execute() { let _ = tracing_subscriber::fmt::try_init(); - let session = create_new_session_builder().build_legacy().await.unwrap(); + let session = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); session.use_keyspace(ks, false).await.unwrap(); - let caching_session: LegacyCachingSession = LegacyCachingSession::from(session, 64); + let caching_session: CachingSession = CachingSession::from(session, 64); caching_session .execute_unpaged( @@ -2272,7 +2337,10 @@ async fn test_unprepared_reprepare_in_caching_session_execute() { .execute_unpaged("SELECT a, b, c FROM tab", &()) .await .unwrap() - .rows_typed::<(i32, i32, i32)>() + .into_rows_result() + .unwrap() + .unwrap() + .rows::<(i32, i32, i32)>() .unwrap() .map(|r| r.unwrap()) .collect(); @@ -2284,7 +2352,7 @@ async fn test_unprepared_reprepare_in_caching_session_execute() { async fn test_views_in_schema_info() { let _ = tracing_subscriber::fmt::try_init(); - let session = create_new_session_builder().build_legacy().await.unwrap(); + let session = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); @@ -2334,12 +2402,15 @@ async fn test_views_in_schema_info() { ) } -async fn assert_test_batch_table_rows_contain(sess: &LegacySession, expected_rows: &[(i32, i32)]) { +async fn assert_test_batch_table_rows_contain(sess: &Session, expected_rows: &[(i32, i32)]) { let selected_rows: BTreeSet<(i32, i32)> = sess .query_unpaged("SELECT a, b FROM test_batch_table", ()) .await .unwrap() - .rows_typed::<(i32, i32)>() + .into_rows_result() + .unwrap() + .unwrap() + .rows::<(i32, i32)>() .unwrap() .map(|r| r.unwrap()) .collect(); @@ -2355,7 +2426,7 @@ async fn assert_test_batch_table_rows_contain(sess: &LegacySession, expected_row #[tokio::test] async fn test_prepare_batch() { - let session = create_new_session_builder().build_legacy().await.unwrap(); + let session = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); @@ -2452,7 +2523,7 @@ async fn test_prepare_batch() { #[tokio::test] async fn test_refresh_metadata_after_schema_agreement() { - let session = create_new_session_builder().build_legacy().await.unwrap(); + let session = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); @@ -2490,7 +2561,7 @@ async fn test_refresh_metadata_after_schema_agreement() { #[tokio::test] async fn test_rate_limit_exceeded_exception() { - let session = create_new_session_builder().build_legacy().await.unwrap(); + let session = create_new_session_builder().build().await.unwrap(); // Typed errors in RPC were introduced along with per-partition rate limiting. // There is no dedicated feature for per-partition rate limiting, so we are @@ -2538,11 +2609,11 @@ async fn test_rate_limit_exceeded_exception() { // Batches containing LWT queries (IF col = som) return rows with information whether the queries were applied. #[tokio::test] async fn test_batch_lwts() { - let session = create_new_session_builder().build_legacy().await.unwrap(); + let session = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); let mut create_ks = format!("CREATE KEYSPACE {} WITH REPLICATION = {{'class': 'NetworkTopologyStrategy', 'replication_factor': 1}}", ks); - if scylla_supports_tablets_legacy(&session).await { + if scylla_supports_tablets(&session).await { create_ks += " and TABLETS = { 'enabled': false}"; } session.query_unpaged(create_ks, &[]).await.unwrap(); @@ -2566,33 +2637,34 @@ async fn test_batch_lwts() { batch.append_statement("INSERT INTO tab (p1, c1, r1, r2) VALUES (0, 123, 321, 312)"); batch.append_statement("UPDATE tab SET r1 = 1 WHERE p1 = 0 AND c1 = 0 IF r2 = 0"); - let batch_res: LegacyQueryResult = session.batch(&batch, ((), (), ())).await.unwrap(); + let batch_res: QueryResult = session.batch(&batch, ((), (), ())).await.unwrap(); + let batch_deserializer = batch_res.into_rows_result().unwrap().unwrap(); // Scylla returns 5 columns, but Cassandra returns only 1 - let is_scylla: bool = batch_res.col_specs().len() == 5; + let is_scylla: bool = batch_deserializer.column_specs().len() == 5; if is_scylla { - test_batch_lwts_for_scylla(&session, &batch, batch_res).await; + test_batch_lwts_for_scylla(&session, &batch, &batch_deserializer).await; } else { - test_batch_lwts_for_cassandra(&session, &batch, batch_res).await; + test_batch_lwts_for_cassandra(&session, &batch, &batch_deserializer).await; } } async fn test_batch_lwts_for_scylla( - session: &LegacySession, + session: &Session, batch: &Batch, - batch_res: LegacyQueryResult, + query_rows_result: &QueryRowsResult, ) { // Alias required by clippy type IntOrNull = Option; // Returned columns are: // [applied], p1, c1, r1, r2 - let batch_res_rows: Vec<(bool, IntOrNull, IntOrNull, IntOrNull, IntOrNull)> = batch_res - .rows_typed() + let batch_res_rows: Vec<(bool, IntOrNull, IntOrNull, IntOrNull, IntOrNull)> = query_rows_result + .rows() .unwrap() - .map(|r| r.unwrap()) - .collect(); + .collect::>() + .unwrap(); let expected_batch_res_rows = vec![ (true, Some(0), Some(0), Some(0), Some(0)), @@ -2603,12 +2675,15 @@ async fn test_batch_lwts_for_scylla( assert_eq!(batch_res_rows, expected_batch_res_rows); let prepared_batch: Batch = session.prepare_batch(batch).await.unwrap(); - let prepared_batch_res: LegacyQueryResult = + let prepared_batch_res: QueryResult = session.batch(&prepared_batch, ((), (), ())).await.unwrap(); let prepared_batch_res_rows: Vec<(bool, IntOrNull, IntOrNull, IntOrNull, IntOrNull)> = prepared_batch_res - .rows_typed() + .into_rows_result() + .unwrap() + .unwrap() + .rows() .unwrap() .map(|r| r.unwrap()) .collect(); @@ -2623,17 +2698,17 @@ async fn test_batch_lwts_for_scylla( } async fn test_batch_lwts_for_cassandra( - session: &LegacySession, + session: &Session, batch: &Batch, - batch_res: LegacyQueryResult, + query_rows_result: &QueryRowsResult, ) { // Alias required by clippy type IntOrNull = Option; // Returned columns are: // [applied] - let batch_res_rows: Vec<(bool,)> = batch_res - .rows_typed() + let batch_res_rows: Vec<(bool,)> = query_rows_result + .rows() .unwrap() .map(|r| r.unwrap()) .collect(); @@ -2643,14 +2718,17 @@ async fn test_batch_lwts_for_cassandra( assert_eq!(batch_res_rows, expected_batch_res_rows); let prepared_batch: Batch = session.prepare_batch(batch).await.unwrap(); - let prepared_batch_res: LegacyQueryResult = + let prepared_batch_res: QueryResult = session.batch(&prepared_batch, ((), (), ())).await.unwrap(); // Returned columns are: // [applied], p1, c1, r1, r2 let prepared_batch_res_rows: Vec<(bool, IntOrNull, IntOrNull, IntOrNull, IntOrNull)> = prepared_batch_res - .rows_typed() + .into_rows_result() + .unwrap() + .unwrap() + .rows() .unwrap() .map(|r| r.unwrap()) .collect(); @@ -2665,7 +2743,7 @@ async fn test_keyspaces_to_fetch() { let ks1 = unique_keyspace_name(); let ks2 = unique_keyspace_name(); - let session_default = create_new_session_builder().build_legacy().await.unwrap(); + let session_default = create_new_session_builder().build().await.unwrap(); for ks in [&ks1, &ks2] { session_default .query_unpaged(format!("CREATE KEYSPACE {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]) @@ -2684,7 +2762,7 @@ async fn test_keyspaces_to_fetch() { let session1 = create_new_session_builder() .keyspaces_to_fetch([&ks1]) - .build_legacy() + .build() .await .unwrap(); assert!(session1.get_cluster_data().keyspaces.contains_key(&ks1)); @@ -2692,7 +2770,7 @@ async fn test_keyspaces_to_fetch() { let session_all = create_new_session_builder() .keyspaces_to_fetch([] as [String; 0]) - .build_legacy() + .build() .await .unwrap(); assert!(session_all.get_cluster_data().keyspaces.contains_key(&ks1)); @@ -2734,7 +2812,7 @@ async fn test_iter_works_when_retry_policy_returns_ignore_write_error() { let session = create_new_session_builder() .default_execution_profile_handle(handle) - .build_legacy() + .build() .await .unwrap(); @@ -2742,7 +2820,7 @@ async fn test_iter_works_when_retry_policy_returns_ignore_write_error() { let cluster_size = session.get_cluster_data().get_nodes_info().len(); let ks = unique_keyspace_name(); let mut create_ks = format!("CREATE KEYSPACE {} WITH REPLICATION = {{'class': 'NetworkTopologyStrategy', 'replication_factor': {}}}", ks, cluster_size + 1); - if scylla_supports_tablets_legacy(&session).await { + if scylla_supports_tablets(&session).await { create_ks += " and TABLETS = { 'enabled': false}"; } session.query_unpaged(create_ks, ()).await.unwrap(); @@ -2758,7 +2836,10 @@ async fn test_iter_works_when_retry_policy_returns_ignore_write_error() { let mut iter = session .query_iter("INSERT INTO t (pk v) VALUES (1, 2)", ()) .await - .unwrap(); + .unwrap() + .into_typed::() + .unwrap() + .into_stream(); assert!(retried_flag.load(Ordering::Relaxed)); while iter.try_next().await.unwrap().is_some() {} @@ -2769,7 +2850,13 @@ async fn test_iter_works_when_retry_policy_returns_ignore_write_error() { .prepare("INSERT INTO t (pk, v) VALUES (?, ?)") .await .unwrap(); - let mut iter = session.execute_iter(p, (1, 2)).await.unwrap(); + let mut iter = session + .execute_iter(p, (1, 2)) + .await + .unwrap() + .into_typed::() + .unwrap() + .into_stream(); assert!(retried_flag.load(Ordering::Relaxed)); while iter.try_next().await.unwrap().is_some() {} @@ -2777,7 +2864,7 @@ async fn test_iter_works_when_retry_policy_returns_ignore_write_error() { #[tokio::test] async fn test_iter_methods_with_modification_statements() { - let session = create_new_session_builder().build_legacy().await.unwrap(); + let session = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); @@ -2799,7 +2886,7 @@ async fn test_iter_methods_with_modification_statements() { query.set_tracing(true); let mut row_iterator = session.query_iter(query, &[]).await.unwrap(); row_iterator.next().await.ok_or(()).unwrap_err(); // assert empty - assert!(!row_iterator.get_tracing_ids().is_empty()); + assert!(!row_iterator.tracing_ids().is_empty()); let prepared_statement = session .prepare(format!("INSERT INTO {}.t (a, b, c) VALUES (?, ?, ?)", ks)) @@ -2818,7 +2905,7 @@ async fn test_get_keyspace_name() { // Create the keyspace // No keyspace is set in config, so get_keyspace() should return None. - let session = create_new_session_builder().build_legacy().await.unwrap(); + let session = create_new_session_builder().build().await.unwrap(); assert_eq!(session.get_keyspace(), None); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); assert_eq!(session.get_keyspace(), None); @@ -2843,7 +2930,7 @@ async fn test_get_keyspace_name() { #[tokio::test] async fn simple_strategy_test() { let ks = unique_keyspace_name(); - let session = create_new_session_builder().build_legacy().await.unwrap(); + let session = create_new_session_builder().build().await.unwrap(); session .query_unpaged( @@ -2895,7 +2982,10 @@ async fn simple_strategy_test() { .query_unpaged(format!("SELECT p, c, r FROM {}.tab", ks), ()) .await .unwrap() - .rows_typed::<(i32, i32, i32)>() + .into_rows_result() + .unwrap() + .unwrap() + .rows::<(i32, i32, i32)>() .unwrap() .map(|r| r.unwrap()) .collect::>(); @@ -2908,12 +2998,12 @@ async fn simple_strategy_test() { async fn test_manual_primary_key_computation() { // Setup session let ks = unique_keyspace_name(); - let session = create_new_session_builder().build_legacy().await.unwrap(); + let session = create_new_session_builder().build().await.unwrap(); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); session.use_keyspace(&ks, true).await.unwrap(); async fn assert_tokens_equal( - session: &LegacySession, + session: &Session, prepared: &PreparedStatement, serialized_pk_values_in_pk_order: &SerializedValues, all_values_in_query_order: impl SerializeRow, diff --git a/scylla/src/transport/silent_prepare_batch_test.rs b/scylla/src/transport/silent_prepare_batch_test.rs index c7ae8e83d..48c0dc1f1 100644 --- a/scylla/src/transport/silent_prepare_batch_test.rs +++ b/scylla/src/transport/silent_prepare_batch_test.rs @@ -2,14 +2,14 @@ use crate::{ batch::Batch, prepared_statement::PreparedStatement, test_utils::{create_new_session_builder, setup_tracing, unique_keyspace_name}, - LegacySession, + Session, }; use std::collections::BTreeSet; #[tokio::test] async fn test_quietly_prepare_batch() { setup_tracing(); - let session = create_new_session_builder().build_legacy().await.unwrap(); + let session = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); @@ -91,12 +91,15 @@ async fn test_quietly_prepare_batch() { } } -async fn assert_test_batch_table_rows_contain(sess: &LegacySession, expected_rows: &[(i32, i32)]) { +async fn assert_test_batch_table_rows_contain(sess: &Session, expected_rows: &[(i32, i32)]) { let selected_rows: BTreeSet<(i32, i32)> = sess .query_unpaged("SELECT a, b FROM test_batch_table", ()) .await .unwrap() - .rows_typed::<(i32, i32)>() + .into_rows_result() + .unwrap() + .unwrap() + .rows::<(i32, i32)>() .unwrap() .map(|r| r.unwrap()) .collect(); diff --git a/scylla/src/utils/test_utils.rs b/scylla/src/utils/test_utils.rs index 37390291d..a04032e39 100644 --- a/scylla/src/utils/test_utils.rs +++ b/scylla/src/utils/test_utils.rs @@ -2,7 +2,7 @@ use scylla_cql::frame::response::result::Row; #[cfg(test)] use crate::transport::session_builder::{GenericSessionBuilder, SessionBuilderKind}; -use crate::{LegacySession, Session}; +use crate::Session; #[cfg(test)] use std::{num::NonZeroU32, time::Duration}; use std::{ @@ -27,7 +27,7 @@ pub fn unique_keyspace_name() -> String { } #[cfg(test)] -pub(crate) async fn supports_feature(session: &LegacySession, feature: &str) -> bool { +pub(crate) async fn supports_feature(session: &Session, feature: &str) -> bool { // Cassandra doesn't have a concept of features, so first detect // if there is the `supported_features` column in system.local @@ -48,7 +48,10 @@ pub(crate) async fn supports_feature(session: &LegacySession, feature: &str) -> .query_unpaged("SELECT supported_features FROM system.local", ()) .await .unwrap() - .single_row_typed() + .into_rows_result() + .unwrap() + .unwrap() + .single_row() .unwrap(); features @@ -94,20 +97,6 @@ pub fn create_new_session_builder() -> GenericSessionBuilder bool { - let result = session - .query_unpaged( - "select column_name from system_schema.columns where - keyspace_name = 'system_schema' - and table_name = 'scylla_keyspaces' - and column_name = 'initial_tablets'", - &[], - ) - .await - .unwrap(); - result.single_row().is_ok() -} - pub async fn scylla_supports_tablets(session: &Session) -> bool { let result = session .query_unpaged( @@ -119,7 +108,7 @@ pub async fn scylla_supports_tablets(session: &Session) -> bool { ) .await .unwrap(); - result.rows_deserializer().map_or(false, |opt| { + result.into_rows_result().map_or(false, |opt| { opt.map_or(false, |deserializer| { deserializer.single_row::().is_ok() }) diff --git a/scylla/tests/integration/consistency.rs b/scylla/tests/integration/consistency.rs index 5531973ec..09780066a 100644 --- a/scylla/tests/integration/consistency.rs +++ b/scylla/tests/integration/consistency.rs @@ -6,8 +6,8 @@ use scylla::prepared_statement::PreparedStatement; use scylla::retry_policy::FallthroughRetryPolicy; use scylla::routing::{Shard, Token}; use scylla::test_utils::unique_keyspace_name; -use scylla::transport::session::LegacySession; use scylla::transport::NodeRef; +use scylla::Session; use scylla_cql::frame::response::result::TableSpec; use tokio::sync::mpsc::{self, UnboundedReceiver, UnboundedSender}; @@ -59,7 +59,7 @@ fn pairs_of_all_consistencies() -> impl Iterator, @@ -81,7 +81,7 @@ async fn query_consistency_set_directly( } async fn execute_consistency_set_directly( - session: &LegacySession, + session: &Session, prepared: &PreparedStatement, c: Consistency, sc: Option, @@ -94,7 +94,7 @@ async fn execute_consistency_set_directly( } async fn batch_consistency_set_directly( - session: &LegacySession, + session: &Session, batch: &Batch, c: Consistency, sc: Option, @@ -107,7 +107,7 @@ async fn batch_consistency_set_directly( // The following functions perform a request with consistencies set on a per-statement execution profile. async fn query_consistency_set_on_exec_profile( - session: &LegacySession, + session: &Session, query: &Query, profile: ExecutionProfileHandle, ) { @@ -118,7 +118,7 @@ async fn query_consistency_set_on_exec_profile( } async fn execute_consistency_set_on_exec_profile( - session: &LegacySession, + session: &Session, prepared: &PreparedStatement, profile: ExecutionProfileHandle, ) { @@ -129,7 +129,7 @@ async fn execute_consistency_set_on_exec_profile( } async fn batch_consistency_set_on_exec_profile( - session: &LegacySession, + session: &Session, batch: &Batch, profile: ExecutionProfileHandle, ) { @@ -159,7 +159,7 @@ async fn check_for_all_consistencies_and_setting_options< let session = session_builder .clone() .default_execution_profile_handle(base_for_every_profile.clone().build().into_handle()) - .build_legacy() + .build() .await .unwrap(); create_schema(&session, ks).await; @@ -212,7 +212,7 @@ async fn check_for_all_consistencies_and_setting_options< let session_with_consistencies = session_builder .clone() .default_execution_profile_handle(handle) - .build_legacy() + .build() .await .unwrap(); session_with_consistencies @@ -473,7 +473,7 @@ async fn consistency_allows_for_paxos_selects() { let session = SessionBuilder::new() .known_node(uri.as_str()) - .build_legacy() + .build() .await .unwrap(); diff --git a/scylla/tests/integration/execution_profiles.rs b/scylla/tests/integration/execution_profiles.rs index 46ca7c7a2..0a49bae78 100644 --- a/scylla/tests/integration/execution_profiles.rs +++ b/scylla/tests/integration/execution_profiles.rs @@ -159,7 +159,7 @@ async fn test_execution_profiles() { .known_node(proxy_uris[0].as_str()) .address_translator(Arc::new(translation_map)) .default_execution_profile_handle(profile1.into_handle()) - .build_legacy() + .build() .await .unwrap(); let ks = unique_keyspace_name(); diff --git a/scylla/tests/integration/lwt_optimisation.rs b/scylla/tests/integration/lwt_optimisation.rs index 508c953bd..ca56cff93 100644 --- a/scylla/tests/integration/lwt_optimisation.rs +++ b/scylla/tests/integration/lwt_optimisation.rs @@ -1,8 +1,8 @@ use crate::utils::{setup_tracing, test_with_3_node_cluster}; use scylla::retry_policy::FallthroughRetryPolicy; -use scylla::test_utils::scylla_supports_tablets_legacy; +use scylla::test_utils::scylla_supports_tablets; use scylla::test_utils::unique_keyspace_name; -use scylla::transport::session::LegacySession; +use scylla::transport::session::Session; use scylla::{ExecutionProfile, SessionBuilder}; use scylla_cql::frame::protocol_features::ProtocolFeatures; use scylla_cql::frame::types; @@ -52,11 +52,11 @@ async fn if_lwt_optimisation_mark_offered_then_negotiatied_and_lwt_routed_optima .into_handle(); // DB preparation phase - let session: LegacySession = SessionBuilder::new() + let session: Session = SessionBuilder::new() .known_node(proxy_uris[0].as_str()) .default_execution_profile_handle(handle) .address_translator(Arc::new(translation_map)) - .build_legacy() + .build() .await .unwrap(); @@ -70,7 +70,7 @@ async fn if_lwt_optimisation_mark_offered_then_negotiatied_and_lwt_routed_optima // Create schema let ks = unique_keyspace_name(); let mut create_ks = format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 3}}", ks); - if scylla_supports_tablets_legacy(&session).await { + if scylla_supports_tablets(&session).await { create_ks += " and TABLETS = { 'enabled': false}"; } session.query_unpaged(create_ks, &[]).await.unwrap(); diff --git a/scylla/tests/integration/new_session.rs b/scylla/tests/integration/new_session.rs index d28fa1d7a..6f734f0da 100644 --- a/scylla/tests/integration/new_session.rs +++ b/scylla/tests/integration/new_session.rs @@ -16,7 +16,7 @@ async fn proceed_if_only_some_hostnames_are_invalid() { let session = SessionBuilder::new() .known_nodes([uri1, uri2, uri3]) - .build_legacy() + .build() .await .unwrap(); session diff --git a/scylla/tests/integration/retries.rs b/scylla/tests/integration/retries.rs index e8bd5477c..43cbf5807 100644 --- a/scylla/tests/integration/retries.rs +++ b/scylla/tests/integration/retries.rs @@ -1,7 +1,7 @@ use crate::utils::{setup_tracing, test_with_3_node_cluster}; use scylla::retry_policy::FallthroughRetryPolicy; use scylla::speculative_execution::SimpleSpeculativeExecutionPolicy; -use scylla::transport::session::LegacySession; +use scylla::transport::session::Session; use scylla::ExecutionProfile; use scylla::SessionBuilder; use scylla::{query::Query, test_utils::unique_keyspace_name}; @@ -27,11 +27,11 @@ async fn speculative_execution_is_fired() { max_retry_count: 2, retry_interval: Duration::from_millis(10), }))).retry_policy(Arc::new(FallthroughRetryPolicy)).build(); - let session: LegacySession = SessionBuilder::new() + let session: Session = SessionBuilder::new() .known_node(proxy_uris[0].as_str()) .default_execution_profile_handle(simple_speculative_no_retry_profile.into_handle()) .address_translator(Arc::new(translation_map)) - .build_legacy() + .build() .await .unwrap(); @@ -104,10 +104,10 @@ async fn retries_occur() { let res = test_with_3_node_cluster(ShardAwareness::QueryNode, |proxy_uris, translation_map, mut running_proxy| async move { // DB preparation phase - let session: LegacySession = SessionBuilder::new() + let session: Session = SessionBuilder::new() .known_node(proxy_uris[0].as_str()) .address_translator(Arc::new(translation_map)) - .build_legacy() + .build() .await .unwrap(); @@ -183,11 +183,11 @@ async fn speculative_execution_panic_regression_test() { .retry_policy(Arc::new(FallthroughRetryPolicy)) .build(); // DB preparation phase - let session: LegacySession = SessionBuilder::new() + let session: Session = SessionBuilder::new() .known_node(proxy_uris[0].as_str()) .address_translator(Arc::new(translation_map)) .default_execution_profile_handle(profile.into_handle()) - .build_legacy() + .build() .await .unwrap(); diff --git a/scylla/tests/integration/self_identity.rs b/scylla/tests/integration/self_identity.rs index e7378c6e2..cba46f717 100644 --- a/scylla/tests/integration/self_identity.rs +++ b/scylla/tests/integration/self_identity.rs @@ -1,5 +1,5 @@ use crate::utils::{setup_tracing, test_with_3_node_cluster}; -use scylla::{LegacySession, SessionBuilder}; +use scylla::{Session, SessionBuilder}; use scylla_cql::frame::request::options; use scylla_cql::frame::types; use std::sync::Arc; @@ -50,11 +50,11 @@ async fn test_given_self_identity(self_identity: SelfIdentity<'static>) { )])); // DB preparation phase - let _session: LegacySession = SessionBuilder::new() + let _session: Session = SessionBuilder::new() .known_node(proxy_uris[0].as_str()) .address_translator(Arc::new(translation_map)) .custom_identity(self_identity.clone()) - .build_legacy() + .build() .await .unwrap(); diff --git a/scylla/tests/integration/shards.rs b/scylla/tests/integration/shards.rs index c3db91ca2..b22cfc397 100644 --- a/scylla/tests/integration/shards.rs +++ b/scylla/tests/integration/shards.rs @@ -1,7 +1,7 @@ use std::sync::Arc; use crate::utils::{setup_tracing, test_with_3_node_cluster}; -use scylla::test_utils::scylla_supports_tablets_legacy; +use scylla::test_utils::scylla_supports_tablets; use scylla::{test_utils::unique_keyspace_name, SessionBuilder}; use tokio::sync::mpsc; @@ -32,14 +32,14 @@ async fn test_consistent_shard_awareness() { let session = SessionBuilder::new() .known_node(proxy_uris[0].as_str()) .address_translator(Arc::new(translation_map)) - .build_legacy() + .build() .await .unwrap(); let ks = unique_keyspace_name(); /* Prepare schema */ let mut create_ks = format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 3}}", ks); - if scylla_supports_tablets_legacy(&session).await { + if scylla_supports_tablets(&session).await { create_ks += " and TABLETS = { 'enabled': false}"; } session.query_unpaged(create_ks, &[]).await.unwrap(); diff --git a/scylla/tests/integration/silent_prepare_query.rs b/scylla/tests/integration/silent_prepare_query.rs index 64da40406..93950206a 100644 --- a/scylla/tests/integration/silent_prepare_query.rs +++ b/scylla/tests/integration/silent_prepare_query.rs @@ -1,5 +1,5 @@ use crate::utils::{setup_tracing, test_with_3_node_cluster}; -use scylla::transport::session::LegacySession; +use scylla::Session; use scylla::SessionBuilder; use scylla::{query::Query, test_utils::unique_keyspace_name}; use scylla_proxy::{ @@ -19,10 +19,10 @@ async fn test_prepare_query_with_values() { let res = test_with_3_node_cluster(ShardAwareness::QueryNode, |proxy_uris, translation_map, mut running_proxy| async move { // DB preparation phase - let session: LegacySession = SessionBuilder::new() + let session: Session = SessionBuilder::new() .known_node(proxy_uris[0].as_str()) .address_translator(Arc::new(translation_map)) - .build_legacy() + .build() .await .unwrap(); @@ -70,10 +70,10 @@ async fn test_query_with_no_values() { let res = test_with_3_node_cluster(ShardAwareness::QueryNode, |proxy_uris, translation_map, mut running_proxy| async move { // DB preparation phase - let session: LegacySession = SessionBuilder::new() + let session: Session = SessionBuilder::new() .known_node(proxy_uris[0].as_str()) .address_translator(Arc::new(translation_map)) - .build_legacy() + .build() .await .unwrap(); diff --git a/scylla/tests/integration/skip_metadata_optimization.rs b/scylla/tests/integration/skip_metadata_optimization.rs index a50e33b8b..17f595400 100644 --- a/scylla/tests/integration/skip_metadata_optimization.rs +++ b/scylla/tests/integration/skip_metadata_optimization.rs @@ -1,7 +1,6 @@ use crate::utils::{setup_tracing, test_with_3_node_cluster}; -use scylla::transport::session::LegacySession; -use scylla::SessionBuilder; use scylla::{prepared_statement::PreparedStatement, test_utils::unique_keyspace_name}; +use scylla::{Session, SessionBuilder}; use scylla_cql::frame::request::query::{PagingState, PagingStateResponse}; use scylla_cql::frame::types; use scylla_proxy::{ @@ -20,10 +19,10 @@ async fn test_skip_result_metadata() { let res = test_with_3_node_cluster(ShardAwareness::QueryNode, |proxy_uris, translation_map, mut running_proxy| async move { // DB preparation phase - let session: LegacySession = SessionBuilder::new() + let session: Session = SessionBuilder::new() .known_node(proxy_uris[0].as_str()) .address_translator(Arc::new(translation_map)) - .build_legacy() + .build() .await .unwrap(); @@ -51,7 +50,7 @@ async fn test_skip_result_metadata() { } async fn test_with_flags_predicate( - session: &LegacySession, + session: &Session, prepared: &PreparedStatement, rx: &mut tokio::sync::mpsc::UnboundedReceiver<(ResponseFrame, Option)>, predicate: impl FnOnce(i32) -> bool @@ -114,7 +113,10 @@ async fn test_skip_result_metadata() { .query_unpaged(select_query, ()) .await .unwrap() - .rows_typed::() + .into_rows_result() + .unwrap() + .unwrap() + .rows::() .unwrap() .collect::, _>>() .unwrap(); @@ -130,8 +132,14 @@ async fn test_skip_result_metadata() { .execute_single_page(&prepared_paged, &[], paging_state) .await .unwrap(); - results_from_manual_paging - .extend(rs_manual.rows_typed::().unwrap().map(Result::unwrap)); + results_from_manual_paging.extend( + rs_manual.into_rows_result() + .unwrap() + .unwrap() + .rows::() + .unwrap() + .map(Result::unwrap) + ); match paging_state_response { PagingStateResponse::HasMorePages { state } => { diff --git a/scylla/tests/integration/tablets.rs b/scylla/tests/integration/tablets.rs index b570f0548..a65d056d9 100644 --- a/scylla/tests/integration/tablets.rs +++ b/scylla/tests/integration/tablets.rs @@ -16,7 +16,7 @@ use scylla::test_utils::unique_keyspace_name; use scylla::transport::ClusterData; use scylla::transport::Node; use scylla::transport::NodeRef; -use scylla::{ExecutionProfile, LegacyQueryResult, LegacySession}; +use scylla::{ExecutionProfile, QueryResult, Session}; use scylla::transport::errors::QueryError; use scylla_proxy::{ @@ -28,7 +28,7 @@ use tokio::sync::mpsc; use tracing::info; use uuid::Uuid; -#[derive(scylla::FromRow)] +#[derive(scylla::DeserializeRow)] struct SelectedTablet { last_token: i64, replicas: Vec<(Uuid, i32)>, @@ -40,7 +40,7 @@ struct Tablet { replicas: Vec<(Arc, i32)>, } -async fn get_tablets(session: &LegacySession, ks: &str, table: &str) -> Vec { +async fn get_tablets(session: &Session, ks: &str, table: &str) -> Vec { let cluster_data = session.get_cluster_data(); let endpoints = cluster_data.get_nodes_info(); for endpoint in endpoints.iter() { @@ -55,8 +55,10 @@ async fn get_tablets(session: &LegacySession, ks: &str, table: &str) -> Vec = selected_tablets_response .into_typed::() + .unwrap() + .into_stream() .try_collect::>() .await .unwrap(); @@ -179,11 +181,11 @@ impl LoadBalancingPolicy for SingleTargetLBP { } async fn send_statement_everywhere( - session: &LegacySession, + session: &Session, cluster: &ClusterData, statement: &PreparedStatement, values: &dyn SerializeRow, -) -> Result, QueryError> { +) -> Result, QueryError> { let tasks = cluster.get_nodes_info().iter().flat_map(|node| { let shard_count: u16 = node.sharder().unwrap().nr_shards.into(); (0..shard_count).map(|shard| { @@ -205,10 +207,10 @@ async fn send_statement_everywhere( } async fn send_unprepared_query_everywhere( - session: &LegacySession, + session: &Session, cluster: &ClusterData, query: &Query, -) -> Result, QueryError> { +) -> Result, QueryError> { let tasks = cluster.get_nodes_info().iter().flat_map(|node| { let shard_count: u16 = node.sharder().unwrap().nr_shards.into(); (0..shard_count).map(|shard| { @@ -247,7 +249,7 @@ fn count_tablet_feedbacks( .count() } -async fn prepare_schema(session: &LegacySession, ks: &str, table: &str, tablet_count: usize) { +async fn prepare_schema(session: &Session, ks: &str, table: &str, tablet_count: usize) { session .query_unpaged( format!( @@ -294,11 +296,11 @@ async fn test_default_policy_is_tablet_aware() { let session = scylla::SessionBuilder::new() .known_node(proxy_uris[0].as_str()) .address_translator(Arc::new(translation_map)) - .build_legacy() + .build() .await .unwrap(); - if !scylla::test_utils::scylla_supports_tablets_legacy(&session).await { + if !scylla::test_utils::scylla_supports_tablets(&session).await { tracing::warn!("Skipping test because this Scylla version doesn't support tablets"); return running_proxy; } @@ -416,6 +418,8 @@ async fn test_default_policy_is_tablet_aware() { #[tokio::test] #[ntest::timeout(30000)] async fn test_tablet_feedback_not_sent_for_unprepared_queries() { + use scylla::test_utils::scylla_supports_tablets; + setup_tracing(); const TABLET_COUNT: usize = 16; @@ -425,11 +429,11 @@ async fn test_tablet_feedback_not_sent_for_unprepared_queries() { let session = scylla::SessionBuilder::new() .known_node(proxy_uris[0].as_str()) .address_translator(Arc::new(translation_map)) - .build_legacy() + .build() .await .unwrap(); - if !scylla::test_utils::scylla_supports_tablets_legacy(&session).await { + if !scylla_supports_tablets(&session).await { tracing::warn!("Skipping test because this Scylla version doesn't support tablets"); return running_proxy; } @@ -488,6 +492,8 @@ async fn test_tablet_feedback_not_sent_for_unprepared_queries() { #[ntest::timeout(30000)] #[ignore] async fn test_lwt_optimization_works_with_tablets() { + use scylla::test_utils::scylla_supports_tablets; + setup_tracing(); const TABLET_COUNT: usize = 16; @@ -497,11 +503,11 @@ async fn test_lwt_optimization_works_with_tablets() { let session = scylla::SessionBuilder::new() .known_node(proxy_uris[0].as_str()) .address_translator(Arc::new(translation_map)) - .build_legacy() + .build() .await .unwrap(); - if !scylla::test_utils::scylla_supports_tablets_legacy(&session).await { + if !scylla_supports_tablets(&session).await { tracing::warn!("Skipping test because this Scylla version doesn't support tablets"); return running_proxy; } From 4fb4922e01bec60683a715c9512713be0b911bfd Mon Sep 17 00:00:00 2001 From: Piotr Dulikowski Date: Tue, 14 Mar 2023 12:04:02 +0100 Subject: [PATCH 38/42] examples: adjust to use the new interface MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This commit goes over all unadjusted examples and changes them to use the new deserialization framework. Again, it contains a lot of changes, but they are quite simple. Co-authored-by: Wojciech Przytuła --- examples/Cargo.toml | 1 + examples/allocations.rs | 10 ++--- examples/auth.rs | 2 +- examples/basic.rs | 31 ++++++++-------- examples/cloud.rs | 2 +- examples/compare-tokens.rs | 8 ++-- examples/cql-time-types.rs | 23 ++++++------ examples/cqlsh-rs.rs | 47 ++++++++++++++---------- examples/custom_deserialization.rs | 7 +++- examples/custom_load_balancing_policy.rs | 6 +-- examples/execution_profile.rs | 12 +++--- examples/get_by_name.rs | 29 ++++++++++----- examples/logging.rs | 4 +- examples/logging_log.rs | 5 +-- examples/parallel-prepared.rs | 4 +- examples/parallel.rs | 4 +- examples/query_history.rs | 5 +-- examples/schema_agreement.rs | 9 ++--- examples/select-paging.rs | 19 +++++++--- examples/speculative-execution.rs | 6 +-- examples/tls.rs | 9 ++--- examples/tower.rs | 25 ++++++++----- examples/tracing.rs | 31 ++++++++-------- examples/user-defined-type.rs | 11 +++--- examples/value_list.rs | 16 +++++--- 25 files changed, 177 insertions(+), 149 deletions(-) diff --git a/examples/Cargo.toml b/examples/Cargo.toml index 5b335fd34..d05034275 100644 --- a/examples/Cargo.toml +++ b/examples/Cargo.toml @@ -19,6 +19,7 @@ scylla = { path = "../scylla", features = [ "num-bigint-04", "bigdecimal-04", ] } +scylla-cql = { path = "../scylla-cql" } tokio = { version = "1.34", features = ["full"] } tracing = { version = "0.1.25", features = ["log"] } tracing-subscriber = { version = "0.3.14", features = ["env-filter"] } diff --git a/examples/allocations.rs b/examples/allocations.rs index f87f7641f..d10ad9771 100644 --- a/examples/allocations.rs +++ b/examples/allocations.rs @@ -1,5 +1,6 @@ use anyhow::Result; -use scylla::{statement::prepared_statement::PreparedStatement, LegacySession, SessionBuilder}; +use scylla::transport::session::Session; +use scylla::{statement::prepared_statement::PreparedStatement, SessionBuilder}; use std::io::Write; use std::sync::atomic::{AtomicUsize, Ordering}; use std::sync::Arc; @@ -65,7 +66,7 @@ fn print_stats(stats: &stats_alloc::Stats, reqs: f64) { } async fn measure( - session: Arc, + session: Arc, prepared: Arc, reqs: usize, parallelism: usize, @@ -128,10 +129,7 @@ async fn main() -> Result<()> { println!("Connecting to {} ...", args.node); - let session: LegacySession = SessionBuilder::new() - .known_node(args.node) - .build_legacy() - .await?; + let session: Session = SessionBuilder::new().known_node(args.node).build().await?; let session = Arc::new(session); session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; diff --git a/examples/auth.rs b/examples/auth.rs index ded1115f3..22fbee007 100644 --- a/examples/auth.rs +++ b/examples/auth.rs @@ -10,7 +10,7 @@ async fn main() -> Result<()> { let session = SessionBuilder::new() .known_node(uri) .user("cassandra", "cassandra") - .build_legacy() + .build() .await .unwrap(); diff --git a/examples/basic.rs b/examples/basic.rs index ad8570db5..83a4fb079 100644 --- a/examples/basic.rs +++ b/examples/basic.rs @@ -1,7 +1,7 @@ use anyhow::Result; -use futures::TryStreamExt; -use scylla::macros::FromRow; -use scylla::transport::session::LegacySession; +use scylla::frame::response::result::Row; +use scylla::transport::session::Session; +use scylla::DeserializeRow; use scylla::SessionBuilder; use std::env; @@ -11,7 +11,7 @@ async fn main() -> Result<()> { println!("Connecting to {} ...", uri); - let session: LegacySession = SessionBuilder::new().known_node(uri).build_legacy().await?; + let session: Session = SessionBuilder::new().known_node(uri).build().await?; session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; @@ -53,23 +53,24 @@ async fn main() -> Result<()> { let mut iter = session .query_iter("SELECT a, b, c FROM examples_ks.basic", &[]) .await? - .into_typed::<(i32, i32, String)>(); + .into_typed::<(i32, i32, String)>()?; while let Some((a, b, c)) = iter.try_next().await? { println!("a, b, c: {}, {}, {}", a, b, c); } - // Or as custom structs that derive FromRow - #[derive(Debug, FromRow)] + // Or as custom structs that derive DeserializeRow + #[allow(unused)] + #[derive(Debug, DeserializeRow)] struct RowData { - _a: i32, - _b: Option, - _c: String, + a: i32, + b: Option, + c: String, } let mut iter = session .query_iter("SELECT a, b, c FROM examples_ks.basic", &[]) .await? - .into_typed::(); + .into_typed::()?; while let Some(row_data) = iter.try_next().await? { println!("row_data: {:?}", row_data); } @@ -77,15 +78,13 @@ async fn main() -> Result<()> { // Or simply as untyped rows let mut iter = session .query_iter("SELECT a, b, c FROM examples_ks.basic", &[]) - .await?; - while let Some(row) = iter.try_next().await? { + .await? + .into_typed::()?; + while let Some(row) = iter.next().await.transpose()? { let a = row.columns[0].as_ref().unwrap().as_int().unwrap(); let b = row.columns[1].as_ref().unwrap().as_int().unwrap(); let c = row.columns[2].as_ref().unwrap().as_text().unwrap(); println!("a, b, c: {}, {}, {}", a, b, c); - - // Alternatively each row can be parsed individually - // let (a2, b2, c2) = row.into_typed::<(i32, i32, String)>() ?; } let metrics = session.get_metrics(); diff --git a/examples/cloud.rs b/examples/cloud.rs index 5859ef12e..63265e41f 100644 --- a/examples/cloud.rs +++ b/examples/cloud.rs @@ -12,7 +12,7 @@ async fn main() -> Result<()> { .unwrap_or("examples/config_data.yaml".to_owned()); let session = CloudSessionBuilder::new(Path::new(&config_path)) .unwrap() - .build_legacy() + .build() .await .unwrap(); diff --git a/examples/compare-tokens.rs b/examples/compare-tokens.rs index 4863608ff..5350006b9 100644 --- a/examples/compare-tokens.rs +++ b/examples/compare-tokens.rs @@ -1,7 +1,7 @@ use anyhow::Result; use scylla::routing::Token; use scylla::transport::NodeAddr; -use scylla::{LegacySession, SessionBuilder}; +use scylla::{Session, SessionBuilder}; use std::env; #[tokio::main] @@ -10,7 +10,7 @@ async fn main() -> Result<()> { println!("Connecting to {} ...", uri); - let session: LegacySession = SessionBuilder::new().known_node(uri).build_legacy().await?; + let session: Session = SessionBuilder::new().known_node(uri).build().await?; session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; @@ -51,7 +51,9 @@ async fn main() -> Result<()> { (pk,), ) .await? - .single_row_typed::<(i64,)>()?; + .into_rows_result()? + .expect("Got not Rows result") + .single_row()?; assert_eq!(t, qt); println!("token for {}: {}", pk, t); } diff --git a/examples/cql-time-types.rs b/examples/cql-time-types.rs index 1b9e475d4..77ddef09f 100644 --- a/examples/cql-time-types.rs +++ b/examples/cql-time-types.rs @@ -3,10 +3,9 @@ use anyhow::Result; use chrono::{DateTime, NaiveDate, NaiveTime, Utc}; -use futures::{StreamExt, TryStreamExt}; use scylla::frame::response::result::CqlValue; use scylla::frame::value::{CqlDate, CqlTime, CqlTimestamp}; -use scylla::transport::session::LegacySession; +use scylla::transport::session::Session; use scylla::SessionBuilder; use std::env; @@ -16,7 +15,7 @@ async fn main() -> Result<()> { println!("Connecting to {} ...", uri); - let session: LegacySession = SessionBuilder::new().known_node(uri).build_legacy().await?; + let session: Session = SessionBuilder::new().known_node(uri).build().await?; session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; @@ -44,7 +43,7 @@ async fn main() -> Result<()> { let mut iter = session .query_iter("SELECT d from examples_ks.dates", &[]) .await? - .into_typed::<(NaiveDate,)>(); + .into_typed::<(NaiveDate,)>()?; while let Some(row_result) = iter.next().await { let (read_date,): (NaiveDate,) = match row_result { Ok(read_date) => read_date, @@ -66,7 +65,7 @@ async fn main() -> Result<()> { let mut iter = session .query_iter("SELECT d from examples_ks.dates", &[]) .await? - .into_typed::<(time::Date,)>(); + .into_typed::<(time::Date,)>()?; while let Some(row_result) = iter.next().await { let (read_date,): (time::Date,) = match row_result { Ok(read_date) => read_date, @@ -88,7 +87,7 @@ async fn main() -> Result<()> { let mut iter = session .query_iter("SELECT d from examples_ks.dates", &[]) .await? - .into_typed::<(CqlValue,)>(); + .into_typed::<(CqlValue,)>()?; while let Some(row_result) = iter.next().await { let read_days: u32 = match row_result { Ok((CqlValue::Date(CqlDate(days)),)) => days, @@ -124,7 +123,7 @@ async fn main() -> Result<()> { let mut iter = session .query_iter("SELECT d from examples_ks.times", &[]) .await? - .into_typed::<(NaiveTime,)>(); + .into_typed::<(NaiveTime,)>()?; while let Some((read_time,)) = iter.try_next().await? { println!("Parsed a time into chrono::NaiveTime: {:?}", read_time); } @@ -139,7 +138,7 @@ async fn main() -> Result<()> { let mut iter = session .query_iter("SELECT d from examples_ks.times", &[]) .await? - .into_typed::<(time::Time,)>(); + .into_typed::<(time::Time,)>()?; while let Some((read_time,)) = iter.try_next().await? { println!("Parsed a time into time::Time: {:?}", read_time); } @@ -154,7 +153,7 @@ async fn main() -> Result<()> { let mut iter = session .query_iter("SELECT d from examples_ks.times", &[]) .await? - .into_typed::<(CqlTime,)>(); + .into_typed::<(CqlTime,)>()?; while let Some((read_time,)) = iter.try_next().await? { println!("Read a time as raw nanos: {:?}", read_time); } @@ -185,7 +184,7 @@ async fn main() -> Result<()> { let mut iter = session .query_iter("SELECT d from examples_ks.timestamps", &[]) .await? - .into_typed::<(DateTime,)>(); + .into_typed::<(DateTime,)>()?; while let Some((read_time,)) = iter.try_next().await? { println!( "Parsed a timestamp into chrono::DateTime: {:?}", @@ -206,7 +205,7 @@ async fn main() -> Result<()> { let mut iter = session .query_iter("SELECT d from examples_ks.timestamps", &[]) .await? - .into_typed::<(time::OffsetDateTime,)>(); + .into_typed::<(time::OffsetDateTime,)>()?; while let Some((read_time,)) = iter.try_next().await? { println!( "Parsed a timestamp into time::OffsetDateTime: {:?}", @@ -227,7 +226,7 @@ async fn main() -> Result<()> { let mut iter = session .query_iter("SELECT d from examples_ks.timestamps", &[]) .await? - .into_typed::<(CqlTimestamp,)>(); + .into_typed::<(CqlTimestamp,)>()?; while let Some((read_time,)) = iter.try_next().await? { println!("Read a timestamp as raw millis: {:?}", read_time); } diff --git a/examples/cqlsh-rs.rs b/examples/cqlsh-rs.rs index a4371909a..1e26750d6 100644 --- a/examples/cqlsh-rs.rs +++ b/examples/cqlsh-rs.rs @@ -3,8 +3,11 @@ use rustyline::completion::{Completer, Pair}; use rustyline::error::ReadlineError; use rustyline::{CompletionType, Config, Context, Editor}; use rustyline_derive::{Helper, Highlighter, Hinter, Validator}; +use scylla::transport::session::Session; use scylla::transport::Compression; -use scylla::{LegacyQueryResult, LegacySession, SessionBuilder}; +use scylla::QueryRowsResult; +use scylla::SessionBuilder; +use scylla_cql::frame::response::result::Row; use std::env; #[derive(Helper, Highlighter, Validator, Hinter)] @@ -173,23 +176,24 @@ impl Completer for CqlHelper { } } -fn print_result(result: &LegacyQueryResult) { - if result.rows.is_none() { - println!("OK"); - return; - } - for row in result.rows.as_ref().unwrap() { - for column in &row.columns { - print!("|"); - print!( - " {:16}", - match column { - None => "null".to_owned(), - Some(value) => format!("{:?}", value), - } - ); +fn print_result(result: Option<&QueryRowsResult>) { + if let Some(rows_result) = result { + for row in rows_result.rows::().unwrap() { + let row = row.unwrap(); + for column in &row.columns { + print!("|"); + print!( + " {:16}", + match column { + None => "null".to_owned(), + Some(value) => format!("{:?}", value), + } + ); + } + println!("|") } - println!("|") + } else { + println!("OK"); } } @@ -199,10 +203,10 @@ async fn main() -> Result<()> { println!("Connecting to {} ...", uri); - let session: LegacySession = SessionBuilder::new() + let session: Session = SessionBuilder::new() .known_node(uri) .compression(Some(Compression::Lz4)) - .build_legacy() + .build() .await?; let config = Config::builder() @@ -222,7 +226,10 @@ async fn main() -> Result<()> { let maybe_res = session.query_unpaged(line, &[]).await; match maybe_res { Err(err) => println!("Error: {}", err), - Ok(res) => print_result(&res), + Ok(res) => { + let rows_res = res.into_rows_result()?; + print_result(rows_res.as_ref()) + } } } Err(ReadlineError::Interrupted) => continue, diff --git a/examples/custom_deserialization.rs b/examples/custom_deserialization.rs index 7bd694c81..66bc4ad80 100644 --- a/examples/custom_deserialization.rs +++ b/examples/custom_deserialization.rs @@ -2,7 +2,8 @@ use anyhow::Result; use scylla::cql_to_rust::{FromCqlVal, FromCqlValError}; use scylla::frame::response::result::CqlValue; use scylla::macros::impl_from_cql_value_from_method; -use scylla::{LegacySession, SessionBuilder}; +use scylla::transport::session::Session; +use scylla::SessionBuilder; use std::env; #[tokio::main] @@ -11,7 +12,7 @@ async fn main() -> Result<()> { println!("Connecting to {} ...", uri); - let session: LegacySession = SessionBuilder::new().known_node(uri).build_legacy().await?; + let session: Session = SessionBuilder::new().known_node(uri).build().await?; session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; session @@ -46,6 +47,7 @@ async fn main() -> Result<()> { (), ) .await? + .into_legacy_result()? .single_row_typed::<(MyType,)>()?; assert_eq!(v, MyType("asdf".to_owned())); @@ -73,6 +75,7 @@ async fn main() -> Result<()> { (), ) .await? + .into_legacy_result()? .single_row_typed::<(MyOtherType,)>()?; assert_eq!(v, MyOtherType("asdf".to_owned())); diff --git a/examples/custom_load_balancing_policy.rs b/examples/custom_load_balancing_policy.rs index e70ed0213..5c279f233 100644 --- a/examples/custom_load_balancing_policy.rs +++ b/examples/custom_load_balancing_policy.rs @@ -6,7 +6,7 @@ use scylla::{ load_balancing::{LoadBalancingPolicy, RoutingInfo}, routing::Shard, transport::{ClusterData, ExecutionProfile}, - LegacySession, SessionBuilder, + Session, SessionBuilder, }; use std::{env, sync::Arc}; @@ -68,10 +68,10 @@ async fn main() -> Result<()> { .load_balancing_policy(Arc::new(custom_load_balancing)) .build(); - let _session: LegacySession = SessionBuilder::new() + let _session: Session = SessionBuilder::new() .known_node(uri) .default_execution_profile_handle(profile.into_handle()) - .build_legacy() + .build() .await?; Ok(()) diff --git a/examples/execution_profile.rs b/examples/execution_profile.rs index 46ae8e03f..3562966ac 100644 --- a/examples/execution_profile.rs +++ b/examples/execution_profile.rs @@ -4,7 +4,7 @@ use scylla::query::Query; use scylla::retry_policy::{DefaultRetryPolicy, FallthroughRetryPolicy}; use scylla::speculative_execution::PercentileSpeculativeExecutionPolicy; use scylla::statement::{Consistency, SerialConsistency}; -use scylla::transport::session::LegacySession; +use scylla::transport::session::Session; use scylla::transport::ExecutionProfile; use scylla::{SessionBuilder, SessionConfig}; use std::env; @@ -42,22 +42,22 @@ async fn main() -> Result<()> { let mut handle2 = profile2.into_handle(); // It is even possible to use multiple sessions interleaved, having them configured with different profiles. - let session1: LegacySession = SessionBuilder::new() + let session1: Session = SessionBuilder::new() .known_node(&uri) .default_execution_profile_handle(handle1.clone()) - .build_legacy() + .build() .await?; - let session2: LegacySession = SessionBuilder::new() + let session2: Session = SessionBuilder::new() .known_node(&uri) .default_execution_profile_handle(handle2.clone()) - .build_legacy() + .build() .await?; // As default execution profile is not provided explicitly, session 3 uses a predefined one. let mut session_3_config = SessionConfig::new(); session_3_config.add_known_node(uri); - let session3: LegacySession = LegacySession::connect(session_3_config).await?; + let session3: Session = Session::connect(session_3_config).await?; session1.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; diff --git a/examples/get_by_name.rs b/examples/get_by_name.rs index a0a21b855..cac5349f1 100644 --- a/examples/get_by_name.rs +++ b/examples/get_by_name.rs @@ -1,6 +1,7 @@ -use anyhow::{anyhow, Result}; -use scylla::transport::session::LegacySession; +use anyhow::{anyhow, Context as _, Result}; +use scylla::transport::session::Session; use scylla::SessionBuilder; +use scylla_cql::frame::response::result::Row; use std::env; #[tokio::main] @@ -10,7 +11,7 @@ async fn main() -> Result<()> { println!("Connecting to {} ...", uri); - let session: LegacySession = SessionBuilder::new().known_node(uri).build_legacy().await?; + let session: Session = SessionBuilder::new().known_node(uri).build().await?; session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; @@ -35,18 +36,26 @@ async fn main() -> Result<()> { ) .await?; - let query_result = session + let rows_result = session .query_unpaged("SELECT pk, ck, value FROM examples_ks.get_by_name", &[]) - .await?; - let (ck_idx, _) = query_result - .get_column_spec("ck") + .await? + .into_rows_result()? + .context("Response is not of Rows type")?; + let col_specs = rows_result.column_specs(); + let (ck_idx, _) = col_specs + .get_by_name("ck") .ok_or_else(|| anyhow!("No ck column found"))?; - let (value_idx, _) = query_result - .get_column_spec("value") + let (value_idx, _) = col_specs + .get_by_name("value") .ok_or_else(|| anyhow!("No value column found"))?; + let rows = rows_result + .rows::() + .unwrap() + .collect::, _>>() + .unwrap(); println!("ck | value"); println!("---------------------"); - for row in query_result.rows.ok_or_else(|| anyhow!("no rows found"))? { + for row in rows { println!("{:?} | {:?}", row.columns[ck_idx], row.columns[value_idx]); } diff --git a/examples/logging.rs b/examples/logging.rs index 37e534b8c..6b090acbc 100644 --- a/examples/logging.rs +++ b/examples/logging.rs @@ -1,5 +1,5 @@ use anyhow::Result; -use scylla::transport::session::LegacySession; +use scylla::transport::session::Session; use scylla::SessionBuilder; use std::env; use tracing::info; @@ -16,7 +16,7 @@ async fn main() -> Result<()> { let uri = env::var("SCYLLA_URI").unwrap_or_else(|_| "127.0.0.1:9042".to_string()); info!("Connecting to {}", uri); - let session: LegacySession = SessionBuilder::new().known_node(uri).build_legacy().await?; + let session: Session = SessionBuilder::new().known_node(uri).build().await?; session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; session.query_unpaged("USE examples_ks", &[]).await?; diff --git a/examples/logging_log.rs b/examples/logging_log.rs index a1f962419..19465018c 100644 --- a/examples/logging_log.rs +++ b/examples/logging_log.rs @@ -1,6 +1,5 @@ use anyhow::Result; -use scylla::transport::session::LegacySession; -use scylla::SessionBuilder; +use scylla::{Session, SessionBuilder}; use std::env; use tracing::info; @@ -18,7 +17,7 @@ async fn main() -> Result<()> { let uri = env::var("SCYLLA_URI").unwrap_or_else(|_| "127.0.0.1:9042".to_string()); info!("Connecting to {}", uri); - let session: LegacySession = SessionBuilder::new().known_node(uri).build_legacy().await?; + let session: Session = SessionBuilder::new().known_node(uri).build().await?; session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; session.query_unpaged("USE examples_ks", &[]).await?; diff --git a/examples/parallel-prepared.rs b/examples/parallel-prepared.rs index 531f6d7b4..167b58394 100644 --- a/examples/parallel-prepared.rs +++ b/examples/parallel-prepared.rs @@ -1,5 +1,5 @@ use anyhow::Result; -use scylla::{LegacySession, SessionBuilder}; +use scylla::{Session, SessionBuilder}; use std::env; use std::sync::Arc; @@ -11,7 +11,7 @@ async fn main() -> Result<()> { println!("Connecting to {} ...", uri); - let session: LegacySession = SessionBuilder::new().known_node(uri).build_legacy().await?; + let session: Session = SessionBuilder::new().known_node(uri).build().await?; let session = Arc::new(session); session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; diff --git a/examples/parallel.rs b/examples/parallel.rs index 5e3f119fb..716225fb7 100644 --- a/examples/parallel.rs +++ b/examples/parallel.rs @@ -1,5 +1,5 @@ use anyhow::Result; -use scylla::{LegacySession, SessionBuilder}; +use scylla::{Session, SessionBuilder}; use std::env; use std::sync::Arc; @@ -11,7 +11,7 @@ async fn main() -> Result<()> { println!("Connecting to {} ...", uri); - let session: LegacySession = SessionBuilder::new().known_node(uri).build_legacy().await?; + let session: Session = SessionBuilder::new().known_node(uri).build().await?; let session = Arc::new(session); session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; diff --git a/examples/query_history.rs b/examples/query_history.rs index 710f9616d..826e10bd0 100644 --- a/examples/query_history.rs +++ b/examples/query_history.rs @@ -1,10 +1,9 @@ //! This example shows how to collect history of query execution. use anyhow::Result; -use futures::StreamExt; use scylla::history::{HistoryCollector, StructuredHistory}; use scylla::query::Query; -use scylla::transport::session::LegacySession; +use scylla::transport::session::Session; use scylla::SessionBuilder; use std::env; use std::sync::Arc; @@ -15,7 +14,7 @@ async fn main() -> Result<()> { println!("Connecting to {} ...", uri); - let session: LegacySession = SessionBuilder::new().known_node(uri).build_legacy().await?; + let session: Session = SessionBuilder::new().known_node(uri).build().await?; session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; diff --git a/examples/schema_agreement.rs b/examples/schema_agreement.rs index 9b9369ac9..3b7a41ed8 100644 --- a/examples/schema_agreement.rs +++ b/examples/schema_agreement.rs @@ -1,7 +1,6 @@ use anyhow::{bail, Result}; -use futures::TryStreamExt; use scylla::transport::errors::QueryError; -use scylla::transport::session::LegacySession; +use scylla::transport::session::Session; use scylla::SessionBuilder; use std::env; use std::time::Duration; @@ -13,10 +12,10 @@ async fn main() -> Result<()> { println!("Connecting to {} ...", uri); - let session: LegacySession = SessionBuilder::new() + let session: Session = SessionBuilder::new() .known_node(uri) .schema_agreement_interval(Duration::from_secs(1)) // check every second for schema agreement if not agreed first check - .build_legacy() + .build() .await?; let schema_version = session.await_schema_agreement().await?; @@ -70,7 +69,7 @@ async fn main() -> Result<()> { let mut iter = session .query_iter("SELECT a, b, c FROM examples_ks.schema_agreement", &[]) .await? - .into_typed::<(i32, i32, String)>(); + .into_typed::<(i32, i32, String)>()?; while let Some((a, b, c)) = iter.try_next().await? { println!("a, b, c: {}, {}, {}", a, b, c); } diff --git a/examples/select-paging.rs b/examples/select-paging.rs index f9027675a..92dd52859 100644 --- a/examples/select-paging.rs +++ b/examples/select-paging.rs @@ -1,7 +1,6 @@ use anyhow::Result; -use futures::stream::StreamExt; use scylla::statement::PagingState; -use scylla::{query::Query, LegacySession, SessionBuilder}; +use scylla::{query::Query, Session, SessionBuilder}; use std::env; use std::ops::ControlFlow; @@ -11,7 +10,7 @@ async fn main() -> Result<()> { println!("Connecting to {} ...", uri); - let session: LegacySession = SessionBuilder::new().known_node(uri).build_legacy().await?; + let session: Session = SessionBuilder::new().known_node(uri).build().await?; session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; @@ -35,7 +34,7 @@ async fn main() -> Result<()> { let mut rows_stream = session .query_iter("SELECT a, b, c FROM examples_ks.select_paging", &[]) .await? - .into_typed::<(i32, i32, String)>(); + .into_typed::<(i32, i32, String)>()?; while let Some(next_row_res) = rows_stream.next().await { let (a, b, c) = next_row_res?; @@ -51,10 +50,14 @@ async fn main() -> Result<()> { .query_single_page(paged_query.clone(), &[], paging_state) .await?; + let res = res + .into_rows_result()? + .expect("Got result different than Rows"); + println!( "Paging state: {:#?} ({} rows)", paging_state_response, - res.rows_num()?, + res.rows_num(), ); match paging_state_response.into_paging_control_flow() { @@ -81,10 +84,14 @@ async fn main() -> Result<()> { .execute_single_page(&paged_prepared, &[], paging_state) .await?; + let res = res + .into_rows_result()? + .expect("Got result different than Rows"); + println!( "Paging state from the prepared statement execution: {:#?} ({} rows)", paging_state_response, - res.rows_num()?, + res.rows_num(), ); match paging_state_response.into_paging_control_flow() { diff --git a/examples/speculative-execution.rs b/examples/speculative-execution.rs index c53285cac..e6c64e3ad 100644 --- a/examples/speculative-execution.rs +++ b/examples/speculative-execution.rs @@ -1,6 +1,6 @@ use scylla::{ speculative_execution::PercentileSpeculativeExecutionPolicy, - transport::execution_profile::ExecutionProfile, LegacySession, SessionBuilder, + transport::execution_profile::ExecutionProfile, Session, SessionBuilder, }; use anyhow::Result; @@ -20,10 +20,10 @@ async fn main() -> Result<()> { .speculative_execution_policy(Some(Arc::new(speculative))) .build(); - let session: LegacySession = SessionBuilder::new() + let session: Session = SessionBuilder::new() .known_node(uri) .default_execution_profile_handle(speculative_profile.into_handle()) - .build_legacy() + .build() .await?; session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; diff --git a/examples/tls.rs b/examples/tls.rs index 1bb354e56..3cb22b249 100644 --- a/examples/tls.rs +++ b/examples/tls.rs @@ -1,6 +1,5 @@ use anyhow::Result; -use futures::TryStreamExt; -use scylla::transport::session::LegacySession; +use scylla::transport::session::Session; use scylla::SessionBuilder; use std::env; use std::fs; @@ -44,10 +43,10 @@ async fn main() -> Result<()> { context_builder.set_ca_file(ca_dir.as_path())?; context_builder.set_verify(SslVerifyMode::PEER); - let session: LegacySession = SessionBuilder::new() + let session: Session = SessionBuilder::new() .known_node(uri) .ssl_context(Some(context_builder.build())) - .build_legacy() + .build() .await?; session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; @@ -90,7 +89,7 @@ async fn main() -> Result<()> { let mut iter = session .query_iter("SELECT a, b, c FROM examples_ks.tls", &[]) .await? - .into_typed::<(i32, i32, String)>(); + .into_typed::<(i32, i32, String)>()?; while let Some((a, b, c)) = iter.try_next().await? { println!("a, b, c: {}, {}, {}", a, b, c); } diff --git a/examples/tower.rs b/examples/tower.rs index 0b6085e00..d206b8a14 100644 --- a/examples/tower.rs +++ b/examples/tower.rs @@ -1,3 +1,5 @@ +use scylla::transport::session::Session; +use scylla_cql::frame::response::result::Row; use std::env; use std::future::Future; use std::pin::Pin; @@ -7,12 +9,12 @@ use std::task::Poll; use tower::Service; struct SessionService { - session: Arc, + session: Arc, } // A trivial service implementation for sending parameterless simple string requests to Scylla. impl Service for SessionService { - type Response = scylla::LegacyQueryResult; + type Response = scylla::QueryResult; type Error = scylla::transport::errors::QueryError; type Future = Pin>>>; @@ -35,14 +37,16 @@ async fn main() -> anyhow::Result<()> { session: Arc::new( scylla::SessionBuilder::new() .known_node(uri) - .build_legacy() + .build() .await?, ), }; - let resp = session + let rows_result = session .call("SELECT keyspace_name, table_name FROM system_schema.tables;".into()) - .await?; + .await? + .into_rows_result()? + .expect("Got result different than Rows"); let print_text = |t: &Option| { t.as_ref() @@ -56,14 +60,15 @@ async fn main() -> anyhow::Result<()> { println!( "Tables:\n{}", - resp.rows()? - .into_iter() - .map(|r| format!( + rows_result + .rows::()? + .map(|r| r.map(|r| format!( "\t{}.{}", print_text(&r.columns[0]), print_text(&r.columns[1]) - )) - .collect::>() + ))) + .collect::, _>>() + .unwrap() .join("\n") ); Ok(()) diff --git a/examples/tracing.rs b/examples/tracing.rs index 435e356c7..f417917b1 100644 --- a/examples/tracing.rs +++ b/examples/tracing.rs @@ -2,15 +2,14 @@ // query() prepare() execute() batch() query_iter() and execute_iter() can be traced use anyhow::{anyhow, Result}; -use futures::StreamExt; use scylla::batch::Batch; use scylla::statement::{ prepared_statement::PreparedStatement, query::Query, Consistency, SerialConsistency, }; use scylla::tracing::TracingInfo; -use scylla::transport::iterator::LegacyRowIterator; -use scylla::LegacyQueryResult; -use scylla::{LegacySession, SessionBuilder}; +use scylla::transport::iterator::RawIterator; +use scylla::QueryResult; +use scylla::{Session, SessionBuilder}; use std::env; use std::num::NonZeroU32; use std::time::Duration; @@ -21,9 +20,9 @@ async fn main() -> Result<()> { let uri = env::var("SCYLLA_URI").unwrap_or_else(|_| "127.0.0.1:9042".to_string()); println!("Connecting to {} ...", uri); - let session: LegacySession = SessionBuilder::new() + let session: Session = SessionBuilder::new() .known_node(uri.as_str()) - .build_legacy() + .build() .await?; session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; @@ -42,9 +41,9 @@ async fn main() -> Result<()> { query.set_serial_consistency(Some(SerialConsistency::LocalSerial)); // QueryResult will contain a tracing_id which can be used to query tracing information - let query_result: LegacyQueryResult = session.query_unpaged(query.clone(), &[]).await?; + let query_result: QueryResult = session.query_unpaged(query.clone(), &[]).await?; let query_tracing_id: Uuid = query_result - .tracing_id + .tracing_id() .ok_or_else(|| anyhow!("Tracing id is None!"))?; // Get tracing information for this query and print it @@ -79,14 +78,14 @@ async fn main() -> Result<()> { // To trace execution of a prepared statement tracing must be enabled for it prepared.set_tracing(true); - let execute_result: LegacyQueryResult = session.execute_unpaged(&prepared, &[]).await?; - println!("Execute tracing id: {:?}", execute_result.tracing_id); + let execute_result: QueryResult = session.execute_unpaged(&prepared, &[]).await?; + println!("Execute tracing id: {:?}", execute_result.tracing_id()); // PAGED QUERY_ITER EXECUTE_ITER // It's also possible to trace paged queries like query_iter or execute_iter // After iterating through all rows iterator.get_tracing_ids() will give tracing ids // for all page queries - let mut row_iterator: LegacyRowIterator = session.query_iter(query, &[]).await?; + let mut row_iterator: RawIterator = session.query_iter(query, &[]).await?; while let Some(_row) = row_iterator.next().await { // Receive rows @@ -95,7 +94,7 @@ async fn main() -> Result<()> { // Now print tracing ids for all page queries: println!( "Paged row iterator tracing ids: {:?}\n", - row_iterator.get_tracing_ids() + row_iterator.tracing_ids() ); // BATCH @@ -105,19 +104,19 @@ async fn main() -> Result<()> { batch.set_tracing(true); // Run the batch and print its tracing_id - let batch_result: LegacyQueryResult = session.batch(&batch, ((),)).await?; - println!("Batch tracing id: {:?}\n", batch_result.tracing_id); + let batch_result: QueryResult = session.batch(&batch, ((),)).await?; + println!("Batch tracing id: {:?}\n", batch_result.tracing_id()); // CUSTOM // Session configuration allows specifying custom settings for querying tracing info. // Tracing info might not immediately be available on queried node // so the driver performs a few attempts with sleeps in between. - let session: LegacySession = SessionBuilder::new() + let session: Session = SessionBuilder::new() .known_node(uri) .tracing_info_fetch_attempts(NonZeroU32::new(8).unwrap()) .tracing_info_fetch_interval(Duration::from_millis(100)) .tracing_info_fetch_consistency(Consistency::One) - .build_legacy() + .build() .await?; let _custom_info: TracingInfo = session.get_tracing_info(&query_tracing_id).await?; diff --git a/examples/user-defined-type.rs b/examples/user-defined-type.rs index 9e01586a7..719a7d8d6 100644 --- a/examples/user-defined-type.rs +++ b/examples/user-defined-type.rs @@ -1,7 +1,6 @@ use anyhow::Result; -use futures::TryStreamExt; -use scylla::macros::FromUserType; -use scylla::{LegacySession, SerializeValue, SessionBuilder}; +use scylla::macros::DeserializeValue; +use scylla::{SerializeValue, Session, SessionBuilder}; use std::env; #[tokio::main] @@ -10,7 +9,7 @@ async fn main() -> Result<()> { println!("Connecting to {} ...", uri); - let session: LegacySession = SessionBuilder::new().known_node(uri).build_legacy().await?; + let session: Session = SessionBuilder::new().known_node(uri).build().await?; session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; @@ -30,7 +29,7 @@ async fn main() -> Result<()> { // Define custom struct that matches User Defined Type created earlier // wrapping field in Option will gracefully handle null field values - #[derive(Debug, FromUserType, SerializeValue)] + #[derive(Debug, DeserializeValue, SerializeValue)] struct MyType { int_val: i32, text_val: Option, @@ -56,7 +55,7 @@ async fn main() -> Result<()> { &[], ) .await? - .into_typed::<(MyType,)>(); + .into_typed::<(MyType,)>()?; while let Some((my_val,)) = iter.try_next().await? { println!("{:?}", my_val); } diff --git a/examples/value_list.rs b/examples/value_list.rs index ce997b70e..59c556eab 100644 --- a/examples/value_list.rs +++ b/examples/value_list.rs @@ -1,5 +1,6 @@ use anyhow::Result; -use scylla::{LegacySession, SessionBuilder}; +use futures::StreamExt; +use scylla::{Session, SessionBuilder}; use std::env; #[tokio::main] @@ -8,7 +9,7 @@ async fn main() -> Result<()> { println!("Connecting to {} ...", uri); - let session: LegacySession = SessionBuilder::new().known_node(uri).build_legacy().await?; + let session: Session = SessionBuilder::new().known_node(uri).build().await.unwrap(); session.query_unpaged("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?; @@ -56,11 +57,14 @@ async fn main() -> Result<()> { ) .await?; - let q = session - .query_unpaged("SELECT * FROM examples_ks.my_type", &[]) - .await?; + let iter = session + .query_iter("SELECT * FROM examples_ks.my_type", &[]) + .await? + .into_typed::<(i32, String)>()? + .into_stream(); - println!("Q: {:?}", q.rows); + let rows = iter.collect::>().await; + println!("Q: {:?}", rows); Ok(()) } From a82fa6d4bd8dfdd63aab356ef0bda35649b3a9ea Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Wojciech=20Przytu=C5=82a?= Date: Wed, 14 Aug 2024 08:33:01 +0200 Subject: [PATCH 39/42] codewide: migrate doctests to new deser API --- scylla/src/lib.rs | 26 ++- scylla/src/transport/execution_profile.rs | 12 +- scylla/src/transport/iterator.rs | 3 +- scylla/src/transport/session.rs | 72 ++++---- scylla/src/transport/session_builder.rs | 212 +++++++++++----------- 5 files changed, 160 insertions(+), 165 deletions(-) diff --git a/scylla/src/lib.rs b/scylla/src/lib.rs index 1dd70e90b..6616c8d1b 100644 --- a/scylla/src/lib.rs +++ b/scylla/src/lib.rs @@ -17,15 +17,15 @@ //! `Session` is created by specifying a few known nodes and connecting to them: //! //! ```rust,no_run -//! use scylla::{LegacySession, SessionBuilder}; +//! use scylla::{Session, SessionBuilder}; //! use std::error::Error; //! //! #[tokio::main] //! async fn main() -> Result<(), Box> { -//! let session: LegacySession = SessionBuilder::new() +//! let session: Session = SessionBuilder::new() //! .known_node("127.0.0.1:9042") //! .known_node("1.2.3.4:9876") -//! .build_legacy() +//! .build() //! .await?; //! //! Ok(()) @@ -50,9 +50,9 @@ //! //! The easiest way to specify bound values in a query is using a tuple: //! ```rust -//! # use scylla::LegacySession; +//! # use scylla::Session; //! # use std::error::Error; -//! # async fn check_only_compiles(session: &LegacySession) -> Result<(), Box> { +//! # async fn check_only_compiles(session: &Session) -> Result<(), Box> { //! // Insert an int and text into the table //! session //! .query_unpaged( @@ -69,22 +69,20 @@ //! The easiest way to read rows returned by a query is to cast each row to a tuple of values: //! //! ```rust -//! # use scylla::LegacySession; +//! # use scylla::Session; //! # use std::error::Error; -//! # async fn check_only_compiles(session: &LegacySession) -> Result<(), Box> { -//! use scylla::IntoTypedRows; +//! # async fn check_only_compiles(session: &Session) -> Result<(), Box> { //! //! // Read rows containing an int and text //! // Keep in mind that all results come in one response (no paging is done!), //! // so the memory footprint and latency may be huge! //! // To prevent that, use `Session::query_iter` or `Session::query_single_page`. -//! let rows_opt = session -//! .query_unpaged("SELECT a, b FROM ks.tab", &[]) -//! .await? -//! .rows; +//! let query_result = session +//! .query_unpaged("SELECT a, b FROM ks.tab", &[]) +//! .await?; //! -//! if let Some(rows) = rows_opt { -//! for row in rows.into_typed::<(i32, String)>() { +//! if let Some(rows) = query_result.maybe_rows::<(i32, String)>()? { +//! for row in rows { //! // Parse row as int and text \ //! let (int_val, text_val): (i32, String) = row?; //! } diff --git a/scylla/src/transport/execution_profile.rs b/scylla/src/transport/execution_profile.rs index 421a7da7e..a94addec5 100644 --- a/scylla/src/transport/execution_profile.rs +++ b/scylla/src/transport/execution_profile.rs @@ -16,7 +16,7 @@ //! # extern crate scylla; //! # use std::error::Error; //! # async fn check_only_compiles() -> Result<(), Box> { -//! use scylla::{LegacySession, SessionBuilder}; +//! use scylla::{Session, SessionBuilder}; //! use scylla::statement::Consistency; //! use scylla::transport::ExecutionProfile; //! @@ -27,10 +27,10 @@ //! //! let handle = profile.into_handle(); //! -//! let session: LegacySession = SessionBuilder::new() +//! let session: Session = SessionBuilder::new() //! .known_node("127.0.0.1:9042") //! .default_execution_profile_handle(handle) -//! .build_legacy() +//! .build() //! .await?; //! # Ok(()) //! # } @@ -109,7 +109,7 @@ //! # extern crate scylla; //! # use std::error::Error; //! # async fn check_only_compiles() -> Result<(), Box> { -//! use scylla::{LegacySession, SessionBuilder}; +//! use scylla::{Session, SessionBuilder}; //! use scylla::query::Query; //! use scylla::statement::Consistency; //! use scylla::transport::ExecutionProfile; @@ -125,10 +125,10 @@ //! let mut handle1 = profile1.clone().into_handle(); //! let mut handle2 = profile2.clone().into_handle(); //! -//! let session: LegacySession = SessionBuilder::new() +//! let session: Session = SessionBuilder::new() //! .known_node("127.0.0.1:9042") //! .default_execution_profile_handle(handle1.clone()) -//! .build_legacy() +//! .build() //! .await?; //! //! let mut query1 = Query::from("SELECT * FROM ks.table"); diff --git a/scylla/src/transport/iterator.rs b/scylla/src/transport/iterator.rs index 28b77f208..c3518ab7d 100644 --- a/scylla/src/transport/iterator.rs +++ b/scylla/src/transport/iterator.rs @@ -576,8 +576,7 @@ impl RawIterator { /// /// # Example /// - /// // FIXME: change `text` to `rust` when Session API is migrated to the new deserialization framework. - /// ```text + /// ```rust /// # use scylla::Session; /// # use std::error::Error; /// # async fn check_only_compiles(session: &Session) -> Result<(), Box> { diff --git a/scylla/src/transport/session.rs b/scylla/src/transport/session.rs index 74e63367a..bcf892534 100644 --- a/scylla/src/transport/session.rs +++ b/scylla/src/transport/session.rs @@ -516,9 +516,9 @@ impl GenericSession { /// /// # Examples /// ```rust - /// # use scylla::LegacySession; + /// # use scylla::Session; /// # use std::error::Error; - /// # async fn check_only_compiles(session: &LegacySession) -> Result<(), Box> { + /// # async fn check_only_compiles(session: &Session) -> Result<(), Box> { /// // Insert an int and text into a table. /// session /// .query_unpaged( @@ -530,23 +530,22 @@ impl GenericSession { /// # } /// ``` /// ```rust - /// # use scylla::LegacySession; + /// # use scylla::Session; /// # use std::error::Error; - /// # async fn check_only_compiles(session: &LegacySession) -> Result<(), Box> { + /// # async fn check_only_compiles(session: &Session) -> Result<(), Box> { /// use scylla::IntoTypedRows; /// /// // Read rows containing an int and text. /// // Keep in mind that all results come in one response (no paging is done!), /// // so the memory footprint and latency may be huge! /// // To prevent that, use `Session::query_iter` or `Session::query_single_page`. - /// let rows_opt = session - /// .query_unpaged("SELECT a, b FROM ks.tab", &[]) - /// .await? - /// .rows; - /// - /// if let Some(rows) = rows_opt { - /// for row in rows.into_typed::<(i32, String)>() { - /// // Parse row as int and text \ + /// let query_result = session + /// .query_unpaged("SELECT a, b FROM ks.tab", &[]) + /// .await?; + /// + /// if let Some(rows) = query_result.maybe_rows::<(i32, String)>()? { + /// for row in rows { + /// // Parse row as int and text. /// let (int_val, text_val): (i32, String) = row?; /// } /// } @@ -576,9 +575,9 @@ impl GenericSession { /// # Example /// /// ```rust - /// # use scylla::LegacySession; + /// # use scylla::Session; /// # use std::error::Error; - /// # async fn check_only_compiles(session: &LegacySession) -> Result<(), Box> { + /// # async fn check_only_compiles(session: &Session) -> Result<(), Box> { /// use std::ops::ControlFlow; /// use scylla::statement::PagingState; /// @@ -590,7 +589,7 @@ impl GenericSession { /// .await?; /// /// // Do something with a single page of results. - /// for row in res.rows_typed::<(i32, String)>()? { + /// for row in res.rows::<(i32, String)>()? { /// let (a, b) = row?; /// } /// @@ -640,16 +639,16 @@ impl GenericSession { /// # Example /// /// ```rust - /// # use scylla::LegacySession; + /// # use scylla::Session; /// # use std::error::Error; - /// # async fn check_only_compiles(session: &LegacySession) -> Result<(), Box> { + /// # async fn check_only_compiles(session: &Session) -> Result<(), Box> { /// use scylla::IntoTypedRows; /// use futures::stream::StreamExt; /// /// let mut rows_stream = session /// .query_iter("SELECT a, b FROM ks.t", &[]) /// .await? - /// .into_typed::<(i32, i32)>(); + /// .into_typed::<(i32, i32)>()?; /// /// while let Some(next_row_res) = rows_stream.next().await { /// let (a, b): (i32, i32) = next_row_res?; @@ -693,9 +692,9 @@ impl GenericSession { /// /// # Example /// ```rust - /// # use scylla::LegacySession; + /// # use scylla::Session; /// # use std::error::Error; - /// # async fn check_only_compiles(session: &LegacySession) -> Result<(), Box> { + /// # async fn check_only_compiles(session: &Session) -> Result<(), Box> { /// use scylla::prepared_statement::PreparedStatement; /// /// // Prepare the query for later execution @@ -729,9 +728,9 @@ impl GenericSession { /// # Example /// /// ```rust - /// # use scylla::LegacySession; + /// # use scylla::Session; /// # use std::error::Error; - /// # async fn check_only_compiles(session: &LegacySession) -> Result<(), Box> { + /// # async fn check_only_compiles(session: &Session) -> Result<(), Box> { /// use std::ops::ControlFlow; /// use scylla::query::Query; /// use scylla::statement::{PagingState, PagingStateResponse}; @@ -751,7 +750,7 @@ impl GenericSession { /// .await?; /// /// // Do something with a single page of results. - /// for row in res.rows_typed::<(i32, String)>()? { + /// for row in res.rows::<(i32, String)>()? { /// let (a, b) = row?; /// } /// @@ -797,12 +796,11 @@ impl GenericSession { /// # Example /// /// ```rust - /// # use scylla::LegacySession; + /// # use scylla::Session; /// # use std::error::Error; - /// # async fn check_only_compiles(session: &LegacySession) -> Result<(), Box> { + /// # async fn check_only_compiles(session: &Session) -> Result<(), Box> { /// use scylla::prepared_statement::PreparedStatement; /// use scylla::IntoTypedRows; - /// use futures::stream::StreamExt; /// /// // Prepare the query for later execution /// let prepared: PreparedStatement = session @@ -813,7 +811,7 @@ impl GenericSession { /// let mut rows_stream = session /// .execute_iter(prepared, &[]) /// .await? - /// .into_typed::<(i32, i32)>(); + /// .into_typed::<(i32, i32)>()?; /// /// while let Some(next_row_res) = rows_stream.next().await { /// let (a, b): (i32, i32) = next_row_res?; @@ -849,9 +847,9 @@ impl GenericSession { /// /// # Example /// ```rust - /// # use scylla::LegacySession; + /// # use scylla::Session; /// # use std::error::Error; - /// # async fn check_only_compiles(session: &LegacySession) -> Result<(), Box> { + /// # async fn check_only_compiles(session: &Session) -> Result<(), Box> { /// use scylla::batch::Batch; /// /// let mut batch: Batch = Default::default(); @@ -978,13 +976,13 @@ where /// ```rust /// # use std::error::Error; /// # async fn check_only_compiles() -> Result<(), Box> { - /// use scylla::{LegacySession, SessionConfig}; + /// use scylla::{Session, SessionConfig}; /// use scylla::transport::KnownNode; /// /// let mut config = SessionConfig::new(); /// config.known_nodes.push(KnownNode::Hostname("127.0.0.1:9042".to_string())); /// - /// let session: LegacySession = LegacySession::connect(config).await?; + /// let session: Session = Session::connect(config).await?; /// # Ok(()) /// # } /// ``` @@ -1316,9 +1314,9 @@ where /// /// # Example /// ```rust - /// # use scylla::LegacySession; + /// # use scylla::Session; /// # use std::error::Error; - /// # async fn check_only_compiles(session: &LegacySession) -> Result<(), Box> { + /// # async fn check_only_compiles(session: &Session) -> Result<(), Box> { /// use scylla::prepared_statement::PreparedStatement; /// /// // Prepare the query for later execution @@ -1645,9 +1643,9 @@ where /// /// # Example /// ```rust /// # extern crate scylla; - /// # use scylla::LegacySession; + /// # use scylla::Session; /// # use std::error::Error; - /// # async fn check_only_compiles(session: &LegacySession) -> Result<(), Box> { + /// # async fn check_only_compiles(session: &Session) -> Result<(), Box> { /// use scylla::batch::Batch; /// /// // Create a batch statement with unprepared statements @@ -1706,10 +1704,10 @@ where /// * `case_sensitive` - if set to true the generated query will put keyspace name in quotes /// # Example /// ```rust - /// # use scylla::{LegacySession, SessionBuilder}; + /// # use scylla::{Session, SessionBuilder}; /// # use scylla::transport::Compression; /// # async fn example() -> Result<(), Box> { - /// # let session = SessionBuilder::new().known_node("127.0.0.1:9042").build_legacy().await?; + /// # let session = SessionBuilder::new().known_node("127.0.0.1:9042").build().await?; /// session /// .query_unpaged("INSERT INTO my_keyspace.tab (a) VALUES ('test1')", &[]) /// .await?; diff --git a/scylla/src/transport/session_builder.rs b/scylla/src/transport/session_builder.rs index 73725d83e..31b653a5c 100644 --- a/scylla/src/transport/session_builder.rs +++ b/scylla/src/transport/session_builder.rs @@ -62,13 +62,13 @@ pub type CloudSessionBuilder = GenericSessionBuilder; /// # Example /// /// ``` -/// # use scylla::{LegacySession, SessionBuilder}; +/// # use scylla::{Session, SessionBuilder}; /// # use scylla::transport::Compression; /// # async fn example() -> Result<(), Box> { -/// let session: LegacySession = SessionBuilder::new() +/// let session: Session = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .compression(Some(Compression::Snappy)) -/// .build_legacy() +/// .build() /// .await?; /// # Ok(()) /// # } @@ -97,22 +97,22 @@ impl GenericSessionBuilder { /// Add a known node with a hostname /// # Examples /// ``` - /// # use scylla::{LegacySession, SessionBuilder}; + /// # use scylla::{Session, SessionBuilder}; /// # async fn example() -> Result<(), Box> { - /// let session: LegacySession = SessionBuilder::new() + /// let session: Session = SessionBuilder::new() /// .known_node("127.0.0.1:9042") - /// .build_legacy() + /// .build() /// .await?; /// # Ok(()) /// # } /// ``` /// /// ``` - /// # use scylla::{LegacySession, SessionBuilder}; + /// # use scylla::{Session, SessionBuilder}; /// # async fn example() -> Result<(), Box> { - /// let session: LegacySession = SessionBuilder::new() + /// let session: Session = SessionBuilder::new() /// .known_node("db1.example.com") - /// .build_legacy() + /// .build() /// .await?; /// # Ok(()) /// # } @@ -125,12 +125,12 @@ impl GenericSessionBuilder { /// Add a known node with an IP address /// # Example /// ``` - /// # use scylla::{LegacySession, SessionBuilder}; + /// # use scylla::{Session, SessionBuilder}; /// # use std::net::{SocketAddr, IpAddr, Ipv4Addr}; /// # async fn example() -> Result<(), Box> { - /// let session: LegacySession = SessionBuilder::new() + /// let session: Session = SessionBuilder::new() /// .known_node_addr(SocketAddr::new(IpAddr::V4(Ipv4Addr::new(127, 0, 0, 1)), 9042)) - /// .build_legacy() + /// .build() /// .await?; /// # Ok(()) /// # } @@ -143,11 +143,11 @@ impl GenericSessionBuilder { /// Add a list of known nodes with hostnames /// # Example /// ``` - /// # use scylla::{LegacySession, SessionBuilder}; + /// # use scylla::{Session, SessionBuilder}; /// # async fn example() -> Result<(), Box> { - /// let session: LegacySession = SessionBuilder::new() + /// let session: Session = SessionBuilder::new() /// .known_nodes(["127.0.0.1:9042", "db1.example.com"]) - /// .build_legacy() + /// .build() /// .await?; /// # Ok(()) /// # } @@ -160,15 +160,15 @@ impl GenericSessionBuilder { /// Add a list of known nodes with IP addresses /// # Example /// ``` - /// # use scylla::{LegacySession, SessionBuilder}; + /// # use scylla::{Session, SessionBuilder}; /// # use std::net::{SocketAddr, IpAddr, Ipv4Addr}; /// # async fn example() -> Result<(), Box> { /// let addr1 = SocketAddr::new(IpAddr::V4(Ipv4Addr::new(172, 17, 0, 3)), 9042); /// let addr2 = SocketAddr::new(IpAddr::V4(Ipv4Addr::new(172, 17, 0, 4)), 9042); /// - /// let session: LegacySession = SessionBuilder::new() + /// let session: Session = SessionBuilder::new() /// .known_nodes_addr([addr1, addr2]) - /// .build_legacy() + /// .build() /// .await?; /// # Ok(()) /// # } @@ -186,14 +186,14 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{LegacySession, SessionBuilder}; + /// # use scylla::{Session, SessionBuilder}; /// # use scylla::transport::Compression; /// # async fn example() -> Result<(), Box> { - /// let session: LegacySession = SessionBuilder::new() + /// let session: Session = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .use_keyspace("my_keyspace_name", false) /// .user("cassandra", "cassandra") - /// .build_legacy() + /// .build() /// .await?; /// # Ok(()) /// # } @@ -212,7 +212,7 @@ impl GenericSessionBuilder { /// ``` /// # use std::sync::Arc; /// use bytes::Bytes; - /// use scylla::{LegacySession, SessionBuilder}; + /// use scylla::{Session, SessionBuilder}; /// use async_trait::async_trait; /// use scylla::authentication::{AuthenticatorProvider, AuthenticatorSession, AuthError}; /// # use scylla::transport::Compression; @@ -240,12 +240,12 @@ impl GenericSessionBuilder { /// } /// /// # async fn example() -> Result<(), Box> { - /// let session: LegacySession = SessionBuilder::new() + /// let session: Session = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .use_keyspace("my_keyspace_name", false) /// .user("cassandra", "cassandra") /// .authenticator_provider(Arc::new(CustomAuthenticatorProvider)) - /// .build_legacy() + /// .build() /// .await?; /// # Ok(()) /// # } @@ -266,7 +266,7 @@ impl GenericSessionBuilder { /// # use async_trait::async_trait; /// # use std::net::SocketAddr; /// # use std::sync::Arc; - /// # use scylla::{LegacySession, SessionBuilder}; + /// # use scylla::{Session, SessionBuilder}; /// # use scylla::transport::session::{AddressTranslator, TranslationError}; /// # use scylla::transport::topology::UntranslatedPeer; /// struct IdentityTranslator; @@ -282,10 +282,10 @@ impl GenericSessionBuilder { /// } /// /// # async fn example() -> Result<(), Box> { - /// let session: LegacySession = SessionBuilder::new() + /// let session: Session = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .address_translator(Arc::new(IdentityTranslator)) - /// .build_legacy() + /// .build() /// .await?; /// # Ok(()) /// # } @@ -296,7 +296,7 @@ impl GenericSessionBuilder { /// # use std::sync::Arc; /// # use std::collections::HashMap; /// # use std::str::FromStr; - /// # use scylla::{LegacySession, SessionBuilder}; + /// # use scylla::{Session, SessionBuilder}; /// # use scylla::transport::session::{AddressTranslator, TranslationError}; /// # /// # async fn example() -> Result<(), Box> { @@ -304,10 +304,10 @@ impl GenericSessionBuilder { /// let addr_before_translation = SocketAddr::from_str("192.168.0.42:19042").unwrap(); /// let addr_after_translation = SocketAddr::from_str("157.123.12.42:23203").unwrap(); /// translation_rules.insert(addr_before_translation, addr_after_translation); - /// let session: LegacySession = SessionBuilder::new() + /// let session: Session = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .address_translator(Arc::new(translation_rules)) - /// .build_legacy() + /// .build() /// .await?; /// # Ok(()) /// # } @@ -327,7 +327,7 @@ impl GenericSessionBuilder { /// ``` /// # use std::fs; /// # use std::path::PathBuf; - /// # use scylla::{LegacySession, SessionBuilder}; + /// # use scylla::{Session, SessionBuilder}; /// # use openssl::ssl::{SslContextBuilder, SslVerifyMode, SslMethod, SslFiletype}; /// # async fn example() -> Result<(), Box> { /// let certdir = fs::canonicalize(PathBuf::from("./examples/certs/scylla.crt"))?; @@ -335,10 +335,10 @@ impl GenericSessionBuilder { /// context_builder.set_certificate_file(certdir.as_path(), SslFiletype::PEM)?; /// context_builder.set_verify(SslVerifyMode::NONE); /// - /// let session: LegacySession = SessionBuilder::new() + /// let session: Session = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .ssl_context(Some(context_builder.build())) - /// .build_legacy() + /// .build() /// .await?; /// # Ok(()) /// # } @@ -350,8 +350,8 @@ impl GenericSessionBuilder { } } -// NOTE: this `impl` block contains configuration options specific for **Cloud** [`LegacySession`]. -// This means that if an option fits both non-Cloud and Cloud `LegacySession`s, it should NOT be put +// NOTE: this `impl` block contains configuration options specific for **Cloud** [`Session`]. +// This means that if an option fits both non-Cloud and Cloud `Session`s, it should NOT be put // here, but rather in `impl GenericSessionBuilder` block. #[cfg(feature = "cloud")] impl CloudSessionBuilder { @@ -386,13 +386,13 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{LegacySession, SessionBuilder}; + /// # use scylla::{Session, SessionBuilder}; /// # use scylla::transport::Compression; /// # async fn example() -> Result<(), Box> { - /// let session: LegacySession = SessionBuilder::new() + /// let session: Session = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .compression(Some(Compression::Snappy)) - /// .build_legacy() + /// .build() /// .await?; /// # Ok(()) /// # } @@ -407,13 +407,13 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{LegacySession, SessionBuilder}; + /// # use scylla::{Session, SessionBuilder}; /// # use std::time::Duration; /// # async fn example() -> Result<(), Box> { - /// let session: LegacySession = SessionBuilder::new() + /// let session: Session = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .schema_agreement_interval(Duration::from_secs(5)) - /// .build_legacy() + /// .build() /// .await?; /// # Ok(()) /// # } @@ -427,17 +427,17 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{statement::Consistency, ExecutionProfile, LegacySession, SessionBuilder}; + /// # use scylla::{statement::Consistency, ExecutionProfile, Session, SessionBuilder}; /// # use std::time::Duration; /// # async fn example() -> Result<(), Box> { /// let execution_profile = ExecutionProfile::builder() /// .consistency(Consistency::All) /// .request_timeout(Some(Duration::from_secs(2))) /// .build(); - /// let session: LegacySession = SessionBuilder::new() + /// let session: Session = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .default_execution_profile_handle(execution_profile.into_handle()) - /// .build_legacy() + /// .build() /// .await?; /// # Ok(()) /// # } @@ -455,12 +455,12 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{LegacySession, SessionBuilder}; + /// # use scylla::{Session, SessionBuilder}; /// # async fn example() -> Result<(), Box> { - /// let session: LegacySession = SessionBuilder::new() + /// let session: Session = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .tcp_nodelay(true) - /// .build_legacy() + /// .build() /// .await?; /// # Ok(()) /// # } @@ -478,12 +478,12 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{LegacySession, SessionBuilder}; + /// # use scylla::{Session, SessionBuilder}; /// # async fn example() -> Result<(), Box> { - /// let session: LegacySession = SessionBuilder::new() + /// let session: Session = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .tcp_keepalive_interval(std::time::Duration::from_secs(42)) - /// .build_legacy() + /// .build() /// .await?; /// # Ok(()) /// # } @@ -506,13 +506,13 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{LegacySession, SessionBuilder}; + /// # use scylla::{Session, SessionBuilder}; /// # use scylla::transport::Compression; /// # async fn example() -> Result<(), Box> { - /// let session: LegacySession = SessionBuilder::new() + /// let session: Session = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .use_keyspace("my_keyspace_name", false) - /// .build_legacy() + /// .build() /// .await?; /// # Ok(()) /// # } @@ -536,7 +536,7 @@ impl GenericSessionBuilder { /// let session: LegacySession = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .compression(Some(Compression::Snappy)) - /// .build_legacy() // Turns SessionBuilder into Session + /// .build_legacy() // Turns SessionBuilder into LegacySession /// .await?; /// # Ok(()) /// # } @@ -550,7 +550,7 @@ impl GenericSessionBuilder { /// Builds the Session after setting all the options. /// /// The new session object uses the new deserialization API. If you wish - /// to use the old API, use [`SessionBuilder::build_legacy`]. + /// to use the old API, use [`SessionBuilder::build`]. /// /// # Example /// ``` @@ -577,13 +577,13 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{LegacySession, SessionBuilder}; + /// # use scylla::{Session, SessionBuilder}; /// # use std::time::Duration; /// # async fn example() -> Result<(), Box> { - /// let session: LegacySession = SessionBuilder::new() + /// let session: Session = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .connection_timeout(Duration::from_secs(30)) - /// .build_legacy() // Turns SessionBuilder into Session + /// .build() // Turns SessionBuilder into Session /// .await?; /// # Ok(()) /// # } @@ -598,17 +598,17 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{LegacySession, SessionBuilder}; + /// # use scylla::{Session, SessionBuilder}; /// # async fn example() -> Result<(), Box> { /// use std::num::NonZeroUsize; /// use scylla::transport::session::PoolSize; /// /// // This session will establish 4 connections to each node. /// // For Scylla clusters, this number will be divided across shards - /// let session: LegacySession = SessionBuilder::new() + /// let session: Session = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .pool_size(PoolSize::PerHost(NonZeroUsize::new(4).unwrap())) - /// .build_legacy() + /// .build() /// .await?; /// # Ok(()) /// # } @@ -642,12 +642,12 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{LegacySession, SessionBuilder}; + /// # use scylla::{Session, SessionBuilder}; /// # async fn example() -> Result<(), Box> { - /// let session: LegacySession = SessionBuilder::new() + /// let session: Session = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .disallow_shard_aware_port(true) - /// .build_legacy() + /// .build() /// .await?; /// # Ok(()) /// # } @@ -662,12 +662,12 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{LegacySession, SessionBuilder}; + /// # use scylla::{Session, SessionBuilder}; /// # async fn example() -> Result<(), Box> { - /// let session: LegacySession = SessionBuilder::new() + /// let session: Session = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .keyspaces_to_fetch(["my_keyspace"]) - /// .build_legacy() + /// .build() /// .await?; /// # Ok(()) /// # } @@ -685,12 +685,12 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{LegacySession, SessionBuilder}; + /// # use scylla::{Session, SessionBuilder}; /// # async fn example() -> Result<(), Box> { - /// let session: LegacySession = SessionBuilder::new() + /// let session: Session = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .fetch_schema_metadata(true) - /// .build_legacy() + /// .build() /// .await?; /// # Ok(()) /// # } @@ -708,12 +708,12 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{LegacySession, SessionBuilder}; + /// # use scylla::{Session, SessionBuilder}; /// # async fn example() -> Result<(), Box> { - /// let session: LegacySession = SessionBuilder::new() + /// let session: Session = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .keepalive_interval(std::time::Duration::from_secs(42)) - /// .build_legacy() + /// .build() /// .await?; /// # Ok(()) /// # } @@ -738,12 +738,12 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{LegacySession, SessionBuilder}; + /// # use scylla::{Session, SessionBuilder}; /// # async fn example() -> Result<(), Box> { - /// let session: LegacySession = SessionBuilder::new() + /// let session: Session = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .keepalive_timeout(std::time::Duration::from_secs(42)) - /// .build_legacy() + /// .build() /// .await?; /// # Ok(()) /// # } @@ -765,12 +765,12 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{LegacySession, SessionBuilder}; + /// # use scylla::{Session, SessionBuilder}; /// # async fn example() -> Result<(), Box> { - /// let session: LegacySession = SessionBuilder::new() + /// let session: Session = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .schema_agreement_timeout(std::time::Duration::from_secs(120)) - /// .build_legacy() + /// .build() /// .await?; /// # Ok(()) /// # } @@ -785,12 +785,12 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{LegacySession, SessionBuilder}; + /// # use scylla::{Session, SessionBuilder}; /// # async fn example() -> Result<(), Box> { - /// let session: LegacySession = SessionBuilder::new() + /// let session: Session = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .auto_await_schema_agreement(false) - /// .build_legacy() + /// .build() /// .await?; /// # Ok(()) /// # } @@ -813,16 +813,16 @@ impl GenericSessionBuilder { /// # use async_trait::async_trait; /// # use std::net::SocketAddr; /// # use std::sync::Arc; - /// # use scylla::{LegacySession, SessionBuilder}; + /// # use scylla::{Session, SessionBuilder}; /// # use scylla::transport::session::{AddressTranslator, TranslationError}; /// # use scylla::transport::host_filter::DcHostFilter; /// /// # async fn example() -> Result<(), Box> { /// // The session will only connect to nodes from "my-local-dc" - /// let session: LegacySession = SessionBuilder::new() + /// let session: Session = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .host_filter(Arc::new(DcHostFilter::new("my-local-dc".to_string()))) - /// .build_legacy() + /// .build() /// .await?; /// # Ok(()) /// # } @@ -837,12 +837,12 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{LegacySession, SessionBuilder}; + /// # use scylla::{Session, SessionBuilder}; /// # async fn example() -> Result<(), Box> { - /// let session: LegacySession = SessionBuilder::new() + /// let session: Session = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .refresh_metadata_on_auto_schema_agreement(true) - /// .build_legacy() + /// .build() /// .await?; /// # Ok(()) /// # } @@ -865,13 +865,13 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{LegacySession, SessionBuilder}; + /// # use scylla::{Session, SessionBuilder}; /// # use std::num::NonZeroU32; /// # async fn example() -> Result<(), Box> { - /// let session: LegacySession = SessionBuilder::new() + /// let session: Session = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .tracing_info_fetch_attempts(NonZeroU32::new(10).unwrap()) - /// .build_legacy() + /// .build() /// .await?; /// # Ok(()) /// # } @@ -894,13 +894,13 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{LegacySession, SessionBuilder}; + /// # use scylla::{Session, SessionBuilder}; /// # use std::time::Duration; /// # async fn example() -> Result<(), Box> { - /// let session: LegacySession = SessionBuilder::new() + /// let session: Session = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .tracing_info_fetch_interval(Duration::from_millis(50)) - /// .build_legacy() + /// .build() /// .await?; /// # Ok(()) /// # } @@ -916,12 +916,12 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{LegacySession, SessionBuilder, statement::Consistency}; + /// # use scylla::{Session, SessionBuilder, statement::Consistency}; /// # async fn example() -> Result<(), Box> { - /// let session: LegacySession = SessionBuilder::new() + /// let session: Session = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .tracing_info_fetch_consistency(Consistency::One) - /// .build_legacy() + /// .build() /// .await?; /// # Ok(()) /// # } @@ -946,13 +946,13 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{LegacySession, SessionBuilder}; + /// # use scylla::{Session, SessionBuilder}; /// # use scylla::transport::Compression; /// # async fn example() -> Result<(), Box> { - /// let session: LegacySession = SessionBuilder::new() + /// let session: Session = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .write_coalescing(false) // Enabled by default - /// .build_legacy() + /// .build() /// .await?; /// # Ok(()) /// # } @@ -971,12 +971,12 @@ impl GenericSessionBuilder { /// means that the metadata is refreshed every 20 seconds. /// # Example /// ``` - /// # use scylla::{LegacySession, SessionBuilder}; + /// # use scylla::{Session, SessionBuilder}; /// # async fn example() -> Result<(), Box> { - /// let session: LegacySession = SessionBuilder::new() + /// let session: Session = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .cluster_metadata_refresh_interval(std::time::Duration::from_secs(20)) - /// .build_legacy() + /// .build() /// .await?; /// # Ok(()) /// # } @@ -994,13 +994,13 @@ impl GenericSessionBuilder { /// /// # Example /// ``` - /// # use scylla::{LegacySession, SessionBuilder}; + /// # use scylla::{Session, SessionBuilder}; /// # use scylla::transport::SelfIdentity; /// # async fn example() -> Result<(), Box> { /// let (app_major, app_minor, app_patch) = (2, 1, 3); /// let app_version = format!("{app_major}.{app_minor}.{app_patch}"); /// - /// let session: LegacySession = SessionBuilder::new() + /// let session: Session = SessionBuilder::new() /// .known_node("127.0.0.1:9042") /// .custom_identity( /// SelfIdentity::new() @@ -1008,7 +1008,7 @@ impl GenericSessionBuilder { /// .with_application_name("my-app") /// .with_application_version(app_version) /// ) - /// .build_legacy() + /// .build() /// .await?; /// # Ok(()) /// # } From a2616585afc7b132ef921ee68de10c6885004e35 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Wojciech=20Przytu=C5=82a?= Date: Tue, 21 May 2024 17:47:45 +0200 Subject: [PATCH 40/42] session_test: regression test empty collections deserialization ScyllaDB does not distinguish empty collections from nulls. That is, INSERTing an empty collection is equivalent to nullifying the corresponding column. As pointed out in [#1001](https://github.com/scylladb/scylla-rust-driver/issues/1001), it's a nice QOL feature to be able to deserialize empty CQL collections to empty Rust collections instead of `None::`. A test is added that checks it. --- scylla/src/transport/session_test.rs | 72 ++++++++++++++++++++++++++-- 1 file changed, 69 insertions(+), 3 deletions(-) diff --git a/scylla/src/transport/session_test.rs b/scylla/src/transport/session_test.rs index 9bb4fc5a5..bde64faaf 100644 --- a/scylla/src/transport/session_test.rs +++ b/scylla/src/transport/session_test.rs @@ -1,4 +1,5 @@ use crate::batch::{Batch, BatchStatement}; +use crate::deserialize::DeserializeValue; use crate::prepared_statement::PreparedStatement; use crate::query::Query; use crate::retry_policy::{QueryInfo, RetryDecision, RetryPolicy, RetrySession}; @@ -25,12 +26,12 @@ use assert_matches::assert_matches; use futures::{FutureExt, TryStreamExt}; use itertools::Itertools; use scylla_cql::frame::request::query::{PagingState, PagingStateResponse}; -use scylla_cql::frame::response::result::ColumnType; -use scylla_cql::frame::response::result::Row; +use scylla_cql::frame::response::result::{ColumnType, Row}; +use scylla_cql::frame::value::CqlVarint; use scylla_cql::types::serialize::row::{SerializeRow, SerializedValues}; use scylla_cql::types::serialize::value::SerializeValue; -use std::collections::BTreeSet; use std::collections::{BTreeMap, HashMap}; +use std::collections::{BTreeSet, HashSet}; use std::sync::atomic::{AtomicBool, Ordering}; use std::sync::Arc; use tokio::net::TcpListener; @@ -3097,3 +3098,68 @@ async fn test_manual_primary_key_computation() { .await; } } + +/// ScyllaDB does not distinguish empty collections from nulls. That is, INSERTing an empty collection +/// is equivalent to nullifying the corresponding column. +/// As pointed out in [#1001](https://github.com/scylladb/scylla-rust-driver/issues/1001), it's a nice +/// QOL feature to be able to deserialize empty CQL collections to empty Rust collections instead of +/// `None::`. This test checks that. +#[tokio::test] +async fn test_deserialize_empty_collections() { + // Setup session. + let ks = unique_keyspace_name(); + let session = create_new_session_builder().build().await.unwrap(); + session.query_unpaged(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); + session.use_keyspace(&ks, true).await.unwrap(); + + async fn deserialize_empty_collection< + Collection: Default + for<'frame> DeserializeValue<'frame, 'frame> + SerializeValue, + >( + session: &Session, + collection_name: &str, + collection_type_params: &str, + ) -> Collection { + // Create a table for the given collection type. + let table_name = "test_empty_".to_owned() + collection_name; + let query = format!( + "CREATE TABLE {} (n int primary key, c {}<{}>)", + table_name, collection_name, collection_type_params + ); + session.query_unpaged(query, ()).await.unwrap(); + + // Populate the table with an empty collection, effectively inserting null as the collection. + session + .query_unpaged( + format!("INSERT INTO {} (n, c) VALUES (?, ?)", table_name,), + (0, Collection::default()), + ) + .await + .unwrap(); + + let query_rows_result = session + .query_unpaged(format!("SELECT c FROM {}", table_name), ()) + .await + .unwrap() + .into_rows_result() + .unwrap() + .unwrap(); + let (collection,) = query_rows_result.first_row::<(Collection,)>().unwrap(); + + // Drop the table + collection + } + + let list = deserialize_empty_collection::>(&session, "list", "int").await; + assert!(list.is_empty()); + + let set = deserialize_empty_collection::>(&session, "set", "bigint").await; + assert!(set.is_empty()); + + let map = deserialize_empty_collection::>( + &session, + "map", + "boolean, varint", + ) + .await; + assert!(map.is_empty()); +} From 5561b1785c0a4ebd93c3fda72c6ba55fdf695dd7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Wojciech=20Przytu=C5=82a?= Date: Mon, 28 Oct 2024 12:51:39 +0100 Subject: [PATCH 41/42] treewide tests: remove needless vec![] allocations There were plenty of places were using a stack-allocated slice &[] suffices. Not to promote bad practice of redundant heap allocation and to possibly quicken our tests, vec![x, ...] was replaced with &[x, ...] where possible. --- scylla-cql/src/frame/mod.rs | 4 +- scylla-cql/src/frame/response/result.rs | 32 +- scylla-cql/src/frame/types.rs | 404 +++++++++--------- scylla-cql/src/frame/value_tests.rs | 142 +++--- .../src/types/deserialize/value_tests.rs | 8 +- scylla-cql/src/types/serialize/row.rs | 8 +- scylla/src/transport/caching_session.rs | 2 +- scylla/src/transport/cql_collections_test.rs | 4 +- scylla/src/transport/cql_types_test.rs | 60 +-- scylla/src/transport/cql_value_test.rs | 12 +- .../downgrading_consistency_retry_policy.rs | 4 +- scylla/src/transport/legacy_query_result.rs | 16 +- scylla/src/transport/locator/mod.rs | 12 +- .../transport/locator/precomputed_replicas.rs | 30 +- .../src/transport/locator/replication_info.rs | 62 +-- scylla/src/transport/locator/token_ring.rs | 30 +- scylla/src/transport/retry_policy.rs | 4 +- scylla/src/transport/session_builder.rs | 10 +- scylla/src/transport/session_test.rs | 54 +-- 19 files changed, 434 insertions(+), 464 deletions(-) diff --git a/scylla-cql/src/frame/mod.rs b/scylla-cql/src/frame/mod.rs index 4f8dc8beb..3b132350d 100644 --- a/scylla-cql/src/frame/mod.rs +++ b/scylla-cql/src/frame/mod.rs @@ -302,9 +302,9 @@ mod test { let mut out = Vec::from(&b"Hello"[..]); let uncomp_body = b", World!"; let compression = Compression::Lz4; - let expect = vec![ + let expect = &[ 72, 101, 108, 108, 111, 0, 0, 0, 8, 128, 44, 32, 87, 111, 114, 108, 100, 33, - ]; + ][..]; compress_append(uncomp_body, compression, &mut out).unwrap(); assert_eq!(expect, out); diff --git a/scylla-cql/src/frame/response/result.rs b/scylla-cql/src/frame/response/result.rs index 34edac89d..173bf43a3 100644 --- a/scylla-cql/src/frame/response/result.rs +++ b/scylla-cql/src/frame/response/result.rs @@ -1730,7 +1730,7 @@ mod tests { #[test] fn test_deserialize_text_types() { - let buf: Vec = vec![0x41]; + let buf = [0x41]; let int_slice = &mut &buf[..]; let ascii_serialized = super::deser_cql_value(&ColumnType::Ascii, int_slice).unwrap(); let text_serialized = super::deser_cql_value(&ColumnType::Text, int_slice).unwrap(); @@ -1742,7 +1742,7 @@ mod tests { fn test_deserialize_uuid_inet_types() { let my_uuid = Uuid::parse_str("00000000000000000000000000000001").unwrap(); - let uuid_buf: Vec = vec![0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1]; + let uuid_buf = [0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1]; let uuid_slice = &mut &uuid_buf[..]; let uuid_serialize = super::deser_cql_value(&ColumnType::Uuid, uuid_slice).unwrap(); assert_eq!(uuid_serialize, CqlValue::Uuid(my_uuid)); @@ -1753,13 +1753,13 @@ mod tests { assert_eq!(time_uuid_serialize, CqlValue::Timeuuid(my_timeuuid)); let my_ip = "::1".parse().unwrap(); - let ip_buf: Vec = vec![0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1]; + let ip_buf = [0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1]; let ip_slice = &mut &ip_buf[..]; let ip_serialize = super::deser_cql_value(&ColumnType::Inet, ip_slice).unwrap(); assert_eq!(ip_serialize, CqlValue::Inet(my_ip)); let max_ip = "ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff".parse().unwrap(); - let max_ip_buf: Vec = vec![ + let max_ip_buf = [ 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, ]; let max_ip_slice = &mut &max_ip_buf[..]; @@ -1772,12 +1772,12 @@ mod tests { let float: f32 = 0.5; let double: f64 = 2.0; - let float_buf: Vec = vec![63, 0, 0, 0]; + let float_buf = [63, 0, 0, 0]; let float_slice = &mut &float_buf[..]; let float_serialize = super::deser_cql_value(&ColumnType::Float, float_slice).unwrap(); assert_eq!(float_serialize, CqlValue::Float(float)); - let double_buf: Vec = vec![64, 0, 0, 0, 0, 0, 0, 0]; + let double_buf = [64, 0, 0, 0, 0, 0, 0, 0]; let double_slice = &mut &double_buf[..]; let double_serialize = super::deser_cql_value(&ColumnType::Double, double_slice).unwrap(); assert_eq!(double_serialize, CqlValue::Double(double)); @@ -1790,7 +1790,7 @@ mod tests { } #[cfg(any(feature = "num-bigint-03", feature = "num-bigint-04"))] - fn varint_test_cases_from_spec() -> Vec { + fn varint_test_cases_from_spec() -> [VarintTestCase; 8] { /* Table taken from CQL Binary Protocol v4 spec @@ -1805,7 +1805,7 @@ mod tests { -128 | 0x80 -129 | 0xFF7F */ - vec![ + [ VarintTestCase { value: 0, encoding: vec![0x00], @@ -1906,7 +1906,7 @@ mod tests { #[test] fn test_deserialize_counter() { - let counter: Vec = vec![0, 0, 0, 0, 0, 0, 1, 0]; + let counter = [0, 0, 0, 0, 0, 0, 1, 0]; let counter_slice = &mut &counter[..]; let counter_serialize = super::deser_cql_value(&ColumnType::Counter, counter_slice).unwrap(); @@ -1923,12 +1923,12 @@ mod tests { #[test] fn test_deserialize_bool() { - let bool_buf: Vec = vec![0x00]; + let bool_buf = [0x00]; let bool_slice = &mut &bool_buf[..]; let bool_serialize = super::deser_cql_value(&ColumnType::Boolean, bool_slice).unwrap(); assert_eq!(bool_serialize, CqlValue::Boolean(false)); - let bool_buf: Vec = vec![0x01]; + let bool_buf = [0x01]; let bool_slice = &mut &bool_buf[..]; let bool_serialize = super::deser_cql_value(&ColumnType::Boolean, bool_slice).unwrap(); assert_eq!(bool_serialize, CqlValue::Boolean(true)); @@ -1936,24 +1936,24 @@ mod tests { #[test] fn test_deserialize_int_types() { - let int_buf: Vec = vec![0, 0, 0, 4]; + let int_buf = [0, 0, 0, 4]; let int_slice = &mut &int_buf[..]; let int_serialized = super::deser_cql_value(&ColumnType::Int, int_slice).unwrap(); assert_eq!(int_serialized, CqlValue::Int(4)); - let smallint_buf: Vec = vec![0, 4]; + let smallint_buf = [0, 4]; let smallint_slice = &mut &smallint_buf[..]; let smallint_serialized = super::deser_cql_value(&ColumnType::SmallInt, smallint_slice).unwrap(); assert_eq!(smallint_serialized, CqlValue::SmallInt(4)); - let tinyint_buf: Vec = vec![4]; + let tinyint_buf = [4]; let tinyint_slice = &mut &tinyint_buf[..]; let tinyint_serialized = super::deser_cql_value(&ColumnType::TinyInt, tinyint_slice).unwrap(); assert_eq!(tinyint_serialized, CqlValue::TinyInt(4)); - let bigint_buf: Vec = vec![0, 0, 0, 0, 0, 0, 0, 4]; + let bigint_buf = [0, 0, 0, 0, 0, 0, 0, 4]; let bigint_slice = &mut &bigint_buf[..]; let bigint_serialized = super::deser_cql_value(&ColumnType::BigInt, bigint_slice).unwrap(); assert_eq!(bigint_serialized, CqlValue::BigInt(4)); @@ -2442,7 +2442,7 @@ mod tests { let mut v = Vec::new(); empty.serialize(&mut v).unwrap(); - assert_eq!(v, vec![0, 0, 0, 0]); + assert_eq!(v, &[0, 0, 0, 0]); } #[test] diff --git a/scylla-cql/src/frame/types.rs b/scylla-cql/src/frame/types.rs index 70f28f6c2..edc304789 100644 --- a/scylla-cql/src/frame/types.rs +++ b/scylla-cql/src/frame/types.rs @@ -461,14 +461,14 @@ pub fn write_string_list( #[test] fn type_string_list() { - let val = vec![ + let val = [ "".to_owned(), "CQL_VERSION".to_owned(), "THROW_ON_OVERLOAD".to_owned(), ]; let mut buf = Vec::new(); - write_string_list(&val, &mut buf).unwrap(); + write_string_list(&val[..], &mut buf).unwrap(); assert_eq!(read_string_list(&mut &buf[..]).unwrap(), val); } @@ -697,218 +697,200 @@ fn zig_zag_decode_test() { #[test] fn unsigned_vint_encode_and_decode_test() { - let unsigned_vint_encoding = vec![ - (0, vec![0]), - (1, vec![1]), - (2, vec![2]), - ((1 << 2) - 1, vec![3]), - (1 << 2, vec![4]), - ((1 << 2) + 1, vec![5]), - ((1 << 3) - 1, vec![7]), - (1 << 3, vec![8]), - ((1 << 3) + 1, vec![9]), - ((1 << 4) - 1, vec![15]), - (1 << 4, vec![16]), - ((1 << 4) + 1, vec![17]), - ((1 << 5) - 1, vec![31]), - (1 << 5, vec![32]), - ((1 << 5) + 1, vec![33]), - ((1 << 6) - 1, vec![63]), - (1 << 6, vec![64]), - ((1 << 6) + 1, vec![65]), - ((1 << 7) - 1, vec![127]), - (1 << 7, vec![128, 128]), - ((1 << 7) + 1, vec![128, 129]), - ((1 << 8) - 1, vec![128, 255]), - (1 << 8, vec![129, 0]), - ((1 << 8) + 1, vec![129, 1]), - ((1 << 9) - 1, vec![129, 255]), - (1 << 9, vec![130, 0]), - ((1 << 9) + 1, vec![130, 1]), - ((1 << 10) - 1, vec![131, 255]), - (1 << 10, vec![132, 0]), - ((1 << 10) + 1, vec![132, 1]), - ((1 << 11) - 1, vec![135, 255]), - (1 << 11, vec![136, 0]), - ((1 << 11) + 1, vec![136, 1]), - ((1 << 12) - 1, vec![143, 255]), - (1 << 12, vec![144, 0]), - ((1 << 12) + 1, vec![144, 1]), - ((1 << 13) - 1, vec![159, 255]), - (1 << 13, vec![160, 0]), - ((1 << 13) + 1, vec![160, 1]), - ((1 << 14) - 1, vec![191, 255]), - (1 << 14, vec![192, 64, 0]), - ((1 << 14) + 1, vec![192, 64, 1]), - ((1 << 15) - 1, vec![192, 127, 255]), - (1 << 15, vec![192, 128, 0]), - ((1 << 15) + 1, vec![192, 128, 1]), - ((1 << 16) - 1, vec![192, 255, 255]), - (1 << 16, vec![193, 0, 0]), - ((1 << 16) + 1, vec![193, 0, 1]), - ((1 << 17) - 1, vec![193, 255, 255]), - (1 << 17, vec![194, 0, 0]), - ((1 << 17) + 1, vec![194, 0, 1]), - ((1 << 18) - 1, vec![195, 255, 255]), - (1 << 18, vec![196, 0, 0]), - ((1 << 18) + 1, vec![196, 0, 1]), - ((1 << 19) - 1, vec![199, 255, 255]), - (1 << 19, vec![200, 0, 0]), - ((1 << 19) + 1, vec![200, 0, 1]), - ((1 << 20) - 1, vec![207, 255, 255]), - (1 << 20, vec![208, 0, 0]), - ((1 << 20) + 1, vec![208, 0, 1]), - ((1 << 21) - 1, vec![223, 255, 255]), - (1 << 21, vec![224, 32, 0, 0]), - ((1 << 21) + 1, vec![224, 32, 0, 1]), - ((1 << 22) - 1, vec![224, 63, 255, 255]), - (1 << 22, vec![224, 64, 0, 0]), - ((1 << 22) + 1, vec![224, 64, 0, 1]), - ((1 << 23) - 1, vec![224, 127, 255, 255]), - (1 << 23, vec![224, 128, 0, 0]), - ((1 << 23) + 1, vec![224, 128, 0, 1]), - ((1 << 24) - 1, vec![224, 255, 255, 255]), - (1 << 24, vec![225, 0, 0, 0]), - ((1 << 24) + 1, vec![225, 0, 0, 1]), - ((1 << 25) - 1, vec![225, 255, 255, 255]), - (1 << 25, vec![226, 0, 0, 0]), - ((1 << 25) + 1, vec![226, 0, 0, 1]), - ((1 << 26) - 1, vec![227, 255, 255, 255]), - (1 << 26, vec![228, 0, 0, 0]), - ((1 << 26) + 1, vec![228, 0, 0, 1]), - ((1 << 27) - 1, vec![231, 255, 255, 255]), - (1 << 27, vec![232, 0, 0, 0]), - ((1 << 27) + 1, vec![232, 0, 0, 1]), - ((1 << 28) - 1, vec![239, 255, 255, 255]), - (1 << 28, vec![240, 16, 0, 0, 0]), - ((1 << 28) + 1, vec![240, 16, 0, 0, 1]), - ((1 << 29) - 1, vec![240, 31, 255, 255, 255]), - (1 << 29, vec![240, 32, 0, 0, 0]), - ((1 << 29) + 1, vec![240, 32, 0, 0, 1]), - ((1 << 30) - 1, vec![240, 63, 255, 255, 255]), - (1 << 30, vec![240, 64, 0, 0, 0]), - ((1 << 30) + 1, vec![240, 64, 0, 0, 1]), - ((1 << 31) - 1, vec![240, 127, 255, 255, 255]), - (1 << 31, vec![240, 128, 0, 0, 0]), - ((1 << 31) + 1, vec![240, 128, 0, 0, 1]), - ((1 << 32) - 1, vec![240, 255, 255, 255, 255]), - (1 << 32, vec![241, 0, 0, 0, 0]), - ((1 << 32) + 1, vec![241, 0, 0, 0, 1]), - ((1 << 33) - 1, vec![241, 255, 255, 255, 255]), - (1 << 33, vec![242, 0, 0, 0, 0]), - ((1 << 33) + 1, vec![242, 0, 0, 0, 1]), - ((1 << 34) - 1, vec![243, 255, 255, 255, 255]), - (1 << 34, vec![244, 0, 0, 0, 0]), - ((1 << 34) + 1, vec![244, 0, 0, 0, 1]), - ((1 << 35) - 1, vec![247, 255, 255, 255, 255]), - (1 << 35, vec![248, 8, 0, 0, 0, 0]), - ((1 << 35) + 1, vec![248, 8, 0, 0, 0, 1]), - ((1 << 36) - 1, vec![248, 15, 255, 255, 255, 255]), - (1 << 36, vec![248, 16, 0, 0, 0, 0]), - ((1 << 36) + 1, vec![248, 16, 0, 0, 0, 1]), - ((1 << 37) - 1, vec![248, 31, 255, 255, 255, 255]), - (1 << 37, vec![248, 32, 0, 0, 0, 0]), - ((1 << 37) + 1, vec![248, 32, 0, 0, 0, 1]), - ((1 << 38) - 1, vec![248, 63, 255, 255, 255, 255]), - (1 << 38, vec![248, 64, 0, 0, 0, 0]), - ((1 << 38) + 1, vec![248, 64, 0, 0, 0, 1]), - ((1 << 39) - 1, vec![248, 127, 255, 255, 255, 255]), - (1 << 39, vec![248, 128, 0, 0, 0, 0]), - ((1 << 39) + 1, vec![248, 128, 0, 0, 0, 1]), - ((1 << 40) - 1, vec![248, 255, 255, 255, 255, 255]), - (1 << 40, vec![249, 0, 0, 0, 0, 0]), - ((1 << 40) + 1, vec![249, 0, 0, 0, 0, 1]), - ((1 << 41) - 1, vec![249, 255, 255, 255, 255, 255]), - (1 << 41, vec![250, 0, 0, 0, 0, 0]), - ((1 << 41) + 1, vec![250, 0, 0, 0, 0, 1]), - ((1 << 42) - 1, vec![251, 255, 255, 255, 255, 255]), - (1 << 42, vec![252, 4, 0, 0, 0, 0, 0]), - ((1 << 42) + 1, vec![252, 4, 0, 0, 0, 0, 1]), - ((1 << 43) - 1, vec![252, 7, 255, 255, 255, 255, 255]), - (1 << 43, vec![252, 8, 0, 0, 0, 0, 0]), - ((1 << 43) + 1, vec![252, 8, 0, 0, 0, 0, 1]), - ((1 << 44) - 1, vec![252, 15, 255, 255, 255, 255, 255]), - (1 << 44, vec![252, 16, 0, 0, 0, 0, 0]), - ((1 << 44) + 1, vec![252, 16, 0, 0, 0, 0, 1]), - ((1 << 45) - 1, vec![252, 31, 255, 255, 255, 255, 255]), - (1 << 45, vec![252, 32, 0, 0, 0, 0, 0]), - ((1 << 45) + 1, vec![252, 32, 0, 0, 0, 0, 1]), - ((1 << 46) - 1, vec![252, 63, 255, 255, 255, 255, 255]), - (1 << 46, vec![252, 64, 0, 0, 0, 0, 0]), - ((1 << 46) + 1, vec![252, 64, 0, 0, 0, 0, 1]), - ((1 << 47) - 1, vec![252, 127, 255, 255, 255, 255, 255]), - (1 << 47, vec![252, 128, 0, 0, 0, 0, 0]), - ((1 << 47) + 1, vec![252, 128, 0, 0, 0, 0, 1]), - ((1 << 48) - 1, vec![252, 255, 255, 255, 255, 255, 255]), - (1 << 48, vec![253, 0, 0, 0, 0, 0, 0]), - ((1 << 48) + 1, vec![253, 0, 0, 0, 0, 0, 1]), - ((1 << 49) - 1, vec![253, 255, 255, 255, 255, 255, 255]), - (1 << 49, vec![254, 2, 0, 0, 0, 0, 0, 0]), - ((1 << 49) + 1, vec![254, 2, 0, 0, 0, 0, 0, 1]), - ((1 << 50) - 1, vec![254, 3, 255, 255, 255, 255, 255, 255]), - (1 << 50, vec![254, 4, 0, 0, 0, 0, 0, 0]), - ((1 << 50) + 1, vec![254, 4, 0, 0, 0, 0, 0, 1]), - ((1 << 51) - 1, vec![254, 7, 255, 255, 255, 255, 255, 255]), - (1 << 51, vec![254, 8, 0, 0, 0, 0, 0, 0]), - ((1 << 51) + 1, vec![254, 8, 0, 0, 0, 0, 0, 1]), - ((1 << 52) - 1, vec![254, 15, 255, 255, 255, 255, 255, 255]), - (1 << 52, vec![254, 16, 0, 0, 0, 0, 0, 0]), - ((1 << 52) + 1, vec![254, 16, 0, 0, 0, 0, 0, 1]), - ((1 << 53) - 1, vec![254, 31, 255, 255, 255, 255, 255, 255]), - (1 << 53, vec![254, 32, 0, 0, 0, 0, 0, 0]), - ((1 << 53) + 1, vec![254, 32, 0, 0, 0, 0, 0, 1]), - ((1 << 54) - 1, vec![254, 63, 255, 255, 255, 255, 255, 255]), - (1 << 54, vec![254, 64, 0, 0, 0, 0, 0, 0]), - ((1 << 54) + 1, vec![254, 64, 0, 0, 0, 0, 0, 1]), - ((1 << 55) - 1, vec![254, 127, 255, 255, 255, 255, 255, 255]), - (1 << 55, vec![254, 128, 0, 0, 0, 0, 0, 0]), - ((1 << 55) + 1, vec![254, 128, 0, 0, 0, 0, 0, 1]), - ((1 << 56) - 1, vec![254, 255, 255, 255, 255, 255, 255, 255]), - (1 << 56, vec![255, 1, 0, 0, 0, 0, 0, 0, 0]), - ((1 << 56) + 1, vec![255, 1, 0, 0, 0, 0, 0, 0, 1]), - ( - (1 << 57) - 1, - vec![255, 1, 255, 255, 255, 255, 255, 255, 255], - ), - (1 << 57, vec![255, 2, 0, 0, 0, 0, 0, 0, 0]), - ((1 << 57) + 1, vec![255, 2, 0, 0, 0, 0, 0, 0, 1]), - ( - (1 << 58) - 1, - vec![255, 3, 255, 255, 255, 255, 255, 255, 255], - ), - (1 << 58, vec![255, 4, 0, 0, 0, 0, 0, 0, 0]), - ((1 << 58) + 1, vec![255, 4, 0, 0, 0, 0, 0, 0, 1]), - ( - (1 << 59) - 1, - vec![255, 7, 255, 255, 255, 255, 255, 255, 255], - ), - (1 << 59, vec![255, 8, 0, 0, 0, 0, 0, 0, 0]), - ((1 << 59) + 1, vec![255, 8, 0, 0, 0, 0, 0, 0, 1]), - ( - (1 << 60) - 1, - vec![255, 15, 255, 255, 255, 255, 255, 255, 255], - ), - (1 << 60, vec![255, 16, 0, 0, 0, 0, 0, 0, 0]), - ((1 << 60) + 1, vec![255, 16, 0, 0, 0, 0, 0, 0, 1]), - ( - (1 << 61) - 1, - vec![255, 31, 255, 255, 255, 255, 255, 255, 255], - ), - (1 << 61, vec![255, 32, 0, 0, 0, 0, 0, 0, 0]), - ((1 << 61) + 1, vec![255, 32, 0, 0, 0, 0, 0, 0, 1]), - ( - (1 << 62) - 1, - vec![255, 63, 255, 255, 255, 255, 255, 255, 255], - ), - (1 << 62, vec![255, 64, 0, 0, 0, 0, 0, 0, 0]), - ((1 << 62) + 1, vec![255, 64, 0, 0, 0, 0, 0, 0, 1]), + let unsigned_vint_encoding = [ + (0, &[0][..]), + (1, &[1]), + (2, &[2]), + ((1 << 2) - 1, &[3]), + (1 << 2, &[4]), + ((1 << 2) + 1, &[5]), + ((1 << 3) - 1, &[7]), + (1 << 3, &[8]), + ((1 << 3) + 1, &[9]), + ((1 << 4) - 1, &[15]), + (1 << 4, &[16]), + ((1 << 4) + 1, &[17]), + ((1 << 5) - 1, &[31]), + (1 << 5, &[32]), + ((1 << 5) + 1, &[33]), + ((1 << 6) - 1, &[63]), + (1 << 6, &[64]), + ((1 << 6) + 1, &[65]), + ((1 << 7) - 1, &[127]), + (1 << 7, &[128, 128]), + ((1 << 7) + 1, &[128, 129]), + ((1 << 8) - 1, &[128, 255]), + (1 << 8, &[129, 0]), + ((1 << 8) + 1, &[129, 1]), + ((1 << 9) - 1, &[129, 255]), + (1 << 9, &[130, 0]), + ((1 << 9) + 1, &[130, 1]), + ((1 << 10) - 1, &[131, 255]), + (1 << 10, &[132, 0]), + ((1 << 10) + 1, &[2, 1]), + ((1 << 11) - 1, &[5, 255]), + (1 << 11, &[136, 0]), + ((1 << 11) + 1, &[136, 1]), + ((1 << 12) - 1, &[143, 255]), + (1 << 12, &[4, 0]), + ((1 << 12) + 1, &[144, 1]), + ((1 << 13) - 1, &[159, 255]), + (1 << 13, &[0, 0]), + ((1 << 13) + 1, &[0, 1]), + ((1 << 14) - 1, &[191, 255]), + (1 << 14, &[192, 64, 0]), + ((1 << 14) + 1, &[192, 64, 1]), + ((1 << 15) - 1, &[192, 127, 255]), + (1 << 15, &[2, 128, 0]), + ((1 << 15) + 1, &[192, 128, 1]), + ((1 << 16) - 1, &[192, 255, 255]), + (1 << 16, &[193, 0, 0]), + ((1 << 16) + 1, &[193, 0, 1]), + ((1 << 17) - 1, &[193, 255, 255]), + (1 << 17, &[194, 0, 0]), + ((1 << 17) + 1, &[194, 0, 1]), + ((1 << 18) - 1, &[195, 255, 255]), + (1 << 18, &[196, 0, 0]), + ((1 << 18) + 1, &[196, 0, 1]), + ((1 << 19) - 1, &[199, 255, 255]), + (1 << 19, &[200, 0, 0]), + ((1 << 19) + 1, &[200, 0, 1]), + ((1 << 20) - 1, &[207, 255, 255]), + (1 << 20, &[208, 0, 0]), + ((1 << 20) + 1, &[208, 0, 1]), + ((1 << 21) - 1, &[223, 255, 255]), + (1 << 21, &[224, 32, 0, 0]), + ((1 << 21) + 1, &[224, 32, 0, 1]), + ((1 << 22) - 1, &[4, 63, 255, 255]), + (1 << 22, &[224, 64, 0, 0]), + ((1 << 22) + 1, &[224, 64, 0, 1]), + ((1 << 23) - 1, &[224, 127, 255, 255]), + (1 << 23, &[128, 0, 0]), + ((1 << 23) + 1, &[4, 128, 0, 1]), + ((1 << 24) - 1, &[224, 255, 255, 255]), + (1 << 24, &[225, 0, 0, 0]), + ((1 << 24) + 1, &[225, 0, 0, 1]), + ((1 << 25) - 1, &[225, 255, 255, 255]), + (1 << 25, &[226, 0, 0, 0]), + ((1 << 25) + 1, &[226, 0, 0, 1]), + ((1 << 26) - 1, &[227, 255, 255, 255]), + (1 << 26, &[8, 0, 0, 0]), + ((1 << 26) + 1, &[228, 0, 0, 1]), + ((1 << 27) - 1, &[231, 255, 255, 255]), + (1 << 27, &[232, 0, 0, 0]), + ((1 << 27) + 1, &[232, 0, 0, 1]), + ((1 << 28) - 1, &[239, 255, 255, 255]), + (1 << 28, &[240, 16, 0, 0, 0]), + ((1 << 28) + 1, &[240, 16, 0, 0, 1]), + ((1 << 29) - 1, &[31, 255, 255, 255]), + (1 << 29, &[240, 32, 0, 0, 0]), + ((1 << 29) + 1, &[240, 32, 0, 0, 1]), + ((1 << 30) - 1, &[240, 63, 255, 255, 255]), + (1 << 30, &[240, 64, 0, 0, 0]), + ((1 << 30) + 1, &[240, 64, 0, 0, 1]), + ((1 << 31) - 1, &[240, 127, 255, 255, 255]), + (1 << 31, &[240, 128, 0, 0, 0]), + ((1 << 31) + 1, &[240, 128, 0, 0, 1]), + ((1 << 32) - 1, &[240, 255, 255, 255, 255]), + (1 << 32, &[241, 0, 0, 0, 0]), + ((1 << 32) + 1, &[241, 0, 0, 0, 1]), + ((1 << 33) - 1, &[241, 255, 255, 255, 255]), + (1 << 33, &[242, 0, 0, 0, 0]), + ((1 << 33) + 1, &[242, 0, 0, 0, 1]), + ((1 << 34) - 1, &[243, 255, 255, 255, 255]), + (1 << 34, &[244, 0, 0, 0, 0]), + ((1 << 34) + 1, &[244, 0, 0, 0, 1]), + ((1 << 35) - 1, &[247, 255, 255, 255, 255]), + (1 << 35, &[248, 8, 0, 0, 0, 0]), + ((1 << 35) + 1, &[248, 8, 0, 0, 0, 1]), + ((1 << 36) - 1, &[248, 15, 255, 255, 255, 255]), + (1 << 36, &[248, 16, 0, 0, 0, 0]), + ((1 << 36) + 1, &[248, 16, 0, 0, 0, 1]), + ((1 << 37) - 1, &[248, 31, 255, 255, 255, 255]), + (1 << 37, &[248, 32, 0, 0, 0, 0]), + ((1 << 37) + 1, &[248, 32, 0, 0, 0, 1]), + ((1 << 38) - 1, &[248, 63, 255, 255, 255, 255]), + (1 << 38, &[248, 64, 0, 0, 0, 0]), + ((1 << 38) + 1, &[248, 64, 0, 0, 0, 1]), + ((1 << 39) - 1, &[248, 127, 255, 255, 255, 255]), + (1 << 39, &[248, 128, 0, 0, 0, 0]), + ((1 << 39) + 1, &[248, 128, 0, 0, 0, 1]), + ((1 << 40) - 1, &[248, 255, 255, 255, 255, 255]), + (1 << 40, &[249, 0, 0, 0, 0, 0]), + ((1 << 40) + 1, &[249, 0, 0, 0, 0, 1]), + ((1 << 41) - 1, &[249, 255, 255, 255, 255, 255]), + (1 << 41, &[250, 0, 0, 0, 0, 0]), + ((1 << 41) + 1, &[250, 0, 0, 0, 0, 1]), + ((1 << 42) - 1, &[251, 255, 255, 255, 255, 255]), + (1 << 42, &[252, 4, 0, 0, 0, 0, 0]), + ((1 << 42) + 1, &[252, 4, 0, 0, 0, 0, 1]), + ((1 << 43) - 1, &[252, 7, 255, 255, 255, 255, 255]), + (1 << 43, &[252, 8, 0, 0, 0, 0, 0]), + ((1 << 43) + 1, &[252, 8, 0, 0, 0, 0, 1]), + ((1 << 44) - 1, &[252, 15, 255, 255, 255, 255, 255]), + (1 << 44, &[252, 16, 0, 0, 0, 0, 0]), + ((1 << 44) + 1, &[252, 16, 0, 0, 0, 0, 1]), + ((1 << 45) - 1, &[252, 31, 255, 255, 255, 255, 255]), + (1 << 45, &[252, 32, 0, 0, 0, 0, 0]), + ((1 << 45) + 1, &[252, 32, 0, 0, 0, 0, 1]), + ((1 << 46) - 1, &[252, 63, 255, 255, 255, 255, 255]), + (1 << 46, &[252, 64, 0, 0, 0, 0, 0]), + ((1 << 46) + 1, &[252, 64, 0, 0, 0, 0, 1]), + ((1 << 47) - 1, &[252, 127, 255, 255, 255, 255, 255]), + (1 << 47, &[252, 128, 0, 0, 0, 0, 0]), + ((1 << 47) + 1, &[252, 128, 0, 0, 0, 0, 1]), + ((1 << 48) - 1, &[252, 255, 255, 255, 255, 255, 255]), + (1 << 48, &[253, 0, 0, 0, 0, 0, 0]), + ((1 << 48) + 1, &[253, 0, 0, 0, 0, 0, 1]), + ((1 << 49) - 1, &[253, 255, 255, 255, 255, 255, 255]), + (1 << 49, &[254, 2, 0, 0, 0, 0, 0, 0]), + ((1 << 49) + 1, &[254, 2, 0, 0, 0, 0, 0, 1]), + ((1 << 50) - 1, &[254, 3, 255, 255, 255, 255, 255, 255]), + (1 << 50, &[254, 4, 0, 0, 0, 0, 0, 0]), + ((1 << 50) + 1, &[254, 4, 0, 0, 0, 0, 0, 1]), + ((1 << 51) - 1, &[254, 7, 255, 255, 255, 255, 255, 255]), + (1 << 51, &[254, 8, 0, 0, 0, 0, 0, 0]), + ((1 << 51) + 1, &[254, 8, 0, 0, 0, 0, 0, 1]), + ((1 << 52) - 1, &[254, 15, 255, 255, 255, 255, 255, 255]), + (1 << 52, &[254, 16, 0, 0, 0, 0, 0, 0]), + ((1 << 52) + 1, &[254, 16, 0, 0, 0, 0, 0, 1]), + ((1 << 53) - 1, &[254, 31, 255, 255, 255, 255, 255, 255]), + (1 << 53, &[254, 32, 0, 0, 0, 0, 0, 0]), + ((1 << 53) + 1, &[254, 32, 0, 0, 0, 0, 0, 1]), + ((1 << 54) - 1, &[254, 63, 255, 255, 255, 255, 255, 255]), + (1 << 54, &[254, 64, 0, 0, 0, 0, 0, 0]), + ((1 << 54) + 1, &[254, 64, 0, 0, 0, 0, 0, 1]), + ((1 << 55) - 1, &[254, 127, 255, 255, 255, 255, 255, 255]), + (1 << 55, &[254, 128, 0, 0, 0, 0, 0, 0]), + ((1 << 55) + 1, &[254, 128, 0, 0, 0, 0, 0, 1]), + ((1 << 56) - 1, &[254, 255, 255, 255, 255, 255, 255, 255]), + (1 << 56, &[255, 1, 0, 0, 0, 0, 0, 0, 0]), + ((1 << 56) + 1, &[255, 1, 0, 0, 0, 0, 0, 0, 1]), + ((1 << 57) - 1, &[255, 1, 255, 255, 255, 255, 255, 255, 255]), + (1 << 57, &[255, 2, 0, 0, 0, 0, 0, 0, 0]), + ((1 << 57) + 1, &[255, 2, 0, 0, 0, 0, 0, 0, 1]), + ((1 << 58) - 1, &[255, 3, 255, 255, 255, 255, 255, 255, 255]), + (1 << 58, &[255, 4, 0, 0, 0, 0, 0, 0, 0]), + ((1 << 58) + 1, &[255, 4, 0, 0, 0, 0, 0, 0, 1]), + ((1 << 59) - 1, &[255, 7, 255, 255, 255, 255, 255, 255, 255]), + (1 << 59, &[255, 8, 0, 0, 0, 0, 0, 0, 0]), + ((1 << 59) + 1, &[255, 8, 0, 0, 0, 0, 0, 0, 1]), + ((1 << 60) - 1, &[5, 15, 255, 255, 255, 255, 255, 255, 255]), + (1 << 60, &[255, 16, 0, 0, 0, 0, 0, 0, 0]), + ((1 << 60) + 1, &[255, 16, 0, 0, 0, 0, 0, 0, 1]), + ((1 << 61) - 1, &[255, 31, 255, 255, 255, 255, 255, 255, 255]), + (1 << 61, &[255, 32, 0, 0, 0, 0, 0, 0, 0]), + ((1 << 61) + 1, &[255, 32, 0, 0, 0, 0, 0, 0, 1]), + ((1 << 62) - 1, &[255, 63, 255, 255, 255, 255, 255, 255, 255]), + (1 << 62, &[255, 64, 0, 0, 0, 0, 0, 0, 0]), + ((1 << 62) + 1, &[255, 64, 0, 0, 0, 0, 0, 0, 1]), ( (1 << 63) - 1, - vec![255, 127, 255, 255, 255, 255, 255, 255, 255], + &[255, 127, 255, 255, 255, 255, 255, 255, 255], ), - (1 << 63, vec![255, 128, 0, 0, 0, 0, 0, 0, 0]), - ((1 << 63) + 1, vec![255, 128, 0, 0, 0, 0, 0, 0, 1]), - (u64::MAX, vec![255, 255, 255, 255, 255, 255, 255, 255, 255]), + (1 << 63, &[255, 128, 0, 0, 0, 0, 0, 0, 0]), + ((1 << 63) + 1, &[255, 128, 0, 0, 0, 0, 0, 0, 1]), + (u64::MAX, &[255, 255, 255, 255, 255, 255, 255, 255, 255]), ]; let mut buf = Vec::new(); diff --git a/scylla-cql/src/frame/value_tests.rs b/scylla-cql/src/frame/value_tests.rs index 62d998cbf..90d5f884d 100644 --- a/scylla-cql/src/frame/value_tests.rs +++ b/scylla-cql/src/frame/value_tests.rs @@ -52,24 +52,24 @@ fn compute_hash(x: &T) -> u64 { #[test] fn boolean_serialization() { - assert_eq!(serialized(true, ColumnType::Boolean), vec![0, 0, 0, 1, 1]); - assert_eq!(serialized(false, ColumnType::Boolean), vec![0, 0, 0, 1, 0]); + assert_eq!(serialized(true, ColumnType::Boolean), &[0, 0, 0, 1, 1]); + assert_eq!(serialized(false, ColumnType::Boolean), &[0, 0, 0, 1, 0]); } #[test] fn fixed_integral_serialization() { - assert_eq!(serialized(8_i8, ColumnType::TinyInt), vec![0, 0, 0, 1, 8]); + assert_eq!(serialized(8_i8, ColumnType::TinyInt), &[0, 0, 0, 1, 8]); assert_eq!( serialized(16_i16, ColumnType::SmallInt), - vec![0, 0, 0, 2, 0, 16] + &[0, 0, 0, 2, 0, 16] ); assert_eq!( serialized(32_i32, ColumnType::Int), - vec![0, 0, 0, 4, 0, 0, 0, 32] + &[0, 0, 0, 4, 0, 0, 0, 32] ); assert_eq!( serialized(64_i64, ColumnType::BigInt), - vec![0, 0, 0, 8, 0, 0, 0, 0, 0, 0, 0, 64] + &[0, 0, 0, 8, 0, 0, 0, 0, 0, 0, 0, 64] ); } @@ -77,7 +77,7 @@ fn fixed_integral_serialization() { fn counter_serialization() { assert_eq!( serialized(0x0123456789abcdef_i64, ColumnType::BigInt), - vec![0, 0, 0, 8, 0x01, 0x23, 0x45, 0x67, 0x89, 0xab, 0xcd, 0xef] + &[0, 0, 0, 8, 0x01, 0x23, 0x45, 0x67, 0x89, 0xab, 0xcd, 0xef] ); } @@ -241,29 +241,26 @@ fn floating_point_serialization() { fn text_serialization() { assert_eq!( serialized("abc", ColumnType::Text), - vec![0, 0, 0, 3, 97, 98, 99] + &[0, 0, 0, 3, 97, 98, 99] ); assert_eq!( serialized("abc".to_string(), ColumnType::Ascii), - vec![0, 0, 0, 3, 97, 98, 99] + &[0, 0, 0, 3, 97, 98, 99] ); } #[test] fn u8_array_serialization() { let val = [1u8; 4]; - assert_eq!( - serialized(val, ColumnType::Blob), - vec![0, 0, 0, 4, 1, 1, 1, 1] - ); + assert_eq!(serialized(val, ColumnType::Blob), &[0, 0, 0, 4, 1, 1, 1, 1]); } #[test] fn u8_slice_serialization() { - let val = vec![1u8, 1, 1, 1]; + let val = &[1u8, 1, 1, 1]; assert_eq!( serialized(val.as_slice(), ColumnType::Blob), - vec![0, 0, 0, 4, 1, 1, 1, 1] + &[0, 0, 0, 4, 1, 1, 1, 1] ); } @@ -271,21 +268,18 @@ fn u8_slice_serialization() { fn cql_date_serialization() { assert_eq!( serialized(CqlDate(0), ColumnType::Date), - vec![0, 0, 0, 4, 0, 0, 0, 0] + &[0, 0, 0, 4, 0, 0, 0, 0] ); assert_eq!( serialized(CqlDate(u32::MAX), ColumnType::Date), - vec![0, 0, 0, 4, 255, 255, 255, 255] + &[0, 0, 0, 4, 255, 255, 255, 255] ); } #[test] fn vec_u8_slice_serialization() { - let val = vec![1u8, 1, 1, 1]; - assert_eq!( - serialized(val, ColumnType::Blob), - vec![0, 0, 0, 4, 1, 1, 1, 1] - ); + let val = &[1u8, 1, 1, 1]; + assert_eq!(serialized(val, ColumnType::Blob), &[0, 0, 0, 4, 1, 1, 1, 1]); } #[test] @@ -293,13 +287,13 @@ fn ipaddr_serialization() { let ipv4 = IpAddr::V4(Ipv4Addr::new(1, 2, 3, 4)); assert_eq!( serialized(ipv4, ColumnType::Inet), - vec![0, 0, 0, 4, 1, 2, 3, 4] + &[0, 0, 0, 4, 1, 2, 3, 4] ); let ipv6 = IpAddr::V6(Ipv6Addr::new(1, 2, 3, 4, 5, 6, 7, 8)); assert_eq!( serialized(ipv6, ColumnType::Inet), - vec![ + &[ 0, 0, 0, 16, // serialized size 0, 1, 0, 2, 0, 3, 0, 4, 0, 5, 0, 6, 0, 7, 0, 8, // contents ] @@ -314,7 +308,7 @@ fn naive_date_04_serialization() { let unix_epoch: NaiveDate = NaiveDate::from_ymd_opt(1970, 1, 1).unwrap(); assert_eq!( serialized(unix_epoch, ColumnType::Date), - vec![0, 0, 0, 4, 128, 0, 0, 0] + &[0, 0, 0, 4, 128, 0, 0, 0] ); assert_eq!(2_u32.pow(31).to_be_bytes(), [128, 0, 0, 0]); @@ -322,7 +316,7 @@ fn naive_date_04_serialization() { let before_epoch: NaiveDate = NaiveDate::from_ymd_opt(1969, 12, 2).unwrap(); assert_eq!( serialized(before_epoch, ColumnType::Date), - vec![0, 0, 0, 4, 127, 255, 255, 226] + &[0, 0, 0, 4, 127, 255, 255, 226] ); assert_eq!((2_u32.pow(31) - 30).to_be_bytes(), [127, 255, 255, 226]); @@ -330,7 +324,7 @@ fn naive_date_04_serialization() { let after_epoch: NaiveDate = NaiveDate::from_ymd_opt(1970, 1, 31).unwrap(); assert_eq!( serialized(after_epoch, ColumnType::Date), - vec![0, 0, 0, 4, 128, 0, 0, 30] + &[0, 0, 0, 4, 128, 0, 0, 30] ); assert_eq!((2_u32.pow(31) + 30).to_be_bytes(), [128, 0, 0, 30]); } @@ -342,7 +336,7 @@ fn date_03_serialization() { let unix_epoch = time_03::Date::from_ordinal_date(1970, 1).unwrap(); assert_eq!( serialized(unix_epoch, ColumnType::Date), - vec![0, 0, 0, 4, 128, 0, 0, 0] + &[0, 0, 0, 4, 128, 0, 0, 0] ); assert_eq!(2_u32.pow(31).to_be_bytes(), [128, 0, 0, 0]); @@ -351,7 +345,7 @@ fn date_03_serialization() { time_03::Date::from_calendar_date(1969, time_03::Month::December, 2).unwrap(); assert_eq!( serialized(before_epoch, ColumnType::Date), - vec![0, 0, 0, 4, 127, 255, 255, 226] + &[0, 0, 0, 4, 127, 255, 255, 226] ); assert_eq!((2_u32.pow(31) - 30).to_be_bytes(), [127, 255, 255, 226]); @@ -359,7 +353,7 @@ fn date_03_serialization() { let after_epoch = time_03::Date::from_calendar_date(1970, time_03::Month::January, 31).unwrap(); assert_eq!( serialized(after_epoch, ColumnType::Date), - vec![0, 0, 0, 4, 128, 0, 0, 30] + &[0, 0, 0, 4, 128, 0, 0, 30] ); assert_eq!((2_u32.pow(31) + 30).to_be_bytes(), [128, 0, 0, 30]); @@ -373,7 +367,7 @@ fn date_03_serialization() { ); assert_eq!( serialized(long_before_epoch, ColumnType::Date), - vec![0, 0, 0, 4, 127, 189, 75, 125] + &[0, 0, 0, 4, 127, 189, 75, 125] ); // Max date represented by time_03::Date (without large-dates feature) @@ -386,7 +380,7 @@ fn date_03_serialization() { ); assert_eq!( serialized(long_after_epoch, ColumnType::Date), - vec![0, 0, 0, 4, 128, 44, 192, 160] + &[0, 0, 0, 4, 128, 44, 192, 160] ); } @@ -653,25 +647,22 @@ fn cqlduration_serialization() { }; assert_eq!( serialized(duration, ColumnType::Duration), - vec![0, 0, 0, 3, 2, 4, 6] + &[0, 0, 0, 3, 2, 4, 6] ); } #[test] fn box_serialization() { let x: Box = Box::new(123); - assert_eq!( - serialized(x, ColumnType::Int), - vec![0, 0, 0, 4, 0, 0, 0, 123] - ); + assert_eq!(serialized(x, ColumnType::Int), &[0, 0, 0, 4, 0, 0, 0, 123]); } #[test] fn vec_set_serialization() { - let m = vec!["ala", "ma", "kota"]; + let m = &["ala", "ma", "kota"][..]; assert_eq!( serialized(m, ColumnType::Set(Box::new(ColumnType::Text))), - vec![ + &[ 0, 0, 0, 25, // 25 bytes 0, 0, 0, 3, // 3 items 0, 0, 0, 3, 97, 108, 97, // ala @@ -683,10 +674,10 @@ fn vec_set_serialization() { #[test] fn slice_set_serialization() { - let m = ["ala", "ma", "kota"]; + let m = &["ala", "ma", "kota"][..]; assert_eq!( serialized(m.as_ref(), ColumnType::Set(Box::new(ColumnType::Text))), - vec![ + &[ 0, 0, 0, 25, // 25 bytes 0, 0, 0, 3, // 3 items 0, 0, 0, 3, 97, 108, 97, // ala @@ -721,7 +712,7 @@ fn hashset_serialization() { let m: HashSet<&'static str, DumbBuildHasher> = ["ala", "ma", "kota"].into_iter().collect(); assert_eq!( serialized(m, ColumnType::Set(Box::new(ColumnType::Text))), - vec![ + &[ 0, 0, 0, 25, // 25 bytes 0, 0, 0, 3, // 3 items 0, 0, 0, 2, 109, 97, // ma @@ -740,7 +731,7 @@ fn hashmap_serialization() { m, ColumnType::Map(Box::new(ColumnType::Text), Box::new(ColumnType::Int)) ), - vec![ + &[ 0, 0, 0, 49, // 49 bytes 0, 0, 0, 3, // 3 items 0, 0, 0, 2, 109, 97, // ma @@ -758,7 +749,7 @@ fn btreeset_serialization() { let m: BTreeSet<&'static str> = ["ala", "ma", "kota"].into_iter().collect(); assert_eq!( serialized(m, ColumnType::Set(Box::new(ColumnType::Text))), - vec![ + &[ 0, 0, 0, 25, // 25 bytes 0, 0, 0, 3, // 3 items 0, 0, 0, 3, 97, 108, 97, // ala @@ -776,7 +767,7 @@ fn btreemap_serialization() { m, ColumnType::Map(Box::new(ColumnType::Text), Box::new(ColumnType::Int)) ), - vec![ + &[ 0, 0, 0, 49, // 49 bytes 0, 0, 0, 3, // 3 items 0, 0, 0, 3, 97, 108, 97, // ala @@ -795,10 +786,7 @@ fn cqlvalue_serialization() { // e.g. UDTs or tuples. // Empty - assert_eq!( - serialized(CqlValue::Empty, ColumnType::Int), - vec![0, 0, 0, 0], - ); + assert_eq!(serialized(CqlValue::Empty, ColumnType::Int), &[0, 0, 0, 0],); // UDTs let udt = CqlValue::UserDefinedType { @@ -820,7 +808,7 @@ fn cqlvalue_serialization() { assert_eq!( serialized(udt, typ.clone()), - vec![ + &[ 0, 0, 0, 12, // size of the whole thing 0, 0, 0, 4, 0, 0, 0, 123, // foo: 123_i32 255, 255, 255, 255, // bar: null @@ -840,7 +828,7 @@ fn cqlvalue_serialization() { assert_eq!( serialized_only_new(udt, typ.clone()), - vec![ + &[ 0, 0, 0, 12, // size of the whole thing 0, 0, 0, 4, 0, 0, 0, 123, // foo: 123_i32 255, 255, 255, 255, // bar: null @@ -852,7 +840,7 @@ fn cqlvalue_serialization() { let typ = ColumnType::Tuple(vec![ColumnType::Int, ColumnType::Text]); assert_eq!( serialized(tup, typ), - vec![ + &[ 0, 0, 0, 12, // size of the whole thing 0, 0, 0, 4, 0, 0, 0, 123, // 123_i32 255, 255, 255, 255, // null @@ -867,7 +855,7 @@ fn cqlvalue_serialization() { let typ = ColumnType::Tuple(vec![ColumnType::Int, ColumnType::Text, ColumnType::Counter]); assert_eq!( serialized(tup, typ), - vec![ + &[ 0, 0, 0, 12, // size of the whole thing 0, 0, 0, 4, 0, 0, 0, 123, // 123_i32 255, 255, 255, 255, // null @@ -881,7 +869,7 @@ fn secret_serialization() { let secret = secrecy_08::Secret::new(987654i32); assert_eq!( serialized(secret, ColumnType::Int), - vec![0, 0, 0, 4, 0x00, 0x0f, 0x12, 0x06] + &[0, 0, 0, 4, 0x00, 0x0f, 0x12, 0x06] ); } @@ -889,7 +877,7 @@ fn secret_serialization() { fn option_value() { assert_eq!( serialized(Some(32_i32), ColumnType::Int), - vec![0, 0, 0, 4, 0, 0, 0, 32] + &[0, 0, 0, 4, 0, 0, 0, 32] ); let null_i32: Option = None; assert_eq!( @@ -914,7 +902,7 @@ fn unset_value() { let set_i32: MaybeUnset = MaybeUnset::Set(32); assert_eq!( serialized(set_i32, ColumnType::Int), - vec![0, 0, 0, 4, 0, 0, 0, 32] + &[0, 0, 0, 4, 0, 0, 0, 32] ); } @@ -945,7 +933,7 @@ fn empty_serialized_values() { let mut empty_request = Vec::::new(); EMPTY.write_to_request(&mut empty_request); - assert_eq!(empty_request, vec![0, 0]); + assert_eq!(empty_request, &[0, 0]); } #[test] @@ -961,11 +949,11 @@ fn serialized_values() { let mut request = Vec::::new(); values.write_to_request(&mut request); - assert_eq!(request, vec![0, 1, 0, 0, 0, 1, 8]); + assert_eq!(request, &[0, 1, 0, 0, 0, 1, 8]); assert_eq!( values.iter().collect::>(), - vec![RawValue::Value([8].as_ref())] + &[RawValue::Value([8].as_ref())] ); } @@ -977,11 +965,11 @@ fn serialized_values() { let mut request = Vec::::new(); values.write_to_request(&mut request); - assert_eq!(request, vec![0, 2, 0, 0, 0, 1, 8, 0, 0, 0, 2, 0, 16]); + assert_eq!(request, &[0, 2, 0, 0, 0, 1, 8, 0, 0, 0, 2, 0, 16]); assert_eq!( values.iter().collect::>(), - vec![ + &[ RawValue::Value([8].as_ref()), RawValue::Value([0, 16].as_ref()) ] @@ -1012,11 +1000,11 @@ fn serialized_values() { let mut request = Vec::::new(); values.write_to_request(&mut request); - assert_eq!(request, vec![0, 2, 0, 0, 0, 1, 8, 0, 0, 0, 2, 0, 16]); + assert_eq!(request, &[0, 2, 0, 0, 0, 1, 8, 0, 0, 0, 2, 0, 16]); assert_eq!( values.iter().collect::>(), - vec![ + &[ RawValue::Value([8].as_ref()), RawValue::Value([0, 16].as_ref()) ] @@ -1051,7 +1039,7 @@ fn slice_value_list() { assert_eq!( serialized.iter().collect::>(), - vec![ + &[ RawValue::Value([0, 0, 0, 1].as_ref()), RawValue::Value([0, 0, 0, 2].as_ref()), RawValue::Value([0, 0, 0, 3].as_ref()) @@ -1061,7 +1049,7 @@ fn slice_value_list() { #[test] fn vec_value_list() { - let values: Vec = vec![1, 2, 3]; + let values = &[1, 2, 3][..]; let cols = &[ col_spec("ala", ColumnType::Int), col_spec("ma", ColumnType::Int), @@ -1071,7 +1059,7 @@ fn vec_value_list() { assert_eq!( serialized.iter().collect::>(), - vec![ + &[ RawValue::Value([0, 0, 0, 1].as_ref()), RawValue::Value([0, 0, 0, 2].as_ref()), RawValue::Value([0, 0, 0, 3].as_ref()) @@ -1222,7 +1210,7 @@ fn map_value_list() { let new_values = serialize_values_only_new(row.clone(), cols); assert_eq!( new_values, - vec![ + &[ 0, 3, // value count: 3 0, 0, 0, 4, 0, 0, 0, 1, // ala: 1 0, 0, 0, 4, 0, 0, 0, 2, // ma: 2 @@ -1251,7 +1239,7 @@ fn ref_value_list() { assert_eq!( serialized.iter().collect::>(), - vec![ + &[ RawValue::Value([0, 0, 0, 1].as_ref()), RawValue::Value([0, 0, 0, 2].as_ref()), RawValue::Value([0, 0, 0, 3].as_ref()) @@ -1343,7 +1331,7 @@ fn slice_batch_values() { col_spec("b", ColumnType::TinyInt), ]; let request = serialize_batch_value_iterators(&mut iters, cols); - assert_eq!(request, vec![0, 2, 0, 0, 0, 1, 1, 0, 0, 0, 1, 2]); + assert_eq!(request, &[0, 2, 0, 0, 0, 1, 1, 0, 0, 0, 1, 2]); } { @@ -1356,14 +1344,14 @@ fn slice_batch_values() { let request = serialize_batch_value_iterators(&mut iters, cols); assert_eq!( request, - vec![0, 4, 0, 0, 0, 1, 2, 0, 0, 0, 1, 3, 0, 0, 0, 1, 4, 0, 0, 0, 1, 5] + &[0, 4, 0, 0, 0, 1, 2, 0, 0, 0, 1, 3, 0, 0, 0, 1, 4, 0, 0, 0, 1, 5] ); } { let cols = &[col_spec("a", ColumnType::TinyInt)]; let request = serialize_batch_value_iterators(&mut iters, cols); - assert_eq!(request, vec![0, 1, 0, 0, 0, 1, 6]); + assert_eq!(request, &[0, 1, 0, 0, 0, 1, 6]); } assert_eq!(iters.0.write_next_to_request(&mut Vec::new()), None); @@ -1386,7 +1374,7 @@ fn vec_batch_values() { col_spec("b", ColumnType::TinyInt), ]; let request = serialize_batch_value_iterators(&mut iters, cols); - assert_eq!(request, vec![0, 2, 0, 0, 0, 1, 1, 0, 0, 0, 1, 2]); + assert_eq!(request, &[0, 2, 0, 0, 0, 1, 1, 0, 0, 0, 1, 2]); } { @@ -1399,14 +1387,14 @@ fn vec_batch_values() { let request = serialize_batch_value_iterators(&mut iters, cols); assert_eq!( request, - vec![0, 4, 0, 0, 0, 1, 2, 0, 0, 0, 1, 3, 0, 0, 0, 1, 4, 0, 0, 0, 1, 5] + &[0, 4, 0, 0, 0, 1, 2, 0, 0, 0, 1, 3, 0, 0, 0, 1, 4, 0, 0, 0, 1, 5] ); } { let cols = &[col_spec("a", ColumnType::TinyInt)]; let request = serialize_batch_value_iterators(&mut iters, cols); - assert_eq!(request, vec![0, 1, 0, 0, 0, 1, 6]); + assert_eq!(request, &[0, 1, 0, 0, 0, 1, 6]); } } @@ -1619,7 +1607,7 @@ fn ref_batch_values() { let mut iters = make_batch_value_iters(&batch_values, &legacy_batch_values); let request = serialize_batch_value_iterators(&mut iters, cols); - assert_eq!(request, vec![0, 2, 0, 0, 0, 1, 1, 0, 0, 0, 1, 2]); + assert_eq!(request, &[0, 2, 0, 0, 0, 1, 1, 0, 0, 0, 1, 2]); } } @@ -1661,7 +1649,7 @@ fn check_batch_values_iterator_is_not_lending() { let mut it = bv.batch_values_iter(); let mut it2 = bv.batch_values_iter(); // Make sure we can hold all these at the same time - let v = vec![ + let v = &[ it.next_serialized().unwrap().unwrap(), it2.next_serialized().unwrap().unwrap(), it.next_serialized().unwrap().unwrap(), @@ -1679,7 +1667,7 @@ fn check_batch_values_iterator_is_not_lending() { let mut writer = RowWriter::new(&mut data); // Make sure we can hold all these at the same time - let v = vec![ + let v = &[ it.serialize_next(&ctx, &mut writer).unwrap().unwrap(), it2.serialize_next(&ctx, &mut writer).unwrap().unwrap(), it.serialize_next(&ctx, &mut writer).unwrap().unwrap(), diff --git a/scylla-cql/src/types/deserialize/value_tests.rs b/scylla-cql/src/types/deserialize/value_tests.rs index 2ce5d0ea4..193b32d31 100644 --- a/scylla-cql/src/types/deserialize/value_tests.rs +++ b/scylla-cql/src/types/deserialize/value_tests.rs @@ -393,8 +393,8 @@ fn test_list_and_set() { assert_eq!(iter.next().transpose().unwrap(), Some("fox")); assert_eq!(iter.next().transpose().unwrap(), None); - let expected_vec_str = vec!["quick", "brown", "fox"]; - let expected_vec_string = vec!["quick".to_string(), "brown".to_string(), "fox".to_string()]; + let expected_vec_str = ["quick", "brown", "fox"]; + let expected_vec_string = ["quick".to_string(), "brown".to_string(), "fox".to_string()]; // list let decoded_vec_str = deserialize::>(&list_typ, &collection).unwrap(); @@ -483,8 +483,8 @@ fn test_map() { assert_eq!(iter.next().transpose().unwrap(), Some((3, "fox"))); assert_eq!(iter.next().transpose().unwrap(), None); - let expected_str = vec![(1, "quick"), (2, "brown"), (3, "fox")]; - let expected_string = vec![ + let expected_str = [(1, "quick"), (2, "brown"), (3, "fox")]; + let expected_string = [ (1, "quick".to_string()), (2, "brown".to_string()), (3, "fox".to_string()), diff --git a/scylla-cql/src/types/serialize/row.rs b/scylla-cql/src/types/serialize/row.rs index 3888a122b..e55b329e3 100644 --- a/scylla-cql/src/types/serialize/row.rs +++ b/scylla-cql/src/types/serialize/row.rs @@ -1026,7 +1026,7 @@ mod tests { col_spec("b", ColumnType::Int), ]; let buf = do_serialize(ValueListAdapter(Foo), columns); - let expected = vec![ + let expected = &[ 0, 0, 0, 4, 0, 0, 0, 123, // First value 0, 0, 0, 4, 0, 0, 1, 65, // Second value ]; @@ -1093,11 +1093,11 @@ mod tests { fn test_slice_errors() { // Non-unit tuple // Count mismatch - let v = vec!["Ala ma kota"]; + let v = &["Ala ma kota"][..]; let spec = [col("a", ColumnType::Text), col("b", ColumnType::Text)]; let err = do_serialize_err(v, &spec); let err = get_typeck_err(&err); - assert_eq!(err.rust_name, std::any::type_name::>()); + assert_eq!(err.rust_name, std::any::type_name::<&[&str]>()); assert_matches!( err.kind, BuiltinTypeCheckErrorKind::WrongColumnCount { @@ -1107,7 +1107,7 @@ mod tests { ); // Serialization of one of the element fails - let v = vec!["Ala ma kota", "Kot ma pchły"]; + let v = &["Ala ma kota", "Kot ma pchły"][..]; let spec = [col("a", ColumnType::Text), col("b", ColumnType::Int)]; let err = do_serialize_err(v, &spec); let err = get_ser_err(&err); diff --git a/scylla/src/transport/caching_session.rs b/scylla/src/transport/caching_session.rs index c813a6e2d..9534366c5 100644 --- a/scylla/src/transport/caching_session.rs +++ b/scylla/src/transport/caching_session.rs @@ -708,7 +708,7 @@ mod tests { .unwrap(); rows.sort_unstable(); - assert_eq!(rows, vec![(1, 1000), (2, 2000)]); + assert_eq!(rows, &[(1, 1000), (2, 2000)]); } // Checks whether the PartitionerName is cached properly. diff --git a/scylla/src/transport/cql_collections_test.rs b/scylla/src/transport/cql_collections_test.rs index f37d28a8f..ae95215c4 100644 --- a/scylla/src/transport/cql_collections_test.rs +++ b/scylla/src/transport/cql_collections_test.rs @@ -117,7 +117,7 @@ async fn test_cql_set() { .await; // HashSet - let set_hashset: HashSet = vec![-1, 1, -2].into_iter().collect(); + let set_hashset: HashSet = [-1, 1, -2].into_iter().collect(); let set_hashset_empty: HashSet = HashSet::new(); let set_hashset_empty_selected: Option> = None; insert_and_select(&session, table_name, &set_hashset, &set_hashset).await; @@ -130,7 +130,7 @@ async fn test_cql_set() { .await; // BTreeSet - let set_btreeset: BTreeSet = vec![0, -2, -1].into_iter().collect(); + let set_btreeset: BTreeSet = [0, -2, -1].into_iter().collect(); let set_btreeset_empty: BTreeSet = BTreeSet::new(); let set_btreeset_empty_selected: Option> = None; insert_and_select(&session, table_name, &set_btreeset, &set_btreeset).await; diff --git a/scylla/src/transport/cql_types_test.rs b/scylla/src/transport/cql_types_test.rs index 0a1833fd7..6009100b5 100644 --- a/scylla/src/transport/cql_types_test.rs +++ b/scylla/src/transport/cql_types_test.rs @@ -109,13 +109,13 @@ where .collect::>(); let expected_value = T::from_str(test).ok().unwrap(); - assert_eq!(read_values, vec![expected_value.clone(), expected_value]); + assert_eq!(read_values, &[expected_value.clone(), expected_value]); } } #[cfg(any(feature = "num-bigint-03", feature = "num-bigint-04"))] -fn varint_test_cases() -> Vec<&'static str> { - vec![ +fn varint_test_cases() -> &'static [&'static str] { + &[ "0", "1", "127", @@ -151,20 +151,20 @@ async fn test_varint04() { #[tokio::test] async fn test_cql_varint() { setup_tracing(); - let tests = [ - vec![0x00], // 0 - vec![0x01], // 1 - vec![0x00, 0x01], // 1 (with leading zeros) - vec![0x7F], // 127 - vec![0x00, 0x80], // 128 - vec![0x00, 0x81], // 129 - vec![0xFF], // -1 - vec![0x80], // -128 - vec![0xFF, 0x7F], // -129 - vec![ + let tests = &[ + &[0x00][..], // 0 + &[0x01], // 1 + &[0x00, 0x01], // 1 (with leading zeros) + &[0x7F], // 127 + &[0x00, 0x80], // 128 + &[0x00, 0x81], // 129 + &[0xFF], // -1 + &[0x80], // -128 + &[0xFF, 0x7F], // -129 + &[ 0x01, 0x8E, 0xE9, 0x0F, 0xF6, 0xC3, 0x73, 0xE0, 0xEE, 0x4E, 0x3F, 0x0A, 0xD2, ], // 123456789012345678901234567890 - vec![ + &[ 0xFE, 0x71, 0x16, 0xF0, 0x09, 0x3C, 0x8C, 0x1F, 0x11, 0xB1, 0xC0, 0xF5, 0x2E, ], // -123456789012345678901234567890 ]; @@ -229,7 +229,7 @@ async fn test_cql_varint() { .map(|row| row.0) .collect::>(); - assert_eq!(read_values, vec![cql_varint]) + assert_eq!(read_values, &[cql_varint]) } } @@ -308,7 +308,7 @@ async fn test_counter() { .collect::>(); let expected_value = Counter(i64::from_str(test).unwrap()); - assert_eq!(read_values, vec![expected_value]); + assert_eq!(read_values, &[expected_value]); } } @@ -1332,7 +1332,7 @@ async fn test_timeuuid_ordering() { .unwrap(); // Timeuuid values, sorted in the same order as Scylla/Cassandra sorts them. - let sorted_timeuuid_vals: Vec = vec![ + let sorted_timeuuid_vals: &[CqlTimeuuid] = &[ CqlTimeuuid::from_str("00000000-0000-1000-8080-808080808080").unwrap(), CqlTimeuuid::from_str("00000000-0000-1000-ffff-ffffffffffff").unwrap(), CqlTimeuuid::from_str("00000000-0000-1000-0000-000000000000").unwrap(), @@ -1481,27 +1481,27 @@ async fn test_blob() { setup_tracing(); let session: Session = init_test("blob_tests", "blob").await; - let long_blob: Vec = vec![0x11; 1234]; + let long_blob: &[u8] = &[0x11; 1234]; let mut long_blob_str: String = "0x".to_string(); long_blob_str.extend(std::iter::repeat('1').take(2 * 1234)); let tests = [ - ("0x", vec![]), - ("0x00", vec![0x00]), - ("0x01", vec![0x01]), - ("0xff", vec![0xff]), - ("0x1122", vec![0x11, 0x22]), - ("0x112233", vec![0x11, 0x22, 0x33]), - ("0x11223344", vec![0x11, 0x22, 0x33, 0x44]), - ("0x1122334455", vec![0x11, 0x22, 0x33, 0x44, 0x55]), - ("0x112233445566", vec![0x11, 0x22, 0x33, 0x44, 0x55, 0x66]), + ("0x", &[][..]), + ("0x00", &[0x00]), + ("0x01", &[0x01]), + ("0xff", &[0xff]), + ("0x1122", &[0x11, 0x22]), + ("0x112233", &[0x11, 0x22, 0x33]), + ("0x11223344", &[0x11, 0x22, 0x33, 0x44]), + ("0x1122334455", &[0x11, 0x22, 0x33, 0x44, 0x55]), + ("0x112233445566", &[0x11, 0x22, 0x33, 0x44, 0x55, 0x66]), ( "0x11223344556677", - vec![0x11, 0x22, 0x33, 0x44, 0x55, 0x66, 0x77], + &[0x11, 0x22, 0x33, 0x44, 0x55, 0x66, 0x77], ), ( "0x1122334455667788", - vec![0x11, 0x22, 0x33, 0x44, 0x55, 0x66, 0x77, 0x88], + &[0x11, 0x22, 0x33, 0x44, 0x55, 0x66, 0x77, 0x88], ), (&long_blob_str, long_blob), ]; diff --git a/scylla/src/transport/cql_value_test.rs b/scylla/src/transport/cql_value_test.rs index 53560ed94..57eb16778 100644 --- a/scylla/src/transport/cql_value_test.rs +++ b/scylla/src/transport/cql_value_test.rs @@ -100,12 +100,12 @@ async fn test_cqlvalue_duration() { nanoseconds: 21372137, }); - let fixture_queries = vec![ - ("CREATE TABLE IF NOT EXISTS cqlvalue_duration_test (pk int, ck int, v duration, primary key (pk, ck))", vec![],), - ("INSERT INTO cqlvalue_duration_test (pk, ck, v) VALUES (0, 0, ?)", vec![&duration_cql_value,],), - ("INSERT INTO cqlvalue_duration_test (pk, ck, v) VALUES (0, 1, 89h4m48s)", vec![],), - ("INSERT INTO cqlvalue_duration_test (pk, ck, v) VALUES (0, 2, PT89H8M53S)", vec![],), - ("INSERT INTO cqlvalue_duration_test (pk, ck, v) VALUES (0, 3, P0000-00-00T89:09:09)", vec![],), + let fixture_queries = [ + ("CREATE TABLE IF NOT EXISTS cqlvalue_duration_test (pk int, ck int, v duration, primary key (pk, ck))", &[][..],), + ("INSERT INTO cqlvalue_duration_test (pk, ck, v) VALUES (0, 0, ?)", &[&duration_cql_value,],), + ("INSERT INTO cqlvalue_duration_test (pk, ck, v) VALUES (0, 1, 89h4m48s)", &[],), + ("INSERT INTO cqlvalue_duration_test (pk, ck, v) VALUES (0, 2, PT89H8M53S)", &[],), + ("INSERT INTO cqlvalue_duration_test (pk, ck, v) VALUES (0, 3, P0000-00-00T89:09:09)", &[],), ]; for query in fixture_queries { diff --git a/scylla/src/transport/downgrading_consistency_retry_policy.rs b/scylla/src/transport/downgrading_consistency_retry_policy.rs index e52a44cbf..a41b82990 100644 --- a/scylla/src/transport/downgrading_consistency_retry_policy.rs +++ b/scylla/src/transport/downgrading_consistency_retry_policy.rs @@ -229,7 +229,7 @@ mod tests { #[test] fn downgrading_consistency_never_retries() { setup_tracing(); - let never_retried_dberrors = vec![ + let never_retried_dberrors = [ DbError::SyntaxError, DbError::Invalid, DbError::AlreadyExists { @@ -321,7 +321,7 @@ mod tests { #[test] fn downgrading_consistency_idempotent_next_retries() { setup_tracing(); - let idempotent_next_errors = vec![ + let idempotent_next_errors = &[ QueryError::DbError(DbError::Overloaded, String::new()), QueryError::DbError(DbError::TruncateError, String::new()), QueryError::DbError(DbError::ServerError, String::new()), diff --git a/scylla/src/transport/legacy_query_result.rs b/scylla/src/transport/legacy_query_result.rs index f1c63c719..fa5a7987a 100644 --- a/scylla/src/transport/legacy_query_result.rs +++ b/scylla/src/transport/legacy_query_result.rs @@ -372,7 +372,7 @@ mod tests { .map(|r| r.unwrap()) .collect(); - assert_eq!(rows0, vec![]); + assert_eq!(rows0, &[]); let rows1: Vec<(i32,)> = make_rows_query_result(1) .rows_typed::<(i32,)>() @@ -380,7 +380,7 @@ mod tests { .map(|r| r.unwrap()) .collect(); - assert_eq!(rows1, vec![(0,)]); + assert_eq!(rows1, &[(0,)]); let rows2: Vec<(i32,)> = make_rows_query_result(2) .rows_typed::<(i32,)>() @@ -388,7 +388,7 @@ mod tests { .map(|r| r.unwrap()) .collect(); - assert_eq!(rows2, vec![(0,), (1,)]); + assert_eq!(rows2, &[(0,), (1,)]); } #[test] @@ -412,7 +412,7 @@ mod tests { #[test] fn rows_or_empty_test() { setup_tracing(); - assert_eq!(make_not_rows_query_result().rows_or_empty(), vec![]); + assert_eq!(make_not_rows_query_result().rows_or_empty(), &[]); assert_eq!(make_rows_query_result(0).rows_or_empty(), make_rows(0)); assert_eq!(make_rows_query_result(1).rows_or_empty(), make_rows(1)); assert_eq!(make_rows_query_result(2).rows_or_empty(), make_rows(2)); @@ -426,28 +426,28 @@ mod tests { .map(|r| r.unwrap()) .collect(); - assert_eq!(rows_empty, vec![]); + assert_eq!(rows_empty, &[]); let rows0: Vec<(i32,)> = make_rows_query_result(0) .rows_typed_or_empty::<(i32,)>() .map(|r| r.unwrap()) .collect(); - assert_eq!(rows0, vec![]); + assert_eq!(rows0, &[]); let rows1: Vec<(i32,)> = make_rows_query_result(1) .rows_typed_or_empty::<(i32,)>() .map(|r| r.unwrap()) .collect(); - assert_eq!(rows1, vec![(0,)]); + assert_eq!(rows1, &[(0,)]); let rows2: Vec<(i32,)> = make_rows_query_result(2) .rows_typed_or_empty::<(i32,)>() .map(|r| r.unwrap()) .collect(); - assert_eq!(rows2, vec![(0,), (1,)]); + assert_eq!(rows2, &[(0,), (1,)]); } #[test] diff --git a/scylla/src/transport/locator/mod.rs b/scylla/src/transport/locator/mod.rs index 2ae46856d..a7e349bee 100644 --- a/scylla/src/transport/locator/mod.rs +++ b/scylla/src/transport/locator/mod.rs @@ -859,21 +859,21 @@ mod tests { None, &metadata.keyspaces.get(KEYSPACE_NTS_RF_3).unwrap().strategy, TABLE_NTS_RF_3, - vec![F, A, C, D, G, E], + &[F, A, C, D, G, E][..], ); check( 160, None, &metadata.keyspaces.get(KEYSPACE_NTS_RF_2).unwrap().strategy, TABLE_NTS_RF_2, - vec![F, A, D, G], + &[F, A, D, G], ); check( 160, None, &metadata.keyspaces.get(KEYSPACE_SS_RF_2).unwrap().strategy, TABLE_SS_RF_2, - vec![F, A], + &[F, A], ); check( @@ -881,21 +881,21 @@ mod tests { Some("eu"), &metadata.keyspaces.get(KEYSPACE_NTS_RF_3).unwrap().strategy, TABLE_NTS_RF_3, - vec![A, C, G], + &[A, C, G], ); check( 160, Some("us"), &metadata.keyspaces.get(KEYSPACE_NTS_RF_3).unwrap().strategy, TABLE_NTS_RF_3, - vec![F, D, E], + &[F, D, E], ); check( 160, Some("eu"), &metadata.keyspaces.get(KEYSPACE_SS_RF_2).unwrap().strategy, TABLE_SS_RF_2, - vec![A], + &[A], ); } } diff --git a/scylla/src/transport/locator/precomputed_replicas.rs b/scylla/src/transport/locator/precomputed_replicas.rs index 69851df50..531efb3c8 100644 --- a/scylla/src/transport/locator/precomputed_replicas.rs +++ b/scylla/src/transport/locator/precomputed_replicas.rs @@ -267,19 +267,19 @@ mod tests { assert_eq!(ids, expected_node_ids); }; - check(160, 0, vec![]); - check(160, 1, vec![F]); - check(160, 2, vec![F, A]); + check(160, 0, &[][..]); + check(160, 1, &[F]); + check(160, 2, &[F, A]); assert_eq!( precomputed_replicas.get_precomputed_simple_strategy_replicas(Token::new(160), 3), None ); - check(200, 1, vec![F]); - check(200, 2, vec![F, A]); + check(200, 1, &[F]); + check(200, 2, &[F, A]); - check(701, 1, vec![E]); - check(701, 2, vec![E, G]); + check(701, 1, &[E]); + check(701, 2, &[E, G]); } #[tokio::test] @@ -312,10 +312,10 @@ mod tests { assert_eq!(ids, expected_node_ids); }; - check(160, "eu", 0, vec![]); - check(160, "eu", 1, vec![A]); - check(160, "eu", 2, vec![A, G]); - check(160, "eu", 3, vec![A, C, G]); + check(160, "eu", 0, &[][..]); + check(160, "eu", 1, &[A]); + check(160, "eu", 2, &[A, G]); + check(160, "eu", 3, &[A, C, G]); assert_eq!( precomputed_replicas.get_precomputed_network_strategy_replicas( Token::new(160), @@ -325,10 +325,10 @@ mod tests { None ); - check(160, "us", 0, vec![]); - check(160, "us", 1, vec![F]); - check(160, "us", 2, vec![F, D]); - check(160, "us", 3, vec![F, D, E]); + check(160, "us", 0, &[][..]); + check(160, "us", 1, &[F]); + check(160, "us", 2, &[F, D]); + check(160, "us", 3, &[F, D, E]); assert_eq!( precomputed_replicas.get_precomputed_network_strategy_replicas( Token::new(160), diff --git a/scylla/src/transport/locator/replication_info.rs b/scylla/src/transport/locator/replication_info.rs index 76747abf2..c0ed6b24a 100644 --- a/scylla/src/transport/locator/replication_info.rs +++ b/scylla/src/transport/locator/replication_info.rs @@ -227,25 +227,25 @@ mod tests { assert_eq!(ids, expected_node_ids); }; - check(160, 0, vec![]); - check(160, 2, vec![F, A]); - - check(200, 1, vec![F]); - check(200, 2, vec![F, A]); - check(200, 3, vec![F, A, C]); - check(200, 4, vec![F, A, C, D]); - check(200, 5, vec![F, A, C, D, G]); - check(200, 6, vec![F, A, C, D, G, B]); - check(200, 7, vec![F, A, C, D, G, B, E]); - - check(701, 1, vec![E]); - check(701, 2, vec![E, G]); - check(701, 3, vec![E, G, B]); - check(701, 4, vec![E, G, B, A]); - check(701, 5, vec![E, G, B, A, F]); - check(701, 6, vec![E, G, B, A, F, C]); - check(701, 7, vec![E, G, B, A, F, C, D]); - check(701, 8, vec![E, G, B, A, F, C, D]); + check(160, 0, &[][..]); + check(160, 2, &[F, A]); + + check(200, 1, &[F]); + check(200, 2, &[F, A]); + check(200, 3, &[F, A, C]); + check(200, 4, &[F, A, C, D]); + check(200, 5, &[F, A, C, D, G]); + check(200, 6, &[F, A, C, D, G, B]); + check(200, 7, &[F, A, C, D, G, B, E]); + + check(701, 1, &[E]); + check(701, 2, &[E, G]); + check(701, 3, &[E, G, B]); + check(701, 4, &[E, G, B, A]); + check(701, 5, &[E, G, B, A, F]); + check(701, 6, &[E, G, B, A, F, C]); + check(701, 7, &[E, G, B, A, F, C, D]); + check(701, 8, &[E, G, B, A, F, C, D]); } #[tokio::test] @@ -261,17 +261,17 @@ mod tests { assert_eq!(ids, expected); }; - check(160, "eu", 0, vec![]); - check(160, "eu", 1, vec![A]); - check(160, "eu", 2, vec![A, G]); - check(160, "eu", 3, vec![A, C, G]); - check(160, "eu", 4, vec![A, C, G, B]); - check(160, "eu", 5, vec![A, C, G, B]); - - check(160, "us", 0, vec![]); - check(160, "us", 1, vec![F]); - check(160, "us", 2, vec![F, D]); - check(160, "us", 3, vec![F, D, E]); - check(160, "us", 4, vec![F, D, E]); + check(160, "eu", 0, &[][..]); + check(160, "eu", 1, &[A]); + check(160, "eu", 2, &[A, G]); + check(160, "eu", 3, &[A, C, G]); + check(160, "eu", 4, &[A, C, G, B]); + check(160, "eu", 5, &[A, C, G, B]); + + check(160, "us", 0, &[]); + check(160, "us", 1, &[F]); + check(160, "us", 2, &[F, D]); + check(160, "us", 3, &[F, D, E]); + check(160, "us", 4, &[F, D, E]); } } diff --git a/scylla/src/transport/locator/token_ring.rs b/scylla/src/transport/locator/token_ring.rs index b78a0d808..7d62ac085 100644 --- a/scylla/src/transport/locator/token_ring.rs +++ b/scylla/src/transport/locator/token_ring.rs @@ -93,105 +93,105 @@ mod tests { ring.ring_range(Token::new(-35)) .cloned() .collect::>(), - vec![-3, -2, -1, 0, 1, 2, 3] + &[-3, -2, -1, 0, 1, 2, 3] ); assert_eq!( ring.ring_range(Token::new(-30)) .cloned() .collect::>(), - vec![-3, -2, -1, 0, 1, 2, 3] + &[-3, -2, -1, 0, 1, 2, 3] ); assert_eq!( ring.ring_range(Token::new(-25)) .cloned() .collect::>(), - vec![-2, -1, 0, 1, 2, 3, -3] + &[-2, -1, 0, 1, 2, 3, -3] ); assert_eq!( ring.ring_range(Token::new(-20)) .cloned() .collect::>(), - vec![-2, -1, 0, 1, 2, 3, -3] + &[-2, -1, 0, 1, 2, 3, -3] ); assert_eq!( ring.ring_range(Token::new(-15)) .cloned() .collect::>(), - vec![-1, 0, 1, 2, 3, -3, -2] + &[-1, 0, 1, 2, 3, -3, -2] ); assert_eq!( ring.ring_range(Token::new(-10)) .cloned() .collect::>(), - vec![-1, 0, 1, 2, 3, -3, -2] + &[-1, 0, 1, 2, 3, -3, -2] ); assert_eq!( ring.ring_range(Token::new(-5)) .cloned() .collect::>(), - vec![0, 1, 2, 3, -3, -2, -1] + &[0, 1, 2, 3, -3, -2, -1] ); assert_eq!( ring.ring_range(Token::new(0)) .cloned() .collect::>(), - vec![0, 1, 2, 3, -3, -2, -1] + &[0, 1, 2, 3, -3, -2, -1] ); assert_eq!( ring.ring_range(Token::new(5)) .cloned() .collect::>(), - vec![1, 2, 3, -3, -2, -1, 0] + &[1, 2, 3, -3, -2, -1, 0] ); assert_eq!( ring.ring_range(Token::new(10)) .cloned() .collect::>(), - vec![1, 2, 3, -3, -2, -1, 0] + &[1, 2, 3, -3, -2, -1, 0] ); assert_eq!( ring.ring_range(Token::new(15)) .cloned() .collect::>(), - vec![2, 3, -3, -2, -1, 0, 1] + &[2, 3, -3, -2, -1, 0, 1] ); assert_eq!( ring.ring_range(Token::new(20)) .cloned() .collect::>(), - vec![2, 3, -3, -2, -1, 0, 1] + &[2, 3, -3, -2, -1, 0, 1] ); assert_eq!( ring.ring_range(Token::new(25)) .cloned() .collect::>(), - vec![3, -3, -2, -1, 0, 1, 2] + &[3, -3, -2, -1, 0, 1, 2] ); assert_eq!( ring.ring_range(Token::new(30)) .cloned() .collect::>(), - vec![3, -3, -2, -1, 0, 1, 2] + &[3, -3, -2, -1, 0, 1, 2] ); assert_eq!( ring.ring_range(Token::new(35)) .cloned() .collect::>(), - vec![-3, -2, -1, 0, 1, 2, 3] + &[-3, -2, -1, 0, 1, 2, 3] ); } } diff --git a/scylla/src/transport/retry_policy.rs b/scylla/src/transport/retry_policy.rs index 54f87380e..2e937a8c2 100644 --- a/scylla/src/transport/retry_policy.rs +++ b/scylla/src/transport/retry_policy.rs @@ -237,7 +237,7 @@ mod tests { #[test] fn default_never_retries() { setup_tracing(); - let never_retried_dberrors = vec![ + let never_retried_dberrors = [ DbError::SyntaxError, DbError::Invalid, DbError::AlreadyExists { @@ -303,7 +303,7 @@ mod tests { #[test] fn default_idempotent_next_retries() { setup_tracing(); - let idempotent_next_errors = vec![ + let idempotent_next_errors = [ QueryError::DbError(DbError::Overloaded, String::new()), QueryError::DbError(DbError::TruncateError, String::new()), QueryError::DbError(DbError::ServerError, String::new()), diff --git a/scylla/src/transport/session_builder.rs b/scylla/src/transport/session_builder.rs index 31b653a5c..c9b63542f 100644 --- a/scylla/src/transport/session_builder.rs +++ b/scylla/src/transport/session_builder.rs @@ -1057,7 +1057,7 @@ mod tests { assert_eq!( builder.config.known_nodes, - vec![KnownNode::Hostname("test_hostname".into())] + &[KnownNode::Hostname("test_hostname".into())] ); assert_eq!(builder.config.compression, None); } @@ -1070,7 +1070,7 @@ mod tests { let addr = SocketAddr::new(IpAddr::V4(Ipv4Addr::new(172, 17, 0, 3)), 1357); builder = builder.known_node_addr(addr); - assert_eq!(builder.config.known_nodes, vec![KnownNode::Address(addr)]); + assert_eq!(builder.config.known_nodes, &[KnownNode::Address(addr)]); assert_eq!(builder.config.compression, None); } @@ -1083,7 +1083,7 @@ mod tests { assert_eq!( builder.config.known_nodes, - vec![ + &[ KnownNode::Hostname("test_hostname1".into()), KnownNode::Hostname("test_hostname2".into()) ] @@ -1103,7 +1103,7 @@ mod tests { assert_eq!( builder.config.known_nodes, - vec![KnownNode::Address(addr1), KnownNode::Address(addr2)] + &[KnownNode::Address(addr1), KnownNode::Address(addr2)] ); assert_eq!(builder.config.compression, None); } @@ -1274,7 +1274,7 @@ mod tests { assert_eq!( builder.config.known_nodes, - vec![ + &[ KnownNode::Hostname("hostname_test".into()), KnownNode::Address(addr), KnownNode::Hostname("hostname_test1".into()), diff --git a/scylla/src/transport/session_test.rs b/scylla/src/transport/session_test.rs index bde64faaf..945088bf5 100644 --- a/scylla/src/transport/session_test.rs +++ b/scylla/src/transport/session_test.rs @@ -126,7 +126,7 @@ async fn test_unprepared_statement() { results.sort(); assert_eq!( results, - vec![ + &[ (1, 2, String::from("abc")), (1, 4, String::from("hello")), (7, 11, String::from("")) @@ -518,7 +518,7 @@ async fn test_batch() { results.sort(); assert_eq!( results, - vec![ + &[ (1, 2, String::from("abc")), (1, 4, String::from("hello")), (7, 11, String::from("")) @@ -555,7 +555,7 @@ async fn test_batch() { .collect::>() .unwrap(); - assert_eq!(results, vec![(4, 20, String::from("foobar"))]); + assert_eq!(results, &[(4, 20, String::from("foobar"))]); } #[tokio::test] @@ -653,7 +653,7 @@ async fn test_token_awareness() { // The default policy should be token aware for size in 1..50usize { - let key = vec!['a'; size].into_iter().collect::(); + let key = std::iter::repeat('a').take(size).collect::(); let values = (&key,); // Execute a query and observe tracing info @@ -724,7 +724,7 @@ async fn test_use_keyspace() { rows.sort(); - assert_eq!(rows, vec!["test1".to_string(), "test2".to_string()]); + assert_eq!(rows, &["test1".to_string(), "test2".to_string()]); // Test that trying to use nonexisting keyspace fails assert!(session @@ -776,7 +776,7 @@ async fn test_use_keyspace() { rows2.sort(); - assert_eq!(rows2, vec!["test1".to_string(), "test2".to_string()]); + assert_eq!(rows2, &["test1".to_string(), "test2".to_string()]); } #[tokio::test] @@ -837,7 +837,7 @@ async fn test_use_keyspace_case_sensitivity() { .map(|row| row.unwrap().0) .collect(); - assert_eq!(rows, vec!["lowercase".to_string()]); + assert_eq!(rows, &["lowercase"]); // Use uppercase keyspace with case sensitivity // Should select the uppercase one @@ -855,7 +855,7 @@ async fn test_use_keyspace_case_sensitivity() { .map(|row| row.unwrap().0) .collect(); - assert_eq!(rows, vec!["uppercase".to_string()]); + assert_eq!(rows, &["uppercase"]); } #[tokio::test] @@ -899,7 +899,7 @@ async fn test_raw_use_keyspace() { .map(|res| res.unwrap().0) .collect(); - assert_eq!(rows, vec!["raw_test".to_string()]); + assert_eq!(rows, &["raw_test"]); // Check if case sensitivity is correctly detected assert!(session @@ -1575,14 +1575,14 @@ async fn test_schema_types_in_metadata() { assert_eq!( tables.keys().sorted().collect::>(), - vec!["table_a", "table_b"] + &["table_a", "table_b"] ); let table_a_columns = &tables["table_a"].columns; assert_eq!( table_a_columns.keys().sorted().collect::>(), - vec!["a", "b", "c", "d", "e"] + &["a", "b", "c", "d", "e"] ); let a = &table_a_columns["a"]; @@ -1709,7 +1709,7 @@ async fn test_user_defined_types_in_metadata() { assert_eq!( user_defined_types.keys().sorted().collect::>(), - vec!["type_a", "type_b", "type_c"] + &["type_a", "type_b", "type_c"] ); let type_a = &user_defined_types["type_a"]; @@ -1812,8 +1812,8 @@ async fn test_primary_key_ordering_in_metadata() { let cluster_data = session.get_cluster_data(); let table = &cluster_data.get_keyspace_info()[&ks].tables["t"]; - assert_eq!(table.partition_key, vec!["c", "e"]); - assert_eq!(table.clustering_key, vec!["b", "a"]); + assert_eq!(table.partition_key, &["c", "e"]); + assert_eq!(table.clustering_key, &["b", "a"]); } #[tokio::test] @@ -1990,9 +1990,9 @@ async fn test_named_bind_markers() { .map(|res| res.unwrap()) .collect(); - assert_eq!(rows, vec![(7, 13, 42), (17, 113, 142)]); + assert_eq!(rows, &[(7, 13, 42), (17, 113, 142)]); - let wrongmaps: Vec> = vec![ + let wrongmaps: &[HashMap<&str, i32>] = &[ HashMap::from([("pk", 7), ("fefe", 42), ("ck", 13)]), HashMap::from([("xxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxx", 7)]), HashMap::new(), @@ -2144,7 +2144,7 @@ async fn test_unprepared_reprepare_in_execute() { .map(|r| r.unwrap()) .collect(); all_rows.sort_unstable(); - assert_eq!(all_rows, vec![(1, 2, 3), (1, 3, 2)]); + assert_eq!(all_rows, &[(1, 2, 3), (1, 3, 2)]); } #[tokio::test] @@ -2170,7 +2170,7 @@ async fn test_unusual_valuelists() { .await .unwrap(); - let values_dyn: Vec<&dyn SerializeValue> = vec![ + let values_dyn: &[&dyn SerializeValue] = &[ &1 as &dyn SerializeValue, &2 as &dyn SerializeValue, &"&dyn" as &dyn SerializeValue, @@ -2180,7 +2180,7 @@ async fn test_unusual_valuelists() { .await .unwrap(); - let values_box_dyn: Vec> = vec![ + let values_box_dyn: &[Box] = &[ Box::new(1) as Box, Box::new(3) as Box, Box::new("Box dyn") as Box, @@ -2204,7 +2204,7 @@ async fn test_unusual_valuelists() { all_rows.sort(); assert_eq!( all_rows, - vec![ + &[ (1i32, 2i32, "&dyn".to_owned()), (1, 3, "Box dyn".to_owned()) ] @@ -2276,7 +2276,7 @@ async fn test_unprepared_reprepare_in_batch() { .map(|r| r.unwrap()) .collect(); all_rows.sort_unstable(); - assert_eq!(all_rows, vec![(1, 2, 3), (1, 3, 2), (4, 5, 6), (4, 6, 5)]); + assert_eq!(all_rows, &[(1, 2, 3), (1, 3, 2), (4, 5, 6), (4, 6, 5)]); } // A tests which checks that Session::execute automatically reprepares PreparedStatemtns if they become unprepared. @@ -2346,7 +2346,7 @@ async fn test_unprepared_reprepare_in_caching_session_execute() { .map(|r| r.unwrap()) .collect(); all_rows.sort_unstable(); - assert_eq!(all_rows, vec![(1, 2, 3), (1, 3, 2)]); + assert_eq!(all_rows, &[(1, 2, 3), (1, 3, 2)]); } #[tokio::test] @@ -2667,7 +2667,7 @@ async fn test_batch_lwts_for_scylla( .collect::>() .unwrap(); - let expected_batch_res_rows = vec![ + let expected_batch_res_rows = &[ (true, Some(0), Some(0), Some(0), Some(0)), (true, None, None, None, None), (true, Some(0), Some(0), Some(0), Some(0)), @@ -2689,7 +2689,7 @@ async fn test_batch_lwts_for_scylla( .map(|r| r.unwrap()) .collect(); - let expected_prepared_batch_res_rows = vec![ + let expected_prepared_batch_res_rows = &[ (false, Some(0), Some(0), Some(1), Some(1)), (false, None, None, None, None), (false, Some(0), Some(0), Some(1), Some(1)), @@ -2714,7 +2714,7 @@ async fn test_batch_lwts_for_cassandra( .map(|r| r.unwrap()) .collect(); - let expected_batch_res_rows = vec![(true,)]; + let expected_batch_res_rows = &[(true,)]; assert_eq!(batch_res_rows, expected_batch_res_rows); @@ -2734,7 +2734,7 @@ async fn test_batch_lwts_for_cassandra( .map(|r| r.unwrap()) .collect(); - let expected_prepared_batch_res_rows = vec![(false, Some(0), Some(0), Some(1), Some(1))]; + let expected_prepared_batch_res_rows = &[(false, Some(0), Some(0), Some(1), Some(1))]; assert_eq!(prepared_batch_res_rows, expected_prepared_batch_res_rows); } @@ -2992,7 +2992,7 @@ async fn simple_strategy_test() { .collect::>(); rows.sort(); - assert_eq!(rows, vec![(1, 2, 3), (4, 5, 6), (7, 8, 9)]); + assert_eq!(rows, &[(1, 2, 3), (4, 5, 6), (7, 8, 9)]); } #[tokio::test] From 0409cbead8f1055a9d21a15eef16ae1d19f8c2f5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Wojciech=20Przytu=C5=82a?= Date: Wed, 2 Oct 2024 11:42:30 +0200 Subject: [PATCH 42/42] partial migration to TableSpec out of ColumnSpec --- scylla-cql/src/frame/response/result.rs | 24 +++++++++++++++------ scylla-cql/src/types/deserialize/result.rs | 4 ++-- scylla/src/statement/prepared_statement.rs | 1 + scylla/src/transport/legacy_query_result.rs | 9 ++++++-- scylla/src/transport/query_result.rs | 6 +++++- 5 files changed, 33 insertions(+), 11 deletions(-) diff --git a/scylla-cql/src/frame/response/result.rs b/scylla-cql/src/frame/response/result.rs index 173bf43a3..55dfaff0b 100644 --- a/scylla-cql/src/frame/response/result.rs +++ b/scylla-cql/src/frame/response/result.rs @@ -525,6 +525,7 @@ impl<'frame> ColumnSpec<'frame> { #[derive(Debug, Clone, Yokeable)] pub struct ResultMetadata<'a> { col_count: usize, + table_spec: Option>, col_specs: Vec>, } @@ -545,6 +546,7 @@ impl<'a> ResultMetadata<'a> { pub fn mock_empty() -> Self { Self { col_count: 0, + table_spec: None, col_specs: Vec::new(), } } @@ -586,6 +588,7 @@ pub struct PreparedMetadata { /// pk_indexes are sorted by `index` and can be reordered in partition key order /// using `sequence` field pub pk_indexes: Vec, + pub table_spec: Option>, pub col_specs: Vec>, } @@ -1041,7 +1044,7 @@ fn deser_col_specs_owned<'frame>( buf: &mut &'frame [u8], global_table_spec: Option>, col_count: usize, -) -> StdResult>, ColumnSpecParseError> { +) -> StdResult<(TableSpec<'static>, Vec>), ColumnSpecParseError> { let result: StdResult>, ColumnSpecParseError> = deser_col_specs_generic( buf, global_table_spec, @@ -1073,18 +1076,21 @@ fn deser_result_metadata( let paging_state = PagingStateResponse::new_from_raw_bytes(raw_paging_state); - let col_specs = if no_metadata { - vec![] + let (table_spec, col_specs) = if no_metadata { + (None, vec![]) } else { let global_table_spec = global_tables_spec .then(|| deser_table_spec(buf)) .transpose()?; - deser_col_specs_owned(buf, global_table_spec, col_count)? + let (table_spec, col_specs) = deser_col_specs_owned(buf, global_table_spec, col_count)?; + let table_spec = table_spec.map(TableSpec::into_owned); + (table_spec, col_specs) }; let metadata = ResultMetadata { col_count, + table_spec, col_specs, }; Ok((metadata, paging_state)) @@ -1262,11 +1268,12 @@ fn deser_prepared_metadata( .then(|| deser_table_spec(buf)) .transpose()?; - let col_specs = deser_col_specs_owned(buf, global_table_spec, col_count)?; + let (table_spec, col_specs) = deser_col_specs_owned(buf, global_table_spec, col_count)?; Ok(PreparedMetadata { flags, col_count, + table_spec: table_spec.map(TableSpec::into_owned), pk_indexes, col_specs, }) @@ -1619,9 +1626,14 @@ mod test_utils { impl<'a> ResultMetadata<'a> { #[inline] #[doc(hidden)] - pub fn new_for_test(col_count: usize, col_specs: Vec>) -> Self { + pub fn new_for_test( + col_count: usize, + table_spec: Option>, + col_specs: Vec>, + ) -> Self { Self { col_count, + table_spec, col_specs, } } diff --git a/scylla-cql/src/types/deserialize/result.rs b/scylla-cql/src/types/deserialize/result.rs index c31c2a2d3..547ebf286 100644 --- a/scylla-cql/src/types/deserialize/result.rs +++ b/scylla-cql/src/types/deserialize/result.rs @@ -298,7 +298,7 @@ mod tests { let row_iter = RowIterator::new(2, specs, FrameSlice::new(raw_data)); let lending_row_iter = RawRowsLendingIterator::new(DeserializedMetadataAndRawRows::new_for_test( - ResultMetadata::new_for_test(specs.len(), specs.to_vec()), + ResultMetadata::new_for_test(specs.len(), None, specs.to_vec()), 2, raw_data.clone(), )); @@ -340,7 +340,7 @@ mod tests { let row_iter = RowIterator::new(2, specs, FrameSlice::new(raw_data)); let lending_row_iter = RawRowsLendingIterator::new(DeserializedMetadataAndRawRows::new_for_test( - ResultMetadata::new_for_test(specs.len(), specs.to_vec()), + ResultMetadata::new_for_test(specs.len(), None, specs.to_vec()), 2, raw_data.clone(), )); diff --git a/scylla/src/statement/prepared_statement.rs b/scylla/src/statement/prepared_statement.rs index 8ecb86a4f..2a52fd442 100644 --- a/scylla/src/statement/prepared_statement.rs +++ b/scylla/src/statement/prepared_statement.rs @@ -636,6 +636,7 @@ mod tests { col_count: col_specs.len(), col_specs, pk_indexes, + table_spec: Some(table_spec), } } diff --git a/scylla/src/transport/legacy_query_result.rs b/scylla/src/transport/legacy_query_result.rs index fa5a7987a..4dae15c6b 100644 --- a/scylla/src/transport/legacy_query_result.rs +++ b/scylla/src/transport/legacy_query_result.rs @@ -303,12 +303,17 @@ mod tests { rows } - fn make_test_metadata() -> ResultMetadata<'static> { + fn make_test_metadata() -> scylla_cql::frame::response::result::ResultMetadata<'static> { let table_spec = TableSpec::borrowed("some_keyspace", "some_table"); + let table_spec_clone = table_spec.clone(); let column_spec = ColumnSpec::borrowed("column0", ColumnType::Int, table_spec); - ResultMetadata::new_for_test(1, vec![column_spec]) + scylla_cql::frame::response::result::ResultMetadata::new_for_test( + 1, + Some(table_spec_clone), + vec![column_spec], + ) } fn make_not_rows_query_result() -> LegacyQueryResult { diff --git a/scylla/src/transport/query_result.rs b/scylla/src/transport/query_result.rs index 1c8579e15..7c16f6311 100644 --- a/scylla/src/transport/query_result.rs +++ b/scylla/src/transport/query_result.rs @@ -506,7 +506,11 @@ mod tests { } fn sample_result_metadata(cols: usize) -> ResultMetadata<'static> { - ResultMetadata::new_for_test(cols, column_spec_infinite_iter().take(cols).collect()) + ResultMetadata::new_for_test( + cols, + None, + column_spec_infinite_iter().take(cols).collect(), + ) } fn sample_raw_rows(cols: usize, rows: usize) -> RawMetadataAndRawRows {