From cddc8f66a7b51d37d99ac3e24257f1d2029d9784 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Karol=20Bary=C5=82a?= Date: Wed, 24 Apr 2024 01:57:42 +0200 Subject: [PATCH 01/16] Make TableSpec lifetime generic This is necessary to avoid unnecessary allocations in some APIs --- scylla-cql/src/frame/response/result.rs | 52 +++++++++++++++++----- scylla-cql/src/frame/value_tests.rs | 5 +-- scylla-cql/src/types/serialize/row.rs | 10 +---- scylla/src/statement/prepared_statement.rs | 9 ++-- scylla/src/transport/query_result.rs | 5 +-- scylla/src/transport/session_test.rs | 4 +- 6 files changed, 51 insertions(+), 34 deletions(-) diff --git a/scylla-cql/src/frame/response/result.rs b/scylla-cql/src/frame/response/result.rs index 75c7a15058..995e55d306 100644 --- a/scylla-cql/src/frame/response/result.rs +++ b/scylla-cql/src/frame/response/result.rs @@ -7,6 +7,7 @@ use crate::frame::value::{ use crate::frame::{frame_errors::ParseError, types}; use byteorder::{BigEndian, ReadBytesExt}; use bytes::{Buf, Bytes}; +use std::borrow::Cow; use std::{ convert::{TryFrom, TryInto}, net::IpAddr, @@ -36,9 +37,9 @@ pub struct SchemaChange { } #[derive(Debug, Clone, PartialEq, Eq)] -pub struct TableSpec { - pub ks_name: String, - pub table_name: String, +pub struct TableSpec<'a> { + ks_name: Cow<'a, str>, + table_name: Cow<'a, str>, } #[derive(Clone, Debug, PartialEq, Eq)] @@ -116,6 +117,40 @@ pub enum CqlValue { Varint(CqlVarint), } +impl<'a> TableSpec<'a> { + pub const fn borrowed(ks: &'a str, table: &'a str) -> Self { + Self { + ks_name: Cow::Borrowed(ks), + table_name: Cow::Borrowed(table), + } + } + + pub fn ks_name(&'a self) -> &'a str { + self.ks_name.as_ref() + } + + pub fn table_name(&'a self) -> &'a str { + self.table_name.as_ref() + } + + pub fn into_owned(self) -> TableSpec<'static> { + TableSpec::owned(self.ks_name.into_owned(), self.table_name.into_owned()) + } + + pub fn to_owned(&self) -> TableSpec<'static> { + TableSpec::owned(self.ks_name().to_owned(), self.table_name().to_owned()) + } +} + +impl TableSpec<'static> { + pub fn owned(ks_name: String, table_name: String) -> Self { + Self { + ks_name: Cow::Owned(ks_name), + table_name: Cow::Owned(table_name), + } + } +} + impl ColumnType { // Returns true if the type allows a special, empty value in addition to its // natural representation. For example, bigint represents a 32-bit integer, @@ -381,7 +416,7 @@ impl CqlValue { #[derive(Debug, Clone, PartialEq, Eq)] pub struct ColumnSpec { - pub table_spec: TableSpec, + pub table_spec: TableSpec<'static>, pub name: String, pub typ: ColumnType, } @@ -441,13 +476,10 @@ pub enum Result { SchemaChange(SchemaChange), } -fn deser_table_spec(buf: &mut &[u8]) -> StdResult { +fn deser_table_spec(buf: &mut &[u8]) -> StdResult, ParseError> { let ks_name = types::read_string(buf)?.to_owned(); let table_name = types::read_string(buf)?.to_owned(); - Ok(TableSpec { - ks_name, - table_name, - }) + Ok(TableSpec::owned(ks_name, table_name)) } fn deser_type(buf: &mut &[u8]) -> StdResult { @@ -521,7 +553,7 @@ fn deser_type(buf: &mut &[u8]) -> StdResult { fn deser_col_specs( buf: &mut &[u8], - global_table_spec: &Option, + global_table_spec: &Option>, col_count: usize, ) -> StdResult, ParseError> { let mut col_specs = Vec::with_capacity(col_count); diff --git a/scylla-cql/src/frame/value_tests.rs b/scylla-cql/src/frame/value_tests.rs index c11f2d74bd..f431aaacce 100644 --- a/scylla-cql/src/frame/value_tests.rs +++ b/scylla-cql/src/frame/value_tests.rs @@ -1077,10 +1077,7 @@ fn vec_value_list() { fn col_spec(name: &str, typ: ColumnType) -> ColumnSpec { ColumnSpec { - table_spec: TableSpec { - ks_name: "ks".to_string(), - table_name: "tbl".to_string(), - }, + table_spec: TableSpec::owned("ks".to_string(), "tbl".to_string()), name: name.to_string(), typ, } diff --git a/scylla-cql/src/types/serialize/row.rs b/scylla-cql/src/types/serialize/row.rs index 6c485ed147..f4c107bad3 100644 --- a/scylla-cql/src/types/serialize/row.rs +++ b/scylla-cql/src/types/serialize/row.rs @@ -869,10 +869,7 @@ mod tests { fn col_spec(name: &str, typ: ColumnType) -> ColumnSpec { ColumnSpec { - table_spec: TableSpec { - ks_name: "ks".to_string(), - table_name: "tbl".to_string(), - }, + table_spec: TableSpec::owned("ks".to_string(), "tbl".to_string()), name: name.to_string(), typ, } @@ -994,10 +991,7 @@ mod tests { fn col(name: &str, typ: ColumnType) -> ColumnSpec { ColumnSpec { - table_spec: TableSpec { - ks_name: "ks".to_string(), - table_name: "tbl".to_string(), - }, + table_spec: TableSpec::owned("ks".to_string(), "tbl".to_string()), name: name.to_string(), typ, } diff --git a/scylla/src/statement/prepared_statement.rs b/scylla/src/statement/prepared_statement.rs index bc0cc1df8f..edee9c3fdd 100644 --- a/scylla/src/statement/prepared_statement.rs +++ b/scylla/src/statement/prepared_statement.rs @@ -213,7 +213,7 @@ impl PreparedStatement { self.get_prepared_metadata() .col_specs .first() - .map(|col_spec| col_spec.table_spec.ks_name.as_str()) + .map(|col_spec| col_spec.table_spec.ks_name()) } /// Returns the name of the table this statement is operating on. @@ -221,7 +221,7 @@ impl PreparedStatement { self.get_prepared_metadata() .col_specs .first() - .map(|col_spec| col_spec.table_spec.table_name.as_str()) + .map(|col_spec| col_spec.table_spec.table_name()) } /// Sets the consistency to be used when executing this statement. @@ -540,10 +540,7 @@ mod tests { cols: impl IntoIterator, idx: impl IntoIterator, ) -> PreparedMetadata { - let table_spec = TableSpec { - ks_name: "ks".to_owned(), - table_name: "t".to_owned(), - }; + let table_spec = TableSpec::owned("ks".to_owned(), "t".to_owned()); let col_specs: Vec<_> = cols .into_iter() .enumerate() diff --git a/scylla/src/transport/query_result.rs b/scylla/src/transport/query_result.rs index 4c0b6e0d27..98be873db4 100644 --- a/scylla/src/transport/query_result.rs +++ b/scylla/src/transport/query_result.rs @@ -293,10 +293,7 @@ mod tests { } fn make_not_rows_query_result() -> QueryResult { - let table_spec = TableSpec { - ks_name: "some_keyspace".to_string(), - table_name: "some_table".to_string(), - }; + let table_spec = TableSpec::owned("some_keyspace".to_string(), "some_table".to_string()); let column_spec = ColumnSpec { table_spec, diff --git a/scylla/src/transport/session_test.rs b/scylla/src/transport/session_test.rs index a00c4f8832..0f7b8ea908 100644 --- a/scylla/src/transport/session_test.rs +++ b/scylla/src/transport/session_test.rs @@ -141,7 +141,7 @@ async fn test_unprepared_statement() { assert_eq!(specs.len(), 3); for (spec, name) in specs.iter().zip(["a", "b", "c"]) { assert_eq!(spec.name, name); // Check column name. - assert_eq!(spec.table_spec.ks_name, ks); + 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); @@ -197,7 +197,7 @@ async fn test_prepared_statement() { assert_eq!(specs.len(), 3); for (spec, name) in specs.iter().zip(["a", "b", "c"]) { assert_eq!(spec.name, name); // Check column name. - assert_eq!(spec.table_spec.ks_name, ks); + assert_eq!(spec.table_spec.ks_name(), ks); } let prepared_statement = session From 0a4420c2da9ce95b73dc4388d7cc211b4f011ab0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Karol=20Bary=C5=82a?= Date: Wed, 24 Apr 2024 01:45:59 +0200 Subject: [PATCH 02/16] Pass keyspace and table down to replica locator No matter what implementation of Tablets is going to be chosen, driver will need to know which table the query uses in order to be able to handle it properly. This commit makes ReplicaLocator receive table name alongside keyspace. --- examples/compare-tokens.rs | 2 +- scylla-cql/src/frame/response/result.rs | 2 +- scylla/src/statement/prepared_statement.rs | 13 +- scylla/src/transport/cluster.rs | 23 +-- scylla/src/transport/iterator.rs | 9 +- .../src/transport/load_balancing/default.rs | 139 ++++++++++-------- scylla/src/transport/load_balancing/mod.rs | 11 +- scylla/src/transport/locator/mod.rs | 14 +- scylla/src/transport/locator/test.rs | 36 ++++- scylla/src/transport/session.rs | 27 ++-- scylla/tests/integration/consistency.rs | 7 +- 11 files changed, 184 insertions(+), 99 deletions(-) diff --git a/examples/compare-tokens.rs b/examples/compare-tokens.rs index d0d6bf2975..bafeb73bdf 100644 --- a/examples/compare-tokens.rs +++ b/examples/compare-tokens.rs @@ -39,7 +39,7 @@ async fn main() -> Result<()> { "Token endpoints for query: {:?}", session .get_cluster_data() - .get_token_endpoints("examples_ks", Token::new(t)) + .get_token_endpoints("examples_ks", "compare_tokens", Token::new(t)) .iter() .map(|(node, _shard)| node.address) .collect::>() diff --git a/scylla-cql/src/frame/response/result.rs b/scylla-cql/src/frame/response/result.rs index 995e55d306..527d481eb2 100644 --- a/scylla-cql/src/frame/response/result.rs +++ b/scylla-cql/src/frame/response/result.rs @@ -36,7 +36,7 @@ pub struct SchemaChange { pub event: SchemaChangeEvent, } -#[derive(Debug, Clone, PartialEq, Eq)] +#[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct TableSpec<'a> { ks_name: Cow<'a, str>, table_name: Cow<'a, str>, diff --git a/scylla/src/statement/prepared_statement.rs b/scylla/src/statement/prepared_statement.rs index edee9c3fdd..c1a0b5efdb 100644 --- a/scylla/src/statement/prepared_statement.rs +++ b/scylla/src/statement/prepared_statement.rs @@ -1,5 +1,8 @@ use bytes::{Bytes, BytesMut}; use scylla_cql::errors::{BadQuery, QueryError}; +use scylla_cql::frame::response::result::{ + ColumnSpec, PartitionKeyIndex, ResultMetadata, TableSpec, +}; use scylla_cql::frame::types::RawValue; use scylla_cql::types::serialize::row::{RowSerializationContext, SerializeRow, SerializedValues}; use scylla_cql::types::serialize::SerializationError; @@ -10,8 +13,6 @@ use std::time::Duration; use thiserror::Error; use uuid::Uuid; -use scylla_cql::frame::response::result::{ColumnSpec, PartitionKeyIndex, ResultMetadata}; - use super::StatementConfig; use crate::frame::response::result::PreparedMetadata; use crate::frame::types::{Consistency, SerialConsistency}; @@ -208,6 +209,14 @@ impl PreparedStatement { .map(|opt| opt.map(|(_pk, token)| token)) } + /// Return keyspace name and table name this statement is operating on. + pub fn get_table_spec(&self) -> Option<&TableSpec> { + self.get_prepared_metadata() + .col_specs + .first() + .map(|spec| &spec.table_spec) + } + /// Returns the name of the keyspace this statement is operating on. pub fn get_keyspace_name(&self) -> Option<&str> { self.get_prepared_metadata() diff --git a/scylla/src/transport/cluster.rs b/scylla/src/transport/cluster.rs index 6985498885..e98b5365f7 100644 --- a/scylla/src/transport/cluster.rs +++ b/scylla/src/transport/cluster.rs @@ -17,6 +17,7 @@ use futures::future::join_all; use futures::{future::RemoteHandle, FutureExt}; use itertools::Itertools; use scylla_cql::errors::{BadQuery, NewSessionError}; +use scylla_cql::frame::response::result::TableSpec; use scylla_cql::types::serialize::row::SerializedValues; use std::collections::HashMap; use std::net::SocketAddr; @@ -409,24 +410,30 @@ impl ClusterData { } /// Access to replicas owning a given token - pub fn get_token_endpoints(&self, keyspace: &str, token: Token) -> Vec<(Arc, Shard)> { - self.get_token_endpoints_iter(keyspace, token) + pub fn get_token_endpoints( + &self, + keyspace: &str, + table: &str, + token: Token, + ) -> Vec<(Arc, Shard)> { + let table_spec = TableSpec::borrowed(keyspace, table); + self.get_token_endpoints_iter(&table_spec, token) .map(|(node, shard)| (node.clone(), shard)) .collect() } pub(crate) fn get_token_endpoints_iter( &self, - keyspace: &str, + table_spec: &TableSpec, token: Token, ) -> impl Iterator, Shard)> { - let keyspace = self.keyspaces.get(keyspace); + let keyspace = self.keyspaces.get(table_spec.ks_name()); let strategy = keyspace .map(|k| &k.strategy) .unwrap_or(&Strategy::LocalStrategy); let replica_set = self .replica_locator() - .replicas_for_token(token, strategy, None); + .replicas_for_token(token, strategy, None, table_spec); replica_set.into_iter() } @@ -438,10 +445,8 @@ impl ClusterData { table: &str, partition_key: &SerializedValues, ) -> Result, Shard)>, BadQuery> { - Ok(self.get_token_endpoints( - keyspace, - self.compute_token(keyspace, table, partition_key)?, - )) + let token = self.compute_token(keyspace, table, partition_key)?; + Ok(self.get_token_endpoints(keyspace, table, token)) } /// Access replica location info diff --git a/scylla/src/transport/iterator.rs b/scylla/src/transport/iterator.rs index f277f5b646..1167d6bdbb 100644 --- a/scylla/src/transport/iterator.rs +++ b/scylla/src/transport/iterator.rs @@ -248,11 +248,12 @@ impl RowIterator { } }; + let table_spec = config.prepared.get_table_spec(); let statement_info = RoutingInfo { consistency, serial_consistency, token, - keyspace: config.prepared.get_keyspace_name(), + table: table_spec, is_confirmed_lwt: config.prepared.is_confirmed_lwt(), }; @@ -273,13 +274,13 @@ impl RowIterator { let serialized_values_size = config.values.buffer_size(); let replicas: Option> = - if let (Some(keyspace), Some(token)) = - (statement_info.keyspace.as_ref(), statement_info.token) + if let (Some(table_spec), Some(token)) = + (statement_info.table, statement_info.token) { Some( config .cluster_data - .get_token_endpoints_iter(keyspace, token) + .get_token_endpoints_iter(table_spec, token) .map(|(node, shard)| (node.clone(), shard)) .collect(), ) diff --git a/scylla/src/transport/load_balancing/default.rs b/scylla/src/transport/load_balancing/default.rs index 1df7328ebb..889ef701aa 100644 --- a/scylla/src/transport/load_balancing/default.rs +++ b/scylla/src/transport/load_balancing/default.rs @@ -9,7 +9,10 @@ use crate::{ use itertools::{Either, Itertools}; use rand::{prelude::SliceRandom, thread_rng, Rng}; use rand_pcg::Pcg32; -use scylla_cql::{errors::QueryError, frame::types::SerialConsistency, Consistency}; +use scylla_cql::errors::QueryError; +use scylla_cql::frame::response::result::TableSpec; +use scylla_cql::frame::types::SerialConsistency; +use scylla_cql::Consistency; use std::hash::{Hash, Hasher}; use std::{fmt, sync::Arc, time::Duration}; use tracing::{debug, warn}; @@ -122,7 +125,7 @@ or refrain from preferring datacenters (which may ban all other datacenters, if } else { StatementType::NonLwt }; - if let Some(ts) = &routing_info.token_with_strategy { + if let (Some(ts), Some(table_spec)) = (&routing_info.token_with_strategy, query.table) { if let NodeLocationPreference::DatacenterAndRack(dc, rack) = &self.preferences { // Try to pick some alive local rack random replica. let local_rack_picked = self.pick_replica( @@ -131,6 +134,7 @@ or refrain from preferring datacenters (which may ban all other datacenters, if |node, shard| (self.pick_predicate)(node, Some(shard)), cluster, statement_type, + table_spec, ); if let Some((alive_local_rack_replica, shard)) = local_rack_picked { @@ -148,6 +152,7 @@ or refrain from preferring datacenters (which may ban all other datacenters, if |node, shard| (self.pick_predicate)(node, Some(shard)), cluster, statement_type, + table_spec, ); if let Some((alive_local_replica, shard)) = picked { @@ -166,6 +171,7 @@ or refrain from preferring datacenters (which may ban all other datacenters, if |node, shard| (self.pick_predicate)(node, Some(shard)), cluster, statement_type, + table_spec, ); if let Some((alive_remote_replica, shard)) = picked { return Some((alive_remote_replica, Some(shard))); @@ -237,7 +243,9 @@ or refrain from preferring datacenters (which may ban all other datacenters, if }; // If token is available, get a shuffled list of alive replicas. - let maybe_replicas = if let Some(ts) = &routing_info.token_with_strategy { + let maybe_replicas = if let (Some(ts), Some(table_spec)) = + (&routing_info.token_with_strategy, query.table) + { let maybe_local_rack_replicas = if let NodeLocationPreference::DatacenterAndRack(dc, rack) = &self.preferences { let local_rack_replicas = self.fallback_replicas( @@ -246,6 +254,7 @@ or refrain from preferring datacenters (which may ban all other datacenters, if |node, shard| Self::is_alive(node, Some(shard)), cluster, statement_type, + table_spec, ); Either::Left(local_rack_replicas) } else { @@ -262,6 +271,7 @@ or refrain from preferring datacenters (which may ban all other datacenters, if |node, shard| Self::is_alive(node, Some(shard)), cluster, statement_type, + table_spec, ); Either::Left(local_replicas) } else { @@ -278,6 +288,7 @@ or refrain from preferring datacenters (which may ban all other datacenters, if |node, shard| Self::is_alive(node, Some(shard)), cluster, statement_type, + table_spec, ); Either::Left(remote_replicas) } else { @@ -459,12 +470,13 @@ impl DefaultPolicy { ts: &TokenWithStrategy<'a>, replica_location: NodeLocationCriteria<'a>, cluster: &'a ClusterData, + table_spec: &TableSpec, ) -> ReplicaSet<'a> { let datacenter = replica_location.datacenter(); cluster .replica_locator() - .replicas_for_token(ts.token, ts.strategy, datacenter) + .replicas_for_token(ts.token, ts.strategy, datacenter, table_spec) } /// Wraps the provided predicate, adding the requirement for rack to match. @@ -502,16 +514,17 @@ impl DefaultPolicy { predicate: impl Fn(NodeRef<'a>, Shard) -> bool + 'a, cluster: &'a ClusterData, order: ReplicaOrder, + table_spec: &TableSpec, ) -> impl Iterator, Shard)> { let predicate = Self::make_sharded_rack_predicate(predicate, replica_location); let replica_iter = match order { ReplicaOrder::Arbitrary => Either::Left( - self.nonfiltered_replica_set(ts, replica_location, cluster) + self.nonfiltered_replica_set(ts, replica_location, cluster, table_spec) .into_iter(), ), ReplicaOrder::RingOrder => Either::Right( - self.nonfiltered_replica_set(ts, replica_location, cluster) + self.nonfiltered_replica_set(ts, replica_location, cluster, table_spec) .into_replicas_ordered() .into_iter(), ), @@ -526,11 +539,14 @@ impl DefaultPolicy { predicate: impl Fn(NodeRef<'a>, Shard) -> bool + 'a, cluster: &'a ClusterData, statement_type: StatementType, + table_spec: &TableSpec, ) -> Option<(NodeRef<'a>, Shard)> { match statement_type { - StatementType::Lwt => self.pick_first_replica(ts, replica_location, predicate, cluster), + StatementType::Lwt => { + self.pick_first_replica(ts, replica_location, predicate, cluster, table_spec) + } StatementType::NonLwt => { - self.pick_random_replica(ts, replica_location, predicate, cluster) + self.pick_random_replica(ts, replica_location, predicate, cluster, table_spec) } } } @@ -553,6 +569,7 @@ impl DefaultPolicy { replica_location: NodeLocationCriteria<'a>, predicate: impl Fn(NodeRef<'a>, Shard) -> bool + 'a, cluster: &'a ClusterData, + table_spec: &TableSpec, ) -> Option<(NodeRef<'a>, Shard)> { match replica_location { NodeLocationCriteria::Any => { @@ -566,7 +583,7 @@ impl DefaultPolicy { // (computation of the remaining ones is expensive), in case that the primary replica // does not satisfy the `predicate`, None is returned. All expensive computation // is to be done only when `fallback()` is called. - self.nonfiltered_replica_set(ts, replica_location, cluster) + self.nonfiltered_replica_set(ts, replica_location, cluster, table_spec) .into_replicas_ordered() .into_iter() .next() @@ -586,6 +603,7 @@ impl DefaultPolicy { predicate, cluster, ReplicaOrder::RingOrder, + table_spec, ) .next() } @@ -598,10 +616,11 @@ impl DefaultPolicy { replica_location: NodeLocationCriteria<'a>, predicate: impl Fn(NodeRef<'a>, Shard) -> bool + 'a, cluster: &'a ClusterData, + table_spec: &TableSpec, ) -> Option<(NodeRef<'a>, Shard)> { let predicate = Self::make_sharded_rack_predicate(predicate, replica_location); - let replica_set = self.nonfiltered_replica_set(ts, replica_location, cluster); + let replica_set = self.nonfiltered_replica_set(ts, replica_location, cluster, table_spec); if let Some(fixed) = self.fixed_seed { let mut gen = Pcg32::new(fixed, 0); @@ -618,13 +637,14 @@ impl DefaultPolicy { predicate: impl Fn(NodeRef<'_>, Shard) -> bool + 'a, cluster: &'a ClusterData, statement_type: StatementType, + table_spec: &TableSpec, ) -> impl Iterator, Shard)> { let order = match statement_type { StatementType::Lwt => ReplicaOrder::RingOrder, StatementType::NonLwt => ReplicaOrder::Arbitrary, }; - let replicas = self.replicas(ts, replica_location, predicate, cluster, order); + let replicas = self.replicas(ts, replica_location, predicate, cluster, order, table_spec); match statement_type { // As an LWT optimisation: in order to reduce contention caused by Paxos conflicts, @@ -930,7 +950,7 @@ struct TokenWithStrategy<'a> { impl<'a> TokenWithStrategy<'a> { fn new(query: &'a RoutingInfo, cluster: &'a ClusterData) -> Option> { let token = query.token?; - let keyspace_name = query.keyspace?; + let keyspace_name = query.table?.ks_name(); let keyspace = cluster.get_keyspace_info().get(keyspace_name)?; let strategy = &keyspace.strategy; Some(TokenWithStrategy { strategy, token }) @@ -946,16 +966,14 @@ mod tests { get_plan_and_collect_node_identifiers, mock_cluster_data_for_token_unaware_tests, ExpectedGroups, ExpectedGroupsBuilder, }; + use crate::transport::locator::test::{TABLE_NTS_RF_2, TABLE_NTS_RF_3, TABLE_SS_RF_2}; use crate::{ load_balancing::{ default::tests::framework::mock_cluster_data_for_token_aware_tests, Plan, RoutingInfo, }, routing::Token, test_utils::setup_tracing, - transport::{ - locator::test::{KEYSPACE_NTS_RF_2, KEYSPACE_NTS_RF_3, KEYSPACE_SS_RF_2}, - ClusterData, - }, + transport::ClusterData, }; use super::{DefaultPolicy, NodeLocationPreference}; @@ -1209,7 +1227,7 @@ mod tests { pub(crate) const EMPTY_ROUTING_INFO: RoutingInfo = RoutingInfo { token: None, - keyspace: None, + table: None, is_confirmed_lwt: false, consistency: Consistency::Quorum, serial_consistency: Some(SerialConsistency::Serial), @@ -1316,7 +1334,7 @@ mod tests { }, routing_info: RoutingInfo { token: Some(Token::new(160)), - keyspace: Some(KEYSPACE_NTS_RF_2), + table: Some(TABLE_NTS_RF_2), consistency: Consistency::Two, ..Default::default() }, @@ -1341,7 +1359,7 @@ mod tests { }, routing_info: RoutingInfo { token: Some(Token::new(160)), - keyspace: Some(KEYSPACE_NTS_RF_2), + table: Some(TABLE_NTS_RF_2), consistency: Consistency::Two, ..Default::default() }, @@ -1365,7 +1383,7 @@ mod tests { }, routing_info: RoutingInfo { token: Some(Token::new(160)), - keyspace: Some(KEYSPACE_NTS_RF_2), + table: Some(TABLE_NTS_RF_2), consistency: Consistency::LocalOne, // local Consistency forbids datacenter failover ..Default::default() }, @@ -1387,7 +1405,7 @@ mod tests { }, routing_info: RoutingInfo { token: Some(Token::new(160)), - keyspace: Some(KEYSPACE_NTS_RF_2), + table: Some(TABLE_NTS_RF_2), consistency: Consistency::One, ..Default::default() }, @@ -1409,7 +1427,7 @@ mod tests { }, routing_info: RoutingInfo { token: Some(Token::new(160)), - keyspace: Some(KEYSPACE_NTS_RF_3), + table: Some(TABLE_NTS_RF_3), consistency: Consistency::Quorum, ..Default::default() }, @@ -1434,7 +1452,7 @@ mod tests { }, routing_info: RoutingInfo { token: Some(Token::new(160)), - keyspace: Some(KEYSPACE_NTS_RF_3), + table: Some(TABLE_NTS_RF_3), consistency: Consistency::Quorum, ..Default::default() }, @@ -1458,7 +1476,7 @@ mod tests { }, routing_info: RoutingInfo { token: Some(Token::new(160)), - keyspace: Some(KEYSPACE_NTS_RF_3), + table: Some(TABLE_NTS_RF_3), consistency: Consistency::Quorum, ..Default::default() }, @@ -1480,7 +1498,7 @@ mod tests { }, routing_info: RoutingInfo { token: Some(Token::new(160)), - keyspace: Some(KEYSPACE_SS_RF_2), + table: Some(TABLE_SS_RF_2), consistency: Consistency::Two, ..Default::default() }, @@ -1504,7 +1522,7 @@ mod tests { }, routing_info: RoutingInfo { token: Some(Token::new(160)), - keyspace: Some(KEYSPACE_SS_RF_2), + table: Some(TABLE_SS_RF_2), consistency: Consistency::LocalOne, // local Consistency forbids datacenter failover ..Default::default() }, @@ -1526,7 +1544,7 @@ mod tests { }, routing_info: RoutingInfo { token: None, // no token - keyspace: Some(KEYSPACE_NTS_RF_3), + table: Some(TABLE_NTS_RF_3), consistency: Consistency::Quorum, ..Default::default() }, @@ -1545,7 +1563,7 @@ mod tests { }, routing_info: RoutingInfo { token: Some(Token::new(160)), - keyspace: None, // no keyspace + table: None, // no keyspace consistency: Consistency::Quorum, ..Default::default() }, @@ -1564,7 +1582,7 @@ mod tests { }, routing_info: RoutingInfo { token: Some(Token::new(160)), - keyspace: Some(KEYSPACE_NTS_RF_2), + table: Some(TABLE_NTS_RF_2), consistency: Consistency::Quorum, ..Default::default() }, @@ -1586,7 +1604,7 @@ mod tests { }, routing_info: RoutingInfo { token: Some(Token::new(160)), - keyspace: Some(KEYSPACE_NTS_RF_2), + table: Some(TABLE_NTS_RF_2), consistency: Consistency::Quorum, ..Default::default() }, @@ -1605,7 +1623,7 @@ mod tests { }, routing_info: RoutingInfo { token: Some(Token::new(160)), - keyspace: Some(KEYSPACE_NTS_RF_2), + table: Some(TABLE_NTS_RF_2), consistency: Consistency::Quorum, ..Default::default() }, @@ -1627,7 +1645,7 @@ mod tests { }, routing_info: RoutingInfo { token: Some(Token::new(160)), - keyspace: Some(KEYSPACE_NTS_RF_2), + table: Some(TABLE_NTS_RF_2), consistency: Consistency::Quorum, ..Default::default() }, @@ -1652,7 +1670,7 @@ mod tests { }, routing_info: RoutingInfo { token: Some(Token::new(160)), - keyspace: Some(KEYSPACE_NTS_RF_3), + table: Some(TABLE_NTS_RF_3), consistency: Consistency::One, ..Default::default() }, @@ -1680,7 +1698,7 @@ mod tests { }, routing_info: RoutingInfo { token: Some(Token::new(560)), - keyspace: Some(KEYSPACE_SS_RF_2), + table: Some(TABLE_SS_RF_2), consistency: Consistency::Two, ..Default::default() }, @@ -1707,7 +1725,7 @@ mod tests { }, routing_info: RoutingInfo { token: Some(Token::new(160)), - keyspace: Some(KEYSPACE_SS_RF_2), + table: Some(TABLE_SS_RF_2), consistency: Consistency::One, ..Default::default() }, @@ -1733,7 +1751,7 @@ mod tests { }, routing_info: RoutingInfo { token: None, - keyspace: Some(KEYSPACE_NTS_RF_3), + table: Some(TABLE_NTS_RF_3), consistency: Consistency::One, ..Default::default() }, @@ -1788,7 +1806,7 @@ mod tests { }, routing_info: RoutingInfo { token: Some(Token::new(160)), - keyspace: Some(KEYSPACE_NTS_RF_2), + table: Some(TABLE_NTS_RF_2), consistency: Consistency::Two, is_confirmed_lwt: true, ..Default::default() @@ -1814,7 +1832,7 @@ mod tests { }, routing_info: RoutingInfo { token: Some(Token::new(160)), - keyspace: Some(KEYSPACE_NTS_RF_2), + table: Some(TABLE_NTS_RF_2), consistency: Consistency::Two, is_confirmed_lwt: true, ..Default::default() @@ -1839,7 +1857,7 @@ mod tests { }, routing_info: RoutingInfo { token: Some(Token::new(160)), - keyspace: Some(KEYSPACE_NTS_RF_2), + table: Some(TABLE_NTS_RF_2), consistency: Consistency::LocalOne, // local Consistency forbids datacenter failover is_confirmed_lwt: true, ..Default::default() @@ -1862,7 +1880,7 @@ mod tests { }, routing_info: RoutingInfo { token: Some(Token::new(160)), - keyspace: Some(KEYSPACE_NTS_RF_2), + table: Some(TABLE_NTS_RF_2), consistency: Consistency::One, is_confirmed_lwt: true, ..Default::default() @@ -1885,7 +1903,7 @@ mod tests { }, routing_info: RoutingInfo { token: Some(Token::new(160)), - keyspace: Some(KEYSPACE_NTS_RF_3), + table: Some(TABLE_NTS_RF_3), consistency: Consistency::Quorum, is_confirmed_lwt: true, ..Default::default() @@ -1911,7 +1929,7 @@ mod tests { }, routing_info: RoutingInfo { token: Some(Token::new(160)), - keyspace: Some(KEYSPACE_NTS_RF_3), + table: Some(TABLE_NTS_RF_3), consistency: Consistency::Quorum, is_confirmed_lwt: true, ..Default::default() @@ -1936,7 +1954,7 @@ mod tests { }, routing_info: RoutingInfo { token: Some(Token::new(160)), - keyspace: Some(KEYSPACE_NTS_RF_3), + table: Some(TABLE_NTS_RF_3), consistency: Consistency::Quorum, is_confirmed_lwt: true, ..Default::default() @@ -1959,7 +1977,7 @@ mod tests { }, routing_info: RoutingInfo { token: Some(Token::new(160)), - keyspace: Some(KEYSPACE_SS_RF_2), + table: Some(TABLE_SS_RF_2), consistency: Consistency::Two, is_confirmed_lwt: true, ..Default::default() @@ -1984,7 +2002,7 @@ mod tests { }, routing_info: RoutingInfo { token: Some(Token::new(160)), - keyspace: Some(KEYSPACE_SS_RF_2), + table: Some(TABLE_SS_RF_2), consistency: Consistency::LocalOne, // local Consistency forbids datacenter failover is_confirmed_lwt: true, ..Default::default() @@ -2007,7 +2025,7 @@ mod tests { }, routing_info: RoutingInfo { token: None, // no token - keyspace: Some(KEYSPACE_NTS_RF_3), + table: Some(TABLE_NTS_RF_3), consistency: Consistency::Quorum, is_confirmed_lwt: true, ..Default::default() @@ -2027,7 +2045,7 @@ mod tests { }, routing_info: RoutingInfo { token: Some(Token::new(160)), - keyspace: None, // no keyspace + table: None, // no keyspace consistency: Consistency::Quorum, is_confirmed_lwt: true, ..Default::default() @@ -2047,7 +2065,7 @@ mod tests { }, routing_info: RoutingInfo { token: Some(Token::new(160)), - keyspace: Some(KEYSPACE_NTS_RF_2), + table: Some(TABLE_NTS_RF_2), consistency: Consistency::Quorum, is_confirmed_lwt: true, ..Default::default() @@ -2070,7 +2088,7 @@ mod tests { }, routing_info: RoutingInfo { token: Some(Token::new(160)), - keyspace: Some(KEYSPACE_NTS_RF_2), + table: Some(TABLE_NTS_RF_2), consistency: Consistency::Quorum, is_confirmed_lwt: true, ..Default::default() @@ -2090,7 +2108,7 @@ mod tests { }, routing_info: RoutingInfo { token: Some(Token::new(160)), - keyspace: Some(KEYSPACE_NTS_RF_2), + table: Some(TABLE_NTS_RF_2), consistency: Consistency::Quorum, is_confirmed_lwt: true, ..Default::default() @@ -2113,7 +2131,7 @@ mod tests { }, routing_info: RoutingInfo { token: Some(Token::new(160)), - keyspace: Some(KEYSPACE_NTS_RF_2), + table: Some(TABLE_NTS_RF_2), consistency: Consistency::Quorum, is_confirmed_lwt: true, ..Default::default() @@ -2139,7 +2157,7 @@ mod tests { }, routing_info: RoutingInfo { token: Some(Token::new(160)), - keyspace: Some(KEYSPACE_NTS_RF_3), + table: Some(TABLE_NTS_RF_3), consistency: Consistency::One, is_confirmed_lwt: true, ..Default::default() @@ -2168,7 +2186,7 @@ mod tests { }, routing_info: RoutingInfo { token: Some(Token::new(760)), - keyspace: Some(KEYSPACE_SS_RF_2), + table: Some(TABLE_SS_RF_2), consistency: Consistency::Two, is_confirmed_lwt: true, ..Default::default() @@ -2195,7 +2213,7 @@ mod tests { }, routing_info: RoutingInfo { token: Some(Token::new(160)), - keyspace: Some(KEYSPACE_SS_RF_2), + table: Some(TABLE_SS_RF_2), consistency: Consistency::One, is_confirmed_lwt: true, ..Default::default() @@ -2804,6 +2822,7 @@ mod latency_awareness { load_balancing::default::NodeLocationPreference, routing::Shard, test_utils::{create_new_session_builder, setup_tracing}, + transport::locator::test::{TABLE_INVALID, TABLE_NTS_RF_2, TABLE_NTS_RF_3}, }; use crate::{ load_balancing::{ @@ -2812,9 +2831,7 @@ mod latency_awareness { }, routing::Token, transport::{ - locator::test::{ - id_to_invalid_addr, A, B, C, D, E, F, G, KEYSPACE_NTS_RF_2, KEYSPACE_NTS_RF_3, - }, + locator::test::{id_to_invalid_addr, A, B, C, D, E, F, G}, ClusterData, NodeAddr, }, ExecutionProfile, @@ -3407,7 +3424,7 @@ mod latency_awareness { ], routing_info: RoutingInfo { token: Some(Token::new(160)), - keyspace: Some(KEYSPACE_NTS_RF_3), + table: Some(TABLE_NTS_RF_3), consistency: Consistency::Quorum, ..Default::default() }, @@ -3427,7 +3444,7 @@ mod latency_awareness { preset_min_avg: Some(100 * min_avg), routing_info: RoutingInfo { token: Some(Token::new(160)), - keyspace: Some(KEYSPACE_NTS_RF_3), + table: Some(TABLE_NTS_RF_3), consistency: Consistency::Quorum, ..Default::default() }, @@ -3456,7 +3473,7 @@ mod latency_awareness { ], routing_info: RoutingInfo { token: Some(Token::new(160)), - keyspace: Some(KEYSPACE_NTS_RF_2), + table: Some(TABLE_NTS_RF_2), consistency: Consistency::Quorum, ..Default::default() }, @@ -3475,7 +3492,7 @@ mod latency_awareness { preset_min_avg: None, routing_info: RoutingInfo { token: Some(Token::new(160)), - keyspace: Some("invalid"), + table: Some(TABLE_INVALID), consistency: Consistency::Quorum, ..Default::default() }, diff --git a/scylla/src/transport/load_balancing/mod.rs b/scylla/src/transport/load_balancing/mod.rs index f1cd5bdf27..691f8657ee 100644 --- a/scylla/src/transport/load_balancing/mod.rs +++ b/scylla/src/transport/load_balancing/mod.rs @@ -4,7 +4,10 @@ use super::{cluster::ClusterData, NodeRef}; use crate::routing::{Shard, Token}; -use scylla_cql::{errors::QueryError, frame::types}; +use scylla_cql::{ + errors::QueryError, + frame::{response::result::TableSpec, types}, +}; use std::time::Duration; @@ -22,9 +25,11 @@ pub struct RoutingInfo<'a> { pub consistency: types::Consistency, pub serial_consistency: Option, - /// Information about token and keyspace is the basis of token-aware routing. + /// Information that are the basis of token-aware routing: + /// - token, keyspace for vnodes-based routing; + /// - token, keyspace, table for tablets-based routing. pub token: Option, - pub keyspace: Option<&'a str>, + pub table: Option<&'a TableSpec<'a>>, /// If, while preparing, we received from the cluster information that the statement is an LWT, /// then we can use this information for routing optimisation. Namely, an optimisation diff --git a/scylla/src/transport/locator/mod.rs b/scylla/src/transport/locator/mod.rs index f87a32770c..b5594f1433 100644 --- a/scylla/src/transport/locator/mod.rs +++ b/scylla/src/transport/locator/mod.rs @@ -6,6 +6,7 @@ pub(crate) mod test; mod token_ring; use rand::{seq::IteratorRandom, Rng}; +use scylla_cql::frame::response::result::TableSpec; pub use token_ring::TokenRing; use super::{topology::Strategy, Node, NodeRef}; @@ -79,6 +80,7 @@ impl ReplicaLocator { token: Token, strategy: &'a Strategy, datacenter: Option<&'a str>, + _table_spec: &TableSpec, ) -> ReplicaSet<'a> { match strategy { Strategy::SimpleStrategy { replication_factor } => { @@ -143,6 +145,7 @@ impl ReplicaLocator { replication_factor: 1, }, datacenter, + _table_spec, ) } @@ -770,8 +773,9 @@ mod tests { // For each case (token, limit_to_dc, strategy), we are checking // that ReplicasOrdered yields replicas in the expected order. - let check = |token, limit_to_dc, strategy, expected| { - let replica_set = locator.replicas_for_token(Token::new(token), strategy, limit_to_dc); + let check = |token, limit_to_dc, strategy, table, expected| { + let replica_set = + locator.replicas_for_token(Token::new(token), strategy, limit_to_dc, table); let replicas_ordered = replica_set.into_replicas_ordered(); let ids: Vec<_> = replicas_ordered .into_iter() @@ -788,18 +792,21 @@ mod tests { 160, None, &metadata.keyspaces.get(KEYSPACE_NTS_RF_3).unwrap().strategy, + TABLE_NTS_RF_3, vec![F, A, C, D, G, E], ); check( 160, None, &metadata.keyspaces.get(KEYSPACE_NTS_RF_2).unwrap().strategy, + TABLE_NTS_RF_2, vec![F, A, D, G], ); check( 160, None, &metadata.keyspaces.get(KEYSPACE_SS_RF_2).unwrap().strategy, + TABLE_SS_RF_2, vec![F, A], ); @@ -807,18 +814,21 @@ mod tests { 160, Some("eu"), &metadata.keyspaces.get(KEYSPACE_NTS_RF_3).unwrap().strategy, + TABLE_NTS_RF_3, vec![A, C, G], ); check( 160, Some("us"), &metadata.keyspaces.get(KEYSPACE_NTS_RF_3).unwrap().strategy, + TABLE_NTS_RF_3, vec![F, D, E], ); check( 160, Some("eu"), &metadata.keyspaces.get(KEYSPACE_SS_RF_2).unwrap().strategy, + TABLE_SS_RF_2, vec![A], ); } diff --git a/scylla/src/transport/locator/test.rs b/scylla/src/transport/locator/test.rs index f93b2b7b8e..ca152c81f1 100644 --- a/scylla/src/transport/locator/test.rs +++ b/scylla/src/transport/locator/test.rs @@ -1,5 +1,6 @@ use rand::SeedableRng; use rand_chacha::ChaCha8Rng; +use scylla_cql::frame::response::result::TableSpec; use uuid::Uuid; use super::{ReplicaLocator, ReplicaSet}; @@ -23,6 +24,16 @@ pub(crate) const KEYSPACE_NTS_RF_2: &str = "keyspace_with_nts_rf_2"; pub(crate) const KEYSPACE_NTS_RF_3: &str = "keyspace_with_nts_rf_3"; pub(crate) const KEYSPACE_SS_RF_2: &str = "keyspace_with_ss_rf_2"; +// Those are references because otherwise I can't use them in Option without +// additional binding. +pub(crate) const TABLE_NTS_RF_2: &TableSpec<'static> = + &TableSpec::borrowed(KEYSPACE_NTS_RF_2, "table"); +pub(crate) const TABLE_NTS_RF_3: &TableSpec<'static> = + &TableSpec::borrowed(KEYSPACE_NTS_RF_3, "table"); +pub(crate) const TABLE_SS_RF_2: &TableSpec<'static> = + &TableSpec::borrowed(KEYSPACE_SS_RF_2, "table"); +pub(crate) const TABLE_INVALID: &TableSpec<'static> = &TableSpec::borrowed("invalid", "invalid"); + pub(crate) const A: u16 = 1; pub(crate) const B: u16 = 2; pub(crate) const C: u16 = 3; @@ -245,6 +256,7 @@ fn test_simple_strategy_replicas(locator: &ReplicaLocator) { replication_factor: 3, }, None, + TABLE_INVALID, ), &[F, G, D], ); @@ -256,6 +268,7 @@ fn test_simple_strategy_replicas(locator: &ReplicaLocator) { replication_factor: 4, }, None, + TABLE_INVALID, ), &[F, G, D, B], ); @@ -267,6 +280,7 @@ fn test_simple_strategy_replicas(locator: &ReplicaLocator) { replication_factor: 4, }, None, + TABLE_INVALID, ), &[A, C, D, F], ); @@ -278,6 +292,7 @@ fn test_simple_strategy_replicas(locator: &ReplicaLocator) { replication_factor: 0, }, None, + TABLE_INVALID, ), &[], ); @@ -291,6 +306,7 @@ fn test_simple_strategy_replicas(locator: &ReplicaLocator) { replication_factor: 1, }, Some("us"), + TABLE_INVALID, ), &[], ); @@ -302,6 +318,7 @@ fn test_simple_strategy_replicas(locator: &ReplicaLocator) { replication_factor: 3, }, Some("us"), + TABLE_INVALID, ), &[E], ); @@ -313,6 +330,7 @@ fn test_simple_strategy_replicas(locator: &ReplicaLocator) { replication_factor: 3, }, Some("eu"), + TABLE_INVALID, ), &[A, B], ); @@ -328,6 +346,7 @@ fn test_network_topology_strategy_replicas(locator: &ReplicaLocator) { .collect(), }, Some("eu"), + TABLE_INVALID, ), &[B], ); @@ -341,6 +360,7 @@ fn test_network_topology_strategy_replicas(locator: &ReplicaLocator) { .collect(), }, Some("us"), + TABLE_INVALID, ), &[E], ); @@ -354,6 +374,7 @@ fn test_network_topology_strategy_replicas(locator: &ReplicaLocator) { .collect(), }, None, + TABLE_INVALID, ), &[B, E], ); @@ -367,6 +388,7 @@ fn test_network_topology_strategy_replicas(locator: &ReplicaLocator) { .collect(), }, None, + TABLE_INVALID, ), // Walking the ring from token 75, [B E F A C D A F G] is encountered. // NTS takes the first 2 nodes from that list - {B, E} and the last one - G because it is @@ -383,6 +405,7 @@ fn test_network_topology_strategy_replicas(locator: &ReplicaLocator) { .collect(), }, None, + TABLE_INVALID, ), &[E], ); @@ -396,6 +419,7 @@ fn test_network_topology_strategy_replicas(locator: &ReplicaLocator) { .collect(), }, None, + TABLE_INVALID, ), &[G, E], ); @@ -411,6 +435,7 @@ fn test_replica_set_len(locator: &ReplicaLocator) { .collect(), }, None, + TABLE_INVALID, ) .len(); assert_eq!(merged_nts_len, 3); @@ -426,6 +451,7 @@ fn test_replica_set_len(locator: &ReplicaLocator) { .collect(), }, None, + TABLE_INVALID, ) .len(); assert_eq!(capped_merged_nts_len, 5); // 5 = all eu nodes + 1 us node = 4 + 1. @@ -439,6 +465,7 @@ fn test_replica_set_len(locator: &ReplicaLocator) { .collect(), }, Some("eu"), + TABLE_INVALID, ) .len(); assert_eq!(filtered_nts_len, 2); @@ -450,6 +477,7 @@ fn test_replica_set_len(locator: &ReplicaLocator) { replication_factor: 3, }, None, + TABLE_INVALID, ) .len(); assert_eq!(ss_len, 3); @@ -462,6 +490,7 @@ fn test_replica_set_len(locator: &ReplicaLocator) { replication_factor: 3, }, Some("eu"), + TABLE_INVALID, ) .len(); assert_eq!(filtered_ss_len, 1) @@ -482,7 +511,8 @@ fn test_replica_set_choose(locator: &ReplicaLocator) { let mut rng = ChaCha8Rng::seed_from_u64(69); for strategy in strategies { - let replica_set_generator = || locator.replicas_for_token(Token::new(75), &strategy, None); + let replica_set_generator = + || locator.replicas_for_token(Token::new(75), &strategy, None, TABLE_INVALID); // Verify that after a certain number of random selections, the set of selected replicas // will contain all nodes in the ring (replica set was created using a strategy with @@ -522,7 +552,8 @@ fn test_replica_set_choose_filtered(locator: &ReplicaLocator) { let mut rng = ChaCha8Rng::seed_from_u64(69); for strategy in strategies { - let replica_set_generator = || locator.replicas_for_token(Token::new(75), &strategy, None); + let replica_set_generator = + || locator.replicas_for_token(Token::new(75), &strategy, None, TABLE_INVALID); // Verify that after a certain number of random selections with a dc filter, the set of // selected replicas will contain all nodes in the specified dc ring. @@ -554,6 +585,7 @@ fn test_replica_set_choose_filtered(locator: &ReplicaLocator) { Token::new(75), &Strategy::LocalStrategy, Some("unknown_dc_name"), + TABLE_INVALID, ) .choose_filtered(&mut rng, |_| true); assert_eq!(empty, None); diff --git a/scylla/src/transport/session.rs b/scylla/src/transport/session.rs index 47ec56f845..5b47804834 100644 --- a/scylla/src/transport/session.rs +++ b/scylla/src/transport/session.rs @@ -910,11 +910,12 @@ impl Session { prepared: &PreparedStatement, cluster_data: &'a ClusterData, ) -> Option<&'a str> { + let table_spec = prepared.get_table_spec()?; cluster_data .keyspaces - .get(prepared.get_keyspace_name()?)? + .get(table_spec.ks_name())? .tables - .get(prepared.get_table_name()?)? + .get(table_spec.table_name())? .partitioner .as_deref() } @@ -989,6 +990,8 @@ impl Session { .unwrap_or_else(|| self.get_default_execution_profile_handle()) .access(); + let table_spec = prepared.get_table_spec(); + let statement_info = RoutingInfo { consistency: prepared .config @@ -999,7 +1002,7 @@ impl Session { .serial_consistency .unwrap_or(execution_profile.serial_consistency), token, - keyspace: prepared.get_keyspace_name(), + table: table_spec, is_confirmed_lwt: prepared.is_confirmed_lwt(), }; @@ -1010,10 +1013,10 @@ impl Session { ); if !span.span().is_disabled() { - if let (Some(keyspace), Some(token)) = (statement_info.keyspace.as_ref(), token) { + if let (Some(table_spec), Some(token)) = (statement_info.table, token) { let cluster_data = self.get_cluster_data(); let replicas: smallvec::SmallVec<[_; 8]> = cluster_data - .get_token_endpoints_iter(keyspace, token) + .get_token_endpoints_iter(table_spec, token) .collect(); span.record_replicas(&replicas) } @@ -1206,20 +1209,22 @@ impl Session { .serial_consistency .unwrap_or(execution_profile.serial_consistency); - let keyspace_name = match batch.statements.first() { - Some(BatchStatement::PreparedStatement(ps)) => ps.get_keyspace_name(), - _ => None, - }; - let (first_value_token, values) = batch_values::peek_first_token(values, batch.statements.first())?; let values_ref = &values; + let table_spec = + if let Some(BatchStatement::PreparedStatement(ps)) = batch.statements.first() { + ps.get_table_spec() + } else { + None + }; + let statement_info = RoutingInfo { consistency, serial_consistency, token: first_value_token, - keyspace: keyspace_name, + table: table_spec, is_confirmed_lwt: false, }; diff --git a/scylla/tests/integration/consistency.rs b/scylla/tests/integration/consistency.rs index a96e4450bb..2962795834 100644 --- a/scylla/tests/integration/consistency.rs +++ b/scylla/tests/integration/consistency.rs @@ -8,6 +8,7 @@ use scylla::routing::{Shard, Token}; use scylla::test_utils::unique_keyspace_name; use scylla::transport::session::Session; use scylla::transport::NodeRef; +use scylla_cql::frame::response::result::TableSpec; use tokio::sync::mpsc::{self, UnboundedReceiver, UnboundedSender}; use scylla::statement::batch::BatchStatement; @@ -341,7 +342,7 @@ pub(crate) struct OwnedRoutingInfo { serial_consistency: Option, #[allow(unused)] - keyspace: Option, + table: Option>, #[allow(unused)] token: Option, #[allow(unused)] @@ -354,14 +355,14 @@ impl OwnedRoutingInfo { consistency, serial_consistency, token, - keyspace, + table, is_confirmed_lwt, } = info; Self { consistency, serial_consistency, token, - keyspace: keyspace.map(ToOwned::to_owned), + table: table.map(TableSpec::to_owned), is_confirmed_lwt, } } From ae40d60864e450fa40a28093d82edc453c65eb1e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Karol=20Bary=C5=82a?= Date: Tue, 5 Mar 2024 14:52:31 +0100 Subject: [PATCH 03/16] scylla-cql: Add support for custom payload --- scylla-cql/src/frame/mod.rs | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/scylla-cql/src/frame/mod.rs b/scylla-cql/src/frame/mod.rs index 0ed81ec593..e0f8309a81 100644 --- a/scylla-cql/src/frame/mod.rs +++ b/scylla-cql/src/frame/mod.rs @@ -15,7 +15,7 @@ use thiserror::Error; use tokio::io::{AsyncRead, AsyncReadExt}; use uuid::Uuid; -use std::convert::TryFrom; +use std::{collections::HashMap, convert::TryFrom}; use request::SerializableRequest; use response::ResponseOpcode; @@ -169,6 +169,7 @@ pub struct ResponseBodyWithExtensions { pub trace_id: Option, pub warnings: Vec, pub body: Bytes, + pub custom_payload: Option>>, } pub fn parse_response_body_extensions( @@ -204,20 +205,22 @@ pub fn parse_response_body_extensions( Vec::new() }; - if flags & FLAG_CUSTOM_PAYLOAD != 0 { - // TODO: Do something useful with the custom payload map - // For now, just skip it + let custom_payload = if flags & FLAG_CUSTOM_PAYLOAD != 0 { let body_len = body.len(); let buf = &mut &*body; - types::read_bytes_map(buf)?; + let payload_map = types::read_bytes_map(buf)?; let buf_len = buf.len(); body.advance(body_len - buf_len); - } + Some(payload_map) + } else { + None + }; Ok(ResponseBodyWithExtensions { trace_id, warnings, body, + custom_payload, }) } From 54030f5dcf9f5a94015a13304c4fdc34b63c2943 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Karol=20Bary=C5=82a?= Date: Mon, 22 Apr 2024 22:04:05 +0200 Subject: [PATCH 04/16] Store custom payload in QueryResponse --- scylla/src/transport/connection.rs | 3 +++ 1 file changed, 3 insertions(+) diff --git a/scylla/src/transport/connection.rs b/scylla/src/transport/connection.rs index 95888822fc..588e1f46e0 100644 --- a/scylla/src/transport/connection.rs +++ b/scylla/src/transport/connection.rs @@ -217,6 +217,8 @@ pub(crate) struct QueryResponse { pub(crate) response: Response, pub(crate) tracing_id: Option, pub(crate) warnings: Vec, + #[allow(dead_code)] // This is not exposed to user (yet?) + pub(crate) custom_payload: Option>>, } // A QueryResponse in which response can not be Response::Error @@ -1034,6 +1036,7 @@ impl Connection { response, warnings: body_with_ext.warnings, tracing_id: body_with_ext.trace_id, + custom_payload: body_with_ext.custom_payload, }) } From cd0582d6b6578b897d935cb91c0826b2877145c8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Karol=20Bary=C5=82a?= Date: Thu, 1 Feb 2024 10:53:02 +0100 Subject: [PATCH 05/16] Pass ConnectionConfig by ref This was required in one of the previous versions of the PR. Now it is not, but this gets rid of one clone, so I decided to leave this commit. --- scylla/src/transport/connection.rs | 14 +++++++------- scylla/src/transport/connection_pool.rs | 6 +++--- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/scylla/src/transport/connection.rs b/scylla/src/transport/connection.rs index 588e1f46e0..ccb9dedb70 100644 --- a/scylla/src/transport/connection.rs +++ b/scylla/src/transport/connection.rs @@ -1461,7 +1461,7 @@ async fn maybe_translated_addr( pub(crate) async fn open_connection( endpoint: UntranslatedEndpoint, source_port: Option, - config: ConnectionConfig, + config: &ConnectionConfig, ) -> Result<(Connection, ErrorReceiver), QueryError> { let addr = maybe_translated_addr(endpoint, config.address_translator.as_deref()).await?; open_named_connection( @@ -1477,7 +1477,7 @@ pub(crate) async fn open_connection( pub(crate) async fn open_named_connection( addr: SocketAddr, source_port: Option, - config: ConnectionConfig, + config: &ConnectionConfig, driver_name: Option, driver_version: Option, ) -> Result<(Connection, ErrorReceiver), QueryError> { @@ -1924,7 +1924,7 @@ mod tests { datacenter: None, }), None, - ConnectionConfig::default(), + &ConnectionConfig::default(), ) .await .unwrap(); @@ -2049,7 +2049,7 @@ mod tests { datacenter: None, }), None, - ConnectionConfig { + &ConnectionConfig { enable_write_coalescing: enable_coalescing, ..ConnectionConfig::default() }, @@ -2178,7 +2178,7 @@ mod tests { // We must interrupt the driver's full connection opening, because our proxy does not interact further after Startup. let (startup_without_lwt_optimisation, _shard) = select! { - _ = open_connection(UntranslatedEndpoint::ContactPoint(ResolvedContactPoint{address: proxy_addr, datacenter: None}), None, config.clone()) => unreachable!(), + _ = open_connection(UntranslatedEndpoint::ContactPoint(ResolvedContactPoint{address: proxy_addr, datacenter: None}), None, &config) => unreachable!(), startup = startup_rx.recv() => startup.unwrap(), }; @@ -2186,7 +2186,7 @@ mod tests { .change_request_rules(Some(make_rules(options_with_lwt_optimisation_support))); let (startup_with_lwt_optimisation, _shard) = select! { - _ = open_connection(UntranslatedEndpoint::ContactPoint(ResolvedContactPoint{address: proxy_addr, datacenter: None}), None, config.clone()) => unreachable!(), + _ = open_connection(UntranslatedEndpoint::ContactPoint(ResolvedContactPoint{address: proxy_addr, datacenter: None}), None, &config) => unreachable!(), startup = startup_rx.recv() => startup.unwrap(), }; @@ -2247,7 +2247,7 @@ mod tests { datacenter: None, }), None, - config, + &config, ) .await .unwrap(); diff --git a/scylla/src/transport/connection_pool.rs b/scylla/src/transport/connection_pool.rs index 59d6e8e13d..84632a4078 100644 --- a/scylla/src/transport/connection_pool.rs +++ b/scylla/src/transport/connection_pool.rs @@ -950,7 +950,8 @@ impl PoolRefiller { .boxed(), _ => async move { let non_shard_aware_endpoint = endpoint_fut.await; - let result = connection::open_connection(non_shard_aware_endpoint, None, cfg).await; + let result = + connection::open_connection(non_shard_aware_endpoint, None, &cfg).await; OpenedConnectionEvent { result, requested_shard: None, @@ -1242,8 +1243,7 @@ async fn open_connection_to_shard_aware_port( for port in source_port_iter { let connect_result = - connection::open_connection(endpoint.clone(), Some(port), connection_config.clone()) - .await; + connection::open_connection(endpoint.clone(), Some(port), connection_config).await; match connect_result { Err(err) if err.is_address_unavailable_for_use() => continue, // If we can't use this port, try the next one From f9f1948c7b71549520838eba19d6cc96107dd419 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Karol=20Bary=C5=82a?= Date: Thu, 1 Feb 2024 16:28:37 +0100 Subject: [PATCH 06/16] Protocol_features: mark TABLETS_ROUTING_V1 as supported --- scylla-cql/src/frame/protocol_features.rs | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/scylla-cql/src/frame/protocol_features.rs b/scylla-cql/src/frame/protocol_features.rs index a3bc01bd85..a1687485c3 100644 --- a/scylla-cql/src/frame/protocol_features.rs +++ b/scylla-cql/src/frame/protocol_features.rs @@ -3,11 +3,14 @@ use std::collections::HashMap; const RATE_LIMIT_ERROR_EXTENSION: &str = "SCYLLA_RATE_LIMIT_ERROR"; pub const SCYLLA_LWT_ADD_METADATA_MARK_EXTENSION: &str = "SCYLLA_LWT_ADD_METADATA_MARK"; pub const LWT_OPTIMIZATION_META_BIT_MASK_KEY: &str = "LWT_OPTIMIZATION_META_BIT_MASK"; +const TABLETS_ROUTING_V1_KEY: &str = "TABLETS_ROUTING_V1"; + #[derive(Default, Clone, Copy, Debug, PartialEq, Eq)] #[non_exhaustive] pub struct ProtocolFeatures { pub rate_limit_error: Option, pub lwt_optimization_meta_bit_mask: Option, + pub tablets_v1_supported: bool, } // TODO: Log information about options which failed to parse @@ -19,6 +22,7 @@ impl ProtocolFeatures { lwt_optimization_meta_bit_mask: Self::maybe_parse_lwt_optimization_meta_bit_mask( supported, ), + tablets_v1_supported: Self::check_tablets_routing_v1_support(supported), } } @@ -37,6 +41,10 @@ impl ProtocolFeatures { mask_str.parse::().ok() } + fn check_tablets_routing_v1_support(supported: &HashMap>) -> bool { + supported.contains_key(TABLETS_ROUTING_V1_KEY) + } + // Looks up a field which starts with `key=` and returns the rest fn get_cql_extension_field<'a>(vals: &'a [String], key: &str) -> Option<&'a str> { vals.iter() @@ -53,6 +61,10 @@ impl ProtocolFeatures { format!("{}={}", LWT_OPTIMIZATION_META_BIT_MASK_KEY, mask), ); } + + if self.tablets_v1_supported { + options.insert(TABLETS_ROUTING_V1_KEY.to_string(), String::new()); + } } pub fn prepared_flags_contain_lwt_mark(&self, flags: u32) -> bool { From 4d3a7e34b914914aed3bb5af097dc00deeea0109 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Karol=20Bary=C5=82a?= Date: Tue, 7 May 2024 09:25:54 +0200 Subject: [PATCH 07/16] Locator: Add tablets module This module contains structs that hold tablet info. Main operations include: - searching for a tablet for given table + token - adding new tablets - parsing tablet from custom payload feedback --- Cargo.lock.msrv | 44 ++ scylla/Cargo.toml | 2 + scylla/src/transport/load_balancing/plan.rs | 5 +- scylla/src/transport/locator/mod.rs | 1 + scylla/src/transport/locator/tablets.rs | 726 ++++++++++++++++++++ scylla/src/transport/node.rs | 10 +- 6 files changed, 784 insertions(+), 4 deletions(-) create mode 100644 scylla/src/transport/locator/tablets.rs diff --git a/Cargo.lock.msrv b/Cargo.lock.msrv index c328de739e..8614dd913f 100644 --- a/Cargo.lock.msrv +++ b/Cargo.lock.msrv @@ -17,6 +17,18 @@ version = "1.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f26201604c87b1e01bd3d98f8d5d9a8fcbb815e8cedb41ffccbeb4bf593a35fe" +[[package]] +name = "ahash" +version = "0.8.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e89da841a80418a9b391ebaea17f5c112ffaaa96f621d2c285b5174da76b9011" +dependencies = [ + "cfg-if", + "once_cell", + "version_check", + "zerocopy", +] + [[package]] name = "aho-corasick" version = "1.0.5" @@ -26,6 +38,12 @@ dependencies = [ "memchr", ] +[[package]] +name = "allocator-api2" +version = "0.2.16" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0942ffc6dcaadf03badf6e6a2d0228460359d5e34b57ccdc720b7382dfbd5ec5" + [[package]] name = "android-tzdata" version = "0.1.1" @@ -656,6 +674,10 @@ name = "hashbrown" version = "0.14.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2c6201b9ff9fd90a5a3bac2e56a830d0caa509576f0e503818ee82c181b3437a" +dependencies = [ + "ahash", + "allocator-api2", +] [[package]] name = "heck" @@ -1431,8 +1453,10 @@ dependencies = [ "criterion", "dashmap", "futures", + "hashbrown 0.14.0", "histogram", "itertools 0.11.0", + "lazy_static", "lz4_flex", "ntest", "num-bigint 0.3.3", @@ -2247,6 +2271,26 @@ dependencies = [ "memchr", ] +[[package]] +name = "zerocopy" +version = "0.7.32" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "74d4d3961e53fa4c9a25a8637fc2bfaf2595b3d3ae34875568a5cf64787716be" +dependencies = [ + "zerocopy-derive", +] + +[[package]] +name = "zerocopy-derive" +version = "0.7.32" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9ce1b18ccd8e73a9321186f97e46f9f04b778851177567b1975109d26a08d2a6" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.32", +] + [[package]] name = "zeroize" version = "1.6.0" diff --git a/scylla/Cargo.toml b/scylla/Cargo.toml index 982c696d9f..4c6112c56e 100644 --- a/scylla/Cargo.toml +++ b/scylla/Cargo.toml @@ -31,6 +31,7 @@ scylla-cql = { version = "0.1.0", path = "../scylla-cql" } byteorder = "1.3.4" bytes = "1.0.1" futures = "0.3.6" +hashbrown = "0.14" histogram = "0.6.9" tokio = { version = "1.27", features = ["net", "time", "io-util", "sync", "rt", "macros"] } snap = "1.0" @@ -53,6 +54,7 @@ url = { version = "2.3.1", optional = true } base64 = { version = "0.21.1", optional = true } rand_pcg = "0.3.1" socket2 = { version = "0.5.3", features = ["all"] } +lazy_static = "1" [dev-dependencies] num-bigint-03 = { package = "num-bigint", version = "0.3" } diff --git a/scylla/src/transport/load_balancing/plan.rs b/scylla/src/transport/load_balancing/plan.rs index 3dc946c58b..d8d2862c7d 100644 --- a/scylla/src/transport/load_balancing/plan.rs +++ b/scylla/src/transport/load_balancing/plan.rs @@ -179,7 +179,10 @@ mod tests { fn expected_nodes() -> Vec<(Arc, Shard)> { vec![( Arc::new(Node::new_for_test( - NodeAddr::Translatable(SocketAddr::from_str("127.0.0.1:9042").unwrap()), + None, + Some(NodeAddr::Translatable( + SocketAddr::from_str("127.0.0.1:9042").unwrap(), + )), None, None, )), diff --git a/scylla/src/transport/locator/mod.rs b/scylla/src/transport/locator/mod.rs index b5594f1433..0be287495a 100644 --- a/scylla/src/transport/locator/mod.rs +++ b/scylla/src/transport/locator/mod.rs @@ -1,6 +1,7 @@ mod precomputed_replicas; mod replicas; mod replication_info; +pub(crate) mod tablets; #[cfg(test)] pub(crate) mod test; mod token_ring; diff --git a/scylla/src/transport/locator/tablets.rs b/scylla/src/transport/locator/tablets.rs new file mode 100644 index 0000000000..19bd7efd50 --- /dev/null +++ b/scylla/src/transport/locator/tablets.rs @@ -0,0 +1,726 @@ +#![allow(dead_code)] +use lazy_static::lazy_static; +use scylla_cql::cql_to_rust::FromCqlVal; +use scylla_cql::frame::frame_errors::ParseError; +use scylla_cql::frame::response::result::{deser_cql_value, ColumnType, TableSpec}; +use thiserror::Error; +use tracing::warn; +use uuid::Uuid; + +use crate::routing::{Shard, Token}; +use crate::transport::Node; + +use std::collections::HashMap; +use std::sync::Arc; + +#[derive(Error, Debug)] +pub(crate) enum TabletParsingError { + #[error(transparent)] + Parse(#[from] ParseError), + #[error("Shard id for tablet is negative: {0}")] + ShardNum(i32), +} + +#[derive(Debug, PartialEq, Eq)] +struct RawTabletReplicas { + replicas: Vec<(Uuid, Shard)>, +} + +#[derive(Debug, PartialEq, Eq)] +pub(crate) struct RawTablet { + /// First token belonging to the tablet, inclusive + first_token: Token, + /// Last token belonging to the tablet, inclusive + last_token: Token, + replicas: RawTabletReplicas, +} + +type RawTabletPayload = (i64, i64, Vec<(Uuid, i32)>); + +lazy_static! { + static ref RAW_TABLETS_CQL_TYPE: ColumnType = ColumnType::Tuple(vec![ + ColumnType::BigInt, + ColumnType::BigInt, + ColumnType::List(Box::new(ColumnType::Tuple(vec![ + ColumnType::Uuid, + ColumnType::Int, + ]))), + ]); +} + +const CUSTOM_PAYLOAD_TABLETS_V1_KEY: &str = "tablets-routing-v1"; + +impl RawTablet { + pub(crate) fn from_custom_payload( + payload: &HashMap>, + ) -> Option> { + let payload = payload.get(CUSTOM_PAYLOAD_TABLETS_V1_KEY)?; + let cql_value = match deser_cql_value(&RAW_TABLETS_CQL_TYPE, &mut payload.as_slice()) { + Ok(r) => r, + Err(e) => return Some(Err(e.into())), + }; + + // This could only fail if the type was wrong, but we do pass correct type + // to `deser_cql_value`. + let (first_token, last_token, replicas): RawTabletPayload = + FromCqlVal::from_cql(cql_value).unwrap(); + + let replicas = match replicas + .into_iter() + .map(|(uuid, shard_num)| match shard_num.try_into() { + Ok(s) => Ok((uuid, s)), + Err(_) => Err(shard_num), + }) + .collect::, _>>() + { + Ok(r) => r, + Err(shard_num) => return Some(Err(TabletParsingError::ShardNum(shard_num))), + }; + + Some(Ok(RawTablet { + // +1 because Scylla sends left-open range, so received + // number is the last token not belonging to this tablet. + first_token: Token::new(first_token + 1), + last_token: Token::new(last_token), + replicas: RawTabletReplicas { replicas }, + })) + } +} + +#[derive(Clone, Debug, Default)] +#[cfg_attr(test, derive(Eq))] +struct TabletReplicas { + all: Vec<(Arc, Shard)>, + per_dc: HashMap, Shard)>>, +} + +impl TabletReplicas { + pub(crate) fn from_raw_replicas( + raw_replicas: &RawTabletReplicas, + replica_translator: impl Fn(Uuid) -> Option>, + ) -> Self { + let all: Vec<_> = raw_replicas.replicas + .iter() + .filter_map(|(replica, shard)| if let Some(r) = replica_translator(*replica) { + Some((r, *shard as Shard)) + } else { + // TODO: Should this be an error? When can this happen? + warn!("Node {replica} from system.tablets not present in ClusterData.known_peers. Skipping this replica"); + None + }) + .collect(); + + let mut per_dc: HashMap, Shard)>> = HashMap::new(); + all.iter().for_each(|(replica, shard)| { + if let Some(dc) = replica.datacenter.as_ref() { + if let Some(replicas) = per_dc.get_mut(dc) { + replicas.push((Arc::clone(replica), *shard)); + } else { + per_dc.insert(dc.to_string(), vec![(Arc::clone(replica), *shard)]); + } + } + }); + + Self { all, per_dc } + } +} + +// We can't use derive because it would use normal comparision while +// comapring replicas needs to use `Arc::ptr_eq`. It's not enough to compare host ids, +// because different `Node` objects nay have the same host id. +// There is not reason to compare this outside of tests, so the `cfg(test)` is there +// to prevent future contributors from doing something stupid like comparing it +// in non-test driver code. +#[cfg(test)] +impl PartialEq for TabletReplicas { + fn eq(&self, other: &Self) -> bool { + if self.all.len() != other.all.len() { + return false; + } + for ((self_node, self_shard), (other_node, other_shard)) in + self.all.iter().zip(other.all.iter()) + { + if self_shard != other_shard { + return false; + } + if !Arc::ptr_eq(self_node, other_node) { + return false; + } + } + + // Implementations of `TableTablets`, `Tablet` and `TabletReplicas` + // guarantee that if `all` is the same then `per_dc` must be too. + // If it isn't then it is a bug. + // Comparing `TabletReplicas` happens only in tests so we can sacrifice + // a small bit of performance to verify this assumption. + assert_eq!(self.per_dc.len(), other.per_dc.len()); + for (self_k, self_v) in self.per_dc.iter() { + let other_v = other.per_dc.get(self_k).unwrap(); + for ((self_node, self_shard), (other_node, other_shard)) in + self_v.iter().zip(other_v.iter()) + { + assert_eq!(self_shard, other_shard); + assert!(Arc::ptr_eq(self_node, other_node)); + } + } + + true + } +} + +#[derive(Clone, Debug)] +#[cfg_attr(test, derive(PartialEq, Eq))] +pub(crate) struct Tablet { + /// First token belonging to the tablet, inclusive + first_token: Token, + /// Last token belonging to the tablet, inclusive + last_token: Token, + replicas: TabletReplicas, +} + +impl Tablet { + pub(crate) fn from_raw_tablet( + raw_tablet: &RawTablet, + replica_translator: impl Fn(Uuid) -> Option>, + ) -> Self { + Self { + first_token: raw_tablet.first_token, + last_token: raw_tablet.last_token, + replicas: TabletReplicas::from_raw_replicas(&raw_tablet.replicas, replica_translator), + } + } +} + +/// Container for tablets of a single table. +/// +/// It can be viewed as a set of non-overlapping Tablet objects. +/// It has 2 basic operations: +/// 1. Find a tablet for given Token +/// 2. Add a new tablet. +/// +/// Adding new Tablet will first remove all tablets that overlap with the new tablet. +#[derive(Clone, Debug)] +#[cfg_attr(test, derive(PartialEq, Eq))] +pub(crate) struct TableTablets { + table_spec: TableSpec<'static>, + tablet_list: Vec, +} + +impl TableTablets { + fn new(table_spec: TableSpec<'static>) -> Self { + Self { + table_spec, + tablet_list: Default::default(), + } + } + + fn tablet_for_token(&self, token: Token) -> Option<&Tablet> { + let idx = self + .tablet_list + .partition_point(|tablet| tablet.last_token < token); + let tablet = self.tablet_list.get(idx); + tablet.filter(|t| t.first_token <= token) + } + + pub(crate) fn replicas_for_token(&self, token: Token) -> Option<&[(Arc, Shard)]> { + self.tablet_for_token(token) + .map(|tablet| tablet.replicas.all.as_ref()) + } + + pub(crate) fn dc_replicas_for_token( + &self, + token: Token, + dc: &str, + ) -> Option<&[(Arc, Shard)]> { + self.tablet_for_token(token).map(|tablet| { + tablet + .replicas + .per_dc + .get(dc) + .map(|x| x.as_slice()) + .unwrap_or(&[]) + }) + } + + /// This method: + /// - first removes all tablets that overlap with `tablet` from `self` + /// - adds `tablet` to `self` + /// + /// This preserves the invariant that all tablets in `self` are non-overlapping. + fn add_tablet(&mut self, tablet: Tablet) { + // Smallest `left_idx` for which `tablet.first_token` is LESS OR EQUAL to `tablet_list[left_idx].last_token`. + // It implies that `tablet_list[left_idx]` overlaps with `tablet` iff `tablet.last_token` + // is GREATER OR EQUAL to `tablet_list[left_idx].first_token`. + let left_idx = self + .tablet_list + .partition_point(|t| t.last_token < tablet.first_token); + // Smallest `right_idx` for which `tablet.last_token` is LESS than `tablet_list[right_idx].first_token`. + // It means that `right_idx` is the index of first tablet that is "to the right" of `tablet` and doesn't overlap with it. + // From this it follows that if `tablet_list[left_idx]` turns out to not overlap with `tablet`, then `left_idx == right_idx` + // and we won't remove any tablets because `tablet` doesn't overlap with any existing tablets. + let right_idx = self + .tablet_list + .partition_point(|t| t.first_token <= tablet.last_token); + self.tablet_list.drain(left_idx..right_idx); + self.tablet_list.insert(left_idx, tablet); + } + + #[cfg(test)] + fn new_for_test() -> Self { + Self::new(TableSpec::borrowed("test_ks", "test_table")) + } +} + +#[derive(Clone, Debug)] +pub(crate) struct TabletsInfo { + // We use hashbrown hashmap instead of std hashmap because with + // std one it is not possible to query map with key `TableSpec<'static>` + // using `TableSpec<'a>` for `'a` other than `'static`. + // This is because `std::hashmap` requires that the key implements `Borrow` + // where `&Q` is an argument to `.get(key)` method. It is not possible to write + // such `Borrow` impl for `TableSpec`. + // HashBrown on the other hand requires only `Q: Hash + Equivalent + ?Sized`, + // and it is easy to create a wrapper type with required `Equivalent` impl. + tablets: hashbrown::HashMap, TableTablets>, +} + +impl TabletsInfo { + pub(crate) fn new() -> Self { + Self { + tablets: hashbrown::HashMap::new(), + } + } + + pub(crate) fn tablets_for_table<'a, 'b>( + &'a self, + table_spec: &'b TableSpec<'b>, + ) -> Option<&'a TableTablets> { + #[derive(Hash)] + struct TableSpecQueryKey<'a> { + table_spec: &'a TableSpec<'a>, + } + + impl<'key, 'query> hashbrown::Equivalent> for TableSpecQueryKey<'query> { + fn equivalent(&self, key: &TableSpec<'key>) -> bool { + self.table_spec == key + } + } + + let query_key = TableSpecQueryKey { table_spec }; + + let table_tablets = self.tablets.get(&query_key); + + table_tablets + } + + pub(crate) fn add_tablet(&mut self, table_spec: TableSpec<'static>, tablet: Tablet) { + self.tablets + .entry(table_spec) + .or_insert_with_key(|k| { + tracing::debug!( + "Found new tablets table: {}.{}", + k.ks_name(), + k.table_name() + ); + TableTablets::new(k.clone()) + }) + .add_tablet(tablet) + } +} + +#[cfg(test)] +mod tests { + use std::collections::HashMap; + use std::sync::Arc; + + use scylla_cql::frame::response::result::{ColumnType, CqlValue}; + use scylla_cql::types::serialize::{value::SerializeCql, CellWriter}; + use tracing::debug; + use uuid::Uuid; + + use crate::routing::Token; + use crate::transport::locator::tablets::{ + RawTablet, RawTabletReplicas, TabletParsingError, CUSTOM_PAYLOAD_TABLETS_V1_KEY, + RAW_TABLETS_CQL_TYPE, + }; + use crate::transport::Node; + + use super::{TableTablets, Tablet, TabletReplicas}; + + const DC1: &str = "dc1"; + const DC2: &str = "dc2"; + const DC3: &str = "dc3"; + + #[test] + fn test_raw_tablet_deser_empty() { + let custom_payload = HashMap::new(); + assert!(RawTablet::from_custom_payload(&custom_payload).is_none()); + } + + #[test] + fn test_raw_tablet_deser_trash() { + let custom_payload = + HashMap::from([(CUSTOM_PAYLOAD_TABLETS_V1_KEY.to_string(), vec![1, 2, 3])]); + assert_matches::assert_matches!( + RawTablet::from_custom_payload(&custom_payload), + Some(Err(TabletParsingError::Parse(_))) + ); + } + + #[test] + fn test_raw_tablet_deser_wrong_type() { + let mut custom_payload = HashMap::new(); + let mut data = vec![]; + + let value = CqlValue::Tuple(vec![ + Some(CqlValue::Ascii("asdderty".to_string())), + Some(CqlValue::BigInt(1234)), + Some(CqlValue::List(vec![])), + ]); + let col_type = ColumnType::Tuple(vec![ + ColumnType::Ascii, + ColumnType::BigInt, + ColumnType::List(Box::new(ColumnType::Tuple(vec![ + ColumnType::Uuid, + ColumnType::Int, + ]))), + ]); + + SerializeCql::serialize(&value, &col_type, CellWriter::new(&mut data)).unwrap(); + debug!("{:?}", data); + + custom_payload.insert(CUSTOM_PAYLOAD_TABLETS_V1_KEY.to_string(), data); + + assert_matches::assert_matches!( + RawTablet::from_custom_payload(&custom_payload), + Some(Err(TabletParsingError::Parse(_))) + ); + } + + #[test] + fn test_raw_tablet_deser_correct() { + let mut custom_payload = HashMap::new(); + let mut data = vec![]; + + const FIRST_TOKEN: i64 = 1234; + const LAST_TOKEN: i64 = 5678; + + let value = CqlValue::Tuple(vec![ + Some(CqlValue::BigInt(FIRST_TOKEN)), + Some(CqlValue::BigInt(LAST_TOKEN)), + Some(CqlValue::List(vec![ + CqlValue::Tuple(vec![ + Some(CqlValue::Uuid(Uuid::from_u64_pair(1, 2))), + Some(CqlValue::Int(15)), + ]), + CqlValue::Tuple(vec![ + Some(CqlValue::Uuid(Uuid::from_u64_pair(3, 4))), + Some(CqlValue::Int(19)), + ]), + ])), + ]); + + SerializeCql::serialize(&value, &RAW_TABLETS_CQL_TYPE, CellWriter::new(&mut data)).unwrap(); + tracing::debug!("{:?}", data); + + custom_payload.insert( + CUSTOM_PAYLOAD_TABLETS_V1_KEY.to_string(), + // Skipping length because `SerializeCql::serialize` adds length at the + // start of serialized value while Scylla sends the value without initial + // length. + data[4..].to_vec(), + ); + + let tablet = RawTablet::from_custom_payload(&custom_payload) + .unwrap() + .unwrap(); + + assert_eq!( + tablet, + RawTablet { + first_token: Token::new(FIRST_TOKEN + 1), + last_token: Token::new(LAST_TOKEN), + replicas: RawTabletReplicas { + replicas: vec![ + (Uuid::from_u64_pair(1, 2), 15), + (Uuid::from_u64_pair(3, 4), 19) + ] + } + } + ); + } + + #[test] + fn raw_replicas_to_replicas_groups_correctly() { + let nodes: HashMap> = [ + Node::new_for_test( + Some(Uuid::from_u64_pair(1, 1)), + None, + Some(DC1.to_string()), + None, + ), + Node::new_for_test( + Some(Uuid::from_u64_pair(1, 2)), + None, + Some(DC2.to_string()), + None, + ), + Node::new_for_test( + Some(Uuid::from_u64_pair(1, 3)), + None, + Some(DC3.to_string()), + None, + ), + Node::new_for_test( + Some(Uuid::from_u64_pair(1, 4)), + None, + Some(DC2.to_string()), + None, + ), + Node::new_for_test( + Some(Uuid::from_u64_pair(1, 5)), + None, + Some(DC2.to_string()), + None, + ), + Node::new_for_test( + Some(Uuid::from_u64_pair(1, 6)), + None, + Some(DC1.to_string()), + None, + ), + ] + .into_iter() + .map(|node| (node.host_id, Arc::new(node))) + .collect(); + + let translator = |uuid| nodes.get(&uuid).cloned(); + + let replicas_uids = [ + Uuid::from_u64_pair(1, 1), + Uuid::from_u64_pair(1, 2), + Uuid::from_u64_pair(1, 3), + Uuid::from_u64_pair(1, 4), + Uuid::from_u64_pair(1, 5), + Uuid::from_u64_pair(1, 6), + ]; + + let raw_replicas = RawTabletReplicas { + replicas: replicas_uids.into_iter().map(|uid| (uid, 1)).collect(), + }; + + let replicas = TabletReplicas::from_raw_replicas(&raw_replicas, translator); + + let mut per_dc = HashMap::new(); + per_dc.insert( + "dc1".to_string(), + vec![ + (translator(Uuid::from_u64_pair(1, 1)).unwrap(), 1), + (translator(Uuid::from_u64_pair(1, 6)).unwrap(), 1), + ], + ); + per_dc.insert( + "dc2".to_string(), + vec![ + (translator(Uuid::from_u64_pair(1, 2)).unwrap(), 1), + (translator(Uuid::from_u64_pair(1, 4)).unwrap(), 1), + (translator(Uuid::from_u64_pair(1, 5)).unwrap(), 1), + ], + ); + per_dc.insert( + "dc3".to_string(), + vec![(translator(Uuid::from_u64_pair(1, 3)).unwrap(), 1)], + ); + + assert_eq!( + replicas, + TabletReplicas { + all: replicas_uids + .iter() + .cloned() + .map(|replica| (translator(replica).unwrap(), 1)) + .collect(), + per_dc + } + ); + } + + #[test] + fn table_tablets_empty() { + let tablets: TableTablets = TableTablets::new_for_test(); + assert_eq!(tablets.tablet_for_token(Token::new(1)), None); + } + + fn verify_ranges(tablets: &TableTablets, ranges: &[(i64, i64)]) { + let mut ranges_iter = ranges.iter(); + for tablet in tablets.tablet_list.iter() { + let range = ranges_iter.next().unwrap(); + assert_eq!(tablet.first_token.value(), range.0); + assert_eq!(tablet.last_token.value(), range.1); + } + assert_eq!(ranges_iter.next(), None) + } + + fn insert_ranges(tablets: &mut TableTablets, ranges: &[(i64, i64)]) { + for (first, last) in ranges.iter() { + tablets.add_tablet(Tablet { + first_token: Token::new(*first), + last_token: Token::new(*last), + replicas: Default::default(), + }); + } + } + + #[test] + fn table_tablets_single() { + let mut tablets = TableTablets::new_for_test(); + + insert_ranges(&mut tablets, &[(-200, 1000)]); + verify_ranges(&tablets, &[(-200, 1000)]); + + assert_eq!( + tablets.tablet_for_token(Token::new(-1)), + Some(&tablets.tablet_list[0]) + ); + assert_eq!( + tablets.tablet_for_token(Token::new(0)), + Some(&tablets.tablet_list[0]) + ); + assert_eq!( + tablets.tablet_for_token(Token::new(1)), + Some(&tablets.tablet_list[0]) + ); + assert_eq!( + tablets.tablet_for_token(Token::new(-200)), + Some(&tablets.tablet_list[0]) + ); + assert_eq!(tablets.tablet_for_token(Token::new(-201)), None); + assert_eq!( + tablets.tablet_for_token(Token::new(1000)), + Some(&tablets.tablet_list[0]) + ); + assert_eq!(tablets.tablet_for_token(Token::new(1001)), None); + } + + #[test] + fn test_adding_tablets_non_overlapping() { + let mut tablets = TableTablets::new_for_test(); + const RANGES: &[(i64, i64)] = &[ + (-2000000, -1900001), + (-1900000, -1700001), + (-1700000, -1), + (0, 19), + (20, 10000), + ]; + + insert_ranges(&mut tablets, RANGES); + verify_ranges(&tablets, RANGES); + } + + #[test] + fn test_adding_tablet_same() { + let mut tablets = TableTablets::new_for_test(); + + insert_ranges(&mut tablets, &[(-2000000, -1800000), (-2000000, -1800000)]); + verify_ranges(&tablets, &[(-2000000, -1800000)]); + } + + #[test] + fn test_adding_tablet_overlapping_one() { + let mut tablets = TableTablets::new_for_test(); + insert_ranges(&mut tablets, &[(-2000000, -1800000)]); + verify_ranges(&tablets, &[(-2000000, -1800000)]); + + // Replacing a tablet, overlaps right part of the old one + insert_ranges(&mut tablets, &[(-1900000, -1700000)]); + verify_ranges(&tablets, &[(-1900000, -1700000)]); + + // Replacing a tablet, overlaps left part of the old one + insert_ranges(&mut tablets, &[(-2000000, -1800000)]); + verify_ranges(&tablets, &[(-2000000, -1800000)]); + } + + #[test] + fn test_adding_tablet_fill_hole() { + let mut tablets = TableTablets::new_for_test(); + + // Fill a hole between two tablets + insert_ranges( + &mut tablets, + &[ + (-2000000, -1800001), + (-1600000, -1400000), // Create a hole + (-1800000, -1600001), // Fully fill this hole + ], + ); + verify_ranges( + &tablets, + &[ + (-2000000, -1800001), + (-1800000, -1600001), + (-1600000, -1400000), + ], + ); + } + + #[test] + fn test_adding_tablet_neighbours_not_removed() { + let mut tablets = TableTablets::new_for_test(); + insert_ranges( + &mut tablets, + &[ + (-2000000, -1800001), + (-1800000, -1600001), + (-1600000, -1400000), + ], + ); + + // Make sure neighbours are not removed when fully replacing tablet in the middle + insert_ranges(&mut tablets, &[(-1800000, -1600001)]); + verify_ranges( + &tablets, + &[ + (-2000000, -1800001), + (-1800000, -1600001), + (-1600000, -1400000), + ], + ); + + // Make sure neighbours are not removed when new tablet is smaller than old one + insert_ranges(&mut tablets, &[(-1750000, -1650000)]); + verify_ranges( + &tablets, + &[ + (-2000000, -1800001), + (-1750000, -1650000), + (-1600000, -1400000), + ], + ); + } + + #[test] + fn replace_multiple_tablets_middle() { + let mut tablets = TableTablets::new_for_test(); + insert_ranges( + &mut tablets, + &[ + (-2000000, -1800001), + (-1800000, -1600001), + (-1600000, -1400001), + (-1400000, -1200001), + (-1200000, -1000000), + ], + ); + + // Replacing 3 middle tablets + insert_ranges(&mut tablets, &[(-1750000, -1250000)]); + verify_ranges( + &tablets, + &[ + (-2000000, -1800001), + (-1750000, -1250000), + (-1200000, -1000000), + ], + ); + } +} diff --git a/scylla/src/transport/node.rs b/scylla/src/transport/node.rs index 07c34e1302..02ca247bc5 100644 --- a/scylla/src/transport/node.rs +++ b/scylla/src/transport/node.rs @@ -331,13 +331,17 @@ mod tests { impl Node { pub(crate) fn new_for_test( - address: NodeAddr, + id: Option, + address: Option, datacenter: Option, rack: Option, ) -> Self { Self { - host_id: Uuid::new_v4(), - address, + host_id: id.unwrap_or(Uuid::new_v4()), + address: address.unwrap_or(NodeAddr::Translatable(SocketAddr::from(( + [255, 255, 255, 255], + 0, + )))), datacenter, rack, pool: None, From 56a26833e2c3093a7315baefa7888b4239f710c1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Karol=20Bary=C5=82a?= Date: Wed, 1 May 2024 18:33:35 +0200 Subject: [PATCH 08/16] Get and store information about tablets from custom payload tokio version is bumped to 1.34 because this code needs `recv_many` method on channel, which was introduced in 1.34. --- Cargo.lock.msrv | 41 +++++++----- scylla/Cargo.toml | 2 +- scylla/src/transport/cluster.rs | 55 ++++++++++++++- scylla/src/transport/connection.rs | 67 ++++++++++++++++--- .../src/transport/load_balancing/default.rs | 25 ++++++- scylla/src/transport/locator/mod.rs | 6 ++ scylla/src/transport/locator/test.rs | 3 +- scylla/src/transport/session.rs | 6 ++ 8 files changed, 175 insertions(+), 30 deletions(-) diff --git a/Cargo.lock.msrv b/Cargo.lock.msrv index 8614dd913f..a42a68aca6 100644 --- a/Cargo.lock.msrv +++ b/Cargo.lock.msrv @@ -465,7 +465,7 @@ checksum = "136526188508e25c6fef639d7927dfb3e0e3084488bf202267829cf7fc23dbdd" dependencies = [ "errno-dragonfly", "libc", - "windows-sys", + "windows-sys 0.48.0", ] [[package]] @@ -518,7 +518,7 @@ checksum = "ef033ed5e9bad94e55838ca0ca906db0e043f517adda0c8b79c7a8c66c93c1b5" dependencies = [ "cfg-if", "rustix", - "windows-sys", + "windows-sys 0.48.0", ] [[package]] @@ -806,9 +806,9 @@ checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" [[package]] name = "libc" -version = "0.2.147" +version = "0.2.153" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b4668fb0ea861c1df094127ac5f1da3409a82116a4ba74fca2e58ef927159bb3" +checksum = "9c198f91728a82281a64e1f4f9eeb25d82cb32a5de251c6bd1b5154d63a8e7bd" [[package]] name = "libm" @@ -891,13 +891,13 @@ dependencies = [ [[package]] name = "mio" -version = "0.8.8" +version = "0.8.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "927a765cd3fc26206e66b296465fa9d3e5ab003e651c1b3c060e7956d96b19d2" +checksum = "a4a650543ca06a924e8b371db273b2756685faae30f8487da1b56505a8f78b0c" dependencies = [ "libc", "wasi", - "windows-sys", + "windows-sys 0.48.0", ] [[package]] @@ -1380,7 +1380,7 @@ dependencies = [ "errno", "libc", "linux-raw-sys", - "windows-sys", + "windows-sys 0.48.0", ] [[package]] @@ -1631,12 +1631,12 @@ checksum = "5e9f0ab6ef7eb7353d9119c170a436d1bf248eea575ac42d19d12f4e34130831" [[package]] name = "socket2" -version = "0.5.3" +version = "0.5.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2538b18701741680e0322a2302176d3253a35388e2e62f172f64f4f16605f877" +checksum = "05ffd9c0a93b7543e062e759284fcf5f5e3b098501104bfbdde4d404db792871" dependencies = [ "libc", - "windows-sys", + "windows-sys 0.52.0", ] [[package]] @@ -1783,9 +1783,9 @@ checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" [[package]] name = "tokio" -version = "1.32.0" +version = "1.36.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "17ed6077ed6cd6c74735e21f37eb16dc3935f96878b1fe961074089cc80893f9" +checksum = "61285f6515fa018fb2d1e46eb21223fff441ee8db5d0f1435e8ab4f5cdb80931" dependencies = [ "backtrace", "bytes", @@ -1797,14 +1797,14 @@ dependencies = [ "signal-hook-registry", "socket2", "tokio-macros", - "windows-sys", + "windows-sys 0.48.0", ] [[package]] name = "tokio-macros" -version = "2.1.0" +version = "2.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "630bdcf245f78637c13ec01ffae6187cca34625e8c63150d424b59e55af2675e" +checksum = "5b8a1e28f2deaa14e508979454cb3a223b10b938b45af148bc0986de36f1923b" dependencies = [ "proc-macro2", "quote", @@ -2148,6 +2148,15 @@ dependencies = [ "windows-targets 0.48.5", ] +[[package]] +name = "windows-sys" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "282be5f36a8ce781fad8c8ae18fa3f9beff57ec1b52cb3de0789201425d9a33d" +dependencies = [ + "windows-targets 0.52.4", +] + [[package]] name = "windows-targets" version = "0.48.5" diff --git a/scylla/Cargo.toml b/scylla/Cargo.toml index 4c6112c56e..2add79fc1b 100644 --- a/scylla/Cargo.toml +++ b/scylla/Cargo.toml @@ -33,7 +33,7 @@ bytes = "1.0.1" futures = "0.3.6" hashbrown = "0.14" histogram = "0.6.9" -tokio = { version = "1.27", features = ["net", "time", "io-util", "sync", "rt", "macros"] } +tokio = { version = "1.34", features = ["net", "time", "io-util", "sync", "rt", "macros"] } snap = "1.0" uuid = { version = "1.0", features = ["v4"] } rand = "0.8.3" diff --git a/scylla/src/transport/cluster.rs b/scylla/src/transport/cluster.rs index e98b5365f7..734d7937b6 100644 --- a/scylla/src/transport/cluster.rs +++ b/scylla/src/transport/cluster.rs @@ -3,6 +3,7 @@ use crate::frame::response::event::{Event, StatusChangeEvent}; use crate::prepared_statement::TokenCalculationError; use crate::routing::{Shard, Token}; use crate::transport::host_filter::HostFilter; +use crate::transport::session::TABLET_CHANNEL_SIZE; use crate::transport::{ connection::{Connection, VerifiedKeyspaceName}, connection_pool::PoolConfig, @@ -27,6 +28,7 @@ use tracing::instrument::WithSubscriber; use tracing::{debug, warn}; use uuid::Uuid; +use super::locator::tablets::{RawTablet, Tablet, TabletsInfo}; use super::node::{KnownNode, NodeAddr}; use super::NodeRef; @@ -116,6 +118,10 @@ struct ClusterWorker { // Channel used to receive signals that control connection is broken control_connection_repair_channel: tokio::sync::broadcast::Receiver<()>, + // Channel used to receive info about new tablets from custom payload in responses + // sent by server. + tablets_channel: tokio::sync::mpsc::Receiver<(TableSpec<'static>, RawTablet)>, + // Keyspace send in "USE " when opening each connection used_keyspace: Option, @@ -147,6 +153,7 @@ impl Cluster { fetch_schema_metadata: bool, host_filter: Option>, cluster_metadata_refresh_interval: Duration, + tablet_receiver: tokio::sync::mpsc::Receiver<(TableSpec<'static>, RawTablet)>, ) -> Result { let (refresh_sender, refresh_receiver) = tokio::sync::mpsc::channel(32); let (use_keyspace_sender, use_keyspace_receiver) = tokio::sync::mpsc::channel(32); @@ -173,6 +180,7 @@ impl Cluster { &HashMap::new(), &None, host_filter.as_deref(), + TabletsInfo::new(), ) .await; cluster_data.wait_until_all_pools_are_initialized().await; @@ -188,6 +196,7 @@ impl Cluster { refresh_channel: refresh_receiver, server_events_channel: server_events_receiver, control_connection_repair_channel: control_connection_repair_receiver, + tablets_channel: tablet_receiver, use_keyspace_channel: use_keyspace_receiver, used_keyspace: None, @@ -276,6 +285,7 @@ impl ClusterData { known_peers: &HashMap>, used_keyspace: &Option, host_filter: Option<&dyn HostFilter>, + tablets: TabletsInfo, ) -> Self { // Create new updated known_peers and ring let mut new_known_peers: HashMap> = @@ -343,7 +353,7 @@ impl ClusterData { let keyspaces = metadata.keyspaces; let (locator, keyspaces) = tokio::task::spawn_blocking(move || { let keyspace_strategies = keyspaces.values().map(|ks| &ks.strategy); - let locator = ReplicaLocator::new(ring.into_iter(), keyspace_strategies); + let locator = ReplicaLocator::new(ring.into_iter(), keyspace_strategies, tablets); (locator, keyspaces) }) .await @@ -479,6 +489,15 @@ impl ClusterData { // By an invariant `self.known_peers` is nonempty, so the returned iterator // is nonempty, too. } + + fn update_tablets(&mut self, raw_tablets: Vec<(TableSpec<'static>, RawTablet)>) { + let replica_translator = |uuid: Uuid| self.known_peers.get(&uuid).cloned(); + + for (table, raw_tablet) in raw_tablets.into_iter() { + let tablet = Tablet::from_raw_tablet(&raw_tablet, replica_translator); + self.locator.tablets.add_tablet(table, tablet); + } + } } impl ClusterWorker { @@ -501,6 +520,8 @@ impl ClusterWorker { }) .unwrap_or_else(Instant::now); + let mut tablets = Vec::new(); + let sleep_future = tokio::time::sleep_until(sleep_until); tokio::pin!(sleep_future); @@ -512,6 +533,37 @@ impl ClusterWorker { None => return, // If refresh_channel was closed then cluster was dropped, we can stop working } } + tablets_count = self.tablets_channel.recv_many(&mut tablets, TABLET_CHANNEL_SIZE) => { + tracing::trace!("Performing tablets update - received {} tablets", tablets_count); + if tablets_count == 0 { + // If the channel was closed then the cluster was dropped, we can stop working + return; + } + // The current tablet implementation collects tablet feedback in a channel + // and then clones the whole ClusterData, updates it with new tablets and replaces + // the old ClusterData - this update procedure happens below. + // This fits the general model of how ClusterData is handled in the driver: + // - ClusterData remains a "simple" struct - without locks etc (apart from Node). + // - Topology information update is similar to tablet update - it creates a new ClusterData + // and replaces the old one. + // The disadvantage is that we need to have 2 copies of ClusterData, but this happens + // anyway during topology update. + // + // An alternative solution would be to use some synchronization primitives to update tablet info + // in place. This solution avoids ClusterData cloning but: + // - ClusterData would be much more complicated + // - Requires using locks in hot path (when sending request) + // - Makes maintenance (which happens during topology update) more complicated and error-prone. + // + // I decided to stick with the approach that fits with the driver. + // Apart from the reasons above, it is much easier to reason about concurrency etc + // when reading the code in other parts of the driver. + let mut new_cluster_data: ClusterData = self.cluster_data.load().as_ref().clone(); + new_cluster_data.update_tablets(tablets); + self.update_cluster_data(Arc::new(new_cluster_data)); + + continue; + } recv_res = self.server_events_channel.recv() => { if let Some(event) = recv_res { debug!("Received server event: {:?}", event); @@ -667,6 +719,7 @@ impl ClusterWorker { &cluster_data.known_peers, &self.used_keyspace, self.host_filter.as_deref(), + cluster_data.locator.tablets.clone(), ) .await, ); diff --git a/scylla/src/transport/connection.rs b/scylla/src/transport/connection.rs index ccb9dedb70..c3f91ec6b4 100644 --- a/scylla/src/transport/connection.rs +++ b/scylla/src/transport/connection.rs @@ -2,7 +2,7 @@ use bytes::Bytes; use futures::{future::RemoteHandle, FutureExt}; use scylla_cql::errors::TranslationError; use scylla_cql::frame::request::options::Options; -use scylla_cql::frame::response::result::ResultMetadata; +use scylla_cql::frame::response::result::{ResultMetadata, TableSpec}; use scylla_cql::frame::response::Error; use scylla_cql::frame::types::SerialConsistency; use scylla_cql::types::serialize::batch::{BatchValues, BatchValuesIterator}; @@ -43,6 +43,7 @@ use std::{ use super::errors::{BadKeyspaceName, DbError, QueryError}; use super::iterator::RowIterator; +use super::locator::tablets::{RawTablet, TabletParsingError}; use super::query_result::SingleRowTypedError; use super::session::AddressTranslator; use super::topology::{PeerEndpoint, UntranslatedEndpoint, UntranslatedPeer}; @@ -369,6 +370,7 @@ pub(crate) struct ConnectionConfig { pub(crate) keepalive_interval: Option, pub(crate) keepalive_timeout: Option, + pub(crate) tablet_sender: Option, RawTablet)>>, } impl Default for ConnectionConfig { @@ -391,6 +393,8 @@ impl Default for ConnectionConfig { // Note: this is different than SessionConfig default values. keepalive_interval: None, keepalive_timeout: None, + + tablet_sender: None, } } } @@ -723,6 +727,15 @@ impl Connection { ) .await?; + if let Some(spec) = prepared_statement.get_table_spec() { + if let Err(e) = self + .update_tablets_from_response(spec, &query_response) + .await + { + tracing::warn!("Error while parsing tablet info from custom payload: {}", e); + } + } + match &query_response.response { Response::Error(frame::response::Error { error: DbError::Unprepared { statement_id }, @@ -732,13 +745,25 @@ impl Connection { // Repreparation of a statement is needed self.reprepare(prepared_statement.get_statement(), prepared_statement) .await?; - self.send_request( - &execute_frame, - true, - prepared_statement.config.tracing, - cached_metadata, - ) - .await + let new_response = self + .send_request( + &execute_frame, + true, + prepared_statement.config.tracing, + cached_metadata, + ) + .await?; + + if let Some(spec) = prepared_statement.get_table_spec() { + if let Err(e) = self.update_tablets_from_response(spec, &new_response).await { + tracing::warn!( + "Error while parsing tablet info from custom payload: {}", + e + ); + } + } + + Ok(new_response) } _ => Ok(query_response), } @@ -1417,6 +1442,32 @@ impl Connection { pub(crate) fn get_connect_address(&self) -> SocketAddr { self.connect_address } + + async fn update_tablets_from_response( + &self, + table: &TableSpec<'_>, + response: &QueryResponse, + ) -> Result<(), TabletParsingError> { + if let (Some(sender), Some(tablet_data)) = ( + self.config.tablet_sender.as_ref(), + response.custom_payload.as_ref(), + ) { + let tablet = match RawTablet::from_custom_payload(tablet_data) { + Some(Ok(v)) => v, + Some(Err(e)) => return Err(e), + None => return Ok(()), + }; + tracing::trace!( + "Received tablet info for table {}.{} in custom payload: {:?}", + table.ks_name(), + table.table_name(), + tablet + ); + let _ = sender.send((table.to_owned(), tablet)).await; + } + + Ok(()) + } } async fn maybe_translated_addr( diff --git a/scylla/src/transport/load_balancing/default.rs b/scylla/src/transport/load_balancing/default.rs index 889ef701aa..fe058a50e2 100644 --- a/scylla/src/transport/load_balancing/default.rs +++ b/scylla/src/transport/load_balancing/default.rs @@ -988,7 +988,10 @@ mod tests { routing::Token, test_utils::setup_tracing, transport::{ - locator::test::{id_to_invalid_addr, mock_metadata_for_token_aware_tests}, + locator::{ + tablets::TabletsInfo, + test::{id_to_invalid_addr, mock_metadata_for_token_aware_tests}, + }, topology::{Metadata, Peer}, ClusterData, }, @@ -1189,7 +1192,15 @@ mod tests { // based on locator mock cluster pub(crate) async fn mock_cluster_data_for_token_aware_tests() -> ClusterData { let metadata = mock_metadata_for_token_aware_tests(); - ClusterData::new(metadata, &Default::default(), &HashMap::new(), &None, None).await + ClusterData::new( + metadata, + &Default::default(), + &HashMap::new(), + &None, + None, + TabletsInfo::new(), + ) + .await } // creates ClusterData with info about 5 nodes living in 2 different datacenters @@ -1211,7 +1222,15 @@ mod tests { keyspaces: HashMap::new(), }; - ClusterData::new(info, &Default::default(), &HashMap::new(), &None, None).await + ClusterData::new( + info, + &Default::default(), + &HashMap::new(), + &None, + None, + TabletsInfo::new(), + ) + .await } pub(crate) fn get_plan_and_collect_node_identifiers( diff --git a/scylla/src/transport/locator/mod.rs b/scylla/src/transport/locator/mod.rs index 0be287495a..f90457c39f 100644 --- a/scylla/src/transport/locator/mod.rs +++ b/scylla/src/transport/locator/mod.rs @@ -10,6 +10,8 @@ use rand::{seq::IteratorRandom, Rng}; use scylla_cql::frame::response::result::TableSpec; pub use token_ring::TokenRing; +use self::tablets::TabletsInfo; + use super::{topology::Strategy, Node, NodeRef}; use crate::routing::{Shard, Token}; use itertools::Itertools; @@ -34,6 +36,8 @@ pub struct ReplicaLocator { precomputed_replicas: PrecomputedReplicas, datacenters: Vec, + + pub(crate) tablets: TabletsInfo, } impl ReplicaLocator { @@ -43,6 +47,7 @@ impl ReplicaLocator { pub(crate) fn new<'a>( ring_iter: impl Iterator)>, precompute_replica_sets_for: impl Iterator, + tablets: TabletsInfo, ) -> Self { let replication_data = ReplicationInfo::new(ring_iter); let precomputed_replicas = @@ -60,6 +65,7 @@ impl ReplicaLocator { replication_data, precomputed_replicas, datacenters, + tablets, } } diff --git a/scylla/src/transport/locator/test.rs b/scylla/src/transport/locator/test.rs index ca152c81f1..2622eb99a6 100644 --- a/scylla/src/transport/locator/test.rs +++ b/scylla/src/transport/locator/test.rs @@ -3,6 +3,7 @@ use rand_chacha::ChaCha8Rng; use scylla_cql::frame::response::result::TableSpec; use uuid::Uuid; +use super::tablets::TabletsInfo; use super::{ReplicaLocator, ReplicaSet}; use crate::routing::Token; use crate::test_utils::setup_tracing; @@ -202,7 +203,7 @@ pub(crate) fn create_locator(metadata: &Metadata) -> ReplicaLocator { let ring = create_ring(metadata); let strategies = metadata.keyspaces.values().map(|ks| &ks.strategy); - ReplicaLocator::new(ring, strategies) + ReplicaLocator::new(ring, strategies, TabletsInfo::new()) } #[tokio::test] diff --git a/scylla/src/transport/session.rs b/scylla/src/transport/session.rs index 5b47804834..7da4c3f54f 100644 --- a/scylla/src/transport/session.rs +++ b/scylla/src/transport/session.rs @@ -79,6 +79,8 @@ use crate::authentication::AuthenticatorProvider; use openssl::ssl::SslContext; use scylla_cql::errors::BadQuery; +pub(crate) const TABLET_CHANNEL_SIZE: usize = 8192; + /// Translates IP addresses received from ScyllaDB nodes into locally reachable addresses. /// /// The driver auto-detects new ScyllaDB nodes added to the cluster through server side pushed @@ -494,6 +496,8 @@ impl Session { return Err(NewSessionError::EmptyKnownNodesList); } + let (tablet_sender, tablet_receiver) = tokio::sync::mpsc::channel(TABLET_CHANNEL_SIZE); + let connection_config = ConnectionConfig { compression: config.compression, tcp_nodelay: config.tcp_nodelay, @@ -510,6 +514,7 @@ impl Session { enable_write_coalescing: config.enable_write_coalescing, keepalive_interval: config.keepalive_interval, keepalive_timeout: config.keepalive_timeout, + tablet_sender: Some(tablet_sender), }; let pool_config = PoolConfig { @@ -526,6 +531,7 @@ impl Session { config.fetch_schema_metadata, config.host_filter, config.cluster_metadata_refresh_interval, + tablet_receiver, ) .await?; From 0a218cb1c18693bb9a5a20c6f1faf0274f8ef958 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Karol=20Bary=C5=82a?= Date: Wed, 24 Apr 2024 01:49:21 +0200 Subject: [PATCH 09/16] Use tablets in locator for relevant tables. This enables shard awareness for tablet tables --- scylla/src/transport/locator/mod.rs | 164 ++++++++++++++++++---------- 1 file changed, 109 insertions(+), 55 deletions(-) diff --git a/scylla/src/transport/locator/mod.rs b/scylla/src/transport/locator/mod.rs index f90457c39f..64b4dc9fa0 100644 --- a/scylla/src/transport/locator/mod.rs +++ b/scylla/src/transport/locator/mod.rs @@ -82,78 +82,99 @@ impl ReplicaLocator { /// parameter of `Self::new`, invocation of this function will trigger a computation of the /// desired replica set (the computation might be delegated in time and start upon interaction /// with the returned `ReplicaSet`). + /// + /// If the requested table uses Tablets, then a separate code path is taken, which ignores + /// replication strategies and only uses tablet information stored in ReplicaLocator. + /// If we don't have info about the tablet that owns the given token, empty set will be returned. pub fn replicas_for_token<'a>( &'a self, token: Token, strategy: &'a Strategy, datacenter: Option<&'a str>, - _table_spec: &TableSpec, + table_spec: &TableSpec, ) -> ReplicaSet<'a> { - match strategy { - Strategy::SimpleStrategy { replication_factor } => { - if let Some(datacenter) = datacenter { - let replicas = self.get_simple_strategy_replicas(token, *replication_factor); - - return ReplicaSet { - inner: ReplicaSetInner::FilteredSimple { - replicas, - datacenter, - }, - token, - }; - } else { - return ReplicaSet { - inner: ReplicaSetInner::Plain( - self.get_simple_strategy_replicas(token, *replication_factor), - ), - token, - }; - } - } - Strategy::NetworkTopologyStrategy { - datacenter_repfactors, - } => { - if let Some(dc) = datacenter { - if let Some(repfactor) = datacenter_repfactors.get(dc) { + if let Some(tablets) = self.tablets.tablets_for_table(table_spec) { + let replicas: Option<&[(Arc, Shard)]> = if let Some(datacenter) = datacenter { + tablets.dc_replicas_for_token(token, datacenter) + } else { + tablets.replicas_for_token(token) + }; + return ReplicaSet { + inner: ReplicaSetInner::PlainSharded(replicas.unwrap_or( + // The table is a tablet table, but we don't have information for given token. + // Let's just return empty set in this case. + &[], + )), + token, + }; + } else { + match strategy { + Strategy::SimpleStrategy { replication_factor } => { + if let Some(datacenter) = datacenter { + let replicas = + self.get_simple_strategy_replicas(token, *replication_factor); + + return ReplicaSet { + inner: ReplicaSetInner::FilteredSimple { + replicas, + datacenter, + }, + token, + }; + } else { return ReplicaSet { inner: ReplicaSetInner::Plain( - self.get_network_strategy_replicas(token, dc, *repfactor), + self.get_simple_strategy_replicas(token, *replication_factor), ), token, }; + } + } + Strategy::NetworkTopologyStrategy { + datacenter_repfactors, + } => { + if let Some(dc) = datacenter { + if let Some(repfactor) = datacenter_repfactors.get(dc) { + return ReplicaSet { + inner: ReplicaSetInner::Plain( + self.get_network_strategy_replicas(token, dc, *repfactor), + ), + token, + }; + } else { + debug!("Datacenter ({}) does not exist!", dc); + return ReplicaSet { + inner: ReplicaSetInner::Plain(EMPTY_REPLICAS), + token, + }; + } } else { - debug!("Datacenter ({}) does not exist!", dc); return ReplicaSet { - inner: ReplicaSetInner::Plain(EMPTY_REPLICAS), + inner: ReplicaSetInner::ChainedNTS { + datacenter_repfactors, + locator: self, + token, + }, token, }; } - } else { - return ReplicaSet { - inner: ReplicaSetInner::ChainedNTS { - datacenter_repfactors, - locator: self, - token, - }, - token, - }; } + Strategy::Other { name, .. } => { + debug!("Unknown strategy ({}), falling back to SimpleStrategy with replication_factor = 1", name) + } + _ => (), } - Strategy::Other { name, .. } => { - debug!("Unknown strategy ({}), falling back to SimpleStrategy with replication_factor = 1", name) - } - _ => (), - } - // Fallback to simple strategy with replication factor = 1. - self.replicas_for_token( - token, - &Strategy::SimpleStrategy { - replication_factor: 1, - }, - datacenter, - _table_spec, - ) + // Fallback to simple strategy with replication factor = 1. + self.replicas_for_token( + token, + &Strategy::SimpleStrategy { + replication_factor: 1, + }, + datacenter, + table_spec, + ) + } } /// Gives access to the token ring, based on which all token ranges/replica sets are computed. @@ -244,6 +265,8 @@ fn with_computed_shard(node: NodeRef, token: Token) -> (NodeRef, Shard) { enum ReplicaSetInner<'a> { Plain(ReplicasArray<'a>), + PlainSharded(&'a [(Arc, Shard)]), + // Represents a set of SimpleStrategy replicas that is limited to a specified datacenter. FilteredSimple { replicas: ReplicasArray<'a>, @@ -301,6 +324,7 @@ impl<'a> ReplicaSet<'a> { pub fn len(&self) -> usize { match &self.inner { ReplicaSetInner::Plain(replicas) => replicas.len(), + ReplicaSetInner::PlainSharded(replicas) => replicas.len(), ReplicaSetInner::FilteredSimple { replicas, datacenter, @@ -345,6 +369,9 @@ impl<'a> ReplicaSet<'a> { ReplicaSetInner::Plain(replicas) => replicas .get(index) .map(|node| with_computed_shard(node, self.token)), + ReplicaSetInner::PlainSharded(replicas) => { + replicas.get(index).map(|(node, shard)| (node, *shard)) + } ReplicaSetInner::FilteredSimple { replicas, datacenter, @@ -400,6 +427,9 @@ impl<'a> IntoIterator for ReplicaSet<'a> { fn into_iter(self) -> Self::IntoIter { let inner = match self.inner { ReplicaSetInner::Plain(replicas) => ReplicaSetIteratorInner::Plain { replicas, idx: 0 }, + ReplicaSetInner::PlainSharded(replicas) => { + ReplicaSetIteratorInner::PlainSharded { replicas, idx: 0 } + } ReplicaSetInner::FilteredSimple { replicas, datacenter, @@ -446,6 +476,10 @@ enum ReplicaSetIteratorInner<'a> { replicas: ReplicasArray<'a>, idx: usize, }, + PlainSharded { + replicas: &'a [(Arc, Shard)], + idx: usize, + }, FilteredSimple { replicas: ReplicasArray<'a>, datacenter: &'a str, @@ -481,6 +515,14 @@ impl<'a> Iterator for ReplicaSetIterator<'a> { None } + ReplicaSetIteratorInner::PlainSharded { replicas, idx } => { + if let Some((replica, shard)) = replicas.get(*idx) { + *idx += 1; + return Some((replica, *shard)); + } + + None + } ReplicaSetIteratorInner::FilteredSimple { replicas, datacenter, @@ -530,6 +572,11 @@ impl<'a> Iterator for ReplicaSetIterator<'a> { (size, Some(size)) } + ReplicaSetIteratorInner::PlainSharded { replicas, idx } => { + let size = replicas.len() - *idx; + + (size, Some(size)) + } ReplicaSetIteratorInner::FilteredSimple { replicas, datacenter: _, @@ -558,7 +605,8 @@ impl<'a> Iterator for ReplicaSetIterator<'a> { fn nth(&mut self, n: usize) -> Option { match &mut self.inner { - ReplicaSetIteratorInner::Plain { replicas: _, idx } => { + ReplicaSetIteratorInner::Plain { replicas: _, idx } + | ReplicaSetIteratorInner::PlainSharded { replicas: _, idx } => { *idx += n; self.next() @@ -581,7 +629,8 @@ impl<'a> ReplicaSet<'a> { } /// Represents a sequence of replicas for a given token and strategy, -/// ordered according to the ring order. +/// ordered according to the ring order (for token-ring tables) or with the +/// order defined by tablet data (for tablet tables). /// /// This container can only be created by calling `ReplicaSet::into_replicas_ordered()`, /// and either it can borrow precomputed replica lists living in the locator (in case of SimpleStrategy) @@ -749,6 +798,11 @@ impl<'a> IntoIterator for ReplicasOrdered<'a> { replica_set_iter: replica_set.into_iter(), } } + ReplicaSetInner::PlainSharded(_) => { + ReplicasOrderedIteratorInner::AlreadyRingOrdered { + replica_set_iter: replica_set.into_iter(), + } + } ReplicaSetInner::ChainedNTS { datacenter_repfactors, locator, From 5b4cada4f42be81c4d8560e3ba245cb0af9a8c80 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Karol=20Bary=C5=82a?= Date: Wed, 24 Apr 2024 02:00:53 +0200 Subject: [PATCH 10/16] Tests: Support running tests on Scylla with tablet support Some tests break when running on Scylla with tablet support, mostly for 2 reasons: - CDC is not yet supported for tablets - Token awareness works differently For affected tests disable tablet awareness for the time being. --- scylla/src/transport/caching_session.rs | 29 +++++++++++------ scylla/src/transport/session_test.rs | 43 ++++++++++++++++++++----- scylla/src/utils/test_utils.rs | 15 ++++++++- scylla/tests/integration/shards.rs | 7 +++- 4 files changed, 74 insertions(+), 20 deletions(-) diff --git a/scylla/src/transport/caching_session.rs b/scylla/src/transport/caching_session.rs index 5a38b47fee..903eb4e346 100644 --- a/scylla/src/transport/caching_session.rs +++ b/scylla/src/transport/caching_session.rs @@ -218,7 +218,7 @@ where #[cfg(test)] mod tests { use crate::query::Query; - use crate::test_utils::{create_new_session_builder, setup_tracing}; + use crate::test_utils::{create_new_session_builder, scylla_supports_tablets, setup_tracing}; use crate::transport::partitioner::PartitionerName; use crate::utils::test_utils::unique_keyspace_name; use crate::{ @@ -229,15 +229,23 @@ mod tests { use futures::TryStreamExt; use std::collections::BTreeSet; - async fn new_for_test() -> Session { + async fn new_for_test(with_tablet_support: bool) -> Session { let session = create_new_session_builder() .build() .await .expect("Could not create session"); let ks = unique_keyspace_name(); + let mut create_ks = format!( + "CREATE KEYSPACE IF NOT EXISTS {ks} + WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}" + ); + if !with_tablet_support && scylla_supports_tablets(&session).await { + create_ks += " AND TABLETS = {'enabled': false}"; + } + session - .query(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]) + .query(create_ks, &[]) .await .expect("Could not create keyspace"); @@ -261,7 +269,7 @@ mod tests { } async fn create_caching_session() -> CachingSession { - let session = CachingSession::from(new_for_test().await, 2); + let session = CachingSession::from(new_for_test(true).await, 2); // Add a row, this makes it easier to check if the caching works combined with the regular execute fn on Session session @@ -419,11 +427,11 @@ mod tests { } let _session: CachingSession = - CachingSession::from(new_for_test().await, 2); + CachingSession::from(new_for_test(true).await, 2); let _session: CachingSession = - CachingSession::from(new_for_test().await, 2); + CachingSession::from(new_for_test(true).await, 2); let _session: CachingSession = - CachingSession::with_hasher(new_for_test().await, 2, Default::default()); + CachingSession::with_hasher(new_for_test(true).await, 2, Default::default()); } #[tokio::test] @@ -552,7 +560,7 @@ mod tests { #[tokio::test] async fn test_parameters_caching() { setup_tracing(); - let session: CachingSession = CachingSession::from(new_for_test().await, 100); + let session: CachingSession = CachingSession::from(new_for_test(true).await, 100); session .execute("CREATE TABLE tbl (a int PRIMARY KEY, b int)", ()) @@ -604,12 +612,13 @@ mod tests { return; } - let session: CachingSession = CachingSession::from(new_for_test().await, 100); + // This test uses CDC which is not yet compatible with Scylla's tablets. + let session: CachingSession = CachingSession::from(new_for_test(false).await, 100); session .execute( "CREATE TABLE tbl (a int PRIMARY KEY) with cdc = {'enabled': true}", - (), + &(), ) .await .unwrap(); diff --git a/scylla/src/transport/session_test.rs b/scylla/src/transport/session_test.rs index 0f7b8ea908..8136477251 100644 --- a/scylla/src/transport/session_test.rs +++ b/scylla/src/transport/session_test.rs @@ -6,7 +6,7 @@ use crate::query::Query; use crate::retry_policy::{QueryInfo, RetryDecision, RetryPolicy, RetrySession}; use crate::routing::Token; use crate::statement::Consistency; -use crate::test_utils::setup_tracing; +use crate::test_utils::{scylla_supports_tablets, setup_tracing}; use crate::tracing::TracingInfo; use crate::transport::cluster::Datacenter; use crate::transport::errors::{BadKeyspaceName, BadQuery, DbError, QueryError}; @@ -518,7 +518,17 @@ async fn test_token_awareness() { let session = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); - session.query(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); + // Need to disable tablets in this test because they make token routing + // work differently, and in this test we want to test the classic token ring + // behavior. + let mut create_ks = format!( + "CREATE KEYSPACE IF NOT EXISTS {ks} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}" + ); + if scylla_supports_tablets(&session).await { + create_ks += " AND TABLETS = {'enabled': false}" + } + + session.query(create_ks, &[]).await.unwrap(); session .query( format!("CREATE TABLE IF NOT EXISTS {}.t (a text primary key)", ks), @@ -1670,10 +1680,15 @@ async fn test_table_partitioner_in_metadata() { let session = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); - session - .query(format!("CREATE KEYSPACE {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]) - .await - .unwrap(); + // This test uses CDC which is not yet compatible with Scylla's tablets. + let mut create_ks = format!( + "CREATE KEYSPACE {ks} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}" + ); + if scylla_supports_tablets(&session).await { + create_ks += " AND TABLETS = {'enabled': false}"; + } + + session.query(create_ks, &[]).await.unwrap(); session.query(format!("USE {}", ks), &[]).await.unwrap(); @@ -1834,7 +1849,14 @@ async fn test_prepared_partitioner() { let session = create_new_session_builder().build().await.unwrap(); let ks = unique_keyspace_name(); - session.query(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}", ks), &[]).await.unwrap(); + // This test uses CDC which is not yet compatible with Scylla's tablets. + let mut create_ks = format!( + "CREATE KEYSPACE IF NOT EXISTS {ks} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}}"); + if scylla_supports_tablets(&session).await { + create_ks += " AND TABLETS = {'enabled': false}" + } + + session.query(create_ks, &[]).await.unwrap(); session.use_keyspace(ks, false).await.unwrap(); session @@ -2551,6 +2573,7 @@ async fn test_keyspaces_to_fetch() { // Reproduces the problem with execute_iter mentioned in #608. #[tokio::test] async fn test_iter_works_when_retry_policy_returns_ignore_write_error() { + setup_tracing(); // It's difficult to reproduce the issue with a real downgrading consistency policy, // as it would require triggering a WriteTimeout. We just need the policy // to return IgnoreWriteError, so we will trigger a different error @@ -2592,7 +2615,11 @@ async fn test_iter_works_when_retry_policy_returns_ignore_write_error() { // Create a keyspace with replication factor that is larger than the cluster size let cluster_size = session.get_cluster_data().get_nodes_info().len(); let ks = unique_keyspace_name(); - session.query(format!("CREATE KEYSPACE {} WITH REPLICATION = {{'class': 'NetworkTopologyStrategy', 'replication_factor': {}}}", ks, cluster_size + 1), ()).await.unwrap(); + let mut create_ks = format!("CREATE KEYSPACE {} WITH REPLICATION = {{'class': 'NetworkTopologyStrategy', 'replication_factor': {}}}", ks, cluster_size + 1); + if scylla_supports_tablets(&session).await { + create_ks += " and TABLETS = { 'enabled': false}"; + } + session.query(create_ks, ()).await.unwrap(); session.use_keyspace(ks, true).await.unwrap(); session .query("CREATE TABLE t (pk int PRIMARY KEY, v int)", ()) diff --git a/scylla/src/utils/test_utils.rs b/scylla/src/utils/test_utils.rs index b0696c7bfb..1b0e0a02d4 100644 --- a/scylla/src/utils/test_utils.rs +++ b/scylla/src/utils/test_utils.rs @@ -1,6 +1,5 @@ #[cfg(test)] use crate::transport::session_builder::{GenericSessionBuilder, SessionBuilderKind}; -#[cfg(test)] use crate::Session; #[cfg(test)] use std::{num::NonZeroU32, time::Duration}; @@ -93,6 +92,20 @@ pub fn create_new_session_builder() -> GenericSessionBuilder bool { + let result = session + .query( + "select column_name from system_schema.columns where + keyspace_name = 'system_schema' + and table_name = 'scylla_keyspaces' + and column_name = 'initial_tablets'", + &[], + ) + .await + .unwrap(); + result.single_row().is_ok() +} + #[cfg(test)] pub(crate) fn setup_tracing() { let _ = tracing_subscriber::fmt::fmt() diff --git a/scylla/tests/integration/shards.rs b/scylla/tests/integration/shards.rs index da6c4f3d2e..ae084cca7a 100644 --- a/scylla/tests/integration/shards.rs +++ b/scylla/tests/integration/shards.rs @@ -1,6 +1,7 @@ use std::sync::Arc; use crate::utils::{setup_tracing, test_with_3_node_cluster}; +use scylla::test_utils::scylla_supports_tablets; use scylla::{test_utils::unique_keyspace_name, SessionBuilder}; use tokio::sync::mpsc; @@ -37,7 +38,11 @@ async fn test_consistent_shard_awareness() { let ks = unique_keyspace_name(); /* Prepare schema */ - session.query(format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 3}}", ks), &[]).await.unwrap(); + let mut create_ks = format!("CREATE KEYSPACE IF NOT EXISTS {} WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 3}}", ks); + if scylla_supports_tablets(&session).await { + create_ks += " and TABLETS = { 'enabled': false}"; + } + session.query(create_ks, &[]).await.unwrap(); session .query( format!( From fad15288440a47c2c7a4a203c013da74803301a2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Karol=20Bary=C5=82a?= Date: Wed, 1 May 2024 17:57:08 +0200 Subject: [PATCH 11/16] Add Tablets CI --- .github/workflows/tablets.yml | 38 +++++++++++++++++++++++++++++++++++ 1 file changed, 38 insertions(+) create mode 100644 .github/workflows/tablets.yml diff --git a/.github/workflows/tablets.yml b/.github/workflows/tablets.yml new file mode 100644 index 0000000000..24e14d66a9 --- /dev/null +++ b/.github/workflows/tablets.yml @@ -0,0 +1,38 @@ +name: Tablets + +on: + push: + branches: + - main + - 'branch-*' + pull_request: + branches: + - main + - 'branch-*' + +env: + CARGO_TERM_COLOR: always + RUST_BACKTRACE: full + +jobs: + build: + runs-on: ubuntu-latest + timeout-minutes: 60 + steps: + - uses: actions/checkout@v3 + - name: Install scylla-ccm + run: pip3 install https://github.com/scylladb/scylla-ccm/archive/master.zip + + - name: Create cluster with tablets enabled + run: | + ccm create tablets -i 127.0.1. -n 3 --scylla -v 'unstable/master:2024-05-01T18:26:10Z' + ccm updateconf 'experimental_features: [consistent-topology-changes, tablets]' + ccm start + + - name: Check + run: cargo check --verbose + - name: Run tablets tests + run: SCYLLA_URI=127.0.1.1:9042 SCYLLA_URI2=127.0.1.2:9042 SCYLLA_URI3=127.0.1.3:9042 cargo test --verbose + + - name: Remove tablets cluster + run: ccm remove tablets From 20fa04ac2d5432ef49fe7c1a77a3d27f31ec3273 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Karol=20Bary=C5=82a?= Date: Wed, 8 May 2024 11:51:51 +0200 Subject: [PATCH 12/16] Add integration tests for tablets --- scylla/tests/integration/main.rs | 1 + scylla/tests/integration/tablets.rs | 614 ++++++++++++++++++++++++++++ 2 files changed, 615 insertions(+) create mode 100644 scylla/tests/integration/tablets.rs diff --git a/scylla/tests/integration/main.rs b/scylla/tests/integration/main.rs index 7f09ae2c5a..06a2ab429a 100644 --- a/scylla/tests/integration/main.rs +++ b/scylla/tests/integration/main.rs @@ -7,4 +7,5 @@ mod retries; mod shards; mod silent_prepare_query; mod skip_metadata_optimization; +mod tablets; pub(crate) mod utils; diff --git a/scylla/tests/integration/tablets.rs b/scylla/tests/integration/tablets.rs new file mode 100644 index 0000000000..00441181a5 --- /dev/null +++ b/scylla/tests/integration/tablets.rs @@ -0,0 +1,614 @@ +use std::sync::Arc; + +use crate::utils::setup_tracing; +use crate::utils::test_with_3_node_cluster; + +use futures::future::try_join_all; +use futures::TryStreamExt; +use itertools::Itertools; +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::ExecutionProfile; +use scylla::QueryResult; +use scylla::Session; + +use scylla_cql::errors::QueryError; +use scylla_proxy::{ + Condition, ProxyError, Reaction, ResponseFrame, ResponseOpcode, ResponseReaction, ResponseRule, + ShardAwareness, TargetShard, WorkerError, +}; + +use tokio::sync::mpsc; +use tracing::info; +use uuid::Uuid; + +#[derive(scylla::FromRow)] +struct SelectedTablet { + last_token: i64, + replicas: Vec<(Uuid, i32)>, +} + +struct Tablet { + first_token: i64, + last_token: i64, + replicas: Vec<(Arc, i32)>, +} + +async fn get_tablets(session: &Session, ks: &str, table: &str) -> Vec { + let cluster_data = session.get_cluster_data(); + let endpoints = cluster_data.get_nodes_info(); + for endpoint in endpoints.iter() { + info!( + "Endpoint id: {}, address: {}", + endpoint.host_id, + endpoint.address.ip() + ); + } + + let selected_tablets_response = session.query_iter( + "select last_token, replicas from system.tablets WHERE keyspace_name = ? and table_name = ? ALLOW FILTERING", + &(ks, table)).await.unwrap(); + + let mut selected_tablets = selected_tablets_response + .into_typed::() + .try_collect::>() + .await + .unwrap(); + selected_tablets.sort_unstable_by(|a, b| a.last_token.cmp(&b.last_token)); + + let (tablets, _) = selected_tablets.iter().fold( + (Vec::new(), i64::MIN), + |(mut tablets, first_token), tablet| { + let replicas = tablet + .replicas + .iter() + .map(|(uuid, shard)| { + ( + Arc::clone( + endpoints + .get(endpoints.iter().position(|e| e.host_id == *uuid).unwrap()) + .unwrap(), + ), + *shard, + ) + }) + .collect(); + let raw_tablet = Tablet { + first_token, + last_token: tablet.last_token, + replicas, + }; + + tablets.push(raw_tablet); + (tablets, tablet.last_token.wrapping_add(1)) + }, + ); + + // Print information about tablets, useful for debugging + for tablet in tablets.iter() { + info!( + "Tablet: [{}, {}]: {:?}", + tablet.first_token, + tablet.last_token, + tablet + .replicas + .iter() + .map(|(replica, shard)| { (replica.address.ip(), shard) }) + .collect::>() + ); + } + + tablets +} + +// Takes a prepared statements which accepts 2 arguments: i32 pk and i32 ck, +// and calculates an example key for each of the tablets in the table. +fn calculate_key_per_tablet(tablets: &[Tablet], prepared: &PreparedStatement) -> Vec<(i32, i32)> { + // Here we calculate a PK per token + let mut present_tablets = vec![false; tablets.len()]; + let mut value_lists = vec![]; + for i in 0..1000 { + let token_value = prepared.calculate_token(&(i, 1)).unwrap().unwrap().value(); + let tablet_idx = tablets + .iter() + .position(|tablet| { + tablet.first_token <= token_value && token_value <= tablet.last_token + }) + .unwrap(); + if !present_tablets[tablet_idx] { + let values = (i, 1); + let tablet = &tablets[tablet_idx]; + info!( + "Values: {:?}, token: {}, tablet index: {}, tablet: [{}, {}], replicas: {:?}", + values, + token_value, + tablet_idx, + tablet.first_token, + tablet.last_token, + tablet + .replicas + .iter() + .map(|(replica, shard)| { (replica.address.ip(), shard) }) + .collect::>() + ); + value_lists.push(values); + present_tablets[tablet_idx] = true; + } + } + + // This function tries 1000 keys and assumes that it is enough to cover all + // tablets. This is just a random number that seems to work in the tests, + // so the assert is here to catch the problem early if 1000 stops being enough + // for some reason. + assert!(present_tablets.iter().all(|x| *x)); + + value_lists +} + +#[derive(Debug)] +struct SingleTargetLBP { + target: (Arc, Option), +} + +impl LoadBalancingPolicy for SingleTargetLBP { + fn pick<'a>( + &'a self, + _query: &'a RoutingInfo, + _cluster: &'a ClusterData, + ) -> Option<(NodeRef<'a>, Option)> { + Some((&self.target.0, self.target.1)) + } + + fn fallback<'a>( + &'a self, + _query: &'a RoutingInfo, + _cluster: &'a ClusterData, + ) -> FallbackPlan<'a> { + Box::new(std::iter::empty()) + } + + fn name(&self) -> String { + "SingleTargetLBP".to_owned() + } +} + +async fn send_statement_everywhere( + session: &Session, + cluster: &ClusterData, + statement: &PreparedStatement, + values: &dyn SerializeRow, +) -> Result, QueryError> { + let tasks = cluster.get_nodes_info().iter().flat_map(|node| { + let shard_count: u16 = node.sharder().unwrap().nr_shards.into(); + (0..shard_count).map(|shard| { + let mut stmt = statement.clone(); + let values_ref = &values; + let policy = SingleTargetLBP { + target: (node.clone(), Some(shard as u32)), + }; + let execution_profile = ExecutionProfile::builder() + .load_balancing_policy(Arc::new(policy)) + .build(); + stmt.set_execution_profile_handle(Some(execution_profile.into_handle())); + + async move { session.execute(&stmt, values_ref).await } + }) + }); + + try_join_all(tasks).await +} + +async fn send_unprepared_query_everywhere( + session: &Session, + cluster: &ClusterData, + query: &Query, +) -> Result, QueryError> { + let tasks = cluster.get_nodes_info().iter().flat_map(|node| { + let shard_count: u16 = node.sharder().unwrap().nr_shards.into(); + (0..shard_count).map(|shard| { + let mut stmt = query.clone(); + let policy = SingleTargetLBP { + target: (node.clone(), Some(shard as u32)), + }; + let execution_profile = ExecutionProfile::builder() + .load_balancing_policy(Arc::new(policy)) + .build(); + stmt.set_execution_profile_handle(Some(execution_profile.into_handle())); + + async move { session.query(stmt, &()).await } + }) + }); + + try_join_all(tasks).await +} + +fn frame_has_tablet_feedback(frame: ResponseFrame) -> bool { + let response = + scylla_cql::frame::parse_response_body_extensions(frame.params.flags, None, frame.body) + .unwrap(); + match response.custom_payload { + Some(map) => map.contains_key("tablets-routing-v1"), + None => false, + } +} + +fn count_tablet_feedbacks( + rx: &mut mpsc::UnboundedReceiver<(ResponseFrame, Option)>, +) -> usize { + std::iter::from_fn(|| rx.try_recv().ok()) + .map(|(frame, _shard)| frame_has_tablet_feedback(frame)) + .filter(|b| *b) + .count() +} + +async fn prepare_schema(session: &Session, ks: &str, table: &str, tablet_count: usize) { + session + .query( + format!( + "CREATE KEYSPACE IF NOT EXISTS {} + WITH REPLICATION = {{'class' : 'NetworkTopologyStrategy', 'replication_factor' : 2}} + AND tablets = {{ 'initial': {} }}", + ks, tablet_count + ), + &[], + ) + .await + .unwrap(); + session + .query( + format!( + "CREATE TABLE IF NOT EXISTS {}.{} (a int, b int, c text, primary key (a, b))", + ks, table + ), + &[], + ) + .await + .unwrap(); +} + +/// Tests that, when using DefaultPolicy with TokenAwareness and querying table +/// that uses tablets: +/// 1. When querying data that belongs to tablet we didn't receive yet we will +/// receive this tablet at some point. +/// 2. When we have all tablets info locally then we'll never receive tablet info. +/// +/// The test first sends 100 queries per tablet and expects to receive tablet info. +/// After that we know we have all the info. The test sends the statements again +/// and expects to not receive any tablet info. +#[cfg(not(scylla_cloud_tests))] +#[tokio::test] +#[ntest::timeout(30000)] +async fn test_default_policy_is_tablet_aware() { + setup_tracing(); + const TABLET_COUNT: usize = 16; + + let res = test_with_3_node_cluster( + ShardAwareness::QueryNode, + |proxy_uris, translation_map, mut running_proxy| async move { + let session = scylla::SessionBuilder::new() + .known_node(proxy_uris[0].as_str()) + .address_translator(Arc::new(translation_map)) + .build() + .await + .unwrap(); + + if !scylla::test_utils::scylla_supports_tablets(&session).await { + tracing::warn!("Skipping test because this Scylla version doesn't support tablets"); + return running_proxy; + } + + let ks = unique_keyspace_name(); + + /* Prepare schema */ + prepare_schema(&session, &ks, "t", TABLET_COUNT).await; + + let tablets = get_tablets(&session, &ks, "t").await; + + let prepared = session + .prepare(format!( + "INSERT INTO {}.t (a, b, c) VALUES (?, ?, 'abc')", + ks + )) + .await + .unwrap(); + + let value_lists = calculate_key_per_tablet(&tablets, &prepared); + + let (feedback_txs, mut feedback_rxs): (Vec<_>, Vec<_>) = (0..3) + .map(|_| mpsc::unbounded_channel::<(ResponseFrame, Option)>()) + .unzip(); + for (i, tx) in feedback_txs.iter().cloned().enumerate() { + running_proxy.running_nodes[i].change_response_rules(Some(vec![ResponseRule( + Condition::ResponseOpcode(ResponseOpcode::Result) + .and(Condition::not(Condition::ConnectionRegisteredAnyEvent)), + ResponseReaction::noop().with_feedback_when_performed(tx), + )])); + } + + // When the driver never received tablet info for any tablet in a given table, + // then it will not be aware that the table is tablet-based and fall back + // to token-ring routing for this table. + // After it receives any tablet for this table: + // - tablet-aware routing will be used for tablets that the driver has locally + // - non-token-aware routing will be used for other tablets (which basically means + // sending the requests to random nodes) + // In the following code I want to make sure that the driver fetches info + // about all the tablets in the table. + // Obvious way to do this would be to, for each tablet, send some requests (here some == 100) + // and expect that at least one will land on non-replica and return tablet feedback. + // This mostly works, but there is a problem: initially driver has no + // tablet information at all for this table so it will fall back to token-ring routing. + // It is possible that token-ring replicas and tablet replicas are the same + // for some tokens. If it happens for the first token that we use in this loop, + // then no matter how many requests we send we are not going to receive any tablet feedback. + // The solution is to iterate over tablets twice. + // + // First iteration guarantees that the driver will receive at least one tablet + // for this table (it is statistically improbable for all tokens used here to have the same + // set of replicas for tablets and token-ring). In practice it will receive all or almost all of the tablets. + // + // Second iteration will not use token-ring routing (because the driver has some tablets + // for this table, so it is aware that the table is tablet based), + // which means that for unknown tablets it will send requests to random nodes, + // and definitely fetch the rest of the tablets. + let mut total_tablets_with_feedback = 0; + for values in value_lists.iter().chain(value_lists.iter()) { + info!( + "First loop, trying key {:?}, token: {}", + values, + prepared.calculate_token(&values).unwrap().unwrap().value() + ); + try_join_all((0..100).map(|_| async { session.execute(&prepared, values).await })) + .await + .unwrap(); + let feedbacks: usize = feedback_rxs.iter_mut().map(count_tablet_feedbacks).sum(); + if feedbacks > 0 { + total_tablets_with_feedback += 1; + } + } + + assert_eq!(total_tablets_with_feedback, TABLET_COUNT); + + // Now we must have info about all the tablets. It should not be + // possible to receive any feedback if DefaultPolicy is properly + // tablet-aware. + for values in value_lists.iter() { + info!( + "Second loop, trying key {:?}, token: {}", + values, + prepared.calculate_token(&values).unwrap().unwrap().value() + ); + try_join_all((0..100).map(|_| async { session.execute(&prepared, values).await })) + .await + .unwrap(); + let feedbacks: usize = feedback_rxs.iter_mut().map(count_tablet_feedbacks).sum(); + assert_eq!(feedbacks, 0); + } + + running_proxy + }, + ) + .await; + match res { + Ok(()) => (), + Err(ProxyError::Worker(WorkerError::DriverDisconnected(_))) => (), + Err(err) => panic!("{}", err), + } +} + +/// This test verifies that Scylla never sends tablet info when receiving unprepared +/// query - as it would make no sens to send it (driver can't do token awareness +/// for unprepared queries). +/// +/// The test sends a query to each shard of every node and verifies that no +/// tablet info was sent in response. +#[cfg(not(scylla_cloud_tests))] +#[tokio::test] +#[ntest::timeout(30000)] +async fn test_tablet_feedback_not_sent_for_unprepared_queries() { + setup_tracing(); + const TABLET_COUNT: usize = 16; + + let res = test_with_3_node_cluster( + ShardAwareness::QueryNode, + |proxy_uris, translation_map, mut running_proxy| async move { + let session = scylla::SessionBuilder::new() + .known_node(proxy_uris[0].as_str()) + .address_translator(Arc::new(translation_map)) + .build() + .await + .unwrap(); + + if !scylla::test_utils::scylla_supports_tablets(&session).await { + tracing::warn!("Skipping test because this Scylla version doesn't support tablets"); + return running_proxy; + } + + let ks = unique_keyspace_name(); + + /* Prepare schema */ + prepare_schema(&session, &ks, "t", TABLET_COUNT).await; + + let (feedback_txs, mut feedback_rxs): (Vec<_>, Vec<_>) = (0..3) + .map(|_| mpsc::unbounded_channel::<(ResponseFrame, Option)>()) + .unzip(); + for (i, tx) in feedback_txs.iter().cloned().enumerate() { + running_proxy.running_nodes[i].change_response_rules(Some(vec![ResponseRule( + Condition::ResponseOpcode(ResponseOpcode::Result) + .and(Condition::not(Condition::ConnectionRegisteredAnyEvent)), + ResponseReaction::noop().with_feedback_when_performed(tx), + )])); + } + + // I expect Scylla to not send feedback for unprepared queries, + // as such queries cannot be token-aware anyway + 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')")), + ) + .await + .unwrap(); + + let feedbacks: usize = feedback_rxs.iter_mut().map(count_tablet_feedbacks).sum(); + assert!(feedbacks == 0); + + running_proxy + }, + ) + .await; + match res { + Ok(()) => (), + Err(ProxyError::Worker(WorkerError::DriverDisconnected(_))) => (), + Err(err) => panic!("{}", err), + } +} + +/// Verifies that LWT optimization (sending LWT queries to the replicas in a fixed order) +/// works correctly for tablet tables when we have all tablet info locally. +/// +/// The test first fetches all tablet info: by sending a query to each shard of each node, +/// for every tablet. +/// After that it sends 100 queries fro each tablet and verifies that only 1 shard on 1 node +/// recevied requests for a given tablet. +#[cfg(not(scylla_cloud_tests))] +#[tokio::test] +#[ntest::timeout(30000)] +async fn test_lwt_optimization_works_with_tablets() { + setup_tracing(); + const TABLET_COUNT: usize = 16; + + let res = test_with_3_node_cluster( + ShardAwareness::QueryNode, + |proxy_uris, translation_map, mut running_proxy| async move { + let session = scylla::SessionBuilder::new() + .known_node(proxy_uris[0].as_str()) + .address_translator(Arc::new(translation_map)) + .build() + .await + .unwrap(); + + if !scylla::test_utils::scylla_supports_tablets(&session).await { + tracing::warn!("Skipping test because this Scylla version doesn't support tablets"); + return running_proxy; + } + + let ks = unique_keyspace_name(); + + /* Prepare schema */ + prepare_schema(&session, &ks, "t", TABLET_COUNT).await; + + let tablets = get_tablets(&session, &ks, "t").await; + + let prepared_insert = session + .prepare(format!( + "INSERT INTO {}.t (a, b, c) VALUES (?, ?, null)", + ks + )) + .await + .unwrap(); + + let prepared_lwt_update = session + .prepare(format!( + "UPDATE {}.t SET c = ? WHERE a = ? and b = ? IF c != null", + ks + )) + .await + .unwrap(); + + let value_lists = calculate_key_per_tablet(&tablets, &prepared_insert); + + let (feedback_txs, mut feedback_rxs): (Vec<_>, Vec<_>) = (0..3) + .map(|_| mpsc::unbounded_channel::<(ResponseFrame, Option)>()) + .unzip(); + for (i, tx) in feedback_txs.iter().cloned().enumerate() { + running_proxy.running_nodes[i].change_response_rules(Some(vec![ResponseRule( + Condition::ResponseOpcode(ResponseOpcode::Result) + .and(Condition::not(Condition::ConnectionRegisteredAnyEvent)), + ResponseReaction::noop().with_feedback_when_performed(tx), + )])); + } + + // Unlike test_tablet_awareness_works_with_full_info I use "send_statement_everywhere", + // in order to make the test faster (it sends one request per shard, not 100 requests). + for values in value_lists.iter() { + info!( + "First loop, trying key {:?}, token: {}", + values, + prepared_insert + .calculate_token(&values) + .unwrap() + .unwrap() + .value() + ); + send_statement_everywhere( + &session, + session.get_cluster_data().as_ref(), + &prepared_insert, + values, + ) + .await + .unwrap(); + let feedbacks: usize = feedback_rxs.iter_mut().map(count_tablet_feedbacks).sum(); + assert!(feedbacks > 0); + } + + // We have all the info about tablets. + // Executing LWT queries should not yield any more feedbacks. + // All queries for given key should also land in a single replica. + for (a, b) in value_lists.iter() { + info!( + "Second loop, trying key {:?}, token: {}", + (a, b), + prepared_insert + .calculate_token(&(a, b)) + .unwrap() + .unwrap() + .value() + ); + try_join_all((0..100).map(|_| async { + session.execute(&prepared_lwt_update, &("abc", a, b)).await + })) + .await + .unwrap(); + + let mut queried_nodes = 0; + feedback_rxs.iter_mut().for_each(|rx| { + let frames = std::iter::from_fn(|| rx.try_recv().ok()).collect::>(); + let feedbacks_count = frames + .iter() + .map(|(frame, _shard)| frame_has_tablet_feedback(frame.clone())) + .filter(|b| *b) + .count(); + assert_eq!(feedbacks_count, 0); + + let queried_shards = + frames.iter().map(|(_frame, shard)| *shard).unique().count(); + assert!(queried_shards <= 1); + + if queried_shards == 1 { + queried_nodes += 1; + } + }); + + assert_eq!(queried_nodes, 1); + } + + running_proxy + }, + ) + .await; + match res { + Ok(()) => (), + Err(ProxyError::Worker(WorkerError::DriverDisconnected(_))) => (), + Err(err) => panic!("{}", err), + } +} From 9fd6f5fa7ff9af3e601c4f1cc250e2ac8ef41def Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Karol=20Bary=C5=82a?= Date: Tue, 26 Mar 2024 15:48:29 +0100 Subject: [PATCH 13/16] ClusterData: Remove `all_nodes` variable in `new()` This variable was unused, probably some leftover. It's interesting that we didn't get any warnings about it not being used (no reads, it was only written). --- scylla/src/transport/cluster.rs | 3 --- 1 file changed, 3 deletions(-) diff --git a/scylla/src/transport/cluster.rs b/scylla/src/transport/cluster.rs index 734d7937b6..fcae48503c 100644 --- a/scylla/src/transport/cluster.rs +++ b/scylla/src/transport/cluster.rs @@ -292,7 +292,6 @@ impl ClusterData { HashMap::with_capacity(metadata.peers.len()); let mut ring: Vec<(Token, Arc)> = Vec::new(); let mut datacenters: HashMap = HashMap::new(); - let mut all_nodes: Vec> = Vec::with_capacity(metadata.peers.len()); for peer in metadata.peers { // Take existing Arc if possible, otherwise create new one @@ -344,8 +343,6 @@ impl ClusterData { for token in peer_tokens { ring.push((token, node.clone())); } - - all_nodes.push(node); } Self::update_rack_count(&mut datacenters); From f763fcd2e13709903875d86d6f89636714decf50 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Karol=20Bary=C5=82a?= Date: Mon, 6 May 2024 14:28:38 +0200 Subject: [PATCH 14/16] Tablets: handle schema / topology changes There are some situations where tablet information can become stale. This can have various consequences: - it would prevent old `Node` objects from being destroyed and make them continue to being used in LBP. - some of the replicas may be ignored indefinitely by the driver, increasing load on the rest. To prevent this we need to update tablet info during topology change with newly fetched data about cluster. --- scylla/src/transport/cluster.rs | 62 ++- scylla/src/transport/locator/tablets.rs | 703 +++++++++++++++++++++++- 2 files changed, 741 insertions(+), 24 deletions(-) diff --git a/scylla/src/transport/cluster.rs b/scylla/src/transport/cluster.rs index fcae48503c..2f576f33ea 100644 --- a/scylla/src/transport/cluster.rs +++ b/scylla/src/transport/cluster.rs @@ -20,7 +20,7 @@ use itertools::Itertools; use scylla_cql::errors::{BadQuery, NewSessionError}; use scylla_cql::frame::response::result::TableSpec; use scylla_cql::types::serialize::row::SerializedValues; -use std::collections::HashMap; +use std::collections::{HashMap, HashSet}; use std::net::SocketAddr; use std::sync::Arc; use std::time::Duration; @@ -285,7 +285,7 @@ impl ClusterData { known_peers: &HashMap>, used_keyspace: &Option, host_filter: Option<&dyn HostFilter>, - tablets: TabletsInfo, + mut tablets: TabletsInfo, ) -> Self { // Create new updated known_peers and ring let mut new_known_peers: HashMap> = @@ -345,6 +345,47 @@ impl ClusterData { } } + { + let removed_nodes = { + let mut removed_nodes = HashSet::new(); + for old_peer in known_peers { + if !new_known_peers.contains_key(old_peer.0) { + removed_nodes.insert(*old_peer.0); + } + } + + removed_nodes + }; + + let table_predicate = |spec: &TableSpec| { + if let Some(ks) = metadata.keyspaces.get(spec.ks_name()) { + ks.tables.contains_key(spec.table_name()) + } else { + false + } + }; + + let recreated_nodes = { + let mut recreated_nodes = HashMap::new(); + for (old_peer_id, old_peer_node) in known_peers { + if let Some(new_peer_node) = new_known_peers.get(old_peer_id) { + if !Arc::ptr_eq(old_peer_node, new_peer_node) { + recreated_nodes.insert(*old_peer_id, Arc::clone(new_peer_node)); + } + } + } + + recreated_nodes + }; + + tablets.perform_maintenance( + &table_predicate, + &removed_nodes, + &new_known_peers, + &recreated_nodes, + ) + } + Self::update_rack_count(&mut datacenters); let keyspaces = metadata.keyspaces; @@ -491,7 +532,22 @@ impl ClusterData { let replica_translator = |uuid: Uuid| self.known_peers.get(&uuid).cloned(); for (table, raw_tablet) in raw_tablets.into_iter() { - let tablet = Tablet::from_raw_tablet(&raw_tablet, replica_translator); + // Should we skip tablets that belong to a keyspace not present in + // self.keyspaces? The keyspace could have been, without driver's knowledge: + // 1. Dropped - in which case we'll remove its info soon (when refreshing + // topology) anyway. + // 2. Created - no harm in storing the info now. + // + // So I think we can safely skip checking keyspace presence. + let tablet = match Tablet::from_raw_tablet(raw_tablet, replica_translator) { + Ok(t) => t, + Err((t, f)) => { + debug!("Nodes ({}) that are replicas for a tablet {{ks: {}, table: {}, range: [{}. {}]}} not present in current ClusterData.known_peers. \ + Skipping these replicas until topology refresh", + f.iter().format(", "), table.ks_name(), table.table_name(), t.range().0.value(), t.range().1.value()); + t + } + }; self.locator.tablets.add_tablet(table, tablet); } } diff --git a/scylla/src/transport/locator/tablets.rs b/scylla/src/transport/locator/tablets.rs index 19bd7efd50..a0899b4652 100644 --- a/scylla/src/transport/locator/tablets.rs +++ b/scylla/src/transport/locator/tablets.rs @@ -1,4 +1,5 @@ #![allow(dead_code)] +use itertools::Itertools; use lazy_static::lazy_static; use scylla_cql::cql_to_rust::FromCqlVal; use scylla_cql::frame::frame_errors::ParseError; @@ -10,7 +11,7 @@ use uuid::Uuid; use crate::routing::{Shard, Token}; use crate::transport::Node; -use std::collections::HashMap; +use std::collections::{HashMap, HashSet}; use std::sync::Arc; #[derive(Error, Debug)] @@ -21,7 +22,7 @@ pub(crate) enum TabletParsingError { ShardNum(i32), } -#[derive(Debug, PartialEq, Eq)] +#[derive(Debug, PartialEq, Eq, Clone)] struct RawTabletReplicas { replicas: Vec<(Uuid, Shard)>, } @@ -95,18 +96,25 @@ struct TabletReplicas { } impl TabletReplicas { + /// Gets raw replica list (which is an array of (Uuid, Shard)), retrieves + /// `Node` objects and groups node replicas by DC to make life easier for LBP. + /// In case of failure this function returns Self, but with the problematic nodes skipped, + /// and a list of skipped uuids - so that the caller can e.g. do some logging. pub(crate) fn from_raw_replicas( raw_replicas: &RawTabletReplicas, replica_translator: impl Fn(Uuid) -> Option>, - ) -> Self { - let all: Vec<_> = raw_replicas.replicas + ) -> Result)> { + let mut failed = Vec::new(); + let all: Vec<_> = raw_replicas + .replicas .iter() - .filter_map(|(replica, shard)| if let Some(r) = replica_translator(*replica) { - Some((r, *shard as Shard)) - } else { - // TODO: Should this be an error? When can this happen? - warn!("Node {replica} from system.tablets not present in ClusterData.known_peers. Skipping this replica"); - None + .filter_map(|(replica, shard)| { + if let Some(r) = replica_translator(*replica) { + Some((r, *shard as Shard)) + } else { + failed.push(*replica); + None + } }) .collect(); @@ -121,6 +129,27 @@ impl TabletReplicas { } }); + if failed.is_empty() { + Ok(Self { all, per_dc }) + } else { + Err((Self { all, per_dc }, failed)) + } + } + + #[cfg(test)] + fn new_for_test(replicas: Vec>) -> Self { + let all = replicas.into_iter().map(|r| (r, 0)).collect::>(); + let mut per_dc: HashMap, Shard)>> = HashMap::new(); + all.iter().for_each(|(replica, shard)| { + if let Some(dc) = replica.datacenter.as_ref() { + if let Some(replicas) = per_dc.get_mut(dc) { + replicas.push((Arc::clone(replica), *shard)); + } else { + per_dc.insert(dc.to_string(), vec![(Arc::clone(replica), *shard)]); + } + } + }); + Self { all, per_dc } } } @@ -176,17 +205,95 @@ pub(crate) struct Tablet { /// Last token belonging to the tablet, inclusive last_token: Token, replicas: TabletReplicas, + /// If any of the replicas failed to resolve to a Node, + /// then this field will contain the original list of replicas. + failed: Option, } impl Tablet { pub(crate) fn from_raw_tablet( - raw_tablet: &RawTablet, + raw_tablet: RawTablet, + replica_translator: impl Fn(Uuid) -> Option>, + ) -> Result)> { + let replicas_result = + TabletReplicas::from_raw_replicas(&raw_tablet.replicas, replica_translator); + match replicas_result { + Ok(replicas) => Ok(Self { + first_token: raw_tablet.first_token, + last_token: raw_tablet.last_token, + replicas, + failed: None, + }), + Err((replicas, failed_replicas)) => Err(( + Self { + first_token: raw_tablet.first_token, + last_token: raw_tablet.last_token, + replicas, + failed: Some(raw_tablet.replicas), + }, + failed_replicas, + )), + } + } + + pub(crate) fn range(&self) -> (Token, Token) { + (self.first_token, self.last_token) + } + + // Returns `Ok(())` if after the operation Tablet replicas are fully resolved. + // Return `Err(replicas)` if some replicas failed to resolve. `replicas` is a + // list of Uuids that failed to resolve. + fn re_resolve_replicas( + &mut self, replica_translator: impl Fn(Uuid) -> Option>, - ) -> Self { + ) -> Result<(), Vec> { + if let Some(failed) = self.failed.as_ref() { + match TabletReplicas::from_raw_replicas(failed, replica_translator) { + Ok(resolved_replicas) => { + // We managed to successfully resolve all replicas, all is well. + self.replicas = resolved_replicas; + self.failed = None; + Ok(()) + } + Err((_, failed)) => Err(failed), + } + } else { + Ok(()) + } + } + + fn update_stale_nodes(&mut self, recreated_nodes: &HashMap>) { + let mut any_updated = false; + for (node, _) in self.replicas.all.iter_mut() { + if let Some(new_node) = recreated_nodes.get(&node.host_id) { + assert!(!Arc::ptr_eq(new_node, node)); + any_updated = true; + *node = Arc::clone(new_node); + } + } + + if any_updated { + // Now that we know we have some nodes to update we need to go over + // per-dc nodes and update them too. + for (_, dc_nodes) in self.replicas.per_dc.iter_mut() { + for (node, _) in dc_nodes.iter_mut() { + if let Some(new_node) = recreated_nodes.get(&node.host_id) { + *node = Arc::clone(new_node); + } + } + } + } + } + + #[cfg(test)] + fn new_for_test(token: i64, replicas: Vec>, failed: Option>) -> Self { Self { - first_token: raw_tablet.first_token, - last_token: raw_tablet.last_token, - replicas: TabletReplicas::from_raw_replicas(&raw_tablet.replicas, replica_translator), + first_token: Token::new(token), + last_token: Token::new(token), + replicas: TabletReplicas::new_for_test(replicas), + failed: failed.map(|vec| RawTabletReplicas { + replicas: vec.into_iter().map(|id| (id, 0)).collect::>(), + }), } } } @@ -204,6 +311,12 @@ impl Tablet { pub(crate) struct TableTablets { table_spec: TableSpec<'static>, tablet_list: Vec, + /// In order to make typical tablet maintance faster + /// we remember if there were any tablets that have unrecognized uuids in replica list. + /// If there were none, and a few other conditions are satisfied, we can skip nearly whole maintanace. + /// This flag may be falsely true: if we add tablet with unknown replica but later + /// overwrite it with some other tablet. + has_unknown_replicas: bool, } impl TableTablets { @@ -211,6 +324,7 @@ impl TableTablets { Self { table_spec, tablet_list: Default::default(), + has_unknown_replicas: false, } } @@ -248,6 +362,9 @@ impl TableTablets { /// /// This preserves the invariant that all tablets in `self` are non-overlapping. fn add_tablet(&mut self, tablet: Tablet) { + if tablet.failed.is_some() { + self.has_unknown_replicas = true; + } // Smallest `left_idx` for which `tablet.first_token` is LESS OR EQUAL to `tablet_list[left_idx].last_token`. // It implies that `tablet_list[left_idx]` overlaps with `tablet` iff `tablet.last_token` // is GREATER OR EQUAL to `tablet_list[left_idx].first_token`. @@ -265,6 +382,54 @@ impl TableTablets { self.tablet_list.insert(left_idx, tablet); } + fn perform_maintenance( + &mut self, + removed_nodes: &HashSet, + all_current_nodes: &HashMap>, + recreated_nodes: &HashMap>, + ) { + // First we need to re-resolve unknown replicas or remove their tablets. + // It will make later checks easier because we'll know that `failed` field + // is `None` for all tablets. + if self.has_unknown_replicas { + self.tablet_list.retain_mut(|tablet| { + let r = tablet.re_resolve_replicas(|id: Uuid| all_current_nodes.get(&id).cloned()); + if let Err(failed) = &r { + warn!("Nodes ({}) listed as replicas for a tablet {{ks: {}, table: {}, range: [{}. {}]}} are not present in ClusterData.known_peers, \ + despite topology refresh. Removing problematic tablet.", + failed.iter().format(", "), self.table_spec.ks_name(), self.table_spec.table_name(), tablet.first_token.value(), tablet.last_token.value()); + } + + r.is_ok() + }); + } + + // Now we remove all tablets that have replicas on removed nodes. + if !removed_nodes.is_empty() { + self.tablet_list.retain(|tablet| { + tablet + .replicas + .all + .iter() + .all(|node| !removed_nodes.contains(&node.0.host_id)) + }); + } + + // The last thing to do is to replace all old `Node` objects. + // Situations where driver requires this don't happen often: + // - Node IP change + // - Node DC change / Rack change + // so I don't think we should be too concerned about performance of this code. + if !recreated_nodes.is_empty() { + for tablet in self.tablet_list.iter_mut() { + tablet.update_stale_nodes(recreated_nodes); + } + } + + // All unknown replicas were either resolved or whole tablets removed. + self.has_unknown_replicas = false; + } + #[cfg(test)] fn new_for_test() -> Self { Self::new(TableSpec::borrowed("test_ks", "test_table")) @@ -272,6 +437,7 @@ impl TableTablets { } #[derive(Clone, Debug)] +#[cfg_attr(test, derive(PartialEq, Eq))] pub(crate) struct TabletsInfo { // We use hashbrown hashmap instead of std hashmap because with // std one it is not possible to query map with key `TableSpec<'static>` @@ -282,12 +448,16 @@ pub(crate) struct TabletsInfo { // HashBrown on the other hand requires only `Q: Hash + Equivalent + ?Sized`, // and it is easy to create a wrapper type with required `Equivalent` impl. tablets: hashbrown::HashMap, TableTablets>, + /// See `has_unknown_replicas` field in `TableTablets`. + /// The field here will be true if it is true for any TableTablets. + has_unknown_replicas: bool, } impl TabletsInfo { pub(crate) fn new() -> Self { Self { tablets: hashbrown::HashMap::new(), + has_unknown_replicas: false, } } @@ -314,6 +484,9 @@ impl TabletsInfo { } pub(crate) fn add_tablet(&mut self, table_spec: TableSpec<'static>, tablet: Tablet) { + if tablet.failed.is_some() { + self.has_unknown_replicas = true; + } self.tablets .entry(table_spec) .or_insert_with_key(|k| { @@ -326,26 +499,107 @@ impl TabletsInfo { }) .add_tablet(tablet) } + + /// This method is supposed to be called when topology is updated. + /// It goes through tablet info and adjusts it to topology changes, to prevent + /// a situation where local tablet info and a real one are permanently different. + /// What is updated: + /// 1. Info for dropped tables is removed. + /// 2. Tablets where a removed node was one of replicas are removed. + /// Can be skipped if no nodes were removed. + /// 3. Tablets with unrecognized uuids in replica list are resolved again. + /// If this is unsuccessful again then the tablet is removed. + /// This can be skipped if we know we have no such tablets. + /// 4. Rarely, the driver may need to re-create `Node` object for a given node. + /// The old object is replaced with the new one in replica lists. + /// This can be skipped if there were no re-created `Node` objects. + /// + /// In order to not perform unnecessary work during typical schema refresh + /// we avoid iterating through tablets at all if steps 2-4 can be skipped. + /// + /// * `removed_nodes`: Nodes that previously were present in ClusterData but are not anymore. + /// For any such node we should remove all tablets that have it in replica list. + /// This is because otherwise: + /// 1. We would keep old `Node` objects, not allowing them to release memory. + /// 2. We would return removed nodes in LBP + /// 3. When a new node joins and becomes replica for this tablet, we would + /// not use it - instead we would keep querying a subset of replicas. + /// + /// * `all_current_nodes`: Map of all nodes. Required to remap unknown replicas. + /// If we didn't try to remap them and instead just skipped them, + /// then we would only query subset of replicas for the tablet, + /// potentially increasing load on those replicas. + /// The alternative is dropping the tablet immediately, but if there are a lot + /// of requests to a range belonging to this tablet, then we would get a + /// lot of unnecessary feedbacks sent. Thus the current solution: + /// skipping unknown replicas and dropping the tablet if we still can't resolve + /// them after topology refresh. + /// + /// * `recreated_nodes`: There are some situations (IP change, DC / Rack change) where the driver + /// will create a new `Node` object for some node and drop the old one. + /// Tablet info would still contain the old object, so the driver would not use + /// new connections. That means if there were such nodes then we need to go over + /// tablets and replace `Arc` objects for recreated nodes. + /// + /// There are some situations not handled by this maintanance procedure that could + /// still result in permanent difference between local and real tablet info: + /// + /// * Extending replica list for a tablet: If a new replica is added to replica list, + /// then we won't learn about it, because we'll keep querying current replicas, which are + /// still replicas. I'm not sure if this can happen. The only scenario where this seems + /// possible is increasing RF - I'm not sure if this would only add replicas or make more changes. + /// We could probably discover it by comparing replication strategy pre and post topology referesh + /// and if it changed then remove tablet info for this keyspace. + /// + /// * Removing the keyspace and recreating it immediately without tablets. This seems so absurd + /// that we most likely don't need to worry about it, but I'm putting it here as a potential problem + /// for completeness. + + pub(crate) fn perform_maintenance( + &mut self, + table_predicate: &impl Fn(&TableSpec) -> bool, + removed_nodes: &HashSet, + all_current_nodes: &HashMap>, + recreated_nodes: &HashMap>, + ) { + // First we remove info for all tables that are no longer present. + self.tablets.retain(|k, _| table_predicate(k)); + + if !removed_nodes.is_empty() || !recreated_nodes.is_empty() || self.has_unknown_replicas { + for (_, table_tablets) in self.tablets.iter_mut() { + table_tablets.perform_maintenance( + removed_nodes, + all_current_nodes, + recreated_nodes, + ); + } + } + + // All unknown replicas were either resolved or whole tablets removed. + self.has_unknown_replicas = false; + } } #[cfg(test)] mod tests { - use std::collections::HashMap; + use std::collections::{HashMap, HashSet}; use std::sync::Arc; - use scylla_cql::frame::response::result::{ColumnType, CqlValue}; - use scylla_cql::types::serialize::{value::SerializeCql, CellWriter}; + use scylla_cql::frame::response::result::{ColumnType, CqlValue, TableSpec}; + use scylla_cql::types::serialize::value::SerializeCql; + use scylla_cql::types::serialize::CellWriter; use tracing::debug; use uuid::Uuid; use crate::routing::Token; + use crate::test_utils::setup_tracing; use crate::transport::locator::tablets::{ RawTablet, RawTabletReplicas, TabletParsingError, CUSTOM_PAYLOAD_TABLETS_V1_KEY, RAW_TABLETS_CQL_TYPE, }; use crate::transport::Node; - use super::{TableTablets, Tablet, TabletReplicas}; + use super::{TableTablets, Tablet, TabletReplicas, TabletsInfo}; const DC1: &str = "dc1"; const DC2: &str = "dc2"; @@ -534,14 +788,14 @@ mod tests { assert_eq!( replicas, - TabletReplicas { + Ok(TabletReplicas { all: replicas_uids .iter() .cloned() .map(|replica| (translator(replica).unwrap(), 1)) .collect(), per_dc - } + }) ); } @@ -567,6 +821,7 @@ mod tests { first_token: Token::new(*first), last_token: Token::new(*last), replicas: Default::default(), + failed: None, }); } } @@ -723,4 +978,410 @@ mod tests { ], ); } + + const SOME_KS: TableSpec<'static> = TableSpec::borrowed("ks", "tbl"); + + fn node_map(nodes: &[&Arc]) -> HashMap> { + nodes.iter().map(|n| (n.host_id, Arc::clone(n))).collect() + } + + #[test] + fn table_maintenance_tests() { + setup_tracing(); + + let node1 = Arc::new(Node::new_for_test( + Some(Uuid::from_u128(1)), + None, + Some(DC1.to_owned()), + None, + )); + let node2 = Arc::new(Node::new_for_test( + Some(Uuid::from_u128(2)), + None, + Some(DC2.to_owned()), + None, + )); + let node2_v2 = Arc::new(Node::new_for_test( + Some(Uuid::from_u128(2)), + None, + Some(DC2.to_owned()), + None, + )); + let node3 = Arc::new(Node::new_for_test( + Some(Uuid::from_u128(3)), + None, + Some(DC3.to_owned()), + None, + )); + let node3_v2 = Arc::new(Node::new_for_test( + Some(Uuid::from_u128(3)), + None, + Some(DC3.to_owned()), + None, + )); + + type MaintenanceArgs = ( + HashSet, + HashMap>, + HashMap>, + ); + let tests: &mut [(TableTablets, MaintenanceArgs, TableTablets)] = &mut [ + ( + // [Case 0] Nothing changes, no maintenance required + TableTablets { + table_spec: SOME_KS, + tablet_list: vec![ + Tablet::new_for_test(0, vec![node1.clone(), node2.clone()], None), + Tablet::new_for_test(1, vec![node2.clone(), node3.clone()], None), + Tablet::new_for_test(2, vec![node3.clone(), node1.clone()], None), + Tablet::new_for_test(3, vec![node3.clone()], None), + ], + has_unknown_replicas: false, + }, + ( + HashSet::new(), + node_map(&[&node1, &node2, &node3]), + HashMap::new(), + ), + TableTablets { + table_spec: SOME_KS, + tablet_list: vec![ + Tablet::new_for_test(0, vec![node1.clone(), node2.clone()], None), + Tablet::new_for_test(1, vec![node2.clone(), node3.clone()], None), + Tablet::new_for_test(2, vec![node3.clone(), node1.clone()], None), + Tablet::new_for_test(3, vec![node3.clone()], None), + ], + has_unknown_replicas: false, + }, + ), + ( + // [Case 1] Removed node + TableTablets { + table_spec: SOME_KS, + tablet_list: vec![ + Tablet::new_for_test(0, vec![node1.clone(), node2.clone()], None), + Tablet::new_for_test(1, vec![node2.clone(), node3.clone()], None), + Tablet::new_for_test(2, vec![node3.clone(), node1.clone()], None), + Tablet::new_for_test(3, vec![node3.clone()], None), + ], + has_unknown_replicas: false, + }, + ( + HashSet::from([node1.host_id]), + node_map(&[&node2, &node3]), + HashMap::new(), + ), + TableTablets { + table_spec: SOME_KS, + tablet_list: vec![ + Tablet::new_for_test(1, vec![node2.clone(), node3.clone()], None), + Tablet::new_for_test(3, vec![node3.clone()], None), + ], + has_unknown_replicas: false, + }, + ), + ( + // [Case 2] Multiple removed nodes + TableTablets { + table_spec: SOME_KS, + tablet_list: vec![ + Tablet::new_for_test(0, vec![node1.clone(), node2.clone()], None), + Tablet::new_for_test(1, vec![node2.clone(), node3.clone()], None), + Tablet::new_for_test(2, vec![node3.clone(), node1.clone()], None), + Tablet::new_for_test(3, vec![node3.clone()], None), + ], + has_unknown_replicas: false, + }, + ( + HashSet::from([node1.host_id, node2.host_id]), + node_map(&[&node3]), + HashMap::new(), + ), + TableTablets { + table_spec: SOME_KS, + tablet_list: vec![Tablet::new_for_test(3, vec![node3.clone()], None)], + has_unknown_replicas: false, + }, + ), + ( + // [Case 3] Nodes with unresolved replicas + TableTablets { + table_spec: SOME_KS, + tablet_list: vec![ + Tablet::new_for_test( + 0, + vec![node2.clone()], + Some(vec![node1.host_id, node2.host_id]), + ), + Tablet::new_for_test( + 1, + vec![node2.clone()], + Some(vec![node2.host_id, node3.host_id]), + ), + Tablet::new_for_test(2, vec![node3.clone(), node1.clone()], None), + Tablet::new_for_test(3, vec![node3.clone()], None), + ], + has_unknown_replicas: true, + }, + ( + HashSet::new(), + node_map(&[&node1, &node2, &node3]), + HashMap::new(), + ), + TableTablets { + table_spec: SOME_KS, + tablet_list: vec![ + Tablet::new_for_test(0, vec![node1.clone(), node2.clone()], None), + Tablet::new_for_test(1, vec![node2.clone(), node3.clone()], None), + Tablet::new_for_test(2, vec![node3.clone(), node1.clone()], None), + Tablet::new_for_test(3, vec![node3.clone()], None), + ], + has_unknown_replicas: false, + }, + ), + ( + // [Case 4] Some replicas still unresolved + TableTablets { + table_spec: SOME_KS, + tablet_list: vec![ + Tablet::new_for_test( + 0, + vec![node2.clone()], + Some(vec![node1.host_id, node2.host_id]), + ), + Tablet::new_for_test( + 1, + vec![node2.clone()], + Some(vec![node2.host_id, node3.host_id]), + ), + Tablet::new_for_test(2, vec![], Some(vec![node3.host_id, node1.host_id])), + Tablet::new_for_test(3, vec![node2.clone()], None), + Tablet::new_for_test(4, vec![], Some(vec![node3.host_id])), + ], + has_unknown_replicas: true, + }, + (HashSet::new(), node_map(&[&node2, &node3]), HashMap::new()), + TableTablets { + table_spec: SOME_KS, + tablet_list: vec![ + Tablet::new_for_test(1, vec![node2.clone(), node3.clone()], None), + Tablet::new_for_test(3, vec![node2.clone()], None), + Tablet::new_for_test(4, vec![node3.clone()], None), + ], + has_unknown_replicas: false, + }, + ), + ( + // [Case 5] Incorrectly set "has_unknown_replicas" - unknown replicas should be ignored, + // because this stip of the maintenance is skipped. + TableTablets { + table_spec: SOME_KS, + tablet_list: vec![ + Tablet::new_for_test( + 0, + vec![node2.clone()], + Some(vec![node1.host_id, node2.host_id]), + ), + Tablet::new_for_test( + 1, + vec![node2.clone()], + Some(vec![node2.host_id, node3.host_id]), + ), + Tablet::new_for_test(2, vec![node3.clone(), node1.clone()], None), + Tablet::new_for_test(3, vec![node3.clone()], None), + ], + has_unknown_replicas: false, + }, + ( + HashSet::new(), + node_map(&[&node1, &node2, &node3]), + HashMap::new(), + ), + TableTablets { + table_spec: SOME_KS, + tablet_list: vec![ + Tablet::new_for_test( + 0, + vec![node2.clone()], + Some(vec![node1.host_id, node2.host_id]), + ), + Tablet::new_for_test( + 1, + vec![node2.clone()], + Some(vec![node2.host_id, node3.host_id]), + ), + Tablet::new_for_test(2, vec![node3.clone(), node1.clone()], None), + Tablet::new_for_test(3, vec![node3.clone()], None), + ], + has_unknown_replicas: false, + }, + ), + ( + // [Case 6] Recreated one of the nodes + TableTablets { + table_spec: SOME_KS, + tablet_list: vec![ + Tablet::new_for_test(0, vec![node1.clone(), node2.clone()], None), + Tablet::new_for_test(1, vec![node2.clone(), node3.clone()], None), + Tablet::new_for_test(2, vec![node3.clone(), node1.clone()], None), + Tablet::new_for_test(3, vec![node3.clone()], None), + ], + has_unknown_replicas: false, + }, + ( + HashSet::new(), + node_map(&[&node1, &node2, &node3_v2]), + HashMap::from([(node3_v2.host_id, node3_v2.clone())]), + ), + TableTablets { + table_spec: SOME_KS, + tablet_list: vec![ + Tablet::new_for_test(0, vec![node1.clone(), node2.clone()], None), + Tablet::new_for_test(1, vec![node2.clone(), node3_v2.clone()], None), + Tablet::new_for_test(2, vec![node3_v2.clone(), node1.clone()], None), + Tablet::new_for_test(3, vec![node3_v2.clone()], None), + ], + has_unknown_replicas: false, + }, + ), + ( + // [Case 7] Recreated multiple nodes + TableTablets { + table_spec: SOME_KS, + tablet_list: vec![ + Tablet::new_for_test(0, vec![node1.clone(), node2.clone()], None), + Tablet::new_for_test(1, vec![node2.clone(), node3.clone()], None), + Tablet::new_for_test(2, vec![node3.clone(), node1.clone()], None), + Tablet::new_for_test(3, vec![node3.clone()], None), + ], + has_unknown_replicas: false, + }, + ( + HashSet::new(), + node_map(&[&node1, &node2, &node3_v2]), + HashMap::from([ + (node3_v2.host_id, node3_v2.clone()), + (node2_v2.host_id, node2_v2.clone()), + ]), + ), + TableTablets { + table_spec: SOME_KS, + tablet_list: vec![ + Tablet::new_for_test(0, vec![node1.clone(), node2_v2.clone()], None), + Tablet::new_for_test(1, vec![node2_v2.clone(), node3_v2.clone()], None), + Tablet::new_for_test(2, vec![node3_v2.clone(), node1.clone()], None), + Tablet::new_for_test(3, vec![node3_v2.clone()], None), + ], + has_unknown_replicas: false, + }, + ), + ( + // [Case 8] Unknown replica and removed node + TableTablets { + table_spec: SOME_KS, + tablet_list: vec![ + Tablet::new_for_test( + 0, + vec![node2.clone()], + Some(vec![node1.host_id, node2.host_id]), + ), + Tablet::new_for_test(1, vec![node2.clone(), node3.clone()], None), + Tablet::new_for_test( + 2, + vec![node3.clone()], + Some(vec![node3.host_id, node1.host_id]), + ), + Tablet::new_for_test(3, vec![node3.clone()], None), + ], + has_unknown_replicas: true, + }, + ( + HashSet::from([node2.host_id]), + node_map(&[&node1, &node3]), + HashMap::new(), + ), + TableTablets { + table_spec: SOME_KS, + tablet_list: vec![ + Tablet::new_for_test(2, vec![node3.clone(), node1.clone()], None), + Tablet::new_for_test(3, vec![node3.clone()], None), + ], + has_unknown_replicas: false, + }, + ), + ( + // [Case 9] Unknown replica, removed node and recreated node + TableTablets { + table_spec: SOME_KS, + tablet_list: vec![ + Tablet::new_for_test( + 0, + vec![node2.clone()], + Some(vec![node1.host_id, node2.host_id]), + ), + Tablet::new_for_test(1, vec![node2.clone(), node3.clone()], None), + Tablet::new_for_test( + 2, + vec![node3.clone()], + Some(vec![node3.host_id, node1.host_id]), + ), + Tablet::new_for_test(3, vec![node3.clone()], None), + ], + has_unknown_replicas: true, + }, + ( + HashSet::from([node2.host_id]), + node_map(&[&node1, &node3]), + HashMap::from([(node3_v2.host_id, node3_v2.clone())]), + ), + TableTablets { + table_spec: SOME_KS, + tablet_list: vec![ + Tablet::new_for_test(2, vec![node3_v2.clone(), node1.clone()], None), + Tablet::new_for_test(3, vec![node3_v2.clone()], None), + ], + has_unknown_replicas: false, + }, + ), + ]; + + for (i, (pre, (removed, all, recreated), post)) in tests.iter_mut().enumerate() { + tracing::info!("Test case {}", i); + pre.perform_maintenance(removed, all, recreated); + assert_eq!(pre, post); + } + } + + #[test] + fn maintenance_keyspace_remove_test() { + const TABLE_1: TableSpec<'static> = TableSpec::borrowed("ks_1", "table_1"); + const TABLE_2: TableSpec<'static> = TableSpec::borrowed("ks_2", "table_2"); + const TABLE_DROP: TableSpec<'static> = TableSpec::borrowed("ks_drop", "table_drop"); + + let mut pre = TabletsInfo { + tablets: hashbrown::HashMap::from([ + (TABLE_1.clone(), TableTablets::new(TABLE_1.clone())), + (TABLE_DROP.clone(), TableTablets::new(TABLE_DROP.clone())), + (TABLE_2.clone(), TableTablets::new(TABLE_2.clone())), + ]), + has_unknown_replicas: false, + }; + + let expected_after = TabletsInfo { + tablets: hashbrown::HashMap::from([ + (TABLE_1.clone(), TableTablets::new(TABLE_1.clone())), + (TABLE_2.clone(), TableTablets::new(TABLE_2.clone())), + ]), + has_unknown_replicas: false, + }; + + pre.perform_maintenance( + &|spec| *spec != TABLE_DROP, + &HashSet::new(), + &HashMap::new(), + &HashMap::new(), + ); + + assert_eq!(pre, expected_after); + } } From b82cc23a5ec00a6b527771bd4214d0b8e51c559f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Karol=20Bary=C5=82a?= Date: Fri, 5 Apr 2024 13:17:40 +0200 Subject: [PATCH 15/16] CI, Makefile: clippy and check with all-features In the previous version of tablet PR tablets were put behind "unstable-tablets" feature. In order to test them the feature had to be enabled, but CI / Makefile didn't do this for cargo check or clippy. While the tablets are no longer hidden behind feature flag, I think checking with all features is still a valid addition, so I decided to leave this commit as-is. --- .github/workflows/rust.yml | 4 ++++ Makefile | 12 ++++++++++-- 2 files changed, 14 insertions(+), 2 deletions(-) diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml index b398bdffdc..8fbfab11e8 100644 --- a/.github/workflows/rust.yml +++ b/.github/workflows/rust.yml @@ -30,10 +30,14 @@ jobs: run: cargo fmt --verbose --all -- --check - name: Clippy check run: cargo clippy --verbose --all-targets + - name: Clippy check with all features + run: cargo clippy --verbose --all-targets --all-features - name: Cargo check without features run: cargo check --all-targets --manifest-path "scylla/Cargo.toml" --features "" - name: Cargo check with all serialization features run: cargo check --all-targets --manifest-path "scylla/Cargo.toml" --features "full-serialization" + - name: Cargo check with all features + run: cargo check --all-targets --manifest-path "scylla/Cargo.toml" --all-features - name: Cargo check with secret feature run: cargo check --all-targets --manifest-path "scylla/Cargo.toml" --features "secret" - name: Cargo check with chrono feature diff --git a/Makefile b/Makefile index 5051d660af..9115122099 100644 --- a/Makefile +++ b/Makefile @@ -4,10 +4,10 @@ COMPOSE := docker compose -f test/cluster/docker-compose.yml all: test .PHONY: ci -ci: fmt-check check check-without-features clippy test build +ci: fmt-check check check-without-features check-all-features clippy clippy-all-features test build .PHONY: dockerized-ci -dockerized-ci: fmt-check check check-without-features clippy dockerized-test build +dockerized-ci: fmt-check check check-without-features check-all-features clippy clippy-all-features dockerized-test build .PHONY: fmt fmt: @@ -25,10 +25,18 @@ check: check-without-features: cargo check --manifest-path "scylla/Cargo.toml" --features "" --all-targets +.PHONY: check-all-features +check-all-features: + cargo check --all-targets --all-features + .PHONY: clippy clippy: RUSTFLAGS=-Dwarnings cargo clippy --all-targets +.PHONY: clippy-all-features +clippy-all-features: + RUSTFLAGS=-Dwarnings cargo clippy --all-targets --all-features + .PHONY: test test: up SCYLLA_URI=172.42.0.2:9042 \ From 426db657816fc4085a2ceb5b4dc88b3017bd23d7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Karol=20Bary=C5=82a?= Date: Tue, 23 Apr 2024 00:55:40 +0200 Subject: [PATCH 16/16] Tablets: Remove `#![allow(dead_code)]` The code is now used, so it's no longer necessary to prevent warnings. --- scylla/src/transport/locator/tablets.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/scylla/src/transport/locator/tablets.rs b/scylla/src/transport/locator/tablets.rs index a0899b4652..9277ce1934 100644 --- a/scylla/src/transport/locator/tablets.rs +++ b/scylla/src/transport/locator/tablets.rs @@ -1,4 +1,3 @@ -#![allow(dead_code)] use itertools::Itertools; use lazy_static::lazy_static; use scylla_cql::cql_to_rust::FromCqlVal;