From a3a098938cea03a01f946b66df214da0c2ecfd32 Mon Sep 17 00:00:00 2001 From: muzarski Date: Mon, 6 May 2024 16:19:12 +0200 Subject: [PATCH 1/7] modules: file query.rs -> unprepared_statement.rs --- examples/execution_profile.rs | 2 +- examples/query_history.rs | 2 +- examples/select-paging.rs | 2 +- examples/tower.rs | 4 ++-- examples/tracing.rs | 3 ++- scylla/src/history.rs | 2 +- scylla/src/lib.rs | 2 +- scylla/src/statement/batch.rs | 2 +- scylla/src/statement/mod.rs | 2 +- scylla/src/statement/{query.rs => unprepared_statement.rs} | 0 scylla/src/transport/caching_session.rs | 4 ++-- scylla/src/transport/connection.rs | 4 ++-- scylla/src/transport/iterator.rs | 2 +- scylla/src/transport/large_batch_statements_test.rs | 2 +- scylla/src/transport/session.rs | 2 +- scylla/src/transport/session_test.rs | 2 +- scylla/src/transport/topology.rs | 2 +- scylla/tests/integration/consistency.rs | 2 +- scylla/tests/integration/execution_profiles.rs | 2 +- scylla/tests/integration/retries.rs | 2 +- scylla/tests/integration/silent_prepare_query.rs | 2 +- scylla/tests/integration/tablets.rs | 2 +- 22 files changed, 25 insertions(+), 24 deletions(-) rename scylla/src/statement/{query.rs => unprepared_statement.rs} (100%) diff --git a/examples/execution_profile.rs b/examples/execution_profile.rs index 317604d9b0..ba5b8b32b7 100644 --- a/examples/execution_profile.rs +++ b/examples/execution_profile.rs @@ -1,11 +1,11 @@ use anyhow::Result; use scylla::load_balancing; -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::ExecutionProfile; +use scylla::unprepared_statement::Query; use scylla::{SessionBuilder, SessionConfig}; use std::env; use std::sync::Arc; diff --git a/examples/query_history.rs b/examples/query_history.rs index e75a30514f..92c8de88c1 100644 --- a/examples/query_history.rs +++ b/examples/query_history.rs @@ -3,8 +3,8 @@ use anyhow::Result; use futures::StreamExt; use scylla::history::{HistoryCollector, StructuredHistory}; -use scylla::query::Query; use scylla::transport::session::Session; +use scylla::unprepared_statement::Query; use scylla::SessionBuilder; use std::env; use std::sync::Arc; diff --git a/examples/select-paging.rs b/examples/select-paging.rs index ea6cb256e0..468f088500 100644 --- a/examples/select-paging.rs +++ b/examples/select-paging.rs @@ -1,6 +1,6 @@ use anyhow::Result; use futures::stream::StreamExt; -use scylla::{query::Query, Session, SessionBuilder}; +use scylla::{unprepared_statement::Query, Session, SessionBuilder}; use std::env; #[tokio::main] diff --git a/examples/tower.rs b/examples/tower.rs index 1c3bb2112a..4abafa098d 100644 --- a/examples/tower.rs +++ b/examples/tower.rs @@ -11,7 +11,7 @@ struct SessionService { } // A trivial service implementation for sending parameterless simple string requests to Scylla. -impl Service for SessionService { +impl Service for SessionService { type Response = scylla::QueryResult; type Error = scylla::transport::errors::QueryError; type Future = Pin>>>; @@ -20,7 +20,7 @@ impl Service for SessionService { Poll::Ready(Ok(())) } - fn call(&mut self, req: scylla::query::Query) -> Self::Future { + fn call(&mut self, req: scylla::unprepared_statement::Query) -> Self::Future { let session = self.session.clone(); Box::pin(async move { session.query(req, &[]).await }) } diff --git a/examples/tracing.rs b/examples/tracing.rs index e4c9eb8047..6448b1b7f1 100644 --- a/examples/tracing.rs +++ b/examples/tracing.rs @@ -5,7 +5,8 @@ use anyhow::{anyhow, Result}; use futures::StreamExt; use scylla::batch::Batch; use scylla::statement::{ - prepared_statement::PreparedStatement, query::Query, Consistency, SerialConsistency, + prepared_statement::PreparedStatement, unprepared_statement::Query, Consistency, + SerialConsistency, }; use scylla::tracing::TracingInfo; use scylla::transport::iterator::RowIterator; diff --git a/scylla/src/history.rs b/scylla/src/history.rs index a43d46423a..adbd222800 100644 --- a/scylla/src/history.rs +++ b/scylla/src/history.rs @@ -456,7 +456,7 @@ mod tests { }; use crate::{ - query::Query, retry_policy::RetryDecision, test_utils::setup_tracing, + retry_policy::RetryDecision, test_utils::setup_tracing, unprepared_statement::Query, utils::test_utils::unique_keyspace_name, }; diff --git a/scylla/src/lib.rs b/scylla/src/lib.rs index 1b46559698..294fa1e6c0 100644 --- a/scylla/src/lib.rs +++ b/scylla/src/lib.rs @@ -147,7 +147,7 @@ pub use utils::test_utils; pub use statement::batch; pub use statement::prepared_statement; -pub use statement::query; +pub use statement::unprepared_statement; pub use frame::response::cql_to_rust; pub use frame::response::cql_to_rust::FromRow; diff --git a/scylla/src/statement/batch.rs b/scylla/src/statement/batch.rs index efe95031e2..a0c62d3ebe 100644 --- a/scylla/src/statement/batch.rs +++ b/scylla/src/statement/batch.rs @@ -3,7 +3,7 @@ use std::sync::Arc; use crate::history::HistoryListener; use crate::retry_policy::RetryPolicy; -use crate::statement::{prepared_statement::PreparedStatement, query::Query}; +use crate::statement::{prepared_statement::PreparedStatement, unprepared_statement::Query}; use crate::transport::execution_profile::ExecutionProfileHandle; use super::StatementConfig; diff --git a/scylla/src/statement/mod.rs b/scylla/src/statement/mod.rs index 642ea06ad3..4fcf98dd14 100644 --- a/scylla/src/statement/mod.rs +++ b/scylla/src/statement/mod.rs @@ -5,7 +5,7 @@ use crate::{history::HistoryListener, retry_policy::RetryPolicy}; pub mod batch; pub mod prepared_statement; -pub mod query; +pub mod unprepared_statement; pub use crate::frame::types::{Consistency, SerialConsistency}; diff --git a/scylla/src/statement/query.rs b/scylla/src/statement/unprepared_statement.rs similarity index 100% rename from scylla/src/statement/query.rs rename to scylla/src/statement/unprepared_statement.rs diff --git a/scylla/src/transport/caching_session.rs b/scylla/src/transport/caching_session.rs index 903eb4e346..888f3acedc 100644 --- a/scylla/src/transport/caching_session.rs +++ b/scylla/src/transport/caching_session.rs @@ -1,9 +1,9 @@ use crate::batch::{Batch, BatchStatement}; use crate::prepared_statement::PreparedStatement; -use crate::query::Query; use crate::transport::errors::QueryError; use crate::transport::iterator::RowIterator; use crate::transport::partitioner::PartitionerName; +use crate::unprepared_statement::Query; use crate::{QueryResult, Session}; use bytes::Bytes; use dashmap::DashMap; @@ -217,9 +217,9 @@ where #[cfg(test)] mod tests { - use crate::query::Query; use crate::test_utils::{create_new_session_builder, scylla_supports_tablets, setup_tracing}; use crate::transport::partitioner::PartitionerName; + use crate::unprepared_statement::Query; use crate::utils::test_utils::unique_keyspace_name; use crate::{ batch::{Batch, BatchStatement}, diff --git a/scylla/src/transport/connection.rs b/scylla/src/transport/connection.rs index d34824f97e..9bcea173a1 100644 --- a/scylla/src/transport/connection.rs +++ b/scylla/src/transport/connection.rs @@ -59,11 +59,11 @@ use crate::frame::{ server_event_type::EventType, FrameParams, SerializedRequest, }; -use crate::query::Query; use crate::routing::ShardInfo; use crate::statement::prepared_statement::PreparedStatement; use crate::statement::Consistency; use crate::transport::Compression; +use crate::unprepared_statement::Query; use crate::QueryResult; // Queries for schema agreement @@ -1929,11 +1929,11 @@ mod tests { use tokio::sync::mpsc; use super::ConnectionConfig; - use crate::query::Query; use crate::test_utils::setup_tracing; use crate::transport::connection::open_connection; use crate::transport::node::ResolvedContactPoint; use crate::transport::topology::UntranslatedEndpoint; + use crate::unprepared_statement::Query; use crate::utils::test_utils::unique_keyspace_name; use crate::{IntoTypedRows, SessionBuilder}; use futures::{StreamExt, TryStreamExt}; diff --git a/scylla/src/transport/iterator.rs b/scylla/src/transport/iterator.rs index 6c340b3453..127b2b3636 100644 --- a/scylla/src/transport/iterator.rs +++ b/scylla/src/transport/iterator.rs @@ -28,7 +28,7 @@ use crate::frame::response::{ }; use crate::history::{self, HistoryListener}; use crate::statement::Consistency; -use crate::statement::{prepared_statement::PreparedStatement, query::Query}; +use crate::statement::{prepared_statement::PreparedStatement, unprepared_statement::Query}; use crate::transport::cluster::ClusterData; use crate::transport::connection::{Connection, NonErrorQueryResponse, QueryResponse}; use crate::transport::load_balancing::{self, RoutingInfo}; diff --git a/scylla/src/transport/large_batch_statements_test.rs b/scylla/src/transport/large_batch_statements_test.rs index 1cd92295e8..ccb1b88555 100644 --- a/scylla/src/transport/large_batch_statements_test.rs +++ b/scylla/src/transport/large_batch_statements_test.rs @@ -3,8 +3,8 @@ use assert_matches::assert_matches; use scylla_cql::errors::{BadQuery, QueryError}; use crate::batch::BatchType; -use crate::query::Query; use crate::test_utils::setup_tracing; +use crate::unprepared_statement::Query; use crate::{ batch::Batch, test_utils::{create_new_session_builder, unique_keyspace_name}, diff --git a/scylla/src/transport/session.rs b/scylla/src/transport/session.rs index a81a115a4b..730eae6272 100644 --- a/scylla/src/transport/session.rs +++ b/scylla/src/transport/session.rs @@ -51,7 +51,6 @@ use crate::cql_to_rust::FromRow; use crate::frame::response::cql_to_rust::FromRowError; use crate::frame::response::result; use crate::prepared_statement::PreparedStatement; -use crate::query::Query; use crate::routing::{Shard, Token}; use crate::statement::Consistency; use crate::tracing::{TracingEvent, TracingInfo}; @@ -67,6 +66,7 @@ use crate::transport::query_result::QueryResult; use crate::transport::retry_policy::{QueryInfo, RetryDecision, RetrySession}; use crate::transport::speculative_execution; use crate::transport::Compression; +use crate::unprepared_statement::Query; use crate::{ batch::{Batch, BatchStatement}, statement::StatementConfig, diff --git a/scylla/src/transport/session_test.rs b/scylla/src/transport/session_test.rs index 8136477251..514afaedf7 100644 --- a/scylla/src/transport/session_test.rs +++ b/scylla/src/transport/session_test.rs @@ -2,7 +2,6 @@ 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; @@ -17,6 +16,7 @@ use crate::transport::topology::Strategy::NetworkTopologyStrategy; use crate::transport::topology::{ CollectionType, ColumnKind, CqlType, NativeType, UserDefinedType, }; +use crate::unprepared_statement::Query; use crate::utils::test_utils::{ create_new_session_builder, supports_feature, unique_keyspace_name, }; diff --git a/scylla/src/transport/topology.rs b/scylla/src/transport/topology.rs index b468050c0b..e5efbfd57f 100644 --- a/scylla/src/transport/topology.rs +++ b/scylla/src/transport/topology.rs @@ -1,6 +1,6 @@ use crate::frame::response::event::Event; use crate::routing::Token; -use crate::statement::query::Query; +use crate::statement::unprepared_statement::Query; use crate::transport::connection::{Connection, ConnectionConfig}; use crate::transport::connection_pool::{NodeConnectionPool, PoolConfig, PoolSize}; use crate::transport::errors::{DbError, QueryError}; diff --git a/scylla/tests/integration/consistency.rs b/scylla/tests/integration/consistency.rs index 2962795834..e96a1f6488 100644 --- a/scylla/tests/integration/consistency.rs +++ b/scylla/tests/integration/consistency.rs @@ -12,7 +12,7 @@ use scylla_cql::frame::response::result::TableSpec; use tokio::sync::mpsc::{self, UnboundedReceiver, UnboundedSender}; use scylla::statement::batch::BatchStatement; -use scylla::statement::query::Query; +use scylla::statement::unprepared_statement::Query; use scylla::{ batch::{Batch, BatchType}, statement::SerialConsistency, diff --git a/scylla/tests/integration/execution_profiles.rs b/scylla/tests/integration/execution_profiles.rs index 59f95dfa88..3afd570cd6 100644 --- a/scylla/tests/integration/execution_profiles.rs +++ b/scylla/tests/integration/execution_profiles.rs @@ -5,10 +5,10 @@ use crate::utils::{setup_tracing, test_with_3_node_cluster}; use assert_matches::assert_matches; use scylla::batch::BatchStatement; use scylla::batch::{Batch, BatchType}; -use scylla::query::Query; use scylla::routing::Shard; use scylla::statement::SerialConsistency; use scylla::transport::NodeRef; +use scylla::unprepared_statement::Query; use scylla::{ load_balancing::{LoadBalancingPolicy, RoutingInfo}, retry_policy::{RetryPolicy, RetrySession}, diff --git a/scylla/tests/integration/retries.rs b/scylla/tests/integration/retries.rs index bd928aca6e..2b5f27da01 100644 --- a/scylla/tests/integration/retries.rs +++ b/scylla/tests/integration/retries.rs @@ -4,7 +4,7 @@ use scylla::speculative_execution::SimpleSpeculativeExecutionPolicy; use scylla::transport::session::Session; use scylla::ExecutionProfile; use scylla::SessionBuilder; -use scylla::{query::Query, test_utils::unique_keyspace_name}; +use scylla::{test_utils::unique_keyspace_name, unprepared_statement::Query}; use std::sync::Arc; use std::time::Duration; use tracing::info; diff --git a/scylla/tests/integration/silent_prepare_query.rs b/scylla/tests/integration/silent_prepare_query.rs index 658aa96edf..3f823220fd 100644 --- a/scylla/tests/integration/silent_prepare_query.rs +++ b/scylla/tests/integration/silent_prepare_query.rs @@ -1,7 +1,7 @@ use crate::utils::{setup_tracing, test_with_3_node_cluster}; use scylla::transport::session::Session; use scylla::SessionBuilder; -use scylla::{query::Query, test_utils::unique_keyspace_name}; +use scylla::{test_utils::unique_keyspace_name, unprepared_statement::Query}; use scylla_proxy::{ Condition, ProxyError, Reaction, RequestOpcode, RequestReaction, RequestRule, ShardAwareness, WorkerError, diff --git a/scylla/tests/integration/tablets.rs b/scylla/tests/integration/tablets.rs index 00441181a5..97c5ca0b24 100644 --- a/scylla/tests/integration/tablets.rs +++ b/scylla/tests/integration/tablets.rs @@ -10,12 +10,12 @@ use scylla::load_balancing::FallbackPlan; use scylla::load_balancing::LoadBalancingPolicy; use scylla::load_balancing::RoutingInfo; use scylla::prepared_statement::PreparedStatement; -use scylla::query::Query; use scylla::serialize::row::SerializeRow; use scylla::test_utils::unique_keyspace_name; use scylla::transport::ClusterData; use scylla::transport::Node; use scylla::transport::NodeRef; +use scylla::unprepared_statement::Query; use scylla::ExecutionProfile; use scylla::QueryResult; use scylla::Session; From e16f0c7bd1610412aeec8de77fb8299bca14c0e3 Mon Sep 17 00:00:00 2001 From: muzarski Date: Mon, 6 May 2024 16:20:57 +0200 Subject: [PATCH 2/7] query: Query -> UnpreparedStatement Note: comments/docs are not yet adjusted. --- examples/execution_profile.rs | 4 +- examples/query_history.rs | 7 +-- examples/select-paging.rs | 10 +++-- examples/tower.rs | 4 +- examples/tracing.rs | 5 ++- scylla/src/history.rs | 10 ++--- scylla/src/statement/batch.rs | 12 +++--- scylla/src/statement/unprepared_statement.rs | 16 +++---- scylla/src/transport/caching_session.rs | 20 +++++---- scylla/src/transport/connection.rs | 43 +++++++++++-------- scylla/src/transport/iterator.rs | 8 ++-- .../transport/large_batch_statements_test.rs | 4 +- scylla/src/transport/session.rs | 23 ++++++---- scylla/src/transport/session_test.rs | 38 +++++++++------- scylla/src/transport/topology.rs | 18 ++++---- scylla/tests/integration/consistency.rs | 13 +++--- .../tests/integration/execution_profiles.rs | 4 +- scylla/tests/integration/retries.rs | 6 +-- .../tests/integration/silent_prepare_query.rs | 6 +-- scylla/tests/integration/tablets.rs | 8 ++-- 20 files changed, 148 insertions(+), 111 deletions(-) diff --git a/examples/execution_profile.rs b/examples/execution_profile.rs index ba5b8b32b7..192bbf349d 100644 --- a/examples/execution_profile.rs +++ b/examples/execution_profile.rs @@ -5,7 +5,7 @@ use scylla::speculative_execution::PercentileSpeculativeExecutionPolicy; use scylla::statement::{Consistency, SerialConsistency}; use scylla::transport::session::Session; use scylla::transport::ExecutionProfile; -use scylla::unprepared_statement::Query; +use scylla::unprepared_statement::UnpreparedStatement; use scylla::{SessionBuilder, SessionConfig}; use std::env; use std::sync::Arc; @@ -68,7 +68,7 @@ async fn main() -> Result<()> { ) .await?; - let mut query_insert: Query = + let mut query_insert: UnpreparedStatement = "INSERT INTO examples_ks.execution_profile (a, b, c) VALUES (?, ?, ?)".into(); // As `query_insert` is set another handle than session1, the execution profile pointed by query's handle diff --git a/examples/query_history.rs b/examples/query_history.rs index 92c8de88c1..0dc6ecabf1 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::transport::session::Session; -use scylla::unprepared_statement::Query; +use scylla::unprepared_statement::UnpreparedStatement; use scylla::SessionBuilder; use std::env; use std::sync::Arc; @@ -27,7 +27,7 @@ async fn main() -> Result<()> { .await?; // Create a query for which we would like to trace the history of its execution - let mut query: Query = Query::new("SELECT * FROM ks.t"); + let mut query: UnpreparedStatement = UnpreparedStatement::new("SELECT * FROM ks.t"); let history_listener = Arc::new(HistoryCollector::new()); query.set_history_listener(history_listener.clone()); @@ -54,7 +54,8 @@ async fn main() -> Result<()> { .await?; } - let mut iter_query: Query = Query::new("SELECT * FROM examples_ks.query_history"); + let mut iter_query: UnpreparedStatement = + UnpreparedStatement::new("SELECT * FROM examples_ks.query_history"); iter_query.set_page_size(8); let iter_history_listener = Arc::new(HistoryCollector::new()); iter_query.set_history_listener(iter_history_listener.clone()); diff --git a/examples/select-paging.rs b/examples/select-paging.rs index 468f088500..ce55a722da 100644 --- a/examples/select-paging.rs +++ b/examples/select-paging.rs @@ -1,6 +1,6 @@ use anyhow::Result; use futures::stream::StreamExt; -use scylla::{unprepared_statement::Query, Session, SessionBuilder}; +use scylla::{unprepared_statement::UnpreparedStatement, Session, SessionBuilder}; use std::env; #[tokio::main] @@ -40,7 +40,8 @@ async fn main() -> Result<()> { println!("a, b, c: {}, {}, {}", a, b, c); } - let paged_query = Query::new("SELECT a, b, c FROM examples_ks.select_paging").with_page_size(6); + let paged_query = + UnpreparedStatement::new("SELECT a, b, c FROM examples_ks.select_paging").with_page_size(6); let res1 = session.query(paged_query.clone(), &[]).await?; println!( "Paging state: {:#?} ({} rows)", @@ -65,7 +66,10 @@ async fn main() -> Result<()> { ); let paged_prepared = session - .prepare(Query::new("SELECT a, b, c FROM examples_ks.select_paging").with_page_size(7)) + .prepare( + UnpreparedStatement::new("SELECT a, b, c FROM examples_ks.select_paging") + .with_page_size(7), + ) .await?; let res4 = session.execute(&paged_prepared, &[]).await?; println!( diff --git a/examples/tower.rs b/examples/tower.rs index 4abafa098d..77bcbbc203 100644 --- a/examples/tower.rs +++ b/examples/tower.rs @@ -11,7 +11,7 @@ struct SessionService { } // A trivial service implementation for sending parameterless simple string requests to Scylla. -impl Service for SessionService { +impl Service for SessionService { type Response = scylla::QueryResult; type Error = scylla::transport::errors::QueryError; type Future = Pin>>>; @@ -20,7 +20,7 @@ impl Service for SessionService { Poll::Ready(Ok(())) } - fn call(&mut self, req: scylla::unprepared_statement::Query) -> Self::Future { + fn call(&mut self, req: scylla::unprepared_statement::UnpreparedStatement) -> Self::Future { let session = self.session.clone(); Box::pin(async move { session.query(req, &[]).await }) } diff --git a/examples/tracing.rs b/examples/tracing.rs index 6448b1b7f1..f21417898f 100644 --- a/examples/tracing.rs +++ b/examples/tracing.rs @@ -5,7 +5,7 @@ use anyhow::{anyhow, Result}; use futures::StreamExt; use scylla::batch::Batch; use scylla::statement::{ - prepared_statement::PreparedStatement, unprepared_statement::Query, Consistency, + prepared_statement::PreparedStatement, unprepared_statement::UnpreparedStatement, Consistency, SerialConsistency, }; use scylla::tracing::TracingInfo; @@ -38,7 +38,8 @@ async fn main() -> Result<()> { // QUERY // Create a simple query and enable tracing for it - let mut query: Query = Query::new("SELECT val from examples_ks.tracing"); + let mut query: UnpreparedStatement = + UnpreparedStatement::new("SELECT val from examples_ks.tracing"); query.set_tracing(true); query.set_serial_consistency(Some(SerialConsistency::LocalSerial)); diff --git a/scylla/src/history.rs b/scylla/src/history.rs index adbd222800..5425739c68 100644 --- a/scylla/src/history.rs +++ b/scylla/src/history.rs @@ -456,8 +456,8 @@ mod tests { }; use crate::{ - retry_policy::RetryDecision, test_utils::setup_tracing, unprepared_statement::Query, - utils::test_utils::unique_keyspace_name, + retry_policy::RetryDecision, test_utils::setup_tracing, + unprepared_statement::UnpreparedStatement, utils::test_utils::unique_keyspace_name, }; use super::{ @@ -919,7 +919,7 @@ mod tests { setup_tracing(); let session = create_new_session_builder().build().await.unwrap(); - let mut query = Query::new("SELECT * FROM system.local"); + let mut query = UnpreparedStatement::new("SELECT * FROM system.local"); let history_collector = Arc::new(HistoryCollector::new()); query.set_history_listener(history_collector.clone()); @@ -986,7 +986,7 @@ mod tests { setup_tracing(); let session = create_new_session_builder().build().await.unwrap(); - let mut query = Query::new("This isnt even CQL"); + let mut query = UnpreparedStatement::new("This isnt even CQL"); let history_collector = Arc::new(HistoryCollector::new()); query.set_history_listener(history_collector.clone()); @@ -1040,7 +1040,7 @@ mod tests { .unwrap(); } - let mut iter_query: Query = Query::new("SELECT * FROM t"); + let mut iter_query: UnpreparedStatement = UnpreparedStatement::new("SELECT * FROM t"); iter_query.set_page_size(8); let history_collector = Arc::new(HistoryCollector::new()); iter_query.set_history_listener(history_collector.clone()); diff --git a/scylla/src/statement/batch.rs b/scylla/src/statement/batch.rs index a0c62d3ebe..567cf74bc5 100644 --- a/scylla/src/statement/batch.rs +++ b/scylla/src/statement/batch.rs @@ -3,7 +3,9 @@ use std::sync::Arc; use crate::history::HistoryListener; use crate::retry_policy::RetryPolicy; -use crate::statement::{prepared_statement::PreparedStatement, unprepared_statement::Query}; +use crate::statement::{ + prepared_statement::PreparedStatement, unprepared_statement::UnpreparedStatement, +}; use crate::transport::execution_profile::ExecutionProfileHandle; use super::StatementConfig; @@ -157,18 +159,18 @@ impl Default for Batch { /// This enum represents a CQL statement, that can be part of batch. #[derive(Clone)] pub enum BatchStatement { - Query(Query), + Query(UnpreparedStatement), PreparedStatement(PreparedStatement), } impl From<&str> for BatchStatement { fn from(s: &str) -> Self { - BatchStatement::Query(Query::from(s)) + BatchStatement::Query(UnpreparedStatement::from(s)) } } -impl From for BatchStatement { - fn from(q: Query) -> Self { +impl From for BatchStatement { + fn from(q: UnpreparedStatement) -> Self { BatchStatement::Query(q) } } diff --git a/scylla/src/statement/unprepared_statement.rs b/scylla/src/statement/unprepared_statement.rs index 913aebd1d7..06d98c71c1 100644 --- a/scylla/src/statement/unprepared_statement.rs +++ b/scylla/src/statement/unprepared_statement.rs @@ -10,14 +10,14 @@ use std::time::Duration; /// /// This represents a CQL query that can be executed on a server. #[derive(Clone)] -pub struct Query { +pub struct UnpreparedStatement { pub(crate) config: StatementConfig, pub contents: String, page_size: Option, } -impl Query { +impl UnpreparedStatement { /// Creates a new `Query` from a CQL query string. pub fn new(query_text: impl Into) -> Self { Self { @@ -158,14 +158,14 @@ impl Query { } } -impl From for Query { - fn from(s: String) -> Query { - Query::new(s) +impl From for UnpreparedStatement { + fn from(s: String) -> UnpreparedStatement { + UnpreparedStatement::new(s) } } -impl<'a> From<&'a str> for Query { - fn from(s: &'a str) -> Query { - Query::new(s.to_owned()) +impl<'a> From<&'a str> for UnpreparedStatement { + fn from(s: &'a str) -> UnpreparedStatement { + UnpreparedStatement::new(s.to_owned()) } } diff --git a/scylla/src/transport/caching_session.rs b/scylla/src/transport/caching_session.rs index 888f3acedc..b4198f3eea 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::transport::errors::QueryError; use crate::transport::iterator::RowIterator; use crate::transport::partitioner::PartitionerName; -use crate::unprepared_statement::Query; +use crate::unprepared_statement::UnpreparedStatement; use crate::{QueryResult, Session}; use bytes::Bytes; use dashmap::DashMap; @@ -71,7 +71,7 @@ where /// Does the same thing as [`Session::execute`] but uses the prepared statement cache pub async fn execute( &self, - query: impl Into, + query: impl Into, values: impl SerializeRow, ) -> Result { let query = query.into(); @@ -82,7 +82,7 @@ where /// Does the same thing as [`Session::execute_iter`] but uses the prepared statement cache pub async fn execute_iter( &self, - query: impl Into, + query: impl Into, values: impl SerializeRow, ) -> Result { let query = query.into(); @@ -93,7 +93,7 @@ where /// Does the same thing as [`Session::execute_paged`] but uses the prepared statement cache pub async fn execute_paged( &self, - query: impl Into, + query: impl Into, values: impl SerializeRow, paging_state: Option, ) -> Result { @@ -151,7 +151,7 @@ where /// Adds a prepared statement to the cache pub async fn add_prepared_statement( &self, - query: impl Into<&Query>, + query: impl Into<&UnpreparedStatement>, ) -> Result { self.add_prepared_statement_owned(query.into().clone()) .await @@ -159,7 +159,7 @@ where async fn add_prepared_statement_owned( &self, - query: impl Into, + query: impl Into, ) -> Result { let query = query.into(); @@ -219,7 +219,7 @@ where mod tests { use crate::test_utils::{create_new_session_builder, scylla_supports_tablets, setup_tracing}; use crate::transport::partitioner::PartitionerName; - use crate::unprepared_statement::Query; + use crate::unprepared_statement::UnpreparedStatement; use crate::utils::test_utils::unique_keyspace_name; use crate::{ batch::{Batch, BatchStatement}, @@ -567,7 +567,7 @@ mod tests { .await .unwrap(); - let q = Query::new("INSERT INTO tbl (a, b) VALUES (?, ?)"); + let q = UnpreparedStatement::new("INSERT INTO tbl (a, b) VALUES (?, ?)"); // Insert one row with timestamp 1000 let mut q1 = q.clone(); @@ -634,7 +634,9 @@ mod tests { // partitioner. It should happen when the query is prepared // and after it is fetched from the cache. let verify_partitioner = || async { - let query = Query::new("SELECT * FROM tbl_scylla_cdc_log WHERE \"cdc$stream_id\" = ?"); + let query = UnpreparedStatement::new( + "SELECT * FROM tbl_scylla_cdc_log WHERE \"cdc$stream_id\" = ?", + ); let prepared = session.add_prepared_statement(&query).await.unwrap(); assert_eq!(prepared.get_partitioner_name(), &PartitionerName::CDC); }; diff --git a/scylla/src/transport/connection.rs b/scylla/src/transport/connection.rs index 9bcea173a1..b54c3f2626 100644 --- a/scylla/src/transport/connection.rs +++ b/scylla/src/transport/connection.rs @@ -63,7 +63,7 @@ use crate::routing::ShardInfo; use crate::statement::prepared_statement::PreparedStatement; use crate::statement::Consistency; use crate::transport::Compression; -use crate::unprepared_statement::Query; +use crate::unprepared_statement::UnpreparedStatement; use crate::QueryResult; // Queries for schema agreement @@ -539,7 +539,10 @@ impl Connection { .response) } - pub(crate) async fn prepare(&self, query: &Query) -> Result { + pub(crate) async fn prepare( + &self, + query: &UnpreparedStatement, + ) -> Result { let query_response = self .send_request( &request::Prepare { @@ -579,10 +582,10 @@ impl Connection { pub(crate) async fn reprepare( &self, - query: impl Into, + query: impl Into, previous_prepared: &PreparedStatement, ) -> Result<(), QueryError> { - let reprepare_query: Query = query.into(); + let reprepare_query: UnpreparedStatement = query.into(); let reprepared = self.prepare(&reprepare_query).await?; // Reprepared statement should keep its id - it's the md5 sum // of statement contents @@ -605,9 +608,9 @@ impl Connection { pub(crate) async fn query_single_page( &self, - query: impl Into, + query: impl Into, ) -> Result { - let query: Query = query.into(); + let query: UnpreparedStatement = query.into(); // This method is used only for driver internal queries, so no need to consult execution profile here. let consistency = query @@ -621,11 +624,11 @@ impl Connection { pub(crate) async fn query_single_page_with_consistency( &self, - query: impl Into, + query: impl Into, consistency: Consistency, serial_consistency: Option, ) -> Result { - let query: Query = query.into(); + let query: UnpreparedStatement = query.into(); self.query_with_consistency(&query, consistency, serial_consistency, None) .await? .into_query_result() @@ -633,7 +636,7 @@ impl Connection { pub(crate) async fn query( &self, - query: &Query, + query: &UnpreparedStatement, paging_state: Option, ) -> Result { // This method is used only for driver internal queries, so no need to consult execution profile here. @@ -650,7 +653,7 @@ impl Connection { pub(crate) async fn query_with_consistency( &self, - query: &Query, + query: &UnpreparedStatement, consistency: Consistency, serial_consistency: Option, paging_state: Option, @@ -772,7 +775,7 @@ impl Connection { /// the asynchronous iterator interface. pub(crate) async fn query_iter( self: Arc, - query: Query, + query: UnpreparedStatement, ) -> Result { let consistency = query .config @@ -910,7 +913,9 @@ impl Connection { let mut prepared_queries = HashMap::<&str, PreparedStatement>::new(); for query in &to_prepare { - let prepared = self.prepare(&Query::new(query.to_string())).await?; + let prepared = self + .prepare(&UnpreparedStatement::new(query.to_string())) + .await?; prepared_queries.insert(query, prepared); } @@ -938,7 +943,7 @@ impl Connection { ) -> Result<(), QueryError> { // Trying to pass keyspace_name as bound value doesn't work // We have to send "USE " + keyspace_name - let query: Query = match keyspace_name.is_case_sensitive { + let query: UnpreparedStatement = match keyspace_name.is_case_sensitive { true => format!("USE \"{}\"", keyspace_name.as_str()).into(), false => format!("USE {}", keyspace_name.as_str()).into(), }; @@ -1933,7 +1938,7 @@ mod tests { use crate::transport::connection::open_connection; use crate::transport::node::ResolvedContactPoint; use crate::transport::topology::UntranslatedEndpoint; - use crate::unprepared_statement::Query; + use crate::unprepared_statement::UnpreparedStatement; use crate::utils::test_utils::unique_keyspace_name; use crate::{IntoTypedRows, SessionBuilder}; use futures::{StreamExt, TryStreamExt}; @@ -2010,7 +2015,8 @@ mod tests { .unwrap(); // 1. SELECT from an empty table returns query result where rows are Some(Vec::new()) - let select_query = Query::new("SELECT p FROM connection_query_iter_tab").with_page_size(7); + let select_query = + UnpreparedStatement::new("SELECT p FROM connection_query_iter_tab").with_page_size(7); let empty_res = connection .clone() .query_iter(select_query.clone()) @@ -2025,7 +2031,8 @@ mod tests { let values: Vec = (0..100).collect(); let mut insert_futures = Vec::new(); let insert_query = - Query::new("INSERT INTO connection_query_iter_tab (p) VALUES (?)").with_page_size(7); + UnpreparedStatement::new("INSERT INTO connection_query_iter_tab (p) VALUES (?)") + .with_page_size(7); let prepared = connection.prepare(&insert_query).await.unwrap(); for v in &values { let prepared_clone = prepared.clone(); @@ -2050,7 +2057,7 @@ mod tests { // 3. INSERT query_iter should work and not return any rows. let insert_res1 = connection - .query_iter(Query::new( + .query_iter(UnpreparedStatement::new( "INSERT INTO connection_query_iter_tab (p) VALUES (0)", )) .await @@ -2125,7 +2132,7 @@ mod tests { let conn = connection.clone(); futs.push(tokio::task::spawn(async move { let futs = (base..base + batch_size).map(|j| { - let q = Query::new("INSERT INTO t (p, v) VALUES (?, ?)"); + let q = UnpreparedStatement::new("INSERT INTO t (p, v) VALUES (?, ?)"); let conn = conn.clone(); async move { let prepared = conn.prepare(&q).await.unwrap(); diff --git a/scylla/src/transport/iterator.rs b/scylla/src/transport/iterator.rs index 127b2b3636..3c31a486ef 100644 --- a/scylla/src/transport/iterator.rs +++ b/scylla/src/transport/iterator.rs @@ -28,7 +28,9 @@ use crate::frame::response::{ }; use crate::history::{self, HistoryListener}; use crate::statement::Consistency; -use crate::statement::{prepared_statement::PreparedStatement, unprepared_statement::Query}; +use crate::statement::{ + prepared_statement::PreparedStatement, unprepared_statement::UnpreparedStatement, +}; use crate::transport::cluster::ClusterData; use crate::transport::connection::{Connection, NonErrorQueryResponse, QueryResponse}; use crate::transport::load_balancing::{self, RoutingInfo}; @@ -124,7 +126,7 @@ impl RowIterator { } pub(crate) async fn new_for_query( - mut query: Query, + mut query: UnpreparedStatement, execution_profile: Arc, cluster_data: Arc, metrics: Arc, @@ -323,7 +325,7 @@ impl RowIterator { } pub(crate) async fn new_for_connection_query_iter( - mut query: Query, + mut query: UnpreparedStatement, connection: Arc, consistency: Consistency, serial_consistency: Option, diff --git a/scylla/src/transport/large_batch_statements_test.rs b/scylla/src/transport/large_batch_statements_test.rs index ccb1b88555..bf4aa8cc09 100644 --- a/scylla/src/transport/large_batch_statements_test.rs +++ b/scylla/src/transport/large_batch_statements_test.rs @@ -4,7 +4,7 @@ use scylla_cql::errors::{BadQuery, QueryError}; use crate::batch::BatchType; use crate::test_utils::setup_tracing; -use crate::unprepared_statement::Query; +use crate::unprepared_statement::UnpreparedStatement; use crate::{ batch::Batch, test_utils::{create_new_session_builder, unique_keyspace_name}, @@ -56,7 +56,7 @@ async fn write_batch(session: &Session, n: usize, ks: &String) -> Result, + query: impl Into, values: impl SerializeRow, ) -> Result { self.query_paged(query, values, None).await @@ -632,11 +632,11 @@ impl Session { /// * `paging_state` - previously received paging state or None pub async fn query_paged( &self, - query: impl Into, + query: impl Into, values: impl SerializeRow, paging_state: Option, ) -> Result { - let query: Query = query.into(); + let query: UnpreparedStatement = query.into(); let execution_profile = query .get_execution_profile_handle() @@ -798,10 +798,10 @@ impl Session { /// ``` pub async fn query_iter( &self, - query: impl Into, + query: impl Into, values: impl SerializeRow, ) -> Result { - let query: Query = query.into(); + let query: UnpreparedStatement = query.into(); let execution_profile = query .get_execution_profile_handle() @@ -869,7 +869,10 @@ impl Session { /// # Ok(()) /// # } /// ``` - pub async fn prepare(&self, query: impl Into) -> Result { + pub async fn prepare( + &self, + query: impl Into, + ) -> Result { let query = query.into(); let query_ref = &query; @@ -1447,12 +1450,14 @@ impl Session { consistency: Option, ) -> Result, QueryError> { // Query system_traces.sessions for TracingInfo - let mut traces_session_query = Query::new(crate::tracing::TRACES_SESSION_QUERY_STR); + let mut traces_session_query = + UnpreparedStatement::new(crate::tracing::TRACES_SESSION_QUERY_STR); traces_session_query.config.consistency = consistency; traces_session_query.set_page_size(1024); // Query system_traces.events for TracingEvents - let mut traces_events_query = Query::new(crate::tracing::TRACES_EVENTS_QUERY_STR); + let mut traces_events_query = + UnpreparedStatement::new(crate::tracing::TRACES_EVENTS_QUERY_STR); traces_events_query.config.consistency = consistency; traces_events_query.set_page_size(1024); diff --git a/scylla/src/transport/session_test.rs b/scylla/src/transport/session_test.rs index 514afaedf7..d32e46b167 100644 --- a/scylla/src/transport/session_test.rs +++ b/scylla/src/transport/session_test.rs @@ -16,7 +16,7 @@ use crate::transport::topology::Strategy::NetworkTopologyStrategy; use crate::transport::topology::{ CollectionType, ColumnKind, CqlType, NativeType, UserDefinedType, }; -use crate::unprepared_statement::Query; +use crate::unprepared_statement::UnpreparedStatement; use crate::utils::test_utils::{ create_new_session_builder, supports_feature, unique_keyspace_name, }; @@ -144,7 +144,7 @@ async fn test_unprepared_statement() { assert_eq!(spec.table_spec.ks_name(), ks); } let mut results_from_manual_paging: Vec = vec![]; - let query = Query::new(format!("SELECT a, b, c FROM {}.t", ks)).with_page_size(1); + let query = UnpreparedStatement::new(format!("SELECT a, b, c FROM {}.t", ks)).with_page_size(1); let mut paging_state: Option = None; let mut watchdog = 0; loop { @@ -279,7 +279,8 @@ async fn test_prepared_statement() { assert_eq!((a, b, c), (17, 16, &String::from("I'm prepared!!!"))); let mut results_from_manual_paging: Vec = vec![]; - let query = Query::new(format!("SELECT a, b, c FROM {}.t2", ks)).with_page_size(1); + let query = + UnpreparedStatement::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: Option = None; let mut watchdog = 0; @@ -883,13 +884,15 @@ 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: UnpreparedStatement = + UnpreparedStatement::new(format!("SELECT * FROM {}.tab", ks)); let untraced_query_result: QueryResult = session.query(untraced_query, &[]).await.unwrap(); 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)); + let mut traced_query: UnpreparedStatement = + UnpreparedStatement::new(format!("SELECT * FROM {}.tab", ks)); traced_query.config.tracing = true; let traced_query_result: QueryResult = session.query(traced_query, &[]).await.unwrap(); @@ -936,7 +939,7 @@ async fn test_tracing_prepare(session: &Session, ks: String) { assert!(untraced_prepared.prepare_tracing_ids.is_empty()); // Preparing a statement with tracing enabled has tracing uuids in result - let mut to_prepare_traced = Query::new(format!("SELECT * FROM {}.tab", ks)); + let mut to_prepare_traced = UnpreparedStatement::new(format!("SELECT * FROM {}.tab", ks)); to_prepare_traced.config.tracing = true; let traced_prepared = session.prepare(to_prepare_traced).await.unwrap(); @@ -950,7 +953,8 @@ async fn test_tracing_prepare(session: &Session, 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)); + let mut traced_query: UnpreparedStatement = + UnpreparedStatement::new(format!("SELECT * FROM {}.tab", ks)); traced_query.config.tracing = true; let traced_query_result: QueryResult = session.query(traced_query, &[]).await.unwrap(); @@ -964,7 +968,8 @@ async fn test_get_tracing_info(session: &Session, 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)); + let untraced_query: UnpreparedStatement = + UnpreparedStatement::new(format!("SELECT * FROM {}.tab", ks)); let mut untraced_row_iter = session.query_iter(untraced_query, &[]).await.unwrap(); while let Some(_row) = untraced_row_iter.next().await { @@ -978,7 +983,8 @@ async fn test_tracing_query_iter(session: &Session, ks: String) { assert!(untraced_typed_row_iter.get_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)); + let mut traced_query: UnpreparedStatement = + UnpreparedStatement::new(format!("SELECT * FROM {}.tab", ks)); traced_query.config.tracing = true; let mut traced_row_iter = session.query_iter(traced_query, &[]).await.unwrap(); @@ -1058,7 +1064,8 @@ async fn test_tracing_batch(session: &Session, ks: String) { } 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 = ?"); + let mut traces_query = + UnpreparedStatement::new("SELECT * FROM system_traces.sessions WHERE session_id = ?"); traces_query.config.consistency = Some(Consistency::One); // Tracing info might not be immediately available @@ -1127,7 +1134,7 @@ async fn test_timestamp() { // test regular query timestamps - let mut regular_query = Query::new(query_str.to_string()); + let mut regular_query = UnpreparedStatement::new(query_str.to_string()); regular_query.set_timestamp(Some(420)); session @@ -1227,7 +1234,8 @@ async fn test_request_timeout() { { let session = create_new_session_builder().build().await.unwrap(); - let mut query: Query = Query::new("SELECT * FROM system_schema.tables"); + let mut query: UnpreparedStatement = + UnpreparedStatement::new("SELECT * FROM system_schema.tables"); query.set_request_timeout(Some(Duration::from_millis(1))); match session.query(query, &[]).await { Ok(_) => panic!("the query should have failed due to a client-side timeout"), @@ -1252,7 +1260,7 @@ async fn test_request_timeout() { .await .unwrap(); - let mut query = Query::new("SELECT * FROM system_schema.tables"); + let mut query = UnpreparedStatement::new("SELECT * FROM system_schema.tables"); match timeouting_session.query(query.clone(), &[]).await { Ok(_) => panic!("the query should have failed due to a client-side timeout"), @@ -1286,7 +1294,7 @@ async fn test_prepared_config() { setup_tracing(); let session = create_new_session_builder().build().await.unwrap(); - let mut query = Query::new("SELECT * FROM system_schema.tables"); + let mut query = UnpreparedStatement::new("SELECT * FROM system_schema.tables"); query.set_is_idempotent(true); query.set_page_size(42); @@ -2666,7 +2674,7 @@ async fn test_iter_methods_with_modification_statements() { .await .unwrap(); - let mut query = Query::from(format!( + let mut query = UnpreparedStatement::from(format!( "INSERT INTO {}.t (a, b, c) VALUES (1, 2, 'abc')", ks )); diff --git a/scylla/src/transport/topology.rs b/scylla/src/transport/topology.rs index e5efbfd57f..ec60b0c5a3 100644 --- a/scylla/src/transport/topology.rs +++ b/scylla/src/transport/topology.rs @@ -1,6 +1,6 @@ use crate::frame::response::event::Event; use crate::routing::Token; -use crate::statement::unprepared_statement::Query; +use crate::statement::unprepared_statement::UnpreparedStatement; use crate::transport::connection::{Connection, ConnectionConfig}; use crate::transport::connection_pool::{NodeConnectionPool, PoolConfig, PoolSize}; use crate::transport::errors::{DbError, QueryError}; @@ -792,8 +792,9 @@ impl NodeInfoSource { } async fn query_peers(conn: &Arc, connect_port: u16) -> Result, QueryError> { - let mut peers_query = - Query::new("select host_id, rpc_address, data_center, rack, tokens from system.peers"); + let mut peers_query = UnpreparedStatement::new( + "select host_id, rpc_address, data_center, rack, tokens from system.peers", + ); peers_query.set_page_size(1024); let peers_query_stream = conn .clone() @@ -802,8 +803,9 @@ async fn query_peers(conn: &Arc, connect_port: u16) -> Result( let fut = async move { if keyspaces_to_fetch.is_empty() { - let mut query = Query::new(query_str); + let mut query = UnpreparedStatement::new(query_str); query.set_page_size(1024); conn.query_iter(query).await @@ -915,7 +917,7 @@ fn query_filter_keyspace_name<'a>( let keyspaces = &[keyspaces_to_fetch] as &[&[String]]; let query_str = format!("{query_str} where keyspace_name in ?"); - let mut query = Query::new(query_str); + let mut query = UnpreparedStatement::new(query_str); query.set_page_size(1024); let prepared = conn.prepare(&query).await?; @@ -1649,7 +1651,7 @@ fn freeze_type(type_: PreCqlType) -> PreCqlType { async fn query_table_partitioners( conn: &Arc, ) -> Result>, QueryError> { - let mut partitioner_query = Query::new( + let mut partitioner_query = UnpreparedStatement::new( "select keyspace_name, table_name, partitioner from system_schema.scylla_tables", ); partitioner_query.set_page_size(1024); diff --git a/scylla/tests/integration/consistency.rs b/scylla/tests/integration/consistency.rs index e96a1f6488..e274d2e8a1 100644 --- a/scylla/tests/integration/consistency.rs +++ b/scylla/tests/integration/consistency.rs @@ -12,7 +12,7 @@ use scylla_cql::frame::response::result::TableSpec; use tokio::sync::mpsc::{self, UnboundedReceiver, UnboundedSender}; use scylla::statement::batch::BatchStatement; -use scylla::statement::unprepared_statement::Query; +use scylla::statement::unprepared_statement::UnpreparedStatement; use scylla::{ batch::{Batch, BatchType}, statement::SerialConsistency, @@ -69,7 +69,7 @@ async fn create_schema(session: &Session, ks: &str) { // The following functions perform a request with consistencies set directly on a statement. async fn query_consistency_set_directly( session: &Session, - query: &Query, + query: &UnpreparedStatement, c: Consistency, sc: Option, ) { @@ -108,7 +108,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, - query: &Query, + query: &UnpreparedStatement, profile: ExecutionProfileHandle, ) { let mut query = query.clone(); @@ -166,11 +166,11 @@ async fn check_for_all_consistencies_and_setting_options< rx = after_session_init(rx).await; // We will be using these requests: - let query = Query::from(QUERY_STR); + let query = UnpreparedStatement::from(QUERY_STR); let prepared = session.prepare(QUERY_STR).await.unwrap(); let batch = Batch::new_with_statements( BatchType::Logged, - vec![BatchStatement::Query(Query::from(QUERY_STR))], + vec![BatchStatement::Query(UnpreparedStatement::from(QUERY_STR))], ); for (consistency, serial_consistency) in pairs_of_all_consistencies() { @@ -477,7 +477,8 @@ async fn consistency_allows_for_paxos_selects() { .await .unwrap(); - let mut query = Query::from("SELECT host_id FROM system.peers WHERE peer = '127.0.0.1'"); + let mut query = + UnpreparedStatement::from("SELECT host_id FROM system.peers WHERE peer = '127.0.0.1'"); query.set_consistency(Consistency::Serial); session.query(query, ()).await.unwrap(); } diff --git a/scylla/tests/integration/execution_profiles.rs b/scylla/tests/integration/execution_profiles.rs index 3afd570cd6..cc3369cab8 100644 --- a/scylla/tests/integration/execution_profiles.rs +++ b/scylla/tests/integration/execution_profiles.rs @@ -8,7 +8,7 @@ use scylla::batch::{Batch, BatchType}; use scylla::routing::Shard; use scylla::statement::SerialConsistency; use scylla::transport::NodeRef; -use scylla::unprepared_statement::Query; +use scylla::unprepared_statement::UnpreparedStatement; use scylla::{ load_balancing::{LoadBalancingPolicy, RoutingInfo}, retry_policy::{RetryPolicy, RetrySession}, @@ -181,7 +181,7 @@ async fn test_execution_profiles() { .await .unwrap(); - let mut query = Query::from(format!("INSERT INTO {}.t (a, b, c) VALUES (1, 2, 'abc')", ks)); + let mut query = UnpreparedStatement::from(format!("INSERT INTO {}.t (a, b, c) VALUES (1, 2, 'abc')", ks)); let mut prepared = session.prepare(format!("INSERT INTO {}.t (a, b, c) VALUES (1, 2, 'abc')", ks)).await.unwrap(); let mut batch = Batch::new_with_statements(BatchType::Unlogged, vec![BatchStatement::Query(query.clone())]); diff --git a/scylla/tests/integration/retries.rs b/scylla/tests/integration/retries.rs index 2b5f27da01..a637a6f325 100644 --- a/scylla/tests/integration/retries.rs +++ b/scylla/tests/integration/retries.rs @@ -4,7 +4,7 @@ use scylla::speculative_execution::SimpleSpeculativeExecutionPolicy; use scylla::transport::session::Session; use scylla::ExecutionProfile; use scylla::SessionBuilder; -use scylla::{test_utils::unique_keyspace_name, unprepared_statement::Query}; +use scylla::{test_utils::unique_keyspace_name, unprepared_statement::UnpreparedStatement}; use std::sync::Arc; use std::time::Duration; use tracing::info; @@ -43,7 +43,7 @@ async fn speculative_execution_is_fired() { .await .unwrap(); - let mut q = Query::from("INSERT INTO t (a) VALUES (?)"); + let mut q = UnpreparedStatement::from("INSERT INTO t (a) VALUES (?)"); q.set_is_idempotent(true); // this is to allow speculative execution to fire let drop_frame_rule = RequestRule( @@ -119,7 +119,7 @@ async fn retries_occur() { .await .unwrap(); - let mut q = Query::from("INSERT INTO t (a) VALUES (?)"); + let mut q = UnpreparedStatement::from("INSERT INTO t (a) VALUES (?)"); q.set_is_idempotent(true); // this is to allow retry to fire let forge_error_rule = RequestRule( diff --git a/scylla/tests/integration/silent_prepare_query.rs b/scylla/tests/integration/silent_prepare_query.rs index 3f823220fd..f6b2e47b44 100644 --- a/scylla/tests/integration/silent_prepare_query.rs +++ b/scylla/tests/integration/silent_prepare_query.rs @@ -1,7 +1,7 @@ use crate::utils::{setup_tracing, test_with_3_node_cluster}; use scylla::transport::session::Session; use scylla::SessionBuilder; -use scylla::{test_utils::unique_keyspace_name, unprepared_statement::Query}; +use scylla::{test_utils::unique_keyspace_name, unprepared_statement::UnpreparedStatement}; use scylla_proxy::{ Condition, ProxyError, Reaction, RequestOpcode, RequestReaction, RequestRule, ShardAwareness, WorkerError, @@ -34,7 +34,7 @@ async fn test_prepare_query_with_values() { .await .unwrap(); - let q = Query::from("INSERT INTO t (a) VALUES (?)"); + let q = UnpreparedStatement::from("INSERT INTO t (a) VALUES (?)"); let drop_unprepared_frame_rule = RequestRule( Condition::RequestOpcode(RequestOpcode::Query) @@ -85,7 +85,7 @@ async fn test_query_with_no_values() { .await .unwrap(); - let q = Query::from("INSERT INTO t (a) VALUES (1)"); + let q = UnpreparedStatement::from("INSERT INTO t (a) VALUES (1)"); let drop_prepared_frame_rule = RequestRule( Condition::RequestOpcode(RequestOpcode::Prepare) diff --git a/scylla/tests/integration/tablets.rs b/scylla/tests/integration/tablets.rs index 97c5ca0b24..baff176d16 100644 --- a/scylla/tests/integration/tablets.rs +++ b/scylla/tests/integration/tablets.rs @@ -15,7 +15,7 @@ use scylla::test_utils::unique_keyspace_name; use scylla::transport::ClusterData; use scylla::transport::Node; use scylla::transport::NodeRef; -use scylla::unprepared_statement::Query; +use scylla::unprepared_statement::UnpreparedStatement; use scylla::ExecutionProfile; use scylla::QueryResult; use scylla::Session; @@ -209,7 +209,7 @@ async fn send_statement_everywhere( async fn send_unprepared_query_everywhere( session: &Session, cluster: &ClusterData, - query: &Query, + query: &UnpreparedStatement, ) -> Result, QueryError> { let tasks = cluster.get_nodes_info().iter().flat_map(|node| { let shard_count: u16 = node.sharder().unwrap().nr_shards.into(); @@ -453,7 +453,9 @@ async fn test_tablet_feedback_not_sent_for_unprepared_queries() { send_unprepared_query_everywhere( &session, session.get_cluster_data().as_ref(), - &Query::new(format!("INSERT INTO {ks}.t (a, b, c) VALUES (1, 1, 'abc')")), + &UnpreparedStatement::new(format!( + "INSERT INTO {ks}.t (a, b, c) VALUES (1, 1, 'abc')" + )), ) .await .unwrap(); From 750385552563cb914126ee91c6f25a3ec8e1cfa6 Mon Sep 17 00:00:00 2001 From: muzarski Date: Mon, 6 May 2024 18:48:25 +0200 Subject: [PATCH 3/7] unprepared: adjust docs, query -> statement --- scylla/src/statement/unprepared_statement.rs | 30 ++++++++++---------- 1 file changed, 15 insertions(+), 15 deletions(-) diff --git a/scylla/src/statement/unprepared_statement.rs b/scylla/src/statement/unprepared_statement.rs index 06d98c71c1..7af5c0fb0c 100644 --- a/scylla/src/statement/unprepared_statement.rs +++ b/scylla/src/statement/unprepared_statement.rs @@ -6,9 +6,9 @@ use crate::transport::execution_profile::ExecutionProfileHandle; use std::sync::Arc; use std::time::Duration; -/// CQL query statement. +/// CQL unprepared 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 UnpreparedStatement { pub(crate) config: StatementConfig, @@ -18,7 +18,7 @@ pub struct UnpreparedStatement { } impl UnpreparedStatement { - /// Creates a new `Query` from a CQL query string. + /// Creates a new [`UnpreparedStatement`] from a CQL statement string. pub fn new(query_text: impl Into) -> Self { Self { contents: query_text.into(), @@ -33,18 +33,18 @@ impl UnpreparedStatement { self } - /// Sets the page size for this CQL query. + /// Sets the page size for this CQL statement. pub fn set_page_size(&mut self, page_size: i32) { assert!(page_size > 0, "page size must be larger than 0"); self.page_size = Some(page_size); } - /// Disables paging for this CQL query. + /// Disables paging for this CQL statement. pub fn disable_paging(&mut self) { self.page_size = None; } - /// Returns the page size for this CQL query. + /// Returns the page size for this CQL statement. pub fn get_page_size(&self) -> Option { self.page_size } @@ -54,7 +54,7 @@ impl UnpreparedStatement { 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 @@ -73,10 +73,10 @@ impl UnpreparedStatement { } /// 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 executed multiple times without changing the result of the initial execution /// 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; } @@ -88,7 +88,7 @@ impl UnpreparedStatement { /// Enable or disable CQL Tracing for this statement /// If enabled session.query() will return a QueryResult containing tracing_id - /// which can be used to query tracing information about the execution of this query + /// which can be used to query tracing information about the execution of this statement pub fn set_tracing(&mut self, should_trace: bool) { self.config.tracing = should_trace; } @@ -119,7 +119,7 @@ impl UnpreparedStatement { 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 } @@ -136,7 +136,7 @@ impl UnpreparedStatement { 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); } @@ -146,13 +146,13 @@ impl UnpreparedStatement { self.config.history_listener.take() } - /// Associates the query with execution profile referred by the provided handle. - /// Handle may be later remapped to another profile, and query will reflect those changes. + /// Associates the statement with execution profile referred by the provided handle. + /// Handle may be later remapped to another profile, and statement will reflect those changes. pub fn set_execution_profile_handle(&mut self, profile_handle: Option) { 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() } From 55a9f1f38ae670a890437f26835695c36a6f57f0 Mon Sep 17 00:00:00 2001 From: muzarski Date: Mon, 6 May 2024 19:38:09 +0200 Subject: [PATCH 4/7] session: simple -> unprepared, query -> statement Adjusted the comments and some parameter names, so we follow the unified naming convention. --- scylla/src/transport/session.rs | 130 ++++++++++++++++---------------- 1 file changed, 65 insertions(+), 65 deletions(-) diff --git a/scylla/src/transport/session.rs b/scylla/src/transport/session.rs index 951077ad5d..432b096e70 100644 --- a/scylla/src/transport/session.rs +++ b/scylla/src/transport/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 statements. use crate::batch::batch_values; #[cfg(feature = "cloud")] @@ -138,7 +138,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, @@ -561,19 +561,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 @@ -613,30 +613,30 @@ impl Session { /// ``` pub async fn query( &self, - query: impl Into, + statement: impl Into, values: impl SerializeRow, ) -> Result { - 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, + statement: impl Into, values: impl SerializeRow, paging_state: Option, ) -> Result { - let query: UnpreparedStatement = query.into(); + let query: UnpreparedStatement = statement.into(); let execution_profile = query .get_execution_profile_handle() @@ -730,7 +730,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) @@ -759,21 +759,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 /// @@ -798,10 +798,10 @@ impl Session { /// ``` pub async fn query_iter( &self, - query: impl Into, + statement: impl Into, values: impl SerializeRow, ) -> Result { - let query: UnpreparedStatement = query.into(); + let query: UnpreparedStatement = statement.into(); let execution_profile = query .get_execution_profile_handle() @@ -834,10 +834,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***\ @@ -849,7 +849,7 @@ impl Session { /// See the documentation of [`PreparedStatement`]. /// /// # 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 @@ -858,12 +858,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(&prepared, (to_insert,)).await?; /// # Ok(()) @@ -871,9 +871,9 @@ impl Session { /// ``` pub async fn prepare( &self, - query: impl Into, + statement: impl Into, ) -> Result { - let query = query.into(); + let query = statement.into(); let query_ref = &query; let cluster_data = self.get_cluster_data(); @@ -930,12 +930,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***\ @@ -947,7 +947,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 @@ -956,12 +956,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(&prepared, (to_insert,)).await?; /// # Ok(()) @@ -979,7 +979,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, @@ -1079,7 +1079,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\ @@ -1090,7 +1090,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 /// @@ -1102,12 +1102,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? @@ -1143,22 +1143,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 @@ -1169,19 +1169,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?; @@ -1196,7 +1196,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( @@ -1294,7 +1294,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)); /// @@ -1352,7 +1352,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?; @@ -1369,7 +1369,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)?; @@ -1505,13 +1505,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>( From da31dc97a4ecf8e4945084093296f7d1d4b9afd6 Mon Sep 17 00:00:00 2001 From: muzarski Date: Mon, 6 May 2024 19:44:48 +0200 Subject: [PATCH 5/7] lib/docs: simple query -> unprepared statement --- scylla/src/lib.rs | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/scylla/src/lib.rs b/scylla/src/lib.rs index 294fa1e6c0..db2798f7c2 100644 --- a/scylla/src/lib.rs +++ b/scylla/src/lib.rs @@ -34,21 +34,21 @@ //! `Session` is usually created using the [SessionBuilder].\ //! All configuration options for a `Session` can be specified while building. //! -//! ### Making queries -//! After successfully connecting to the cluster we can make queries.\ -//! The driver supports multiple query types: -//! * [Simple](crate::Session::query) -//! * [Simple paged](crate::Session::query_iter) -//! * [Prepare](crate::Session::execute) (need to be [prepared](crate::Session::prepare) before use) +//! ### Executing statements +//! After successfully connecting to the cluster we can execute statements.\ +//! The driver supports multiple statement types: +//! * [Unprepared](crate::Session::query) +//! * [Unprepared paged](crate::Session::query_iter) +//! * [Prepared](crate::Session::execute) (need to be [prepared](crate::Session::prepare) before use) //! * [Prepared paged](crate::Session::execute_iter) //! * [Batch](crate::Session::batch) //! -//! To specify options for a single query create the query object and configure it: -//! * For simple: [Query](crate::query::Query) +//! To specify options for a single statement, create the statement object and configure it: +//! * For unprepared: [UnpreparedStatement](crate::unprepared_statement::UnpreparedStatement) //! * For prepared: [PreparedStatement](crate::prepared_statement::PreparedStatement) //! * For batch: [Batch](crate::batch::Batch) //! -//! The easiest way to specify bound values in a query is using a tuple: +//! The easiest way to specify bound values in a statement is using a tuple: //! ```rust //! # use scylla::Session; //! # use std::error::Error; From 689d6f64d07f2ce932fcfb6bb9e2e3519798cf32 Mon Sep 17 00:00:00 2001 From: muzarski Date: Mon, 6 May 2024 21:12:32 +0200 Subject: [PATCH 6/7] exec_profile/docs: query -> statement --- scylla/src/transport/execution_profile.rs | 36 +++++++++++------------ 1 file changed, 18 insertions(+), 18 deletions(-) diff --git a/scylla/src/transport/execution_profile.rs b/scylla/src/transport/execution_profile.rs index 7b7a14faaf..cd6f0765e4 100644 --- a/scylla/src/transport/execution_profile.rs +++ b/scylla/src/transport/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 statement execution. //! //! Profiles can be created to represent different workloads, which thanks to them //! can be run conveniently on a single session. @@ -37,12 +37,12 @@ //! ``` //! //! ### Example -//! To create an `ExecutionProfile` and attach it to a `Query`: +//! To create an `ExecutionProfile` and attach it to an `UnpreparedStatement`: //! ``` //! # extern crate scylla; //! # use std::error::Error; //! # async fn check_only_compiles() -> Result<(), Box> { -//! use scylla::query::Query; +//! use scylla::unprepared_statement::UnpreparedStatement; //! use scylla::statement::Consistency; //! use scylla::transport::ExecutionProfile; //! use std::time::Duration; @@ -54,10 +54,10 @@ //! //! let handle = profile.into_handle(); //! -//! let mut query1 = Query::from("SELECT * FROM ks.table"); +//! let mut query1 = UnpreparedStatement::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 = UnpreparedStatement::from("SELECT pk FROM ks.table WHERE pk = ?"); //! query2.set_execution_profile_handle(Some(handle)); //! # Ok(()) //! # } @@ -110,7 +110,7 @@ //! # use std::error::Error; //! # async fn check_only_compiles() -> Result<(), Box> { //! use scylla::{Session, SessionBuilder}; -//! use scylla::query::Query; +//! use scylla::unprepared_statement::UnpreparedStatement; //! use scylla::statement::Consistency; //! use scylla::transport::ExecutionProfile; //! @@ -131,8 +131,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 = UnpreparedStatement::from("SELECT * FROM ks.table"); +//! let mut query2 = UnpreparedStatement::from("SELECT pk FROM ks.table WHERE pk = ?"); //! //! query1.set_execution_profile_handle(Some(handle1.clone())); //! query2.set_execution_profile_handle(Some(handle2.clone())); @@ -256,16 +256,16 @@ impl ExecutionProfileBuilder { self } - /// Specify a default consistency to be used for queries. - /// It's possible to override it by explicitly setting a consistency on the chosen query. + /// Specify a default consistency to be used for statements. + /// It's possible to override it by explicitly setting a consistency on the chosen statement. 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 statements. /// 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 @@ -294,7 +294,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::transport::retry_policy::DefaultRetryPolicy). /// It is possible to implement a custom retry policy by implementing the trait [`RetryPolicy`]. /// @@ -385,11 +385,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); @@ -458,7 +458,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 @@ -481,7 +481,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) } From 43703b78f133d7328539cc2eda3b5df3e5858718 Mon Sep 17 00:00:00 2001 From: muzarski Date: Mon, 6 May 2024 23:10:34 +0200 Subject: [PATCH 7/7] docs: simple -> unprepared, query -> statement Still need to be adjusted in some places --- docs/source/SUMMARY.md | 24 ++++++------- docs/source/data-types/data-types.md | 4 +-- docs/source/data-types/udt.md | 6 ++-- .../execution-profiles/create-and-use.md | 8 ++--- .../execution-profiles/execution-profiles.md | 2 +- .../execution-profiles/maximal-example.md | 4 +-- docs/source/execution-profiles/priority.md | 16 ++++----- docs/source/execution-profiles/remap.md | 6 ++-- docs/source/index.md | 12 +++---- docs/source/load-balancing/default-policy.md | 4 +-- docs/source/load-balancing/load-balancing.md | 20 +++++------ docs/source/logging/logging.md | 2 +- docs/source/metrics/metrics.md | 10 +++--- .../migration-guides/0.11-serialization.md | 8 ++--- docs/source/queries/batch.md | 21 +++++------ docs/source/queries/lwt.md | 19 +++++----- docs/source/queries/paged.md | 30 ++++++++-------- docs/source/queries/prepared.md | 32 ++++++++--------- docs/source/queries/queries.md | 10 +++--- docs/source/queries/simple.md | 35 ++++++++++--------- docs/source/quickstart/example.md | 2 +- docs/source/retry-policy/default.md | 7 ++-- .../retry-policy/downgrading-consistency.md | 9 ++--- docs/source/retry-policy/fallthrough.md | 9 ++--- docs/source/retry-policy/retry-policy.md | 5 +-- docs/source/tracing/basic.md | 19 +++++----- docs/source/tracing/paged.md | 9 ++--- docs/source/tracing/prepare.md | 5 +-- docs/source/tracing/query-history.md | 7 ++-- 29 files changed, 179 insertions(+), 166 deletions(-) diff --git a/docs/source/SUMMARY.md b/docs/source/SUMMARY.md index 63639364fa..32f266c149 100644 --- a/docs/source/SUMMARY.md +++ b/docs/source/SUMMARY.md @@ -5,7 +5,7 @@ - [Quick start](quickstart/quickstart.md) - [Creating a project](quickstart/create-project.md) - [Running Scylla using Docker](quickstart/scylla-docker.md) - - [Connecting and running a simple query](quickstart/example.md) + - [Connecting and executing an unprepared statement](quickstart/example.md) - [Migration guides](migration-guides/migration-guides.md) - [Adjusting code to changes in serialization API introduced in 0.11](migration-guides/0.11-serialization.md) @@ -15,17 +15,17 @@ - [Authentication](connecting/authentication.md) - [TLS](connecting/tls.md) -- [Making queries](queries/queries.md) - - [Simple query](queries/simple.md) - - [Query values](queries/values.md) +- [Executing statements](queries/queries.md) + - [Unprepared statement](queries/simple.md) + - [Statement bound values](queries/values.md) - [Query result](queries/result.md) - - [Prepared query](queries/prepared.md) + - [Prepared statement](queries/prepared.md) - [Batch statement](queries/batch.md) - - [Paged query](queries/paged.md) - - [Lightweight transaction query (LWT)](queries/lwt.md) + - [Paged statement execution](queries/paged.md) + - [Lightweight transaction statement (LWT)](queries/lwt.md) - [USE keyspace](queries/usekeyspace.md) - [Schema agreement](queries/schema-agreement.md) - - [Query timeouts](queries/timeouts.md) + - [Statement execution timeouts](queries/timeouts.md) - [Execution profiles](execution-profiles/execution-profiles.md) - [Creating a profile and setting it](execution-profiles/create-and-use.md) @@ -67,10 +67,10 @@ - [Logging](logging/logging.md) -- [Query tracing](tracing/tracing.md) - - [Tracing a simple/prepared query](tracing/basic.md) - - [Tracing a paged query](tracing/paged.md) +- [Statement execution tracing](tracing/tracing.md) + - [Tracing a statement execution](tracing/basic.md) + - [Tracing a paged statement execution](tracing/paged.md) - [Tracing `Session::prepare`](tracing/prepare.md) - - [Query Execution History](tracing/query-history.md) + - [Statement Execution History](tracing/query-history.md) - [Database schema](schema/schema.md) diff --git a/docs/source/data-types/data-types.md b/docs/source/data-types/data-types.md index 0d9696e765..e35d387b96 100644 --- a/docs/source/data-types/data-types.md +++ b/docs/source/data-types/data-types.md @@ -5,8 +5,8 @@ to achieve seamless sending and receiving of CQL values. See the following chapters for examples on how to send and receive each data type. -See [Query values](../queries/values.md) for more information about sending values in queries.\ -See [Query result](../queries/result.md) for more information about reading values from queries +See [Statement bound values](../queries/values.md) for more information about sending values in statements.\ +See [Statement execution result](../queries/result.md) for more information about reading values after executing a statement. Database types and their Rust equivalents: * `Boolean` <----> `bool` diff --git a/docs/source/data-types/udt.md b/docs/source/data-types/udt.md index c2ed650738..a6226ddd07 100644 --- a/docs/source/data-types/udt.md +++ b/docs/source/data-types/udt.md @@ -3,17 +3,17 @@ Scylla allows users to define their own data types with named fields (See [the o To use user defined types in the driver, you can create a corresponding struct in Rust, and use it to read and write UDT values. -For example let's say `my_type` was created using this query: +For example let's say `my_type` was created using this statement: ```sql CREATE TYPE ks.my_type (int_val int, text_val text) ``` To use this type in the driver, create a matching struct and derive: -- `SerializeCql`: in order to be able to use this struct in query parameters. \ +- `SerializeCql`: in order to be able to use this struct in statement parameters. \ This macro requires fields of UDT and struct to have matching names, but the order of the fields is not required to be the same. \ Note: you can use different name using `rename` attribute - see `SerializeCql` macro documentation. -- `FromUserType`: in order to be able to use this struct in query results. \ +- `FromUserType`: in order to be able to use this struct in statement execution results. \ This macro requires fields of UDT and struct to be in the same *ORDER*. \ This mismatch between `SerializeCql` and `FromUserType` requirements is a temporary situation - in the future `FromUserType` (or the macro that replaces it) will also require matching names. diff --git a/docs/source/execution-profiles/create-and-use.md b/docs/source/execution-profiles/create-and-use.md index 01bfa52d99..05e157985f 100644 --- a/docs/source/execution-profiles/create-and-use.md +++ b/docs/source/execution-profiles/create-and-use.md @@ -27,12 +27,12 @@ let session: Session = SessionBuilder::new() ``` ### Example -To create an `ExecutionProfile` and attach it to a `Query`: +To create an `ExecutionProfile` and attach it to an `UnpreparedStatement`: ```rust # extern crate scylla; # use std::error::Error; # async fn check_only_compiles() -> Result<(), Box> { -use scylla::query::Query; +use scylla::unprepared_statement::UnpreparedStatement; use scylla::statement::Consistency; use scylla::transport::ExecutionProfile; use std::time::Duration; @@ -44,10 +44,10 @@ let profile = ExecutionProfile::builder() let handle = profile.into_handle(); -let mut query1 = Query::from("SELECT * FROM ks.table"); +let mut query1 = UnpreparedStatement::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 = UnpreparedStatement::from("SELECT pk FROM ks.table WHERE pk = ?"); query2.set_execution_profile_handle(Some(handle)); # Ok(()) # } diff --git a/docs/source/execution-profiles/execution-profiles.md b/docs/source/execution-profiles/execution-profiles.md index 6f36726aff..ca6f984300 100644 --- a/docs/source/execution-profiles/execution-profiles.md +++ b/docs/source/execution-profiles/execution-profiles.md @@ -1,6 +1,6 @@ # Execution profiles -Execution profiles are a way to group various query execution configuration options together. Profiles can be created to represent different workloads, which can be run conveniently on a single session. +Execution profiles are a way to group various statement execution configuration options together. Profiles can be created to represent different workloads, which can be run conveniently on a single session. The settings that an execution profile encapsulates are [as follows](maximal-example.md): * consistency diff --git a/docs/source/execution-profiles/maximal-example.md b/docs/source/execution-profiles/maximal-example.md index 8209b926a1..8f7d1633e0 100644 --- a/docs/source/execution-profiles/maximal-example.md +++ b/docs/source/execution-profiles/maximal-example.md @@ -6,7 +6,7 @@ # extern crate scylla; # use std::error::Error; # async fn check_only_compiles() -> Result<(), Box> { -use scylla::query::Query; +use scylla::unprepared_statement::UnpreparedStatement; use scylla::speculative_execution::SimpleSpeculativeExecutionPolicy; use scylla::statement::{Consistency, SerialConsistency}; use scylla::transport::ExecutionProfile; @@ -32,7 +32,7 @@ let profile = ExecutionProfile::builder() ) .build(); -let mut query = Query::from("SELECT * FROM ks.table"); +let mut query = UnpreparedStatement::from("SELECT * FROM ks.table"); query.set_execution_profile_handle(Some(profile.into_handle())); # Ok(()) diff --git a/docs/source/execution-profiles/priority.md b/docs/source/execution-profiles/priority.md index 4ae22d2c86..78bba0b263 100644 --- a/docs/source/execution-profiles/priority.md +++ b/docs/source/execution-profiles/priority.md @@ -14,7 +14,7 @@ Priorities of execution profiles and directly set options: # use std::error::Error; # async fn check_only_compiles() -> Result<(), Box> { use scylla::{Session, SessionBuilder}; -use scylla::query::Query; +use scylla::unprepared_statement::UnpreparedStatement; use scylla::statement::Consistency; use scylla::transport::ExecutionProfile; @@ -32,20 +32,20 @@ let session: Session = SessionBuilder::new() .build() .await?; -let mut query = Query::from("SELECT * FROM ks.table"); +let mut query = UnpreparedStatement::from("SELECT * FROM ks.table"); -// Query is not assigned any specific profile, so session's profile is applied. -// Therefore, the query will be executed with Consistency::One. +// Statement is not assigned any specific profile, so session's profile is applied. +// Therefore, the statement will be executed with Consistency::One. session.query(query.clone(), ()).await?; query.set_execution_profile_handle(Some(query_profile.into_handle())); -// Query's profile is applied. -// Therefore, the query will be executed with Consistency::Two. +// Statement's profile is applied. +// Therefore, the statement will be executed with Consistency::Two. session.query(query.clone(), ()).await?; query.set_consistency(Consistency::Three); -// An option is set directly on the query. -// Therefore, the query will be executed with Consistency::Three. +// An option is set directly on the statement. +// Therefore, the statement will be executed with Consistency::Three. session.query(query, ()).await?; # Ok(()) diff --git a/docs/source/execution-profiles/remap.md b/docs/source/execution-profiles/remap.md index a64aee3916..87bcd64177 100644 --- a/docs/source/execution-profiles/remap.md +++ b/docs/source/execution-profiles/remap.md @@ -24,7 +24,7 @@ Below, the remaps described above are followed in code. # use std::error::Error; # async fn check_only_compiles() -> Result<(), Box> { use scylla::{Session, SessionBuilder}; -use scylla::query::Query; +use scylla::unprepared_statement::UnpreparedStatement; use scylla::statement::Consistency; use scylla::transport::ExecutionProfile; @@ -45,8 +45,8 @@ let session: Session = SessionBuilder::new() .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 = UnpreparedStatement::from("SELECT * FROM ks.table"); +let mut query2 = UnpreparedStatement::from("SELECT pk FROM ks.table WHERE pk = ?"); query1.set_execution_profile_handle(Some(handle1.clone())); query2.set_execution_profile_handle(Some(handle2.clone())); diff --git a/docs/source/index.md b/docs/source/index.md index d2a6b79313..0066312503 100644 --- a/docs/source/index.md +++ b/docs/source/index.md @@ -12,15 +12,15 @@ Although optimized for Scylla, the driver is also compatible with [Apache Cassan ## Contents -* [Quick start](quickstart/quickstart.md) - Setting up a Rust project using `scylla-rust-driver` and running a few queries +* [Quick start](quickstart/quickstart.md) - Setting up a Rust project using `scylla-rust-driver` and executing a few statements * [Migration guides](migration-guides/migration-guides.md) - How to update the code that used an older version of this driver * [Connecting to the cluster](connecting/connecting.md) - Configuring a connection to scylla cluster -* [Making queries](queries/queries.md) - Making different types of queries (simple, prepared, batch, paged) -* [Execution profiles](execution-profiles/execution-profiles.md) - Grouping query execution configuration options together and switching them all at once +* [Executing statements](queries/queries.md) - Creating and executing different types of statements (unprepared, prepared, batch, paged) +* [Execution profiles](execution-profiles/execution-profiles.md) - Grouping statement execution configuration options together and switching them all at once * [Data Types](data-types/data-types.md) - How to use various column data types * [Load balancing](load-balancing/load-balancing.md) - Load balancing configuration -* [Retry policy configuration](retry-policy/retry-policy.md) - What to do when a query fails, query idempotence -* [Driver metrics](metrics/metrics.md) - Statistics about the driver - number of queries, latency etc. +* [Retry policy configuration](retry-policy/retry-policy.md) - What to do when a statement execution fails, statement idempotence +* [Driver metrics](metrics/metrics.md) - Statistics about the driver - number of executed statements, latency etc. * [Logging](logging/logging.md) - Viewing and integrating logs produced by the driver -* [Query tracing](tracing/tracing.md) - Tracing query execution +* [Statement execution tracing](tracing/tracing.md) - Tracing statement execution * [Database schema](schema/schema.md) - Fetching and inspecting database schema diff --git a/docs/source/load-balancing/default-policy.md b/docs/source/load-balancing/default-policy.md index 4f8310a36f..04e8492575 100644 --- a/docs/source/load-balancing/default-policy.md +++ b/docs/source/load-balancing/default-policy.md @@ -2,7 +2,7 @@ `DefaultPolicy` is the default load balancing policy in Scylla Rust Driver. It can be configured to be datacenter-aware and token-aware. Datacenter failover -for queries with non-local consistency mode is also supported. +for statements with non-local consistency mode is also supported. ## Creating a DefaultPolicy @@ -107,7 +107,7 @@ Every `update_rate` the global minimum average latency is computed, and all nodes whose average latency is worse than `exclusion_threshold` times the global minimum average latency become penalised for `retry_period`. Penalisation involves putting those nodes at the very end -of the query plan. As it is often not truly beneficial to prefer +of the statement execution plan. As it is often not truly beneficial to prefer faster non-replica than replicas lagging behind the non-replicas, this mechanism may as well worsen latencies and/or throughput. diff --git a/docs/source/load-balancing/load-balancing.md b/docs/source/load-balancing/load-balancing.md index 3ec27dd7e1..e02db1d2f9 100644 --- a/docs/source/load-balancing/load-balancing.md +++ b/docs/source/load-balancing/load-balancing.md @@ -3,9 +3,9 @@ ## Introduction The driver uses a load balancing policy to determine which node(s) and shard(s) -to contact when executing a query. Load balancing policies implement the +to contact when executing a statement. Load balancing policies implement the `LoadBalancingPolicy` trait, which contains methods to generate a load -balancing plan based on the query information and the state of the cluster. +balancing plan based on the statement information and the state of the cluster. Load balancing policies do not influence to which nodes connections are being opened. For a node connection blacklist configuration refer to @@ -16,9 +16,9 @@ In this chapter, "target" will refer to a pair ``. ## Plan -When a query is prepared to be sent to the database, the load balancing policy +When a statement is prepared to be sent to the database, the load balancing policy constructs a load balancing plan. This plan is essentially a list of targets to -which the driver will try to send the query. The first elements of the plan are +which the driver will try to send the statement. The first elements of the plan are the targets which are the best to contact (e.g. they might be replicas for the requested data or have the best latency). @@ -73,7 +73,7 @@ let session: Session = SessionBuilder::new() In addition to being able to configure load balancing policies through execution profiles at the session level, the driver also allow for setting -execution profile handles on a per-query basis. This means that for each query, +execution profile handles on a per-statement basis. This means that for each statement, a specific execution profile can be selected with a customized load balancing settings. @@ -81,12 +81,12 @@ settings. ### `pick` and `fallback`: -Most queries are sent successfully on the first try. In such cases, only the +Most statements are executed successfully on the first try. In such cases, only the first element of the load balancing plan is needed, so it's usually unnecessary to compute entire load balancing plan. To optimize this common case, the `LoadBalancingPolicy` trait provides two methods: `pick` and `fallback`. -`pick` returns the first target to contact for a given query, which is usually +`pick` returns the first target to contact for a given statement, which is usually the best based on a particular load balancing policy. `fallback`, returns an iterator that provides the rest of the targets in the @@ -94,7 +94,7 @@ load balancing plan. `fallback` is called when using the initial picked target fails (or when executing speculatively) or when `pick` returned `None`. It's possible for the `fallback` method to include the same target that was -returned by the `pick` method. In such cases, the query execution layer filters +returned by the `pick` method. In such cases, the statement execution layer filters out the picked target from the iterator returned by `fallback`. ### `on_query_success` and `on_query_failure`: @@ -103,13 +103,13 @@ The `on_query_success` and `on_query_failure` methods are useful for load balancing policies because they provide feedback on the performance and health of the nodes in the cluster. -When a query is successfully executed, the `on_query_success` method is called +When a statement is successfully executed, the `on_query_success` method is called and can be used by the load balancing policy to update its internal state. For example, a policy might use the latency of the successful query to update its latency statistics for each node in the cluster. This information can be used to make decisions about which nodes to contact in the future. -On the other hand, when a query fails to execute, the `on_query_failure` method +On the other hand, when a statement fails to execute, the `on_query_failure` method is called and provides information about the failure. The error message returned by Cassandra can help determine the cause of the failure, such as a node being down or overloaded. The load balancing policy can use this diff --git a/docs/source/logging/logging.md b/docs/source/logging/logging.md index c99d1b1e0f..0de3370405 100644 --- a/docs/source/logging/logging.md +++ b/docs/source/logging/logging.md @@ -39,7 +39,7 @@ async fn main() -> Result<(), Box> { ) .await?; - // This query should generate a warning message + // This statement should generate a warning message session.query("USE ks", &[]).await?; Ok(()) diff --git a/docs/source/metrics/metrics.md b/docs/source/metrics/metrics.md index ce52615383..3393420535 100644 --- a/docs/source/metrics/metrics.md +++ b/docs/source/metrics/metrics.md @@ -5,11 +5,11 @@ During operation the driver collects various metrics. They can be accessed at any moment using `Session::get_metrics()` ### Collected metrics: -* Query latencies -* Total number of nonpaged queries -* Number of errors during nonpaged queries -* Total number of paged queries -* Number of errors during paged queries +* Statement execution latencies +* Total number of nonpaged statements +* Number of errors during nonpaged statements' execution +* Total number of paged statements +* Number of errors during paged statements' execution * Number of retries ### Example diff --git a/docs/source/migration-guides/0.11-serialization.md b/docs/source/migration-guides/0.11-serialization.md index 4df222592c..dea3af5d3b 100644 --- a/docs/source/migration-guides/0.11-serialization.md +++ b/docs/source/migration-guides/0.11-serialization.md @@ -4,7 +4,7 @@ When executing a statement through the CQL protocol, values for the bind markers are sent in a serialized, untyped form. In order to implement a safer and more robust interface, drivers can use the information returned after preparing a statement to check the type of data provided by the user against the actual types of the bind markers. -Before 0.11, the driver couldn't do this kind of type checking. For example, in the case of non-batch queries, the only information about the user data it has is that it implements `ValueList` - defined as follows: +Before 0.11, the driver couldn't do this kind of type checking. For example, in the case of non-batch statements, the only information about the user data it has is that it implements `ValueList` - defined as follows: ```rust # extern crate scylla; @@ -29,7 +29,7 @@ In version 0.11, a new set of traits is introduced and the old ones are deprecat Both the old and the new APIs are based on three core traits: - `Value` - called `SerializeCql` in the new API. A type that can serialize itself to a single CQL value. For example, `i32` serializes itself into a representation that is compatible with the CQL `int` type. -- `ValueList` - called `SerializeRow` in the new API. A type that can serialize itself as a list of values for a CQL statement. For example, a `(i32, &str)` produces a list of two values which can be used in a query with two bind markers, e.g. `SELECT * FROM table WHERE pk = ? AND ck = ?`. Optionally, values in the produced list may be associated with names which is useful when using it with a query with named bind markers, e.g. `SELECT * FROM table WHERE pk = :pk AND ck = :ck`. +- `ValueList` - called `SerializeRow` in the new API. A type that can serialize itself as a list of values for a CQL statement. For example, a `(i32, &str)` produces a list of two values which can be used in a statement with two bind markers, e.g. `SELECT * FROM table WHERE pk = ? AND ck = ?`. Optionally, values in the produced list may be associated with names which is useful when using it with a statement with named bind markers, e.g. `SELECT * FROM table WHERE pk = :pk AND ck = :ck`. - `LegacyBatchValues`, previously named `BatchValues` - in new API replaced with new trait called (again) `BatchValues`. Represents a source of data for a batch request. It is essentially equivalent to a list of `ValueList`, one for each statement in the batch. For example, `((1, 2), (3, 4, 5))` can be used for a batch with two statements, the first one having two bind markers and the second one having three. All methods which take one of the old traits were changed to take the new trait - notably, this includes `Session::query`, `(Caching)Session::execute`, `(Caching)Session::batch`. @@ -71,12 +71,12 @@ As explained in the [Background](#background) section, the driver uses data retu > **NOTE:** The driver will skip preparation if it detects that the list of values for the statement is empty, as there is nothing to be type checked. -If you send simple statements along with non-empty lists of values, the slowdown will be as follows: +If you send unprepared statements along with non-empty lists of values, the slowdown will be as follows: - For `Session::query`, the driver will prepare the statement before sending it, incurring an additional round-trip. - For `Session::batch`, the driver will send a prepare request for each *unique* unprepared statement with a non-empty list of values. **This is done serially!** -In both cases, if the additional roundtrips are unacceptable, you should prepare the statements beforehand and reuse them - which aligns with our general recommendation against use of simple statements in performance sensitive scenarios. +In both cases, if the additional roundtrips are unacceptable, you should prepare the statements beforehand and reuse them - which aligns with our general recommendation against use of unprepared statements in performance sensitive scenarios. ### Migrating from old to new traits *gradually* diff --git a/docs/source/queries/batch.md b/docs/source/queries/batch.md index 4d9694c45e..6fa2bbb1c5 100644 --- a/docs/source/queries/batch.md +++ b/docs/source/queries/batch.md @@ -1,7 +1,7 @@ # Batch statement A batch statement allows to execute many data-modifying statements at once.\ -These statements can be [simple](simple.md) or [prepared](prepared.md).\ +These statements can be [unprepared](simple.md) or [prepared](prepared.md).\ Only `INSERT`, `UPDATE` and `DELETE` statements are allowed. ```rust @@ -10,17 +10,18 @@ Only `INSERT`, `UPDATE` and `DELETE` statements are allowed. # use std::error::Error; # async fn check_only_compiles(session: &Session) -> Result<(), Box> { use scylla::batch::Batch; -use scylla::query::Query; +use scylla::unprepared_statement::UnpreparedStatement; use scylla::prepared_statement::PreparedStatement; // Create a batch statement let mut batch: Batch = Default::default(); -// Add a simple statement to the batch using its text +// Add an unprepared statement to the batch using its text batch.append_statement("INSERT INTO ks.tab(a, b) VALUES(1, 2)"); -// Add a simple statement created manually to the batch -let simple: Query = Query::new("INSERT INTO ks.tab (a, b) VALUES(3, 4)"); +// Add an unprepared statement created manually to the batch +let simple: UnpreparedStatement = + UnpreparedStatement::new("INSERT INTO ks.tab (a, b) VALUES(3, 4)"); batch.append_statement(simple); // Add a prepared statement to the batch @@ -41,8 +42,8 @@ session.batch(&batch, batch_values).await?; ``` > ***Warning***\ -> Using simple statements with bind markers in batches is strongly discouraged. -> For each simple statement with a non-empty list of values in the batch, +> Using unprepared statements with bind markers in batches is strongly discouraged. +> For each unprepared statement with a non-empty list of values in the batch, > the driver will send a prepare request, and it will be done **sequentially**. > Results of preparation are not cached between `Session::batch` calls. > Consider preparing the statements before putting them into the batch. @@ -103,7 +104,7 @@ See [Batch API documentation](https://docs.rs/scylla/latest/scylla/statement/bat for more options ### Batch values -Batch takes a tuple of values specified just like in [simple](simple.md) or [prepared](prepared.md) queries. +Batch takes a tuple of values specified just like in [unprepared](simple.md) or [prepared](prepared.md) statements. Length of batch values must be equal to the number of statements in a batch.\ Each statement must have its values specified, even if they are empty. @@ -142,10 +143,10 @@ session.batch(&batch, batch_values).await?; # Ok(()) # } ``` -For more information about sending values in a statement see [Query values](values.md) +For more information about sending values in a statement see [Statement bound values](values.md) ### Performance Batch statements do not use token/shard aware load balancing, batches are sent to a random node. -Use [prepared queries](prepared.md) for best performance +Use [prepared statements](prepared.md) for best performance diff --git a/docs/source/queries/lwt.md b/docs/source/queries/lwt.md index afe0d9dc5c..3537d14392 100644 --- a/docs/source/queries/lwt.md +++ b/docs/source/queries/lwt.md @@ -1,20 +1,21 @@ -# Lightweight transaction (LWT) query +# Lightweight transaction (LWT) statement -A lightweight transaction query can be expressed just like any other query, via `Session`, with the notable difference of having an additional consistency level parameter - the `serial_consistency_level`. +A lightweight transaction statement can be expressed just like any other statement, via `Session`, with the notable difference of having an additional consistency level parameter - the `serial_consistency_level`. -### Format of the query -A lightweight transaction query is not a separate type - it can be expressed just like any other queries: via `SimpleQuery`, `PreparedStatement`, batches, and so on. The difference lays in the query string itself - when it contains a condition (e.g. `IF NOT EXISTS`), it becomes a lightweight transaction. It's important to remember that CQL specification requires a separate, additional consistency level to be defined for LWT queries - `serial_consistency_level`. The serial consistency level can only be set to two values: `SerialConsistency::Serial` or `SerialConsistency::LocalSerial`. The "local" variant makes the transaction consistent only within the same datacenter. For convenience, Scylla Rust Driver sets the default consistency level to `LocalSerial`, as it's more commonly used. For cross-datacenter consistency, please remember to always override the default with `SerialConsistency::Serial`. +### Format of the statement +A lightweight transaction statement is not a separate type - it can be expressed just like any other statements: via `UnpreparedStatement`, `PreparedStatement`, batches, and so on. The difference lays in the statement string itself - when it contains a condition (e.g. `IF NOT EXISTS`), it becomes a lightweight transaction. It's important to remember that CQL specification requires a separate, additional consistency level to be defined for LWT statements - `serial_consistency_level`. The serial consistency level can only be set to two values: `SerialConsistency::Serial` or `SerialConsistency::LocalSerial`. The "local" variant makes the transaction consistent only within the same datacenter. For convenience, Scylla Rust Driver sets the default consistency level to `LocalSerial`, as it's more commonly used. For cross-datacenter consistency, please remember to always override the default with `SerialConsistency::Serial`. ```rust # extern crate scylla; # use scylla::Session; # use std::error::Error; # async fn check_only_compiles(session: &Session) -> Result<(), Box> { -use scylla::query::Query; +use scylla::unprepared_statement::UnpreparedStatement; use scylla::statement::{Consistency, SerialConsistency}; -// Create a Query manually to change the Consistency to ONE -let mut my_query: Query = Query::new("INSERT INTO ks.tab (a) VALUES(?) IF NOT EXISTS".to_string()); +// Create an UnpreparedStatement manually to change the Consistency to ONE +let mut my_query: UnpreparedStatement = + UnpreparedStatement::new("INSERT INTO ks.tab (a) VALUES(?) IF NOT EXISTS".to_string()); my_query.set_consistency(Consistency::One); // Use cross-datacenter serial consistency my_query.set_serial_consistency(Some(SerialConsistency::Serial)); @@ -26,7 +27,7 @@ session.query(my_query, (to_insert,)).await?; # } ``` -The rest of the API remains identical for LWT and non-LWT queries. +The rest of the API remains identical for LWT and non-LWT statements. -See [Query API documentation](https://docs.rs/scylla/latest/scylla/statement/query/struct.Query.html) for more options +See [Statement API documentation](https://docs.rs/scylla/latest/scylla/statement/unprepared_statement/struct.UnpreparedStatement.html) for more options diff --git a/docs/source/queries/paged.md b/docs/source/queries/paged.md index 8112c9308b..57d3aa4e55 100644 --- a/docs/source/queries/paged.md +++ b/docs/source/queries/paged.md @@ -2,19 +2,19 @@ Sometimes query results might not fit in a single page. Paged queries allow to receive the whole result page by page. -`Session::query_iter` and `Session::execute_iter` take a [simple query](simple.md) or a [prepared query](prepared.md) +`Session::query_iter` and `Session::execute_iter` take an [unprepared statement](simple.md) or a [prepared statement](prepared.md) and return an `async` iterator over result `Rows`. > ***Warning***\ > In case of unprepared variant (`Session::query_iter`) if the values are not empty -> driver will first fully prepare a query (which means issuing additional request to each +> driver will first fully prepare a statement (which means issuing additional request to each > node in a cluster). This will have a performance penalty - how big it is depends on > the size of your cluster (more nodes - more requests) and the size of returned > result (more returned pages - more amortized penalty). In any case, it is preferable to > use `Session::execute_iter`. ### Examples -Use `query_iter` to perform a [simple query](simple.md) with paging: +Use `query_iter` to execute an [unprepared statement](simple.md) with paging: ```rust # extern crate scylla; # extern crate futures; @@ -37,7 +37,7 @@ while let Some(next_row_res) = rows_stream.next().await { # } ``` -Use `execute_iter` to perform a [prepared query](prepared.md) with paging: +Use `execute_iter` to execute a [prepared statement](prepared.md) with paging: ```rust # extern crate scylla; # extern crate futures; @@ -65,20 +65,21 @@ while let Some(next_row_res) = rows_stream.next().await { # } ``` -Query values can be passed to `query_iter` and `execute_iter` just like in a [simple query](simple.md) +Bound values can be passed to `query_iter` and `execute_iter` just like in an [unprepared statement](simple.md) ### Configuring page size It's possible to configure the size of a single page. -On a `Query`: +On an `UnpreparedStatement`: ```rust # extern crate scylla; # use scylla::Session; # use std::error::Error; # async fn check_only_compiles(session: &Session) -> Result<(), Box> { -use scylla::query::Query; +use scylla::unprepared_statement::UnpreparedStatement; -let mut query: Query = Query::new("SELECT a, b FROM ks.t"); +let mut query: UnpreparedStatement = + UnpreparedStatement::new("SELECT a, b FROM ks.t"); query.set_page_size(16); let _ = session.query_iter(query, &[]).await?; // ... @@ -110,15 +111,16 @@ It's possible to fetch a single page from the table, extract the paging state from the result and manually pass it to the next query. That way, the next query will start fetching the results from where the previous one left off. -On a `Query`: +On an `UnpreparedStatement`: ```rust # extern crate scylla; # use scylla::Session; # use std::error::Error; # async fn check_only_compiles(session: &Session) -> Result<(), Box> { -use scylla::query::Query; +use scylla::unprepared_statement::UnpreparedStatement; -let paged_query = Query::new("SELECT a, b, c FROM ks.t").with_page_size(6); +let paged_query = UnpreparedStatement::new("SELECT a, b, c FROM ks.t") + .with_page_size(6); let res1 = session.query(paged_query.clone(), &[]).await?; let res2 = session .query_paged(paged_query.clone(), &[], res1.paging_state) @@ -138,10 +140,10 @@ On a `PreparedStatement`: # use scylla::Session; # use std::error::Error; # async fn check_only_compiles(session: &Session) -> Result<(), Box> { -use scylla::query::Query; +use scylla::unprepared_statement::UnpreparedStatement; let paged_prepared = session - .prepare(Query::new("SELECT a, b, c FROM ks.t").with_page_size(7)) + .prepare(UnpreparedStatement::new("SELECT a, b, c FROM ks.t").with_page_size(7)) .await?; let res1 = session.execute(&paged_prepared, &[]).await?; let res2 = session @@ -153,4 +155,4 @@ let res2 = session ### Performance Performance is the same as in non-paged variants.\ -For the best performance use [prepared queries](prepared.md). \ No newline at end of file +For the best performance use [prepared statements](prepared.md). \ No newline at end of file diff --git a/docs/source/queries/prepared.md b/docs/source/queries/prepared.md index caddb3c913..4fb9de8ed6 100644 --- a/docs/source/queries/prepared.md +++ b/docs/source/queries/prepared.md @@ -1,6 +1,6 @@ -# Prepared query +# Prepared statement -Prepared queries provide much better performance than simple queries, +Prepared statements provide much better performance than unprepared statements, but they need to be prepared before use. Benefits that prepared statements have to offer: @@ -16,12 +16,12 @@ prepares the statement before execution. The reason for this is to provide type # 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(&prepared, (to_insert,)).await?; # Ok(()) @@ -40,14 +40,14 @@ session.execute(&prepared, (to_insert,)).await?; > When page size is set, `execute` will return only the first page of results. ### `Session::prepare` -`Session::prepare` takes query text and prepares the query on all nodes and shards. +`Session::prepare` takes statement text and prepares the statement on all nodes and shards. If at least one succeeds returns success. ### `Session::execute` -`Session::execute` takes a prepared query and bound values and runs the query. -Passing values and the result is the same as in [simple query](simple.md). +`Session::execute` takes a prepared statement and bound values and executes the statement. +Passing values and the result is the same as in [unprepared statement](simple.md). -### Query options +### Statement options To specify custom options, set them on the `PreparedStatement` before execution. For example to change the consistency: @@ -60,16 +60,16 @@ For example to change the consistency: use scylla::prepared_statement::PreparedStatement; use scylla::statement::Consistency; -// Prepare the query for later execution +// Prepare the statement for later execution let mut prepared: PreparedStatement = session .prepare("INSERT INTO ks.tab (a) VALUES(?)") .await?; -// Set prepared query consistency to One -// This is the consistency with which this query will be executed +// Set prepared statement consistency to One +// This is the consistency with which this statement will be executed prepared.set_consistency(Consistency::One); -// 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(()) @@ -80,13 +80,13 @@ See [PreparedStatement API documentation](https://docs.rs/scylla/latest/scylla/s for more options. > ***Note*** -> Prepared statements can be created from `Query` structs and will inherit from -> the custom options that the `Query` was created with. +> Prepared statements can be created from `UnpreparedStatement` structs and will inherit from +> the custom options that the `UnpreparedStatement` was created with. > This is especially useful when using `CachingSession::execute` for example. ### Performance -Prepared queries have good performance, much better than simple queries. +Prepared statements have good performance, much better than unprepared statements. By default they use shard/token aware load balancing. > **Always** pass partition key values as bound values. @@ -111,7 +111,7 @@ TABLE ks.prepare_table ( # async fn check_only_compiles(session: &Session) -> Result<(), Box> { use scylla::prepared_statement::PreparedStatement; -// WRONG - partition key value is passed in query string +// WRONG - partition key value is passed in statement string // Load balancing will compute the wrong partition key let wrong_prepared: PreparedStatement = session .prepare("INSERT INTO ks.prepare_table (a, b, c) VALUES(12345, ?, 16)") diff --git a/docs/source/queries/queries.md b/docs/source/queries/queries.md index b75810d0ad..ca875833c9 100644 --- a/docs/source/queries/queries.md +++ b/docs/source/queries/queries.md @@ -1,11 +1,11 @@ -# Making queries +# Creating and executing statements -This driver supports all query types available in Scylla: -* [Simple queries](simple.md) +This driver supports all statement types available in Scylla: +* [Unprepared statements](simple.md) * Easy to use * Poor performance * Primitive load balancing -* [Prepared queries](prepared.md) +* [Prepared statements](prepared.md) * Need to be prepared before use * Fast * Properly load balanced @@ -16,7 +16,7 @@ This driver supports all query types available in Scylla: * Allows to read result in multiple pages when it doesn't fit in a single response * Can be prepared for better performance and load balancing -Additionally there is special functionality to enable `USE KEYSPACE` queries: +Additionally there is special functionality to enable `USE KEYSPACE` statements: [USE keyspace](usekeyspace.md) Queries are fully asynchronous - you can run as many of them in parallel as you wish. diff --git a/docs/source/queries/simple.md b/docs/source/queries/simple.md index d065f52d2d..c74b77ecdc 100644 --- a/docs/source/queries/simple.md +++ b/docs/source/queries/simple.md @@ -1,11 +1,11 @@ -# Simple query +# Unprepared statement -Simple query takes query text and values and simply executes them on a `Session`: +Unprepared statement takes statement text and values and simply executes them on a `Session`: ```rust # extern crate scylla; # use scylla::Session; # use std::error::Error; -# async fn simple_query_example(session: &Session) -> Result<(), Box> { +# async fn unprepared_statement_example(session: &Session) -> Result<(), Box> { // Insert a value into the table let to_insert: i32 = 12345; session @@ -16,11 +16,11 @@ session ``` > ***Warning***\ -> Don't use simple query to receive large amounts of data.\ -> By default the query is unpaged and might cause heavy load on the cluster.\ -> In such cases set a page size and use [paged query](paged.md) instead.\ +> Don't use unprepared statements to receive large amounts of data.\ +> By default the statement execution is unpaged and might cause heavy load on the cluster.\ +> In such cases set a page size and use [paged statement](paged.md) instead.\ > -> When page size is set, `query` will return only the first page of results. +> When page size is set, `Session::query` will return only the first page of results. > ***Warning***\ > If the values are not empty, driver first needs to send a `PREPARE` request @@ -35,11 +35,12 @@ You can create a query manually to set custom options. For example to change que # use scylla::Session; # use std::error::Error; # async fn check_only_compiles(session: &Session) -> Result<(), Box> { -use scylla::query::Query; +use scylla::unprepared_statement::UnpreparedStatement; use scylla::statement::Consistency; -// Create a Query manually to change the Consistency to ONE -let mut my_query: Query = Query::new("INSERT INTO ks.tab (a) VALUES(?)"); +// Create an UnpreparedStatement manually to change the Consistency to ONE +let mut my_query: UnpreparedStatement = + UnpreparedStatement::new("INSERT INTO ks.tab (a) VALUES(?)"); my_query.set_consistency(Consistency::One); // Insert a value into the table @@ -48,7 +49,7 @@ session.query(my_query, (to_insert,)).await?; # Ok(()) # } ``` -See [Query API documentation](https://docs.rs/scylla/latest/scylla/statement/query/struct.Query.html) for more options +See [Query API documentation](https://docs.rs/scylla/latest/scylla/statement/unprepared_statement/struct.UnpreparedStatement.html) for more options ### Second argument - the values Query text is constant, but the values might change. @@ -92,15 +93,15 @@ while let Some(read_row) = iter.next().transpose()? { # Ok(()) # } ``` -> In cases where page size is set, simple query returns only a single page of results.\ -> To receive all pages use a [paged query](paged.md) instead.\ +> In cases where page size is set, unprepared statement returns only a single page of results.\ +> To receive all pages use a [paged statement](paged.md) instead.\ See [Query result](result.md) for more information about handling query results ### Performance -Simple queries should not be used in places where performance matters.\ -If performance matters use a [Prepared query](prepared.md) instead. +Unprepared statements should not be used in places where performance matters.\ +If performance matters use a [Prepared statement](prepared.md) instead. -With simple query the database has to parse query text each time it's executed, which worsens performance.\ +With unprepared statement the database has to parse statement text each time it's executed, which worsens performance.\ -Additionally token and shard aware load balancing does not work with simple queries. They are sent to random nodes. +Additionally token and shard aware load balancing does not work with unprepared statements. They are sent to random nodes. diff --git a/docs/source/quickstart/example.md b/docs/source/quickstart/example.md index 32e01b2d5f..6431894fff 100644 --- a/docs/source/quickstart/example.md +++ b/docs/source/quickstart/example.md @@ -1,4 +1,4 @@ -# Connecting and running a simple query +# Connecting and executing an unprepared statement Now everything is ready to use the driver. Here is a small example: diff --git a/docs/source/retry-policy/default.md b/docs/source/retry-policy/default.md index e1f8514ed4..d448d2b9b9 100644 --- a/docs/source/retry-policy/default.md +++ b/docs/source/retry-policy/default.md @@ -28,19 +28,20 @@ let session: Session = SessionBuilder::new() # } ``` -To use in a [simple query](../queries/simple.md): +To use in an [unprepared statement](../queries/simple.md): ```rust # extern crate scylla; # use scylla::Session; # use std::error::Error; # use std::sync::Arc; # async fn check_only_compiles(session: &Session) -> Result<(), Box> { -use scylla::query::Query; +use scylla::unprepared_statement::UnpreparedStatement; use scylla::transport::ExecutionProfile; use scylla::transport::retry_policy::DefaultRetryPolicy; // Create a Query manually and set the retry policy -let mut my_query: Query = Query::new("INSERT INTO ks.tab (a) VALUES(?)"); +let mut my_query: UnpreparedStatement = + UnpreparedStatement::new("INSERT INTO ks.tab (a) VALUES(?)"); my_query.set_retry_policy(Some(Arc::new(DefaultRetryPolicy::new()))); // You can also set retry policy in an execution profile diff --git a/docs/source/retry-policy/downgrading-consistency.md b/docs/source/retry-policy/downgrading-consistency.md index 711329ec80..cfe230641c 100644 --- a/docs/source/retry-policy/downgrading-consistency.md +++ b/docs/source/retry-policy/downgrading-consistency.md @@ -69,13 +69,13 @@ let session: Session = SessionBuilder::new() # } ``` -To use in a [simple query](../queries/simple.md): +To use in an [unprepared statement](../queries/simple.md): ```rust # extern crate scylla; # use scylla::Session; # use std::error::Error; # async fn check_only_compiles(session: &Session) -> Result<(), Box> { -use scylla::query::Query; +use scylla::unprepared_statement::UnpreparedStatement; use scylla::transport::ExecutionProfile; use scylla::transport::downgrading_consistency_retry_policy::DowngradingConsistencyRetryPolicy; @@ -84,8 +84,9 @@ let handle = ExecutionProfile::builder() .build() .into_handle(); -// Create a Query manually and set the retry policy -let mut my_query: Query = Query::new("INSERT INTO ks.tab (a) VALUES(?)"); +// Create an UnpreparedStatement manually and set the retry policy +let mut my_query: UnpreparedStatement = + UnpreparedStatement::new("INSERT INTO ks.tab (a) VALUES(?)"); my_query.set_execution_profile_handle(Some(handle)); // Run the query using this retry policy diff --git a/docs/source/retry-policy/fallthrough.md b/docs/source/retry-policy/fallthrough.md index 0f6ab59388..f4a32427fa 100644 --- a/docs/source/retry-policy/fallthrough.md +++ b/docs/source/retry-policy/fallthrough.md @@ -27,13 +27,13 @@ let session: Session = SessionBuilder::new() # } ``` -To use in a [simple query](../queries/simple.md): +To use in an [unprepared statement](../queries/simple.md): ```rust # extern crate scylla; # use scylla::Session; # use std::error::Error; # async fn check_only_compiles(session: &Session) -> Result<(), Box> { -use scylla::query::Query; +use scylla::unprepared_statement::UnpreparedStatement; use scylla::transport::ExecutionProfile; use scylla::transport::retry_policy::FallthroughRetryPolicy; @@ -42,8 +42,9 @@ let handle = ExecutionProfile::builder() .build() .into_handle(); -// Create a Query manually and set the retry policy -let mut my_query: Query = Query::new("INSERT INTO ks.tab (a) VALUES(?)"); +// Create an UnpreparedStatement manually and set the retry policy +let mut my_query: UnpreparedStatement = + UnpreparedStatement::new("INSERT INTO ks.tab (a) VALUES(?)"); my_query.set_execution_profile_handle(Some(handle)); // Run the query using this retry policy diff --git a/docs/source/retry-policy/retry-policy.md b/docs/source/retry-policy/retry-policy.md index b4859b0c89..d0e4619a94 100644 --- a/docs/source/retry-policy/retry-policy.md +++ b/docs/source/retry-policy/retry-policy.md @@ -24,11 +24,12 @@ Idempotence has to be specified manually, the driver is not able to figure it ou # use scylla::Session; # use std::error::Error; # async fn check_only_compiles(session: &Session) -> Result<(), Box> { -use scylla::query::Query; +use scylla::unprepared_statement::UnpreparedStatement; use scylla::prepared_statement::PreparedStatement; // Specify that a Query is idempotent -let mut my_query: Query = Query::new("SELECT a FROM ks.tab"); +let mut my_query: UnpreparedStatement = + UnpreparedStatement::new("SELECT a FROM ks.tab"); my_query.set_is_idempotent(true); diff --git a/docs/source/tracing/basic.md b/docs/source/tracing/basic.md index 4ee5bc5737..3ef504a639 100644 --- a/docs/source/tracing/basic.md +++ b/docs/source/tracing/basic.md @@ -1,22 +1,23 @@ -# Tracing a simple/prepared/batch query +# Tracing an unprepared/prepared/batch statement execution -[Simple query](../queries/simple.md), [prepared query](../queries/prepared.md) and [batch query](../queries/batch.md) +[Unprepared statement](../queries/simple.md), [prepared statement](../queries/prepared.md) and [batch statement](../queries/batch.md) return a `QueryResult` which contains a `tracing_id` if tracing was enabled. -### Tracing a simple query +### Tracing an unprepared statement execution ```rust # extern crate scylla; # extern crate uuid; # use scylla::Session; # use std::error::Error; # async fn check_only_compiles(session: &Session) -> Result<(), Box> { -use scylla::query::Query; +use scylla::unprepared_statement::UnpreparedStatement; use scylla::QueryResult; use scylla::tracing::TracingInfo; use uuid::Uuid; -// Create a Query manually and enable tracing -let mut query: Query = Query::new("INSERT INTO ks.tab (a) VALUES(4)"); +// Create an UnpreparedStatement manually and enable tracing +let mut query: UnpreparedStatement = + UnpreparedStatement::new("INSERT INTO ks.tab (a) VALUES(4)"); query.set_tracing(true); let res: QueryResult = session.query(query, &[]).await?; @@ -31,7 +32,7 @@ if let Some(id) = tracing_id { # } ``` -### Tracing a prepared query +### Tracing a prepared statement execution ```rust # extern crate scylla; # extern crate uuid; @@ -43,12 +44,12 @@ use scylla::QueryResult; use scylla::tracing::TracingInfo; use uuid::Uuid; -// Prepare the query +// Prepare the statement let mut prepared: PreparedStatement = session .prepare("SELECT a FROM ks.tab") .await?; -// Enable tracing for the prepared query +// Enable tracing for the prepared statement prepared.set_tracing(true); let res: QueryResult = session.execute(&prepared, &[]).await?; diff --git a/docs/source/tracing/paged.md b/docs/source/tracing/paged.md index e69d4f3361..16341928d8 100644 --- a/docs/source/tracing/paged.md +++ b/docs/source/tracing/paged.md @@ -1,7 +1,7 @@ # Tracing a paged query -A paged query performs multiple simple/prepared queries to query subsequent pages.\ -If tracing is enabled the row iterator will contain a list of tracing ids for all performed queries. +A paged query performs multiple unprepared/prepared statements to query subsequent pages.\ +If tracing is enabled the row iterator will contain a list of tracing ids for all executed statements. ### Tracing `Session::query_iter` @@ -12,14 +12,15 @@ If tracing is enabled the row iterator will contain a list of tracing ids for al # use scylla::Session; # use std::error::Error; # async fn check_only_compiles(session: &Session) -> Result<(), Box> { -use scylla::query::Query; +use scylla::unprepared_statement::UnpreparedStatement; use scylla::transport::iterator::RowIterator; use scylla::tracing::TracingInfo; use futures::StreamExt; use uuid::Uuid; // Create a Query manually and enable tracing -let mut query: Query = Query::new("INSERT INTO ks.tab (a) VALUES(4)"); +let mut query: UnpreparedStatement = + UnpreparedStatement::new("INSERT INTO ks.tab (a) VALUES(4)"); query.set_tracing(true); // Create a paged query iterator and fetch pages diff --git a/docs/source/tracing/prepare.md b/docs/source/tracing/prepare.md index 2f3850cbde..e714606e29 100644 --- a/docs/source/tracing/prepare.md +++ b/docs/source/tracing/prepare.md @@ -8,13 +8,14 @@ # use scylla::Session; # use std::error::Error; # async fn check_only_compiles(session: &Session) -> Result<(), Box> { -use scylla::query::Query; +use scylla::unprepared_statement::UnpreparedStatement; use scylla::prepared_statement::PreparedStatement; use scylla::tracing::TracingInfo; use uuid::Uuid; // Prepare the query with tracing enabled -let mut to_prepare: Query = Query::new("SELECT a FROM ks.tab"); +let mut to_prepare: UnpreparedStatement = + UnpreparedStatement::new("SELECT a FROM ks.tab"); to_prepare.set_tracing(true); let mut prepared: PreparedStatement = session diff --git a/docs/source/tracing/query-history.md b/docs/source/tracing/query-history.md index 1c0779259e..a776ab1045 100644 --- a/docs/source/tracing/query-history.md +++ b/docs/source/tracing/query-history.md @@ -11,12 +11,13 @@ This history includes all requests sent, decisions to retry and speculative exec # use scylla::Session; # use std::error::Error; # async fn check_only_compiles(session: &Session) -> Result<(), Box> { -use scylla::query::Query; +use scylla::unprepared_statement::UnpreparedStatement; use scylla::history::{HistoryCollector, StructuredHistory}; use std::sync::Arc; -// Create a query for which we would like to trace the history of its execution -let mut query: Query = Query::new("SELECT * FROM ks.t"); +// Create an UnpreparedStatement for which we would like to trace the history of its execution +let mut query: UnpreparedStatement = + UnpreparedStatement::new("SELECT * FROM ks.t"); // Create a history collector and pass it to the query let history_listener = Arc::new(HistoryCollector::new());