diff --git a/scylla/src/client/execution_profile.rs b/scylla/src/client/execution_profile.rs index 6bdbc42321..c9b5ae518b 100644 --- a/scylla/src/client/execution_profile.rs +++ b/scylla/src/client/execution_profile.rs @@ -1,4 +1,4 @@ -//! `ExecutionProfile` is a grouping of configurable options regarding query execution. +//! `ExecutionProfile` is a grouping of configurable options regarding CQL statement execution. //! //! Profiles can be created to represent different workloads, which thanks to them //! can be run conveniently on a single session. @@ -38,12 +38,12 @@ //! ``` //! //! ### Example -//! To create an `ExecutionProfile` and attach it to a `Query`: +//! To create an [`ExecutionProfile`] and attach it to a [`Statement`](crate::statement::Statement): //! ``` //! # extern crate scylla; //! # use std::error::Error; //! # async fn check_only_compiles() -> Result<(), Box> { -//! use scylla::query::Query; +//! use scylla::statement::Statement; //! use scylla::statement::Consistency; //! use scylla::client::execution_profile::ExecutionProfile; //! use std::time::Duration; @@ -55,10 +55,10 @@ //! //! let handle = profile.into_handle(); //! -//! let mut query1 = Query::from("SELECT * FROM ks.table"); +//! let mut query1 = Statement::from("SELECT * FROM ks.table"); //! query1.set_execution_profile_handle(Some(handle.clone())); //! -//! let mut query2 = Query::from("SELECT pk FROM ks.table WHERE pk = ?"); +//! let mut query2 = Statement::from("SELECT pk FROM ks.table WHERE pk = ?"); //! query2.set_execution_profile_handle(Some(handle)); //! # Ok(()) //! # } @@ -82,7 +82,7 @@ //! let profile = base_profile.to_builder() //! .consistency(Consistency::All) //! .build(); -// +//! //! # Ok(()) //! # } //! ``` @@ -112,7 +112,7 @@ //! # async fn check_only_compiles() -> Result<(), Box> { //! use scylla::client::session::Session; //! use scylla::client::session_builder::SessionBuilder; -//! use scylla::query::Query; +//! use scylla::statement::Statement; //! use scylla::statement::Consistency; //! use scylla::client::execution_profile::ExecutionProfile; //! @@ -133,8 +133,8 @@ //! .build() //! .await?; //! -//! let mut query1 = Query::from("SELECT * FROM ks.table"); -//! let mut query2 = Query::from("SELECT pk FROM ks.table WHERE pk = ?"); +//! let mut query1 = Statement::from("SELECT * FROM ks.table"); +//! let mut query2 = Statement::from("SELECT pk FROM ks.table WHERE pk = ?"); //! //! query1.set_execution_profile_handle(Some(handle1.clone())); //! query2.set_execution_profile_handle(Some(handle2.clone())); @@ -259,16 +259,16 @@ impl ExecutionProfileBuilder { self } - /// Specify a default consistency to be used for queries. + /// Specify a default consistency to be used for statement executions. /// It's possible to override it by explicitly setting a consistency on the chosen query. pub fn consistency(mut self, consistency: Consistency) -> Self { self.consistency = Some(consistency); self } - /// Specify a default serial consistency to be used for queries. + /// Specify a default serial consistency to be used for statement executions. /// It's possible to override it by explicitly setting a serial consistency - /// on the chosen query. + /// on the chosen statement. pub fn serial_consistency(mut self, serial_consistency: Option) -> Self { self.serial_consistency = Some(serial_consistency); self @@ -297,7 +297,7 @@ impl ExecutionProfileBuilder { self } - /// Sets the [`RetryPolicy`] to use by default on queries. + /// Sets the [`RetryPolicy`] to use by default on statements. /// The default is [DefaultRetryPolicy](crate::policies::retry::DefaultRetryPolicy). /// It is possible to implement a custom retry policy by implementing the trait [`RetryPolicy`]. /// @@ -390,11 +390,11 @@ impl Default for ExecutionProfileBuilder { } } -/// A profile that groups configurable options regarding query execution. +/// A profile that groups configurable options regarding statement execution. /// /// Execution profile is immutable as such, but the driver implements double indirection of form: -/// query/Session -> ExecutionProfileHandle -> ExecutionProfile -/// which enables on-fly changing the actual profile associated with all entities (query/Session) +/// statement/Session -> [`ExecutionProfileHandle`] -> [`ExecutionProfile`] +/// which enables on-fly changing the actual profile associated with all entities (statement/Session) /// by the same handle. #[derive(Debug, Clone)] pub struct ExecutionProfile(pub(crate) Arc); @@ -493,7 +493,7 @@ impl ExecutionProfile { /// A handle that points to an ExecutionProfile. /// -/// Its goal is to enable remapping all associated entities (query/Session) +/// Its goal is to enable remapping all associated entities (statement/Session) /// to another execution profile at once. /// Note: Cloned handles initially point to the same Arc'ed execution profile. /// However, as the mapping has yet another level of indirection - through @@ -521,7 +521,7 @@ impl ExecutionProfileHandle { } /// Makes the handle point to a new execution profile. - /// All entities (queries/Session) holding this handle will reflect the change. + /// All entities (statement/Session) holding this handle will reflect the change. pub fn map_to_another_profile(&mut self, profile: ExecutionProfile) { self.0 .0.store(profile.0) } diff --git a/scylla/src/client/pager.rs b/scylla/src/client/pager.rs index b1813faf52..19df341ac6 100644 --- a/scylla/src/client/pager.rs +++ b/scylla/src/client/pager.rs @@ -662,23 +662,23 @@ impl QueryPager { } pub(crate) async fn new_for_query( - query: Statement, + statement: Statement, execution_profile: Arc, cluster_state: Arc, metrics: Arc, ) -> Result { let (sender, receiver) = mpsc::channel::>(1); - let consistency = query + let consistency = statement .config .consistency .unwrap_or(execution_profile.consistency); - let serial_consistency = query + let serial_consistency = statement .config .serial_consistency .unwrap_or(execution_profile.serial_consistency); - let page_size = query.get_validated_page_size(); + let page_size = statement.get_validated_page_size(); let routing_info = RoutingInfo { consistency, @@ -686,7 +686,7 @@ impl QueryPager { ..Default::default() }; - let retry_session = query + let retry_session = statement .get_retry_policy() .map(|rp| &**rp) .unwrap_or(&*execution_profile.retry_policy) @@ -694,7 +694,7 @@ impl QueryPager { let parent_span = tracing::Span::current(); let worker_task = async move { - let query_ref = &query; + let statement_ref = &statement; let page_query = |connection: Arc, consistency: Consistency, @@ -702,7 +702,7 @@ impl QueryPager { async move { connection .query_raw_with_consistency( - query_ref, + statement_ref, consistency, serial_consistency, Some(page_size), @@ -712,7 +712,7 @@ impl QueryPager { } }; - let query_ref = &query; + let query_ref = &statement; let span_creator = move || { let span = RequestSpan::new_query(&query_ref.contents); @@ -724,13 +724,13 @@ impl QueryPager { sender: sender.into(), page_query, statement_info: routing_info, - query_is_idempotent: query.config.is_idempotent, + query_is_idempotent: statement.config.is_idempotent, query_consistency: consistency, retry_session, execution_profile, metrics, paging_state: PagingState::start(), - history_listener: query.config.history_listener.clone(), + history_listener: statement.config.history_listener.clone(), current_request_id: None, current_attempt_id: None, parent_span, diff --git a/scylla/src/client/session.rs b/scylla/src/client/session.rs index 240ddb8c6c..8ab2296107 100644 --- a/scylla/src/client/session.rs +++ b/scylla/src/client/session.rs @@ -1,5 +1,5 @@ //! `Session` is the main object used in the driver.\ -//! It manages all connections to the cluster and allows to perform queries. +//! It manages all connections to the cluster and allows to execute CQL requests. use super::execution_profile::{ExecutionProfile, ExecutionProfileHandle, ExecutionProfileInner}; use super::pager::{PreparedIteratorConfig, QueryPager}; @@ -61,7 +61,7 @@ pub(crate) const TABLET_CHANNEL_SIZE: usize = 8192; const TRACING_QUERY_PAGE_SIZE: i32 = 1024; -/// `Session` manages connections to the cluster and allows to perform queries +/// `Session` manages connections to the cluster and allows to execute CQL requests. pub struct Session { cluster: Cluster, default_execution_profile_handle: ExecutionProfileHandle, @@ -347,24 +347,24 @@ pub(crate) enum RunRequestResult { impl Session { /// Sends a request to the database and receives a response.\ - /// Performs an unpaged query, i.e. all results are received in a single response. + /// Executes an unprepared CQL statement without paging, i.e. all results are received in a single response. /// - /// This is the easiest way to make a query, but performance is worse than that of prepared queries. + /// This is the easiest way to execute a CQL statement, but performance is worse than that of prepared statements. /// - /// 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 + /// It is discouraged to use this method with non-empty values argument ([`SerializeRow::is_empty()`] + /// trait method returns false). In such case, statement first needs to be prepared (on a single connection), so /// driver will perform 2 round trips instead of 1. Please use [`Session::execute_unpaged()`] instead. /// /// 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: + /// it is best to use paged requests: /// - to receive multiple pages and transparently iterate through them, use [query_iter](Session::query_iter). /// - to manually receive multiple pages and iterate through them, use [query_single_page](Session::query_single_page). /// /// See [the book](https://rust-driver.docs.scylladb.com/stable/queries/simple.html) for more information /// # Arguments - /// * `query` - statement to be executed, can be just a `&str` or the [`Statement`] struct. - /// * `values` - values bound to the query, the easiest way is to use a tuple of bound values. + /// * `statement` - statement to be executed, can be just a `&str` or the [`Statement`] struct. + /// * `values` - values bound to the statement, the easiest way is to use a tuple of bound values. /// /// # Examples /// ```rust @@ -404,22 +404,22 @@ impl Session { /// ``` pub async fn query_unpaged( &self, - query: impl Into, + statement: impl Into, values: impl SerializeRow, ) -> Result { - self.do_query_unpaged(&query.into(), values).await + self.do_query_unpaged(&statement.into(), values).await } /// Queries a single page from the database, optionally continuing from a saved point. /// - /// 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 + /// It is discouraged to use this method with non-empty values argument ([`SerializeRow::is_empty()`] + /// trait method returns false). In such case, CQL statement first needs to be prepared (on a single connection), so /// driver will perform 2 round trips instead of 1. Please use [`Session::execute_single_page()`] instead. /// /// # Arguments /// - /// * `query` - statement to be executed - /// * `values` - values bound to the query + /// * `statement` - statement to be executed + /// * `values` - values bound to the statement /// * `paging_state` - previously received paging state or [PagingState::start()] /// /// # Example @@ -463,29 +463,29 @@ impl Session { /// ``` pub async fn query_single_page( &self, - query: impl Into, + statement: impl Into, values: impl SerializeRow, paging_state: PagingState, ) -> Result<(QueryResult, PagingStateResponse), ExecutionError> { - self.do_query_single_page(&query.into(), values, paging_state) + self.do_query_single_page(&statement.into(), values, paging_state) .await } - /// Run an unprepared query with paging\ + /// Execute an unprepared CQL statement 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 [`Statement`] 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 + /// It is discouraged to use this method with non-empty values argument ([`SerializeRow::is_empty()`] + /// trait method returns false). In such case, statement 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 [`Statement`] struct. - /// * `values` - values bound to the query, the easiest way is to use a tuple of bound values. + /// * `statement` - statement to be executed, can be just a `&str` or the [`Statement`] struct. + /// * `values` - values bound to the statement, the easiest way is to use a tuple of bound values. /// /// # Example /// @@ -509,24 +509,24 @@ impl Session { /// ``` pub async fn query_iter( &self, - query: impl Into, + statement: impl Into, values: impl SerializeRow, ) -> Result { - self.do_query_iter(query.into(), values).await + self.do_query_iter(statement.into(), 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. + /// Performs an unpaged request, 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: + /// it is best to use paged requests: /// - 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 + /// Prepared statements are much faster than unprepared statements: + /// * Database doesn't need to parse the statement string /// * They are properly load balanced using token aware routing /// /// > ***Warning***\ @@ -538,7 +538,7 @@ impl Session { /// /// # 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 + /// * `values` - values bound to the statement, the easiest way is to use a tuple of bound values /// /// # Example /// ```rust @@ -547,12 +547,12 @@ impl Session { /// # async fn check_only_compiles(session: &Session) -> Result<(), Box> { /// use scylla::prepared_statement::PreparedStatement; /// - /// // Prepare the query for later execution + /// // Prepare the statement 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. + /// // Execute the prepared statement with some values, just like an unprepared statement. /// let to_insert: i32 = 12345; /// session.execute_unpaged(&prepared, (to_insert,)).await?; /// # Ok(()) @@ -572,7 +572,7 @@ impl Session { /// # Arguments /// /// * `prepared` - a statement prepared with [prepare](crate::client::session::Session::prepare) - /// * `values` - values bound to the query + /// * `values` - values bound to the statement /// * `paging_state` - continuation based on a paging state received from a previous paged query or None /// /// # Example @@ -582,12 +582,12 @@ impl Session { /// # use std::error::Error; /// # async fn check_only_compiles(session: &Session) -> Result<(), Box> { /// use std::ops::ControlFlow; - /// use scylla::query::Query; + /// use scylla::statement::Statement; /// use scylla::response::{PagingState, PagingStateResponse}; /// /// let paged_prepared = session /// .prepare( - /// Query::new("SELECT a, b FROM ks.tbl") + /// Statement::new("SELECT a, b FROM ks.tbl") /// .with_page_size(100.try_into().unwrap()), /// ) /// .await?; @@ -632,7 +632,7 @@ impl Session { .await } - /// Run a prepared query with paging.\ + /// Execute a prepared statement with paging.\ /// This method will query all pages of the result.\ /// /// Returns an async iterator (stream) over all received rows.\ @@ -642,7 +642,7 @@ impl Session { /// /// # 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 + /// * `values` - values bound to the statement, the easiest way is to use a tuple of bound values /// /// # Example /// @@ -653,12 +653,12 @@ impl Session { /// # async fn check_only_compiles(session: &Session) -> Result<(), Box> { /// use scylla::prepared_statement::PreparedStatement; /// - /// // Prepare the query for later execution + /// // Prepare the statement for later execution /// let prepared: PreparedStatement = session /// .prepare("SELECT a, b FROM ks.t") /// .await?; /// - /// // Execute the query and receive all pages + /// // Execute the statement and receive all pages /// let mut rows_stream = session /// .execute_iter(prepared, &[]) /// .await? @@ -679,22 +679,22 @@ impl Session { self.do_execute_iter(prepared.into(), values).await } - /// Perform a batch query\ - /// Batch contains many `simple` or `prepared` queries which are executed at once\ - /// Batch doesn't return any rows + /// Execute a batch statement\ + /// Batch contains many `unprepared` or `prepared` statements which are executed at once\ + /// Batch doesn't return any rows. /// - /// Batch values must contain values for each of the queries + /// Batch values must contain values for each of the statements. /// - /// 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 + /// 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 + /// * `values` - List of values for each statements, it's the easiest to use a tuple of tuples /// /// # Example /// ```rust @@ -705,19 +705,19 @@ impl Session { /// /// let mut batch: Batch = Default::default(); /// - /// // A query with two bound values + /// // A statement with two bound values /// batch.append_statement("INSERT INTO ks.tab(a, b) VALUES(?, ?)"); /// - /// // A query with one bound value + /// // A statement with one bound value /// batch.append_statement("INSERT INTO ks.tab(a, b) VALUES(3, ?)"); /// - /// // A query with no bound values + /// // A statement 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 + /// // Batch values is a tuple of 3 tuples containing values for each statement + /// let batch_values = ((1_i32, 2_i32), // Tuple with two values for the first statement + /// (4_i32,), // Tuple with one value for the second statement + /// ()); // Empty tuple/unit for the third statement /// /// // Run the batch /// session.batch(&batch, batch_values).await?; @@ -859,11 +859,11 @@ impl Session { async fn do_query_unpaged( &self, - query: &Statement, + statement: &Statement, values: impl SerializeRow, ) -> Result { let (result, paging_state_response) = self - .query(query, values, None, PagingState::start()) + .query(statement, 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."); @@ -876,14 +876,14 @@ impl Session { async fn do_query_single_page( &self, - query: &Statement, + statement: &Statement, values: impl SerializeRow, paging_state: PagingState, ) -> Result<(QueryResult, PagingStateResponse), ExecutionError> { self.query( - query, + statement, values, - Some(query.get_validated_page_size()), + Some(statement.get_validated_page_size()), paging_state, ) .await @@ -902,44 +902,44 @@ impl Session { /// should be made. async fn query( &self, - query: &Statement, + statement: &Statement, values: impl SerializeRow, page_size: Option, paging_state: PagingState, ) -> Result<(QueryResult, PagingStateResponse), ExecutionError> { - let execution_profile = query + let execution_profile = statement .get_execution_profile_handle() .unwrap_or_else(|| self.get_default_execution_profile_handle()) .access(); let statement_info = RoutingInfo { - consistency: query + consistency: statement .config .consistency .unwrap_or(execution_profile.consistency), - serial_consistency: query + serial_consistency: statement .config .serial_consistency .unwrap_or(execution_profile.serial_consistency), ..Default::default() }; - let span = RequestSpan::new_query(&query.contents); + let span = RequestSpan::new_query(&statement.contents); let span_ref = &span; let run_request_result = self .run_request( statement_info, - &query.config, + &statement.config, execution_profile, |connection: Arc, consistency: Consistency, execution_profile: &ExecutionProfileInner| { - let serial_consistency = query + let serial_consistency = statement .config .serial_consistency .unwrap_or(execution_profile.serial_consistency); // Needed to avoid moving query and values into async move block - let query_ref = &query; + let statement_ref = &statement; let values_ref = &values; let paging_state_ref = &paging_state; async move { @@ -947,7 +947,7 @@ impl Session { span_ref.record_request_size(0); connection .query_raw_with_consistency( - query_ref, + statement_ref, consistency, serial_consistency, page_size, @@ -956,7 +956,7 @@ impl Session { .await .and_then(QueryResponse::into_non_error_query_response) } else { - let prepared = connection.prepare(query_ref).await?; + let prepared = connection.prepare(statement_ref).await?; let serialized = prepared.serialize_values(values_ref)?; span_ref.record_request_size(serialized.buffer_size()); connection @@ -1033,17 +1033,17 @@ impl Session { async fn do_query_iter( &self, - query: Statement, + statement: Statement, values: impl SerializeRow, ) -> Result { - let execution_profile = query + let execution_profile = statement .get_execution_profile_handle() .unwrap_or_else(|| self.get_default_execution_profile_handle()) .access(); if values.is_empty() { QueryPager::new_for_query( - query, + statement, execution_profile, self.cluster.get_state(), self.metrics.clone(), @@ -1054,7 +1054,7 @@ impl Session { // Making QueryPager::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 prepared = self.prepare(statement).await?; let values = prepared.serialize_values(&values)?; QueryPager::new_for_prepared_statement(PreparedIteratorConfig { prepared, @@ -1069,10 +1069,10 @@ impl Session { } /// Prepares a statement on the server side and returns a prepared statement, - /// which can later be used to perform more efficient queries + /// which can later be used to perform more efficient requests. /// - /// Prepared queries are much faster than simple queries: - /// * Database doesn't need to parse the query + /// Prepared statements are much faster than unprepared statements: + /// * Database doesn't need to parse the statement string /// * They are properly load balanced using token aware routing /// /// > ***Warning***\ @@ -1084,7 +1084,7 @@ impl Session { /// See the documentation of [`PreparedStatement`]. /// /// # Arguments - /// * `query` - query to prepare, can be just a `&str` or the [`Statement`] struct. + /// * `statement` - statement to prepare, can be just a `&str` or the [`Statement`] struct. /// /// # Example /// ```rust @@ -1093,12 +1093,12 @@ impl Session { /// # async fn check_only_compiles(session: &Session) -> Result<(), Box> { /// use scylla::prepared_statement::PreparedStatement; /// - /// // Prepare the query for later execution + /// // Prepare the statement 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 + /// // Execute the prepared statement with some values, just like an unprepared statement. /// let to_insert: i32 = 12345; /// session.execute_unpaged(&prepared, (to_insert,)).await?; /// # Ok(()) @@ -1106,16 +1106,16 @@ impl Session { /// ``` pub async fn prepare( &self, - query: impl Into, + statement: impl Into, ) -> Result { - let query = query.into(); - let query_ref = &query; + let statement = statement.into(); + let statement_ref = &statement; let cluster_state = self.get_cluster_state(); let connections_iter = cluster_state.iter_working_connections()?; // Prepare statements on all connections concurrently - let handles = connections_iter.map(|c| async move { c.prepare(query_ref).await }); + let handles = connections_iter.map(|c| async move { c.prepare(statement_ref).await }); let mut results = join_all(handles).await.into_iter(); // If at least one prepare was successful, `prepare()` returns Ok. @@ -1437,7 +1437,7 @@ impl Session { /// // Prepare all statements in the batch at once /// let prepared_batch: Batch = session.prepare_batch(&batch).await?; /// - /// // Specify bound values to use with each query + /// // Specify bound values to use with each statement /// let batch_values = ((1_i32, 2_i32), /// (3_i32, 4_i32)); /// @@ -1482,7 +1482,7 @@ impl Session { /// /// * `keyspace_name` - keyspace name to use, /// keyspace names can have up to 48 alphanumeric characters and contain underscores - /// * `case_sensitive` - if set to true the generated query will put keyspace name in quotes + /// * `case_sensitive` - if set to true the generated statement will put keyspace name in quotes /// # Example /// ```rust /// # use scylla::client::session::Session; @@ -1496,7 +1496,7 @@ impl Session { /// /// session.use_keyspace("my_keyspace", false).await?; /// - /// // Now we can omit keyspace name in the query + /// // Now we can omit keyspace name in the statement /// session /// .query_unpaged("INSERT INTO tab (a) VALUES ('test2')", &[]) /// .await?; diff --git a/scylla/src/network/connection.rs b/scylla/src/network/connection.rs index 668f0cd060..44188c8a6b 100644 --- a/scylla/src/network/connection.rs +++ b/scylla/src/network/connection.rs @@ -820,27 +820,27 @@ impl Connection { #[allow(dead_code)] pub(crate) async fn query_unpaged( &self, - query: impl Into, + statement: impl Into, ) -> Result { // This method is used only for driver internal queries, so no need to consult execution profile here. - let query: Statement = query.into(); + let statement: Statement = statement.into(); - self.query_raw_unpaged(&query) + self.query_raw_unpaged(&statement) .await .and_then(QueryResponse::into_query_result) } pub(crate) async fn query_raw_unpaged( &self, - query: &Statement, + statement: &Statement, ) -> Result { // This method is used only for driver internal queries, so no need to consult execution profile here. self.query_raw_with_consistency( - query, - query + statement, + statement .config .determine_consistency(self.config.default_consistency), - query.config.serial_consistency.flatten(), + statement.config.serial_consistency.flatten(), None, PagingState::start(), ) @@ -849,7 +849,7 @@ impl Connection { pub(crate) async fn query_raw_with_consistency( &self, - query: &Statement, + statement: &Statement, consistency: Consistency, serial_consistency: Option, page_size: Option, @@ -861,10 +861,10 @@ impl Connection { .as_ref() .map(|gen| gen.next_timestamp()) }; - let timestamp = query.get_timestamp().or_else(get_timestamp_from_gen); + let timestamp = statement.get_timestamp().or_else(get_timestamp_from_gen); let query_frame = query::Query { - contents: Cow::Borrowed(&query.contents), + contents: Cow::Borrowed(&statement.contents), parameters: query::QueryParameters { consistency, serial_consistency, @@ -877,7 +877,7 @@ impl Connection { }; let response = self - .send_request(&query_frame, true, query.config.tracing, None) + .send_request(&query_frame, true, statement.config.tracing, None) .await?; Ok(response) diff --git a/scylla/src/statement/statement.rs b/scylla/src/statement/statement.rs index 31771e31a0..4c41cf8eec 100644 --- a/scylla/src/statement/statement.rs +++ b/scylla/src/statement/statement.rs @@ -6,9 +6,9 @@ use crate::policies::retry::RetryPolicy; use std::sync::Arc; use std::time::Duration; -/// CQL query statement. +/// **Unprepared** CQL statement. /// -/// This represents a CQL query that can be executed on a server. +/// This represents a CQL statement that can be executed on a server. #[derive(Clone)] pub struct Statement { pub(crate) config: StatementConfig, @@ -18,7 +18,7 @@ pub struct Statement { } impl Statement { - /// Creates a new [`Statement`] from a CQL query string. + /// Creates a new [`Statement`] from a CQL statement string. pub fn new(query_text: impl Into) -> Self { Self { contents: query_text.into(), @@ -35,7 +35,7 @@ impl Statement { self } - /// Sets the page size for this CQL query. + /// Sets the page size for this CQL statement. /// /// Panics if given number is nonpositive. pub fn set_page_size(&mut self, page_size: i32) { @@ -44,12 +44,12 @@ impl Statement { .unwrap_or_else(|err| panic!("Query::set_page_size: {err}")); } - /// Returns the page size for this CQL query. + /// Returns the page size for this CQL statement. pub(crate) fn get_validated_page_size(&self) -> PageSize { self.page_size } - /// Returns the page size for this CQL query. + /// Returns the page size for this CQL statement. pub fn get_page_size(&self) -> i32 { self.page_size.inner() } @@ -59,7 +59,7 @@ impl Statement { self.config.consistency = Some(c); } - /// Gets the consistency to be used when executing this query if it is filled. + /// Gets the consistency to be used when executing this statement if it is filled. /// If this is empty, the default_consistency of the session will be used. pub fn get_consistency(&self) -> Option { self.config.consistency @@ -78,10 +78,10 @@ impl Statement { } /// Sets the idempotence of this statement - /// A query is idempotent if it can be applied multiple times without changing the result of the initial application + /// A statement is idempotent if it can be applied multiple times without changing the result of the initial application /// If set to `true` we can be sure that it is idempotent /// If set to `false` it is unknown whether it is idempotent - /// This is used in [`RetryPolicy`] to decide if retrying a query is safe + /// This is used in [`RetryPolicy`] to decide if retrying a statement execution is safe pub fn set_is_idempotent(&mut self, is_idempotent: bool) { self.config.is_idempotent = is_idempotent; } @@ -124,7 +124,7 @@ impl Statement { self.config.request_timeout = timeout } - /// Gets client timeout associated with this query + /// Gets client timeout associated with this statement. pub fn get_request_timeout(&self) -> Option { self.config.request_timeout } @@ -141,7 +141,7 @@ impl Statement { self.config.retry_policy.as_ref() } - /// Sets the listener capable of listening what happens during query execution. + /// Sets the listener capable of listening what happens during statement execution. pub fn set_history_listener(&mut self, history_listener: Arc) { self.config.history_listener = Some(history_listener); } @@ -157,7 +157,7 @@ impl Statement { self.config.execution_profile_handle = profile_handle; } - /// Borrows the execution profile handle associated with this query. + /// Borrows the execution profile handle associated with this statement. pub fn get_execution_profile_handle(&self) -> Option<&ExecutionProfileHandle> { self.config.execution_profile_handle.as_ref() }