Skip to content

Commit

Permalink
session: simple -> unprepared, query -> statement
Browse files Browse the repository at this point in the history
Adjusted the comments and some parameter names,
so we follow the unified naming convention.
  • Loading branch information
muzarski committed May 6, 2024
1 parent 3df17f6 commit 96e258b
Showing 1 changed file with 65 additions and 65 deletions.
130 changes: 65 additions & 65 deletions scylla/src/transport/session.rs
Original file line number Diff line number Diff line change
@@ -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 statements.
use crate::batch::batch_values;
#[cfg(feature = "cloud")]
Expand Down Expand Up @@ -136,7 +136,7 @@ impl AddressTranslator for HashMap<&'static str, &'static str> {
}
}

/// `Session` manages connections to the cluster and allows to perform queries
/// `Session` manages connections to the cluster and allows to execute statements
pub struct Session {
cluster: Cluster,
default_execution_profile_handle: ExecutionProfileHandle,
Expand Down Expand Up @@ -555,19 +555,19 @@ impl Session {
Ok(session)
}

/// Sends a query to the database and receives a response.\
/// Executes an unprepared statement and receives a response.\
/// Returns only a single page of results, to receive multiple pages use [query_iter](Session::query_iter)
///
/// 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 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
/// trait 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()`] instead.
///
/// See [the book](https://rust-driver.docs.scylladb.com/stable/queries/simple.html) for more information
/// # Arguments
/// * `query` - query to perform, can be just a `&str` or the [Query] struct.
/// * `values` - values bound to the query, easiest way is to use a tuple of bound values
/// * `statement` - statement to execute, can be just a `&str` or the [UnpreparedStatement] struct.
/// * `values` - values bound to the statement, easiest way is to use a tuple of bound values
///
/// # Examples
/// ```rust
Expand Down Expand Up @@ -607,30 +607,30 @@ impl Session {
/// ```
pub async fn query(
&self,
query: impl Into<UnpreparedStatement>,
statement: impl Into<UnpreparedStatement>,
values: impl SerializeRow,
) -> Result<QueryResult, QueryError> {
self.query_paged(query, values, None).await
self.query_paged(statement, values, None).await
}

/// Queries the database with a custom paging state.
/// Executes an unprepared statement with a custom paging state.
///
/// 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
/// trait 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_paged()`] instead.
///
/// # Arguments
///
/// * `query` - query to be performed
/// * `values` - values bound to the query
/// * `statement` - statement to be executed
/// * `values` - values bound to the statement
/// * `paging_state` - previously received paging state or None
pub async fn query_paged(
&self,
query: impl Into<UnpreparedStatement>,
statement: impl Into<UnpreparedStatement>,
values: impl SerializeRow,
paging_state: Option<Bytes>,
) -> Result<QueryResult, QueryError> {
let query: UnpreparedStatement = query.into();
let query: UnpreparedStatement = statement.into();

let execution_profile = query
.get_execution_profile_handle()
Expand Down Expand Up @@ -724,7 +724,7 @@ impl Session {
) -> Result<(), QueryError> {
if let Some(set_keyspace) = response.as_set_keyspace() {
debug!(
"Detected USE KEYSPACE query, setting session's keyspace to {}",
"Detected USE KEYSPACE statement, setting session's keyspace to {}",
set_keyspace.keyspace_name
);
self.use_keyspace(set_keyspace.keyspace_name.clone(), true)
Expand Down Expand Up @@ -753,21 +753,21 @@ impl Session {
Ok(())
}

/// Run a simple query with paging\
/// Execute an unprepared 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 [Query] passed to the function
/// Page size can be specified in the [UnpreparedStatement] 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
/// trait 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` - query to perform, can be just a `&str` or the [Query] struct.
/// * `values` - values bound to the query, easiest way is to use a tuple of bound values
/// * `statement` - statement to execute, can be just a `&str` or the [UnpreparedStatement] struct.
/// * `values` - values bound to the statement, easiest way is to use a tuple of bound values
///
/// # Example
///
Expand All @@ -792,10 +792,10 @@ impl Session {
/// ```
pub async fn query_iter(
&self,
query: impl Into<UnpreparedStatement>,
statement: impl Into<UnpreparedStatement>,
values: impl SerializeRow,
) -> Result<RowIterator, QueryError> {
let query: UnpreparedStatement = query.into();
let query: UnpreparedStatement = statement.into();

let execution_profile = query
.get_execution_profile_handle()
Expand Down Expand Up @@ -828,10 +828,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 for more efficient execution
///
/// 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
/// * They are properly load balanced using token aware routing
///
/// > ***Warning***\
Expand All @@ -842,7 +842,7 @@ impl Session {
/// See [the book](https://rust-driver.docs.scylladb.com/stable/queries/prepared.html) for more information
///
/// # Arguments
/// * `query` - query to prepare, can be just a `&str` or the [Query] struct.
/// * `statement` - statement to prepare, can be just a `&str` or the [UnpreparedStatement] struct.
///
/// # Example
/// ```rust
Expand All @@ -851,22 +851,22 @@ impl Session {
/// # async fn check_only_compiles(session: &Session) -> Result<(), Box<dyn Error>> {
/// 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(&prepared, (to_insert,)).await?;
/// # Ok(())
/// # }
/// ```
pub async fn prepare(
&self,
query: impl Into<UnpreparedStatement>,
statement: impl Into<UnpreparedStatement>,
) -> Result<PreparedStatement, QueryError> {
let query = query.into();
let query = statement.into();
let query_ref = &query;

let cluster_data = self.get_cluster_data();
Expand Down Expand Up @@ -922,12 +922,12 @@ impl Session {
.as_deref()
}

/// Execute a prepared query. Requires a [PreparedStatement]
/// Execute a prepared statement. Requires a [PreparedStatement]
/// generated using [`Session::prepare`](Session::prepare)\
/// Returns only a single page of results, to receive multiple pages use [execute_iter](Session::execute_iter)
///
/// 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
/// * They are properly load balanced using token aware routing
///
/// > ***Warning***\
Expand All @@ -939,7 +939,7 @@ impl Session {
///
/// # Arguments
/// * `prepared` - the prepared statement to execute, generated using [`Session::prepare`](Session::prepare)
/// * `values` - values bound to the query, easiest way is to use a tuple of bound values
/// * `values` - values bound to the statement, easiest way is to use a tuple of bound values
///
/// # Example
/// ```rust
Expand All @@ -948,12 +948,12 @@ impl Session {
/// # async fn check_only_compiles(session: &Session) -> Result<(), Box<dyn Error>> {
/// 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(&prepared, (to_insert,)).await?;
/// # Ok(())
Expand All @@ -971,7 +971,7 @@ impl Session {
/// # Arguments
///
/// * `prepared` - a statement prepared with [prepare](crate::transport::session::Session::prepare)
/// * `values` - values bound to the query
/// * `values` - values bound to the statement
/// * `paging_state` - paging state from the previous query or None
pub async fn execute_paged(
&self,
Expand Down Expand Up @@ -1069,7 +1069,7 @@ impl Session {
Ok(result)
}

/// 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\
Expand All @@ -1080,7 +1080,7 @@ impl Session {
///
/// # Arguments
/// * `prepared` - the prepared statement to execute, generated using [`Session::prepare`](Session::prepare)
/// * `values` - values bound to the query, easiest way is to use a tuple of bound values
/// * `values` - values bound to the statement, easiest way is to use a tuple of bound values
///
/// # Example
///
Expand All @@ -1092,12 +1092,12 @@ impl Session {
/// use scylla::IntoTypedRows;
/// use futures::stream::StreamExt;
///
/// // 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?
Expand Down Expand Up @@ -1133,22 +1133,22 @@ impl Session {
.await
}

/// Perform a batch query\
/// Batch contains many `simple` or `prepared` queries which are executed at once\
/// 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 (number_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 statement, it's the easiest to use a tuple of tuples
///
/// # Example
/// ```rust
Expand All @@ -1159,19 +1159,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?;
Expand All @@ -1186,7 +1186,7 @@ impl Session {
// 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

// check to ensure that we don't send a batch statement with more than u16::MAX queries
// check to ensure that we don't send a batch statement with more than u16::MAX statements
let batch_statements_length = batch.statements.len();
if batch_statements_length > u16::MAX as usize {
return Err(QueryError::BadQuery(
Expand Down Expand Up @@ -1282,7 +1282,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));
///
Expand Down Expand Up @@ -1340,7 +1340,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("INSERT INTO tab (a) VALUES ('test2')", &[])
/// .await?;
Expand All @@ -1357,7 +1357,7 @@ impl Session {
.store(Some(Arc::new(keyspace_name.clone())));

// Trying to pass keyspace as bound value in "USE ?" doesn't work
// So we have to create a string for query: "USE " + new_keyspace
// So we have to create a string for statement: "USE " + new_keyspace
// To avoid any possible CQL injections it's good to verify that the name is valid
let verified_ks_name = VerifiedKeyspaceName::new(keyspace_name, case_sensitive)?;

Expand Down Expand Up @@ -1493,13 +1493,13 @@ impl Session {
Ok(Some(tracing_info))
}

// This method allows to easily run a query using load balancing, retry policy etc.
// Requires some information about the query and a closure.
// The closure is used to do the query itself on a connection.
// This method allows to easily execute a statement using load balancing, retry policy etc.
// Requires some information about the statement and a closure.
// The closure is used to execute a statement itself on a connection.
// - query will use connection.query()
// - execute will use connection.execute()
// If this query closure fails with some errors retry policy is used to perform retries
// On success this query's result is returned
// If this closure fails with some errors retry policy is used to perform retries
// On success this statement's result is returned
// I tried to make this closures take a reference instead of an Arc but failed
// maybe once async closures get stabilized this can be fixed
async fn run_query<'a, QueryFut, ResT>(
Expand Down

0 comments on commit 96e258b

Please sign in to comment.