From ddd98cc7f5ea3d93786cfdb9f3f94e31bc5c33fd Mon Sep 17 00:00:00 2001 From: adarovadya Date: Mon, 18 Nov 2024 11:32:57 +0200 Subject: [PATCH 01/11] Core: add az awareness to read strategy (#2539) * redis-core: add az awareness to read strategy * remove unessecrry get and set and fix tests * remove unessecrry az return type * add version check with mutex to skip tests in runtime Signed-off-by: Adar Ovadia --------- Signed-off-by: Adar Ovadia Co-authored-by: Adar Ovadia --- .github/workflows/redis-rs.yml | 1 + CHANGELOG.md | 3 +- glide-core/redis-rs/redis/Cargo.toml | 2 + .../redis-rs/redis/src/aio/connection.rs | 2 +- glide-core/redis-rs/redis/src/aio/mod.rs | 51 ++- .../redis/src/aio/multiplexed_connection.rs | 33 +- glide-core/redis-rs/redis/src/client.rs | 5 +- glide-core/redis-rs/redis/src/cluster.rs | 7 +- .../cluster_async/connections_container.rs | 304 ++++++++++++++++-- .../src/cluster_async/connections_logic.rs | 52 +-- .../redis-rs/redis/src/cluster_async/mod.rs | 14 +- .../redis-rs/redis/src/cluster_client.rs | 15 + .../redis-rs/redis/src/cluster_slotmap.rs | 20 +- glide-core/redis-rs/redis/src/lib.rs | 3 +- .../test_async_cluster_connections_logic.rs | 20 +- .../redis/tests/test_cluster_async.rs | 273 +++++++++++++++- glide-core/src/client/mod.rs | 14 +- .../src/client/reconnecting_connection.rs | 5 +- glide-core/src/client/standalone_client.rs | 79 ++++- glide-core/src/client/types.rs | 18 +- .../src/protobuf/connection_request.proto | 1 + glide-core/tests/test_standalone_client.rs | 12 + glide-core/tests/utilities/cluster.rs | 19 ++ glide-core/tests/utilities/mod.rs | 5 + node/src/BaseClient.ts | 6 +- package.json | 2 +- 26 files changed, 871 insertions(+), 95 deletions(-) diff --git a/.github/workflows/redis-rs.yml b/.github/workflows/redis-rs.yml index d0c1b00830..5d3d82855a 100644 --- a/.github/workflows/redis-rs.yml +++ b/.github/workflows/redis-rs.yml @@ -65,6 +65,7 @@ jobs: working-directory: ./glide-core/redis-rs/redis - name: Test + # TODO remove the concurrency limit after we fix test flakyness. run: | cargo test --release -- --test-threads=1 | tee ../test-results.xml echo "### Tests passed :v:" >> $GITHUB_STEP_SUMMARY diff --git a/CHANGELOG.md b/CHANGELOG.md index 910a557635..23e0dc32f4 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -79,9 +79,10 @@ * Node: Add `JSON.STRLEN` and `JSON.STRAPPEND` command ([#2537](https://github.com/valkey-io/valkey-glide/pull/2537)) * Node: Add `FT.SEARCH` ([#2551](https://github.com/valkey-io/valkey-glide/pull/2551)) * Python: Fix example ([#2556](https://github.com/valkey-io/valkey-glide/issues/2556)) -* Core: Add support for sending multi-slot JSON.MSET and JSON.MGET commands ([#2587]https://github.com/valkey-io/valkey-glide/pull/2587) +* Core: Add support for sending multi-slot JSON.MSET and JSON.MGET commands ([#2587](https://github.com/valkey-io/valkey-glide/pull/2587)) * Node: Add `JSON.DEBUG` command ([#2572](https://github.com/valkey-io/valkey-glide/pull/2572)) * Node: Add `JSON.NUMINCRBY` and `JSON.NUMMULTBY` command ([#2555](https://github.com/valkey-io/valkey-glide/pull/2555)) +* Core: Add support to Availability Zone Affinity read strategy ([#2539](https://github.com/valkey-io/valkey-glide/pull/2539)) * Core: Fix list of readonly commands ([#2634](https://github.com/valkey-io/valkey-glide/pull/2634), [#2649](https://github.com/valkey-io/valkey-glide/pull/2649)) * Core: Improve retry logic and update unmaintained dependencies for Rust lint CI ([#2673](https://github.com/valkey-io/valkey-glide/pull/2643)) * Core: Release the read lock while creating connections in `refresh_connections` ([#2630](https://github.com/valkey-io/valkey-glide/issues/2630)) diff --git a/glide-core/redis-rs/redis/Cargo.toml b/glide-core/redis-rs/redis/Cargo.toml index 579c1da799..25b06f64c2 100644 --- a/glide-core/redis-rs/redis/Cargo.toml +++ b/glide-core/redis-rs/redis/Cargo.toml @@ -99,6 +99,8 @@ uuid = { version = "1.6.1", optional = true } telemetrylib = { path = "../../telemetry" } +lazy_static = "1" + [features] default = [ "acl", diff --git a/glide-core/redis-rs/redis/src/aio/connection.rs b/glide-core/redis-rs/redis/src/aio/connection.rs index bc63456875..86c28d09ee 100644 --- a/glide-core/redis-rs/redis/src/aio/connection.rs +++ b/glide-core/redis-rs/redis/src/aio/connection.rs @@ -65,7 +65,7 @@ where pubsub: false, protocol: connection_info.protocol, }; - setup_connection(connection_info, &mut rv).await?; + setup_connection(connection_info, &mut rv, false).await?; Ok(rv) } diff --git a/glide-core/redis-rs/redis/src/aio/mod.rs b/glide-core/redis-rs/redis/src/aio/mod.rs index 34c098d600..077046feba 100644 --- a/glide-core/redis-rs/redis/src/aio/mod.rs +++ b/glide-core/redis-rs/redis/src/aio/mod.rs @@ -3,7 +3,10 @@ use crate::cmd::{cmd, Cmd}; use crate::connection::{ get_resp3_hello_command_error, PubSubSubscriptionKind, RedisConnectionInfo, }; -use crate::types::{ErrorKind, ProtocolVersion, RedisFuture, RedisResult, Value}; +use crate::types::{ + ErrorKind, FromRedisValue, InfoDict, ProtocolVersion, RedisError, RedisFuture, RedisResult, + Value, +}; use crate::PushKind; use ::tokio::io::{AsyncRead, AsyncWrite}; use async_trait::async_trait; @@ -84,6 +87,14 @@ pub trait ConnectionLike { /// Returns the state of the connection fn is_closed(&self) -> bool; + + /// Get the connection availibility zone + fn get_az(&self) -> Option { + None + } + + /// Set the connection availibility zone + fn set_az(&mut self, _az: Option) {} } /// Implements ability to notify about disconnection events @@ -105,8 +116,40 @@ impl Clone for Box { } } +// Helper function to extract and update availability zone from INFO command +async fn update_az_from_info(con: &mut C) -> RedisResult<()> +where + C: ConnectionLike, +{ + let info_res = con.req_packed_command(&cmd("INFO")).await; + + match info_res { + Ok(value) => { + let info_dict: InfoDict = FromRedisValue::from_redis_value(&value)?; + if let Some(node_az) = info_dict.get::("availability_zone") { + con.set_az(Some(node_az)); + } + Ok(()) + } + Err(e) => { + // Handle the error case for the INFO command + Err(RedisError::from(( + ErrorKind::ResponseError, + "Failed to execute INFO command. ", + format!("{:?}", e), + ))) + } + } +} + // Initial setup for every connection. -async fn setup_connection(connection_info: &RedisConnectionInfo, con: &mut C) -> RedisResult<()> +async fn setup_connection( + connection_info: &RedisConnectionInfo, + con: &mut C, + // This parameter is set to 'true' if ReadFromReplica strategy is set to AZAffinity. + // An INFO command will be triggered in the connection's setup to update the 'availability_zone' property. + discover_az: bool, +) -> RedisResult<()> where C: ConnectionLike, { @@ -181,6 +224,10 @@ where } } + if discover_az { + update_az_from_info(con).await?; + } + // result is ignored, as per the command's instructions. // https://redis.io/commands/client-setinfo/ #[cfg(not(feature = "disable-client-setinfo"))] diff --git a/glide-core/redis-rs/redis/src/aio/multiplexed_connection.rs b/glide-core/redis-rs/redis/src/aio/multiplexed_connection.rs index b31c817817..98b3667cc9 100644 --- a/glide-core/redis-rs/redis/src/aio/multiplexed_connection.rs +++ b/glide-core/redis-rs/redis/src/aio/multiplexed_connection.rs @@ -417,6 +417,7 @@ pub struct MultiplexedConnection { response_timeout: Duration, protocol: ProtocolVersion, push_manager: PushManager, + availability_zone: Option, password: Option, } @@ -479,11 +480,16 @@ impl MultiplexedConnection { .with_push_manager(pm) .with_protocol(connection_info.redis.protocol) .with_password(connection_info.redis.password.clone()) + .with_availability_zone(None) .build() .await?; let driver = { - let auth = setup_connection(&connection_info.redis, &mut con); + let auth = setup_connection( + &connection_info.redis, + &mut con, + glide_connection_options.discover_az, + ); futures_util::pin_mut!(auth); @@ -575,6 +581,11 @@ impl MultiplexedConnection { self.pipeline.set_push_manager(push_manager).await; } + /// For external visibilty (glide-core) + pub fn get_availability_zone(&self) -> Option { + self.availability_zone.clone() + } + /// Replace the password used to authenticate with the server. /// If `None` is provided, the password will be removed. pub async fn update_connection_password( @@ -599,6 +610,8 @@ pub struct MultiplexedConnectionBuilder { push_manager: Option, protocol: Option, password: Option, + /// Represents the node's availability zone + availability_zone: Option, } impl MultiplexedConnectionBuilder { @@ -611,6 +624,7 @@ impl MultiplexedConnectionBuilder { push_manager: None, protocol: None, password: None, + availability_zone: None, } } @@ -644,6 +658,12 @@ impl MultiplexedConnectionBuilder { self } + /// Sets the avazilability zone for the `MultiplexedConnectionBuilder`. + pub fn with_availability_zone(mut self, az: Option) -> Self { + self.availability_zone = az; + self + } + /// Builds and returns a new `MultiplexedConnection` instance using the configured settings. pub async fn build(self) -> RedisResult { let db = self.db.unwrap_or_default(); @@ -661,6 +681,7 @@ impl MultiplexedConnectionBuilder { push_manager, protocol, password, + availability_zone: self.availability_zone, }; Ok(con) @@ -688,6 +709,16 @@ impl ConnectionLike for MultiplexedConnection { fn is_closed(&self) -> bool { self.pipeline.is_closed() } + + /// Get the node's availability zone + fn get_az(&self) -> Option { + self.availability_zone.clone() + } + + /// Set the node's availability zone + fn set_az(&mut self, az: Option) { + self.availability_zone = az; + } } impl MultiplexedConnection { /// Subscribes to a new channel. diff --git a/glide-core/redis-rs/redis/src/client.rs b/glide-core/redis-rs/redis/src/client.rs index f6a7b4ef91..6ac3f40bcf 100644 --- a/glide-core/redis-rs/redis/src/client.rs +++ b/glide-core/redis-rs/redis/src/client.rs @@ -86,6 +86,9 @@ pub struct GlideConnectionOptions { #[cfg(feature = "aio")] /// Passive disconnect notifier pub disconnect_notifier: Option>, + /// If ReadFromReplica strategy is set to AZAffinity, this parameter will be set to 'true'. + /// In this case, an INFO command will be triggered in the connection's setup to update the connection's 'availability_zone' property. + pub discover_az: bool, } /// To enable async support you need to enable the feature: `tokio-comp` @@ -164,7 +167,7 @@ impl Client { /// For Unix connections, returns (async connection, None) #[cfg(feature = "tokio-comp")] #[cfg_attr(docsrs, doc(cfg(feature = "tokio-comp")))] - pub async fn get_multiplexed_async_connection_and_ip( + pub async fn get_multiplexed_async_connection_ip( &self, glide_connection_options: GlideConnectionOptions, ) -> RedisResult<(crate::aio::MultiplexedConnection, Option)> { diff --git a/glide-core/redis-rs/redis/src/cluster.rs b/glide-core/redis-rs/redis/src/cluster.rs index 6ccbaa9c8c..ffd537152a 100644 --- a/glide-core/redis-rs/redis/src/cluster.rs +++ b/glide-core/redis-rs/redis/src/cluster.rs @@ -228,7 +228,10 @@ where ) -> RedisResult { let connection = Self { connections: RefCell::new(HashMap::new()), - slots: RefCell::new(SlotMap::new(vec![], cluster_params.read_from_replicas)), + slots: RefCell::new(SlotMap::new( + vec![], + cluster_params.read_from_replicas.clone(), + )), auto_reconnect: RefCell::new(true), cluster_params, read_timeout: RefCell::new(None), @@ -384,7 +387,7 @@ where "can't parse node address", )))?; match parse_and_count_slots(&value, self.cluster_params.tls, addr).map(|slots_data| { - SlotMap::new(slots_data.1, self.cluster_params.read_from_replicas) + SlotMap::new(slots_data.1, self.cluster_params.read_from_replicas.clone()) }) { Ok(new_slots) => { result = Ok(new_slots); diff --git a/glide-core/redis-rs/redis/src/cluster_async/connections_container.rs b/glide-core/redis-rs/redis/src/cluster_async/connections_container.rs index 44e21848ef..955d24d9e9 100644 --- a/glide-core/redis-rs/redis/src/cluster_async/connections_container.rs +++ b/glide-core/redis-rs/redis/src/cluster_async/connections_container.rs @@ -6,6 +6,7 @@ use dashmap::DashMap; use futures::FutureExt; use rand::seq::IteratorRandom; use std::net::IpAddr; +use std::sync::atomic::Ordering; use std::sync::Arc; use telemetrylib::Telemetry; @@ -24,49 +25,57 @@ macro_rules! count_connections { }}; } -/// A struct that encapsulates a network connection along with its associated IP address. +/// A struct that encapsulates a network connection along with its associated IP address and AZ. #[derive(Clone, Eq, PartialEq, Debug)] -pub struct ConnectionWithIp { +pub struct ConnectionDetails { /// The actual connection pub conn: Connection, /// The IP associated with the connection pub ip: Option, + /// The availability zone associated with the connection + pub az: Option, } -impl ConnectionWithIp +impl ConnectionDetails where Connection: Clone + Send + 'static, { - /// Consumes the current instance and returns a new `ConnectionWithIp` + /// Consumes the current instance and returns a new `ConnectionDetails` /// where the connection is wrapped in a future. #[doc(hidden)] - pub fn into_future(self) -> ConnectionWithIp> { - ConnectionWithIp { + pub fn into_future(self) -> ConnectionDetails> { + ConnectionDetails { conn: async { self.conn }.boxed().shared(), ip: self.ip, + az: self.az, } } } -impl From<(Connection, Option)> for ConnectionWithIp { - fn from(val: (Connection, Option)) -> Self { - ConnectionWithIp { +impl From<(Connection, Option, Option)> + for ConnectionDetails +{ + fn from(val: (Connection, Option, Option)) -> Self { + ConnectionDetails { conn: val.0, ip: val.1, + az: val.2, } } } -impl From> for (Connection, Option) { - fn from(val: ConnectionWithIp) -> Self { - (val.conn, val.ip) +impl From> + for (Connection, Option, Option) +{ + fn from(val: ConnectionDetails) -> Self { + (val.conn, val.ip, val.az) } } #[derive(Clone, Eq, PartialEq, Debug)] pub struct ClusterNode { - pub user_connection: ConnectionWithIp, - pub management_connection: Option>, + pub user_connection: ConnectionDetails, + pub management_connection: Option>, } impl ClusterNode @@ -74,8 +83,8 @@ where Connection: Clone, { pub fn new( - user_connection: ConnectionWithIp, - management_connection: Option>, + user_connection: ConnectionDetails, + management_connection: Option>, ) -> Self { Self { user_connection, @@ -198,6 +207,13 @@ where Telemetry::incr_total_connections(conn_count_after.saturating_sub(conn_count_before)); } + /// Returns the availability zone associated with the connection in address + pub(crate) fn az_for_address(&self, address: &str) -> Option { + self.connection_map + .get(address) + .map(|item| item.value().user_connection.az.clone())? + } + /// Returns true if the address represents a known primary node. pub(crate) fn is_primary(&self, address: &String) -> bool { self.connection_for_address(address).is_some() && self.slot_map.is_primary(address) @@ -208,9 +224,7 @@ where slot_map_value: &SlotMapValue, ) -> Option> { let addrs = &slot_map_value.addrs; - let initial_index = slot_map_value - .last_used_replica - .load(std::sync::atomic::Ordering::Relaxed); + let initial_index = slot_map_value.last_used_replica.load(Ordering::Relaxed); let mut check_count = 0; loop { check_count += 1; @@ -225,14 +239,55 @@ where let _ = slot_map_value.last_used_replica.compare_exchange_weak( initial_index, index, - std::sync::atomic::Ordering::Relaxed, - std::sync::atomic::Ordering::Relaxed, + Ordering::Relaxed, + Ordering::Relaxed, ); return Some(connection); } } } + /// Returns the node's connection in the same availability zone as `client_az` in round robin strategy if exits, + /// if not, will fall back to any available replica or primary. + pub(crate) fn round_robin_read_from_replica_with_az_awareness( + &self, + slot_map_value: &SlotMapValue, + client_az: String, + ) -> Option> { + let addrs = &slot_map_value.addrs; + let initial_index = slot_map_value.last_used_replica.load(Ordering::Relaxed); + let mut retries = 0usize; + + loop { + retries = retries.saturating_add(1); + // Looped through all replicas; no connected replica found in the same availability zone. + if retries > addrs.replicas().len() { + // Attempt a fallback to any available replica or primary if needed. + return self.round_robin_read_from_replica(slot_map_value); + } + + // Calculate index based on initial index and check count. + let index = (initial_index + retries) % addrs.replicas().len(); + let replica = &addrs.replicas()[index]; + + // Check if this replica’s availability zone matches the user’s availability zone. + if let Some((address, connection_details)) = + self.connection_details_for_address(replica.as_str()) + { + if self.az_for_address(&address) == Some(client_az.clone()) { + // Attempt to update `latest_used_replica` with the index of this replica. + let _ = slot_map_value.last_used_replica.compare_exchange_weak( + initial_index, + index, + Ordering::Relaxed, + Ordering::Relaxed, + ); + return Some((address, connection_details.conn)); + } + } + } + } + fn lookup_route(&self, route: &Route) -> Option> { let slot_map_value = self.slot_map.slot_value_for_route(route)?; let addrs = &slot_map_value.addrs; @@ -241,16 +296,31 @@ where } match route.slot_addr() { + // Master strategy will be in use when the command is not read_only SlotAddr::Master => self.connection_for_address(addrs.primary().as_str()), - SlotAddr::ReplicaOptional => match self.read_from_replica_strategy { + // ReplicaOptional strategy will be in use when the command is read_only + SlotAddr::ReplicaOptional => match &self.read_from_replica_strategy { ReadFromReplicaStrategy::AlwaysFromPrimary => { self.connection_for_address(addrs.primary().as_str()) } ReadFromReplicaStrategy::RoundRobin => { self.round_robin_read_from_replica(slot_map_value) } + ReadFromReplicaStrategy::AZAffinity(az) => self + .round_robin_read_from_replica_with_az_awareness( + slot_map_value, + az.to_string(), + ), + }, + // when the user strategy per command is replica_preffered + SlotAddr::ReplicaRequired => match &self.read_from_replica_strategy { + ReadFromReplicaStrategy::AZAffinity(az) => self + .round_robin_read_from_replica_with_az_awareness( + slot_map_value, + az.to_string(), + ), + _ => self.round_robin_read_from_replica(slot_map_value), }, - SlotAddr::ReplicaRequired => self.round_robin_read_from_replica(slot_map_value), } } @@ -301,6 +371,16 @@ where }) } + pub(crate) fn connection_details_for_address( + &self, + address: &str, + ) -> Option>> { + self.connection_map.get(address).map(|item| { + let (address, conn) = (item.key(), item.value()); + (address.clone(), conn.user_connection.clone()) + }) + } + pub(crate) fn random_connections( &self, amount: usize, @@ -378,8 +458,9 @@ mod tests { { pub(crate) fn new_only_with_user_conn(user_connection: Connection) -> Self { let ip = None; + let az = None; Self { - user_connection: (user_connection, ip).into(), + user_connection: (user_connection, ip, az).into(), management_connection: None, } } @@ -414,18 +495,86 @@ mod tests { fn create_cluster_node( connection: usize, use_management_connections: bool, + node_az: Option, ) -> ClusterNode { let ip = None; ClusterNode::new( - (connection, ip).into(), + (connection, ip, node_az.clone()).into(), if use_management_connections { - Some((connection * 10, ip).into()) + Some((connection * 10, ip, node_az).into()) } else { None }, ) } + fn create_container_with_az_strategy( + use_management_connections: bool, + ) -> ConnectionsContainer { + let slot_map = SlotMap::new( + vec![ + Slot::new(1, 1000, "primary1".to_owned(), Vec::new()), + Slot::new( + 1002, + 2000, + "primary2".to_owned(), + vec!["replica2-1".to_owned()], + ), + Slot::new( + 2001, + 3000, + "primary3".to_owned(), + vec![ + "replica3-1".to_owned(), + "replica3-2".to_owned(), + "replica3-3".to_owned(), + ], + ), + ], + ReadFromReplicaStrategy::AlwaysFromPrimary, // this argument shouldn't matter, since we overload the RFR strategy. + ); + let connection_map = DashMap::new(); + connection_map.insert( + "primary1".into(), + create_cluster_node(1, use_management_connections, None), + ); + connection_map.insert( + "primary2".into(), + create_cluster_node(2, use_management_connections, None), + ); + connection_map.insert( + "primary3".into(), + create_cluster_node(3, use_management_connections, None), + ); + connection_map.insert( + "replica2-1".into(), + create_cluster_node(21, use_management_connections, None), + ); + connection_map.insert( + "replica3-1".into(), + create_cluster_node(31, use_management_connections, Some("use-1a".to_string())), + ); + connection_map.insert( + "replica3-2".into(), + create_cluster_node(32, use_management_connections, Some("use-1b".to_string())), + ); + connection_map.insert( + "replica3-3".into(), + create_cluster_node(33, use_management_connections, Some("use-1a".to_string())), + ); + connection_map.insert( + "replica3-3".into(), + create_cluster_node(33, use_management_connections, Some("use-1a".to_string())), + ); + + ConnectionsContainer { + slot_map, + connection_map, + read_from_replica_strategy: ReadFromReplicaStrategy::AZAffinity("use-1a".to_string()), + topology_hash: 0, + } + } + fn create_container_with_strategy( strategy: ReadFromReplicaStrategy, use_management_connections: bool, @@ -451,27 +600,27 @@ mod tests { let connection_map = DashMap::new(); connection_map.insert( "primary1".into(), - create_cluster_node(1, use_management_connections), + create_cluster_node(1, use_management_connections, None), ); connection_map.insert( "primary2".into(), - create_cluster_node(2, use_management_connections), + create_cluster_node(2, use_management_connections, None), ); connection_map.insert( "primary3".into(), - create_cluster_node(3, use_management_connections), + create_cluster_node(3, use_management_connections, None), ); connection_map.insert( "replica2-1".into(), - create_cluster_node(21, use_management_connections), + create_cluster_node(21, use_management_connections, None), ); connection_map.insert( "replica3-1".into(), - create_cluster_node(31, use_management_connections), + create_cluster_node(31, use_management_connections, None), ); connection_map.insert( "replica3-2".into(), - create_cluster_node(32, use_management_connections), + create_cluster_node(32, use_management_connections, None), ); ConnectionsContainer { @@ -650,6 +799,95 @@ mod tests { ); } + #[test] + fn get_connection_for_az_affinity_route() { + let container = create_container_with_az_strategy(false); + + // slot number is not exits + assert!(container + .connection_for_route(&Route::new(1001, SlotAddr::ReplicaOptional)) + .is_none()); + // Get the replica that holds the slot 1002 + assert_eq!( + 21, + container + .connection_for_route(&Route::new(1002, SlotAddr::ReplicaOptional)) + .unwrap() + .1 + ); + + // Get the Primary that holds the slot 1500 + assert_eq!( + 2, + container + .connection_for_route(&Route::new(1500, SlotAddr::Master)) + .unwrap() + .1 + ); + + // receive one of the replicas that holds the slot 2001 and is in the availability zone of the client ("use-1a") + assert!(one_of( + container.connection_for_route(&Route::new(2001, SlotAddr::ReplicaRequired)), + &[31, 33], + )); + + // remove the replica in the same client's az and get the other replica in the same az + remove_nodes(&container, &["replica3-3"]); + assert_eq!( + 31, + container + .connection_for_route(&Route::new(2001, SlotAddr::ReplicaOptional)) + .unwrap() + .1 + ); + + // remove the replica in the same clients az and get the other replica + remove_nodes(&container, &["replica3-1"]); + assert_eq!( + 32, + container + .connection_for_route(&Route::new(2001, SlotAddr::ReplicaOptional)) + .unwrap() + .1 + ); + + // remove the last replica and get the primary + remove_nodes(&container, &["replica3-2"]); + assert_eq!( + 3, + container + .connection_for_route(&Route::new(2001, SlotAddr::ReplicaOptional)) + .unwrap() + .1 + ); + } + + #[test] + fn get_connection_for_az_affinity_route_round_robin() { + let container = create_container_with_az_strategy(false); + + let mut addresses = vec![ + container + .connection_for_route(&Route::new(2001, SlotAddr::ReplicaOptional)) + .unwrap() + .1, + container + .connection_for_route(&Route::new(2001, SlotAddr::ReplicaOptional)) + .unwrap() + .1, + container + .connection_for_route(&Route::new(2001, SlotAddr::ReplicaOptional)) + .unwrap() + .1, + container + .connection_for_route(&Route::new(2001, SlotAddr::ReplicaOptional)) + .unwrap() + .1, + ]; + addresses.sort(); + assert_eq!(addresses, vec![31, 31, 33, 33]); + } + #[test] fn get_connection_by_address() { let container = create_container(); @@ -936,7 +1174,7 @@ mod tests { assert!(container.connection_for_address(&new_node).is_none()); // Create new connection map let new_connection_map = DashMap::new(); - new_connection_map.insert(new_node.clone(), create_cluster_node(1, false)); + new_connection_map.insert(new_node.clone(), create_cluster_node(1, false, None)); // Extend the current connection map container.extend_connection_map(ConnectionsMap(new_connection_map)); diff --git a/glide-core/redis-rs/redis/src/cluster_async/connections_logic.rs b/glide-core/redis-rs/redis/src/cluster_async/connections_logic.rs index 25dfcbb9d1..4f9b3f0d4e 100644 --- a/glide-core/redis-rs/redis/src/cluster_async/connections_logic.rs +++ b/glide-core/redis-rs/redis/src/cluster_async/connections_logic.rs @@ -1,7 +1,8 @@ use super::{ - connections_container::{ClusterNode, ConnectionWithIp}, + connections_container::{ClusterNode, ConnectionDetails}, Connect, }; +use crate::cluster_slotmap::ReadFromReplicaStrategy; use crate::{ aio::{ConnectionLike, DisconnectNotifier}, client::GlideConnectionOptions, @@ -33,7 +34,7 @@ pub enum RefreshConnectionType { fn failed_management_connection( addr: &str, - user_conn: ConnectionWithIp>, + user_conn: ConnectionDetails>, err: RedisError, ) -> ConnectAndCheckResult where @@ -90,8 +91,8 @@ where } fn create_async_node( - user_conn: ConnectionWithIp, - management_conn: Option>, + user_conn: ConnectionDetails, + management_conn: Option>, ) -> AsyncClusterNode where C: ConnectionLike + Connect + Send + Sync + 'static + Clone, @@ -133,9 +134,9 @@ where { (Ok(conn_1), Ok(conn_2)) => { // Both connections were successfully established - let mut user_conn: ConnectionWithIp = conn_1; - let mut management_conn: ConnectionWithIp = conn_2; - if let Err(err) = setup_user_connection(&mut user_conn.conn, params).await { + let mut user_conn: ConnectionDetails = conn_1; + let mut management_conn: ConnectionDetails = conn_2; + if let Err(err) = setup_user_connection(&mut user_conn, params).await { return err.into(); } match setup_management_connection(&mut management_conn.conn).await { @@ -148,7 +149,7 @@ where } (Ok(mut connection), Err(err)) | (Err(err), Ok(mut connection)) => { // Only a single connection was successfully established. Use it for the user connection - match setup_user_connection(&mut connection.conn, params).await { + match setup_user_connection(&mut connection, params).await { Ok(_) => failed_management_connection(addr, connection.into_future(), err), Err(err) => err.into(), } @@ -178,6 +179,11 @@ async fn connect_and_check_only_management_conn( where C: ConnectionLike + Connect + Send + Sync + 'static + Clone, { + let discover_az = matches!( + params.read_from_replicas, + crate::cluster_slotmap::ReadFromReplicaStrategy::AZAffinity(_) + ); + match create_connection::( addr, params.clone(), @@ -186,6 +192,7 @@ where GlideConnectionOptions { push_sender: None, disconnect_notifier, + discover_az, }, ) .await @@ -322,11 +329,11 @@ async fn create_and_setup_user_connection( params: ClusterParams, socket_addr: Option, glide_connection_options: GlideConnectionOptions, -) -> RedisResult> +) -> RedisResult> where C: ConnectionLike + Connect + Send + 'static, { - let mut connection: ConnectionWithIp = create_connection( + let mut connection: ConnectionDetails = create_connection( node, params.clone(), socket_addr, @@ -334,22 +341,28 @@ where glide_connection_options, ) .await?; - setup_user_connection(&mut connection.conn, params).await?; + setup_user_connection(&mut connection, params).await?; Ok(connection) } -async fn setup_user_connection(conn: &mut C, params: ClusterParams) -> RedisResult<()> +async fn setup_user_connection( + conn_details: &mut ConnectionDetails, + params: ClusterParams, +) -> RedisResult<()> where C: ConnectionLike + Connect + Send + 'static, { - let read_from_replicas = params.read_from_replicas - != crate::cluster_slotmap::ReadFromReplicaStrategy::AlwaysFromPrimary; + let read_from_replicas = + params.read_from_replicas != ReadFromReplicaStrategy::AlwaysFromPrimary; let connection_timeout = params.connection_timeout; - check_connection(conn, connection_timeout).await?; + check_connection(&mut conn_details.conn, connection_timeout).await?; if read_from_replicas { // If READONLY is sent to primary nodes, it will have no effect - crate::cmd("READONLY").query_async(conn).await?; + crate::cmd("READONLY") + .query_async(&mut conn_details.conn) + .await?; } + Ok(()) } @@ -373,7 +386,7 @@ async fn create_connection( socket_addr: Option, is_management: bool, mut glide_connection_options: GlideConnectionOptions, -) -> RedisResult> +) -> RedisResult> where C: ConnectionLike + Connect + Send + 'static, { @@ -396,7 +409,10 @@ where glide_connection_options, ) .await - .map(|conn| conn.into()) + .map(|conn| { + let az = conn.0.get_az(); + (conn.0, conn.1, az).into() + }) } /// The function returns None if the checked connection/s are healthy. Otherwise, it returns the type of the unhealthy connection/s. diff --git a/glide-core/redis-rs/redis/src/cluster_async/mod.rs b/glide-core/redis-rs/redis/src/cluster_async/mod.rs index 39e547c85b..e9d432b6f2 100644 --- a/glide-core/redis-rs/redis/src/cluster_async/mod.rs +++ b/glide-core/redis-rs/redis/src/cluster_async/mod.rs @@ -26,7 +26,7 @@ mod connections_container; mod connections_logic; /// Exposed only for testing. pub mod testing { - pub use super::connections_container::ConnectionWithIp; + pub use super::connections_container::ConnectionDetails; pub use super::connections_logic::*; } use crate::{ @@ -1153,9 +1153,15 @@ where None }; + let discover_az = matches!( + cluster_params.read_from_replicas, + crate::cluster_slotmap::ReadFromReplicaStrategy::AZAffinity(_) + ); + let glide_connection_options = GlideConnectionOptions { push_sender, disconnect_notifier, + discover_az, }; let connections = Self::create_initial_connections( @@ -1171,7 +1177,7 @@ where conn_lock: StdRwLock::new(ConnectionsContainer::new( Default::default(), connections, - cluster_params.read_from_replicas, + cluster_params.read_from_replicas.clone(), 0, )), cluster_params: StdRwLock::new(cluster_params.clone()), @@ -1957,7 +1963,7 @@ where // Reset the current slot map and connection vector with the new ones let mut write_guard = inner.conn_lock.write().expect(MUTEX_WRITE_ERR); let read_from_replicas = inner - .get_cluster_param(|params| params.read_from_replicas) + .get_cluster_param(|params| params.read_from_replicas.clone()) .expect(MUTEX_READ_ERR); *write_guard = ConnectionsContainer::new( new_slots, @@ -2777,7 +2783,7 @@ where .expect(MUTEX_READ_ERR); let read_from_replicas = inner - .get_cluster_param(|params| params.read_from_replicas) + .get_cluster_param(|params| params.read_from_replicas.clone()) .expect(MUTEX_READ_ERR); ( calculate_topology( diff --git a/glide-core/redis-rs/redis/src/cluster_client.rs b/glide-core/redis-rs/redis/src/cluster_client.rs index 185b8547ad..c4dc0103dc 100644 --- a/glide-core/redis-rs/redis/src/cluster_client.rs +++ b/glide-core/redis-rs/redis/src/cluster_client.rs @@ -387,6 +387,21 @@ impl ClusterClientBuilder { self } + /// Set the read strategy for this client. + /// + /// The parameter `read_strategy` can be one of: + /// `ReadFromReplicaStrategy::AZAffinity(availability_zone)` - attempt to access replicas in the same availability zone. + /// If no suitable replica is found (i.e. no replica could be found in the requested availability zone), choose any replica. Falling back to primary if needed. + /// `ReadFromReplicaStrategy::RoundRobin` - reads are distributed across replicas for load balancing using round-robin algorithm. Falling back to primary if needed. + /// `ReadFromReplicaStrategy::AlwaysFromPrimary` ensures all read and write queries are directed to the primary node. + /// + /// # Parameters + /// - `read_strategy`: defines the replica routing strategy. + pub fn read_from(mut self, read_strategy: ReadFromReplicaStrategy) -> ClusterClientBuilder { + self.builder_params.read_from_replicas = read_strategy; + self + } + /// Enables periodic topology checks for this client. /// /// If enabled, periodic topology checks will be executed at the configured intervals to examine whether there diff --git a/glide-core/redis-rs/redis/src/cluster_slotmap.rs b/glide-core/redis-rs/redis/src/cluster_slotmap.rs index 68d6a5c2be..88e7549323 100644 --- a/glide-core/redis-rs/redis/src/cluster_slotmap.rs +++ b/glide-core/redis-rs/redis/src/cluster_slotmap.rs @@ -20,11 +20,18 @@ pub(crate) struct SlotMapValue { pub(crate) last_used_replica: Arc, } -#[derive(Debug, Default, Clone, PartialEq, Copy)] -pub(crate) enum ReadFromReplicaStrategy { +#[derive(Debug, Default, Clone, PartialEq)] +/// Represents the client's read from strategy. +pub enum ReadFromReplicaStrategy { #[default] + /// Always get from primary, in order to get the freshest data. AlwaysFromPrimary, + /// Spread the read requests between all replicas in a round robin manner. + /// If no replica is available, route the requests to the primary. RoundRobin, + /// Spread the read requests between replicas in the same client's Aviliablity zone in a round robin manner, + /// falling back to other replicas or the primary if needed. + AZAffinity(String), } #[derive(Debug, Default)] @@ -52,6 +59,7 @@ fn get_address_from_slot( % addrs.replicas().len(); addrs.replicas()[index].clone() } + ReadFromReplicaStrategy::AZAffinity(_az) => todo!(), // Drop sync client } } @@ -129,7 +137,11 @@ impl SlotMap { pub fn slot_addr_for_route(&self, route: &Route) -> Option> { self.slot_value_for_route(route).map(|slot_value| { - get_address_from_slot(slot_value, self.read_from_replica, route.slot_addr()) + get_address_from_slot( + slot_value, + self.read_from_replica.clone(), + route.slot_addr(), + ) }) } @@ -199,7 +211,7 @@ impl SlotMap { if slot_value.start <= slot { Some(get_address_from_slot( slot_value, - self.read_from_replica, + self.read_from_replica.clone(), slot_addr, )) } else { diff --git a/glide-core/redis-rs/redis/src/lib.rs b/glide-core/redis-rs/redis/src/lib.rs index 4f138c2bb6..0c960f3b4e 100644 --- a/glide-core/redis-rs/redis/src/lib.rs +++ b/glide-core/redis-rs/redis/src/lib.rs @@ -448,7 +448,8 @@ pub mod cluster; #[cfg(feature = "cluster")] #[cfg_attr(docsrs, doc(cfg(feature = "cluster")))] -mod cluster_slotmap; +/// Used for ReadFromReplicaStrategy information. +pub mod cluster_slotmap; #[cfg(feature = "cluster-async")] pub use crate::commands::ScanStateRC; diff --git a/glide-core/redis-rs/redis/tests/test_async_cluster_connections_logic.rs b/glide-core/redis-rs/redis/tests/test_async_cluster_connections_logic.rs index 0230d1de17..356c5bfc8c 100644 --- a/glide-core/redis-rs/redis/tests/test_async_cluster_connections_logic.rs +++ b/glide-core/redis-rs/redis/tests/test_async_cluster_connections_logic.rs @@ -20,7 +20,7 @@ mod test_connect_and_check { use super::*; use crate::support::{get_mock_connection_handler, ShouldReturnConnectionError}; use redis::cluster_async::testing::{ - connect_and_check, ConnectAndCheckResult, ConnectionWithIp, + connect_and_check, ConnectAndCheckResult, ConnectionDetails, }; fn assert_partial_result( @@ -233,7 +233,7 @@ mod test_connect_and_check { port: 6379, }; let node = AsyncClusterNode::new( - ConnectionWithIp { + ConnectionDetails { conn: user_conn, ip: Some(ip), } @@ -280,7 +280,7 @@ mod test_connect_and_check { }; let prev_ip = Some(IpAddr::V4(Ipv4Addr::new(1, 1, 1, 1))); let node = AsyncClusterNode::new( - ConnectionWithIp { + ConnectionDetails { conn: user_conn, ip: prev_ip, } @@ -336,13 +336,13 @@ mod test_connect_and_check { }; let node = AsyncClusterNode::new( - ConnectionWithIp { + ConnectionDetails { conn: old_user_conn, ip: Some(prev_ip), } .into_future(), Some( - ConnectionWithIp { + ConnectionDetails { conn: management_conn, ip: Some(prev_ip), } @@ -373,17 +373,17 @@ mod test_connect_and_check { mod test_check_node_connections { use super::*; - use redis::cluster_async::testing::{check_node_connections, ConnectionWithIp}; + use redis::cluster_async::testing::{check_node_connections, ConnectionDetails}; fn create_node_with_all_connections(name: &str) -> AsyncClusterNode { let ip = None; AsyncClusterNode::new( - ConnectionWithIp { + ConnectionDetails { conn: get_mock_connection_with_port(name, 1, 6380), ip, } .into_future(), Some( - ConnectionWithIp { + ConnectionDetails { conn: get_mock_connection_with_port(name, 2, 6381), ip, } @@ -460,7 +460,7 @@ mod test_check_node_connections { let ip = None; let node = AsyncClusterNode::new( - ConnectionWithIp { + ConnectionDetails { conn: get_mock_connection(name, 1), ip, } @@ -544,7 +544,7 @@ mod test_check_node_connections { ); let node = AsyncClusterNode::new( - ConnectionWithIp { + ConnectionDetails { conn: get_mock_connection(name, 1), ip: None, } diff --git a/glide-core/redis-rs/redis/tests/test_cluster_async.rs b/glide-core/redis-rs/redis/tests/test_cluster_async.rs index 968451a1fe..7273f98702 100644 --- a/glide-core/redis-rs/redis/tests/test_cluster_async.rs +++ b/glide-core/redis-rs/redis/tests/test_cluster_async.rs @@ -2,6 +2,92 @@ #![cfg(feature = "cluster-async")] mod support; +use std::cell::Cell; +use tokio::sync::Mutex; + +use lazy_static::lazy_static; + +lazy_static! { + static ref CLUSTER_VERSION: Mutex> = Mutex::>::default(); +} + +/// Check if the current cluster version is less than `min_version`. +/// At first, the func check for the Valkey version and if none exists, then the Redis version is checked. +async fn engine_version_less_than(min_version: &str) -> bool { + let test_version = crate::get_cluster_version().await; + let min_version_usize = crate::version_to_usize(min_version).unwrap(); + if test_version < min_version_usize { + println!( + "The engine version is {:?}, which is lower than {:?}", + test_version, min_version + ); + return true; + } + return false; +} + +/// Static function to get the engine version. When version looks like 8.0.0 -> 80000 and 12.0.1 -> 120001. +async fn get_cluster_version() -> usize { + let cluster_version = CLUSTER_VERSION.lock().await; + if cluster_version.get() == 0 { + let cluster = crate::support::TestClusterContext::new(3, 0); + + let mut connection = cluster.async_connection(None).await; + + let cmd = redis::cmd("INFO"); + let info = connection + .route_command( + &cmd, + redis::cluster_routing::RoutingInfo::SingleNode( + redis::cluster_routing::SingleNodeRoutingInfo::Random, + ), + ) + .await + .unwrap(); + + let info_result = redis::from_owned_redis_value::(info).unwrap(); + + cluster_version.set( + parse_version_from_info(info_result.clone()) + .expect(format!("Invalid version string in INFO : {info_result}").as_str()), + ); + } + return cluster_version.get(); +} + +fn parse_version_from_info(info: String) -> Option { + // check for valkey_version + if let Some(version) = info + .lines() + .find_map(|line| line.strip_prefix("valkey_version:")) + { + return version_to_usize(version); + } + + // check for redis_version if no valkey_version was found + if let Some(version) = info + .lines() + .find_map(|line| line.strip_prefix("redis_version:")) + { + return version_to_usize(version); + } + None +} + +/// Takes a version string (e.g., 8.2.1) and converts it to a usize (e.g., 80201) +/// version 12.10.0 will became 121000 +fn version_to_usize(version: &str) -> Option { + version + .split('.') + .enumerate() + .map(|(index, part)| { + part.parse::() + .ok() + .map(|num| num * 10_usize.pow(2 * (2 - index) as u32)) + }) + .sum() +} + #[cfg(test)] mod cluster_async { use std::{ @@ -35,7 +121,6 @@ mod cluster_async { }; use crate::support::*; - use tokio::sync::mpsc; fn broken_pipe_error() -> RedisError { RedisError::from(std::io::Error::new( @@ -121,6 +206,192 @@ mod cluster_async { .unwrap(); } + #[tokio::test] + async fn test_routing_by_slot_to_replica_with_az_affinity_strategy_to_half_replicas() { + // Skip test if version is less then Valkey 8.0 + if crate::engine_version_less_than("8.0").await { + return; + } + + let replica_num: u16 = 4; + let primaries_num: u16 = 3; + let replicas_num_in_client_az = replica_num / 2; + let cluster = + TestClusterContext::new((replica_num * primaries_num) + primaries_num, replica_num); + let az: String = "us-east-1a".to_string(); + + let mut connection = cluster.async_connection(None).await; + let cluster_addresses: Vec<_> = cluster + .cluster + .servers + .iter() + .map(|server| server.connection_info()) + .collect(); + + let mut cmd = redis::cmd("CONFIG"); + cmd.arg(&["SET", "availability-zone", &az.clone()]); + + for _ in 0..replicas_num_in_client_az { + connection + .route_command( + &cmd, + RoutingInfo::SingleNode(SingleNodeRoutingInfo::SpecificNode(Route::new( + 12182, // foo key is mapping to 12182 slot + SlotAddr::ReplicaRequired, + ))), + ) + .await + .unwrap(); + } + + let mut client = ClusterClient::builder(cluster_addresses.clone()) + .read_from(redis::cluster_slotmap::ReadFromReplicaStrategy::AZAffinity( + az.clone(), + )) + .build() + .unwrap() + .get_async_connection(None) + .await + .unwrap(); + + // Each replica in the client az will return the value of foo n times + let n = 4; + for _ in 0..n * replicas_num_in_client_az { + let mut cmd = redis::cmd("GET"); + cmd.arg("foo"); + let _res: RedisResult = cmd.query_async(&mut client).await; + } + + let mut cmd = redis::cmd("INFO"); + cmd.arg("ALL"); + let info = connection + .route_command( + &cmd, + RoutingInfo::MultiNode((MultipleNodeRoutingInfo::AllNodes, None)), + ) + .await + .unwrap(); + + let info_result = redis::from_owned_redis_value::>(info).unwrap(); + let get_cmdstat = format!("cmdstat_get:calls="); + let n_get_cmdstat = format!("cmdstat_get:calls={}", n); + let client_az = format!("availability_zone:{}", az); + + let mut matching_entries_count: usize = 0; + + for value in info_result.values() { + if value.contains(&get_cmdstat) { + if value.contains(&client_az) && value.contains(&n_get_cmdstat) { + matching_entries_count += 1; + } else { + panic!( + "Invalid entry found: {}. Expected cmdstat_get:calls={} and availability_zone={}", + value, n, az); + } + } + } + + assert_eq!( + (matching_entries_count.try_into() as Result).unwrap(), + replicas_num_in_client_az, + "Test failed: expected exactly '{}' entries with '{}' and '{}', found {}", + replicas_num_in_client_az, + get_cmdstat, + client_az, + matching_entries_count + ); + } + + #[tokio::test] + async fn test_routing_by_slot_to_replica_with_az_affinity_strategy_to_all_replicas() { + // Skip test if version is less then Valkey 8.0 + if crate::engine_version_less_than("8.0").await { + return; + } + + let replica_num: u16 = 4; + let primaries_num: u16 = 3; + let cluster = + TestClusterContext::new((replica_num * primaries_num) + primaries_num, replica_num); + let az: String = "us-east-1a".to_string(); + + let mut connection = cluster.async_connection(None).await; + let cluster_addresses: Vec<_> = cluster + .cluster + .servers + .iter() + .map(|server| server.connection_info()) + .collect(); + + let mut cmd = redis::cmd("CONFIG"); + cmd.arg(&["SET", "availability-zone", &az.clone()]); + + connection + .route_command( + &cmd, + RoutingInfo::MultiNode((MultipleNodeRoutingInfo::AllNodes, None)), + ) + .await + .unwrap(); + + let mut client = ClusterClient::builder(cluster_addresses.clone()) + .read_from(redis::cluster_slotmap::ReadFromReplicaStrategy::AZAffinity( + az.clone(), + )) + .build() + .unwrap() + .get_async_connection(None) + .await + .unwrap(); + + // Each replica will return the value of foo n times + let n = 4; + for _ in 0..(n * replica_num) { + let mut cmd = redis::cmd("GET"); + cmd.arg("foo"); + let _res: RedisResult = cmd.query_async(&mut client).await; + } + + let mut cmd = redis::cmd("INFO"); + cmd.arg("ALL"); + let info = connection + .route_command( + &cmd, + RoutingInfo::MultiNode((MultipleNodeRoutingInfo::AllNodes, None)), + ) + .await + .unwrap(); + + let info_result = redis::from_owned_redis_value::>(info).unwrap(); + let get_cmdstat = format!("cmdstat_get:calls="); + let n_get_cmdstat = format!("cmdstat_get:calls={}", n); + let client_az = format!("availability_zone:{}", az); + + let mut matching_entries_count: usize = 0; + + for value in info_result.values() { + if value.contains(&get_cmdstat) { + if value.contains(&client_az) && value.contains(&n_get_cmdstat) { + matching_entries_count += 1; + } else { + panic!( + "Invalid entry found: {}. Expected cmdstat_get:calls={} and availability_zone={}", + value, n, az); + } + } + } + + assert_eq!( + (matching_entries_count.try_into() as Result).unwrap(), + replica_num, + "Test failed: expected exactly '{}' entries with '{}' and '{}', found {}", + replica_num.to_string(), + get_cmdstat, + client_az, + matching_entries_count + ); + } + #[test] #[serial_test::serial] fn test_async_cluster_basic_eval() { diff --git a/glide-core/src/client/mod.rs b/glide-core/src/client/mod.rs index ffbdc60d4e..78558f8e28 100644 --- a/glide-core/src/client/mod.rs +++ b/glide-core/src/client/mod.rs @@ -12,6 +12,7 @@ use redis::cluster_async::ClusterConnection; use redis::cluster_routing::{ MultipleNodeRoutingInfo, ResponsePolicy, Routable, RoutingInfo, SingleNodeRoutingInfo, }; +use redis::cluster_slotmap::ReadFromReplicaStrategy; use redis::{Cmd, ErrorKind, ObjectType, PushInfo, RedisError, RedisResult, ScanStateRC, Value}; pub use standalone_client::StandaloneClient; use std::io; @@ -542,8 +543,6 @@ async fn create_cluster_client( .into_iter() .map(|address| get_connection_info(&address, tls_mode, redis_connection_info.clone())) .collect(); - let read_from = request.read_from.unwrap_or_default(); - let read_from_replicas = !matches!(read_from, ReadFrom::Primary); // TODO - implement different read from replica strategies. let periodic_topology_checks = match request.periodic_checks { Some(PeriodicCheck::Disabled) => None, Some(PeriodicCheck::Enabled) => Some(DEFAULT_PERIODIC_TOPOLOGY_CHECKS_INTERVAL), @@ -553,9 +552,12 @@ async fn create_cluster_client( let mut builder = redis::cluster::ClusterClientBuilder::new(initial_nodes) .connection_timeout(INTERNAL_CONNECTION_TIMEOUT) .retries(DEFAULT_RETRIES); - if read_from_replicas { - builder = builder.read_from_replicas(); - } + let read_from_strategy = request.read_from.unwrap_or_default(); + builder = builder.read_from(match read_from_strategy { + ReadFrom::AZAffinity(az) => ReadFromReplicaStrategy::AZAffinity(az), + ReadFrom::PreferReplica => ReadFromReplicaStrategy::RoundRobin, + ReadFrom::Primary => ReadFromReplicaStrategy::AlwaysFromPrimary, + }); if let Some(interval_duration) = periodic_topology_checks { builder = builder.periodic_topology_checks(interval_duration); } @@ -649,12 +651,14 @@ fn sanitized_request_string(request: &ConnectionRequest) -> String { let database_id = format!("\ndatabase ID: {}", request.database_id); let rfr_strategy = request .read_from + .clone() .map(|rfr| { format!( "\nRead from Replica mode: {}", match rfr { ReadFrom::Primary => "Only primary", ReadFrom::PreferReplica => "Prefer replica", + ReadFrom::AZAffinity(_) => "Prefer replica in user's availability zone", } ) }) diff --git a/glide-core/src/client/reconnecting_connection.rs b/glide-core/src/client/reconnecting_connection.rs index c567b6b5a6..39a4c1db62 100644 --- a/glide-core/src/client/reconnecting_connection.rs +++ b/glide-core/src/client/reconnecting_connection.rs @@ -113,6 +113,7 @@ async fn create_connection( connection_backend: ConnectionBackend, retry_strategy: RetryStrategy, push_sender: Option>, + discover_az: bool, ) -> Result { let client = &connection_backend.connection_info; let connection_options = GlideConnectionOptions { @@ -120,6 +121,7 @@ async fn create_connection( disconnect_notifier: Some::>(Box::new( TokioDisconnectNotifier::new(), )), + discover_az, }; let action = || async { get_multiplexed_connection(client, &connection_options) @@ -204,6 +206,7 @@ impl ReconnectingConnection { redis_connection_info: RedisConnectionInfo, tls_mode: TlsMode, push_sender: Option>, + discover_az: bool, ) -> Result { log_debug( "connection creation", @@ -216,7 +219,7 @@ impl ReconnectingConnection { connection_available_signal: ManualResetEvent::new(true), client_dropped_flagged: AtomicBool::new(false), }; - create_connection(backend, connection_retry_strategy, push_sender).await + create_connection(backend, connection_retry_strategy, push_sender, discover_az).await } pub(crate) fn node_address(&self) -> String { diff --git a/glide-core/src/client/standalone_client.rs b/glide-core/src/client/standalone_client.rs index 2a6dbd0e77..c5e69fd6dd 100644 --- a/glide-core/src/client/standalone_client.rs +++ b/glide-core/src/client/standalone_client.rs @@ -4,6 +4,7 @@ use super::get_redis_connection_info; use super::reconnecting_connection::{ReconnectReason, ReconnectingConnection}; use super::{ConnectionRequest, NodeAddress, TlsMode}; +use crate::client::types::ReadFrom as ClientReadFrom; use crate::retry_strategies::RetryStrategy; use futures::{future, stream, StreamExt}; use logger_core::log_debug; @@ -13,6 +14,7 @@ use redis::aio::ConnectionLike; use redis::cluster_routing::{self, is_readonly_cmd, ResponsePolicy, Routable, RoutingInfo}; use redis::{PushInfo, RedisError, RedisResult, Value}; use std::sync::atomic::AtomicUsize; +use std::sync::atomic::Ordering; use std::sync::Arc; use telemetrylib::Telemetry; use tokio::sync::mpsc; @@ -22,7 +24,11 @@ use tokio::task; enum ReadFrom { Primary, PreferReplica { - latest_read_replica_index: Arc, + latest_read_replica_index: Arc, + }, + AZAffinity { + client_az: String, + last_read_replica_index: Arc, }, } @@ -120,6 +126,11 @@ impl StandaloneClient { // randomize pubsub nodes, maybe a batter option is to always use the primary let pubsub_node_index = rand::thread_rng().gen_range(0..node_count); let pubsub_addr = &connection_request.addresses[pubsub_node_index]; + let discover_az = matches!( + connection_request.read_from, + Some(ClientReadFrom::AZAffinity(_)) + ); + let mut stream = stream::iter(connection_request.addresses.iter()) .map(|address| async { get_connection_and_replication_info( @@ -132,6 +143,7 @@ impl StandaloneClient { }, tls_mode.unwrap_or(TlsMode::NoTls), &push_sender, + discover_az, ) .await .map_err(|err| (format!("{}:{}", address.host, address.port), err)) @@ -221,7 +233,7 @@ impl StandaloneClient { &self, latest_read_replica_index: &Arc, ) -> &ReconnectingConnection { - let initial_index = latest_read_replica_index.load(std::sync::atomic::Ordering::Relaxed); + let initial_index = latest_read_replica_index.load(Ordering::Relaxed); let mut check_count = 0; loop { check_count += 1; @@ -241,15 +253,53 @@ impl StandaloneClient { let _ = latest_read_replica_index.compare_exchange_weak( initial_index, index, - std::sync::atomic::Ordering::Relaxed, - std::sync::atomic::Ordering::Relaxed, + Ordering::Relaxed, + Ordering::Relaxed, ); return connection; } } } - fn get_connection(&self, readonly: bool) -> &ReconnectingConnection { + async fn round_robin_read_from_replica_az_awareness( + &self, + latest_read_replica_index: &Arc, + client_az: String, + ) -> &ReconnectingConnection { + let initial_index = latest_read_replica_index.load(Ordering::Relaxed); + let mut retries = 0usize; + + loop { + retries = retries.saturating_add(1); + // Looped through all replicas; no connected replica found in the same AZ. + if retries > self.inner.nodes.len() { + // Attempt a fallback to any available replica in other AZs or primary. + return self.round_robin_read_from_replica(latest_read_replica_index); + } + + // Calculate index based on initial index and check count. + let index = (initial_index + retries) % self.inner.nodes.len(); + let replica = &self.inner.nodes[index]; + + // Attempt to get a connection and retrieve the replica's AZ. + if let Ok(connection) = replica.get_connection().await { + if let Some(replica_az) = connection.get_az().as_deref() { + if replica_az == client_az { + // Update `latest_used_replica` with the index of this replica. + let _ = latest_read_replica_index.compare_exchange_weak( + initial_index, + index, + Ordering::Relaxed, + Ordering::Relaxed, + ); + return replica; + } + } + } + } + } + + async fn get_connection(&self, readonly: bool) -> &ReconnectingConnection { if self.inner.nodes.len() == 1 || !readonly { return self.get_primary_connection(); } @@ -259,6 +309,16 @@ impl StandaloneClient { ReadFrom::PreferReplica { latest_read_replica_index, } => self.round_robin_read_from_replica(latest_read_replica_index), + ReadFrom::AZAffinity { + client_az, + last_read_replica_index, + } => { + self.round_robin_read_from_replica_az_awareness( + last_read_replica_index, + client_az.to_string(), + ) + .await + } } } @@ -354,7 +414,7 @@ impl StandaloneClient { cmd: &redis::Cmd, readonly: bool, ) -> RedisResult { - let reconnecting_connection = self.get_connection(readonly); + let reconnecting_connection = self.get_connection(readonly).await; Self::send_request(cmd, reconnecting_connection).await } @@ -478,6 +538,7 @@ impl StandaloneClient { password: Option, ) -> RedisResult { self.get_connection(false) + .await .get_connection() .await? .update_connection_password(password.clone()) @@ -491,6 +552,7 @@ async fn get_connection_and_replication_info( connection_info: &redis::RedisConnectionInfo, tls_mode: TlsMode, push_sender: &Option>, + discover_az: bool, ) -> Result<(ReconnectingConnection, Value), (ReconnectingConnection, RedisError)> { let result = ReconnectingConnection::new( address, @@ -498,6 +560,7 @@ async fn get_connection_and_replication_info( connection_info.clone(), tls_mode, push_sender.clone(), + discover_az, ) .await; let reconnecting_connection = match result { @@ -532,6 +595,10 @@ fn get_read_from(read_from: Option) -> ReadFrom { Some(super::ReadFrom::PreferReplica) => ReadFrom::PreferReplica { latest_read_replica_index: Default::default(), }, + Some(super::ReadFrom::AZAffinity(az)) => ReadFrom::AZAffinity { + client_az: az, + last_read_replica_index: Default::default(), + }, None => ReadFrom::Primary, } } diff --git a/glide-core/src/client/types.rs b/glide-core/src/client/types.rs index ef4be661e6..0c7680b3a6 100644 --- a/glide-core/src/client/types.rs +++ b/glide-core/src/client/types.rs @@ -53,11 +53,12 @@ impl ::std::fmt::Display for NodeAddress { } } -#[derive(PartialEq, Eq, Clone, Copy, Default)] +#[derive(PartialEq, Eq, Clone, Default)] pub enum ReadFrom { #[default] Primary, PreferReplica, + AZAffinity(String), } #[derive(PartialEq, Eq, Clone, Copy, Default)] @@ -99,7 +100,20 @@ impl From for ConnectionRequest { protobuf::ReadFrom::Primary => ReadFrom::Primary, protobuf::ReadFrom::PreferReplica => ReadFrom::PreferReplica, protobuf::ReadFrom::LowestLatency => todo!(), - protobuf::ReadFrom::AZAffinity => todo!(), + protobuf::ReadFrom::AZAffinity => { + if let Some(client_az) = chars_to_string_option(&value.client_az) { + ReadFrom::AZAffinity(client_az) + } else { + log_warn( + "types", + format!( + "Failed to convert availability zone string: '{:?}'. Falling back to `ReadFrom::PreferReplica`", + value.client_az + ), + ); + ReadFrom::PreferReplica + } + } }); let client_name = chars_to_string_option(&value.client_name); diff --git a/glide-core/src/protobuf/connection_request.proto b/glide-core/src/protobuf/connection_request.proto index e8f54c042a..5f4db44b00 100644 --- a/glide-core/src/protobuf/connection_request.proto +++ b/glide-core/src/protobuf/connection_request.proto @@ -70,6 +70,7 @@ message ConnectionRequest { } PubSubSubscriptions pubsub_subscriptions = 13; uint32 inflight_requests_limit = 14; + string client_az = 15; } message ConnectionRetryStrategy { diff --git a/glide-core/tests/test_standalone_client.rs b/glide-core/tests/test_standalone_client.rs index 8001ccab0c..c118d6d28f 100644 --- a/glide-core/tests/test_standalone_client.rs +++ b/glide-core/tests/test_standalone_client.rs @@ -273,6 +273,18 @@ mod standalone_client_tests { }); } + #[rstest] + #[serial_test::serial] + #[timeout(SHORT_STANDALONE_TEST_TIMEOUT)] + fn test_read_from_replica_az_affinity() { + test_read_from_replica(ReadFromReplicaTestConfig { + read_from: ReadFrom::AZAffinity, + expected_primary_reads: 0, + expected_replica_reads: vec![1, 1, 1], + ..Default::default() + }); + } + #[rstest] #[serial_test::serial] #[timeout(SHORT_STANDALONE_TEST_TIMEOUT)] diff --git a/glide-core/tests/utilities/cluster.rs b/glide-core/tests/utilities/cluster.rs index 7660026d9c..8f7ed6aca0 100644 --- a/glide-core/tests/utilities/cluster.rs +++ b/glide-core/tests/utilities/cluster.rs @@ -298,6 +298,25 @@ pub async fn setup_default_client(cluster: &RedisCluster) -> Client { create_cluster_client(Some(cluster), test_config).await } +pub async fn setup_cluster_with_replicas( + configuration: TestConfiguration, + replicas_num: u16, + primaries_num: u16, +) -> ClusterTestBasics { + let cluster = if !configuration.shared_server { + Some(RedisCluster::new( + configuration.use_tls, + &configuration.connection_info, + Some(primaries_num), + Some(replicas_num), + )) + } else { + None + }; + let client = create_cluster_client(cluster.as_ref(), configuration).await; + ClusterTestBasics { cluster, client } +} + pub async fn setup_test_basics(use_tls: bool) -> ClusterTestBasics { setup_test_basics_internal(TestConfiguration { use_tls, diff --git a/glide-core/tests/utilities/mod.rs b/glide-core/tests/utilities/mod.rs index 765c1cffb1..7318d6e640 100644 --- a/glide-core/tests/utilities/mod.rs +++ b/glide-core/tests/utilities/mod.rs @@ -659,6 +659,10 @@ pub fn create_connection_request( connection_request.client_name = client_name.deref().into(); } + if let Some(client_az) = &configuration.client_az { + connection_request.client_az = client_az.deref().into(); + } + connection_request } @@ -673,6 +677,7 @@ pub struct TestConfiguration { pub read_from: Option, pub database_id: u32, pub client_name: Option, + pub client_az: Option, pub protocol: ProtocolVersion, } diff --git a/node/src/BaseClient.ts b/node/src/BaseClient.ts index 665acb4fae..dcd9da4e0c 100644 --- a/node/src/BaseClient.ts +++ b/node/src/BaseClient.ts @@ -498,7 +498,10 @@ export type ReadFrom = | "primary" /** Spread the requests between all replicas in a round robin manner. If no replica is available, route the requests to the primary.*/ - | "preferReplica"; + | "preferReplica" + /** Spread the requests between replicas in the same client's Aviliablity zone in a round robin manner. + If no replica is available, route the requests to the primary.*/ + | "AZAffinity"; /** * Configuration settings for creating a client. Shared settings for standalone and cluster clients. @@ -5993,6 +5996,7 @@ export class BaseClient { > = { primary: connection_request.ReadFrom.Primary, preferReplica: connection_request.ReadFrom.PreferReplica, + AZAffinity: connection_request.ReadFrom.AZAffinity, }; /** diff --git a/package.json b/package.json index 2f59fcc5a8..3f61298feb 100644 --- a/package.json +++ b/package.json @@ -3,7 +3,7 @@ "@eslint/js": "^9.10.0", "@types/eslint__js": "^8.42.3", "@types/eslint-config-prettier": "^6.11.3", - "eslint": "^9.10.0", + "eslint": "9.14.0", "eslint-config-prettier": "^9.1.0", "prettier": "^3.3.3", "typescript": "^5.6.2", From b8dc6a54d2df06a5fd80d5465ace57b1112db6b4 Mon Sep 17 00:00:00 2001 From: ikolomi Date: Sun, 10 Nov 2024 20:58:16 +0200 Subject: [PATCH 02/11] ORT workflows rework: 1. Added ort-sweeper workflow that triggers periodic (once a day) ORT runs for branches main and release-* 2. Fixed ORT invocation for glide-core/redis-rs by creating a virtual workspace 3. Updated license-generating script to prevent the creation of unapproved_package_list.txt file for unapproved/unknown licenses 4. Configured ORT checker workflow to run on each pull request without file filters, producing warnings if licenses need to be updated 5. TODO: Modify ORT checker to fail pull requests introducing unapproved/unknown licenses, deferred until unsupported licenses are removed to avoid blocking PRs 6. Added support to trigger the ORT checker manually with a specified target branch Signed-off-by: ikolomi --- .github/workflows/ort-sweeper.yml | 57 ++++ .github/workflows/ort.yml | 432 ++++++++++++++++-------------- glide-core/redis-rs/Cargo.toml | 8 +- glide-core/redis-rs/src/main.rs | 3 + utils/get_licenses_from_ort.py | 4 + 5 files changed, 299 insertions(+), 205 deletions(-) create mode 100644 .github/workflows/ort-sweeper.yml create mode 100644 glide-core/redis-rs/src/main.rs diff --git a/.github/workflows/ort-sweeper.yml b/.github/workflows/ort-sweeper.yml new file mode 100644 index 0000000000..91889c2239 --- /dev/null +++ b/.github/workflows/ort-sweeper.yml @@ -0,0 +1,57 @@ +name: ORT - Trigger periodic checks for relevant branches + +on: + schedule: + - cron: '0 0 * * *' # Runs daily at 00:00 UTC + +jobs: + trigger-ort-check: + runs-on: ubuntu-latest + + steps: + - name: Checkout repository + uses: actions/checkout@v4 + + - name: Fetch relevant branches + id: get-branches + run: | + # Get all branches matching 'release-*' and include 'main' + branches=$(git ls-remote --heads origin | awk -F'/' '/refs\/heads\/release-/ {print $NF}') + branches="main $branches" + echo "::set-output name=branches::$branches" + env: + GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} + + - name: Trigger ORT Check workflows + uses: actions/github-script@v6 + with: + script: | + const branches = "${{ steps.get-branches.outputs.branches }}".split(" "); + const workflowFile = "ort.yml"; + + const triggerWorkflow = async (branch) => { + try { + console.log(`Triggering workflow for branch: ${branch}`); + await github.rest.actions.createWorkflowDispatch({ + owner: context.repo.owner, + repo: context.repo.repo, + workflow_id: workflowFile, + ref: branch, // The branch where workflow_dispatch is triggered + inputs: { + branch_name: branch + } + }); + console.log(`Successfully triggered workflow for branch: ${branch}`); + } catch (error) { + core.setFailed(error.message); + } + }; + + // Fire all workflow dispatch requests concurrently + const promises = branches + .filter(branch => branch) // Skip empty branches + .map(branch => triggerWorkflow(branch)); + + await Promise.allSettled(promises); + env: + GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} diff --git a/.github/workflows/ort.yml b/.github/workflows/ort.yml index 2eff2a3f1a..486229e733 100644 --- a/.github/workflows/ort.yml +++ b/.github/workflows/ort.yml @@ -1,236 +1,260 @@ name: The OSS Review Toolkit (ORT) on: - schedule: - - cron: "0 0 * * *" pull_request: - paths: - - .github/workflows/ort.yml - - .github/workflows/run-ort-tools/action.yml - - utils/get_licenses_from_ort.py + workflow_dispatch: inputs: - branch: - description: "The branch to run against the ORT tool" - required: true - version: - description: "The release version of GLIDE" + branch_name: + description: 'The branch to run against the ORT tool' required: true + default: 'main' + jobs: run-ort: - if: github.repository_owner == 'valkey-io' name: Create attribution files runs-on: ubuntu-latest + + # 1. For workflow_dispatch, always allow + # 2. For pull_request, run if branch is not autogenerated ort-diff-for- branches + if: > + github.event_name == 'workflow_dispatch' || + (github.event_name == 'pull_request' && !startsWith(github.head_ref, 'ort-diff-for-')) strategy: fail-fast: false - env: + env: PYTHON_ATTRIBUTIONS: "python/THIRD_PARTY_LICENSES_PYTHON" NODE_ATTRIBUTIONS: "node/THIRD_PARTY_LICENSES_NODE" RUST_ATTRIBUTIONS: "glide-core/THIRD_PARTY_LICENSES_RUST" JAVA_ATTRIBUTIONS: "java/THIRD_PARTY_LICENSES_JAVA" + INPUT_TARGET_BRANCH: ${{ github.event.inputs.branch_name }} + EVENT_NAME: ${{ github.event_name }} + HEAD_REF: ${{ github.head_ref }} + steps: - - name: Set the release version - shell: bash - run: | - export version=`if [ "$EVENT_NAME" == 'schedule' ] || [ "$EVENT_NAME" == 'pull_request' ]; then echo '255.255.255'; else echo "$INPUT_VERSION"; fi` - echo "RELEASE_VERSION=${version}" >> $GITHUB_ENV - env: - EVENT_NAME: ${{ github.event_name }} - INPUT_VERSION: ${{ github.event.inputs.version }} - - - name: Set the base branch - run: | - export BASE_BRANCH=`if [ "$EVENT_NAME" == 'schedule' ]; then echo 'main'; elif [ "$EVENT_NAME" == 'workflow_dispatch' ]; then echo "$INPUT_BRANCH"; else echo ""; fi` - echo "Base branch is: ${BASE_BRANCH}" - echo "BASE_BRANCH=${BASE_BRANCH}" >> $GITHUB_ENV - env: - EVENT_NAME: ${{ github.event_name }} - INPUT_BRANCH: ${{ github.event.inputs.branch }} - - - name: Checkout - uses: actions/checkout@v4 - with: - submodules: "true" - ref: ${{ env.BASE_BRANCH }} - - - name: Set up JDK 11 for the ORT package - uses: actions/setup-java@v4 - with: - distribution: "temurin" - java-version: 11 - - - name: Cache ORT and Gradle packages - uses: actions/cache@v4 - id: cache-ort - with: - path: | - ./ort - ~/.gradle/caches - ~/.gradle/wrapper - key: ${{ runner.os }}-ort - - - name: Checkout ORT Repository - if: steps.cache-ort.outputs.cache-hit != 'true' - uses: actions/checkout@v4 - with: - repository: "oss-review-toolkit/ort" - path: "./ort" - ref: "26.0.0" - submodules: recursive - - - name: Install Rust toolchain - uses: dtolnay/rust-toolchain@1.78 - - - name: Install ORT - if: steps.cache-ort.outputs.cache-hit != 'true' - working-directory: ./ort/ - run: | - export JAVA_OPTS="$JAVA_OPTS -Xmx8g" - ./gradlew installDist - - - name: Create ORT config file - run: | - mkdir -p ~/.ort/config - cat << EOF > ~/.ort/config/config.yml - ort: - analyzer: - allowDynamicVersions: true - enabledPackageManagers: [Cargo, NPM, PIP, GradleInspector] - EOF - cat ~/.ort/config/config.yml + - name: Setup target branch and commit + run: | + if [ "$EVENT_NAME" == 'workflow_dispatch' ]; then + echo "TARGET_BRANCH=$INPUT_TARGET_BRANCH" >> $GITHUB_ENV + elif [ "$EVENT_NAME" == 'pull_request' ]; then + echo "TARGET_BRANCH=$HEAD_REF" >> $GITHUB_ENV + fi + + - name: Checkout target branch + uses: actions/checkout@v4 + with: + ref: ${{ env.TARGET_BRANCH }} + + - name: Setup target commit + run: | + echo "TARGET_COMMIT=`git rev-parse HEAD`" >> $GITHUB_ENV + + - name: Set up JDK 11 for the ORT package + uses: actions/setup-java@v4 + with: + distribution: "temurin" + java-version: 11 + + - name: Cache ORT and Gradle packages + uses: actions/cache@v4 + id: cache-ort + with: + path: | + ./ort + ~/.gradle/caches + ~/.gradle/wrapper + key: ${{ runner.os }}-ort + + - name: Checkout ORT Repository + if: steps.cache-ort.outputs.cache-hit != 'true' + uses: actions/checkout@v4 + with: + repository: "oss-review-toolkit/ort" + path: "./ort" + ref: "26.0.0" + submodules: recursive + + - name: Install Rust toolchain + uses: dtolnay/rust-toolchain@1.78 + + - name: Build and install ORT + if: steps.cache-ort.outputs.cache-hit != 'true' + working-directory: ./ort/ + run: | + export JAVA_OPTS="$JAVA_OPTS -Xmx8g" + ./gradlew installDist + + - name: Create ORT config file + run: | + mkdir -p ~/.ort/config + cat << EOF > ~/.ort/config/config.yml + ort: + analyzer: + allowDynamicVersions: true + enabledPackageManagers: [Cargo, NPM, PIP, GradleInspector] + EOF + cat ~/.ort/config/config.yml ### NodeJS ### + - name: Set up Node.js 16.x + uses: actions/setup-node@v4 + with: + node-version: 16.x - - name: Set up Node.js 16.x - uses: actions/setup-node@v4 - with: - node-version: 16.x - - - name: Create package.json file for the Node wrapper - uses: ./.github/workflows/node-create-package-file - with: - release_version: ${{ env.RELEASE_VERSION }} - os: "ubuntu-latest" - - - name: Fix Node base NPM package.json file for ORT - working-directory: ./node/npm/glide - run: | - # Remove the glide-rs dependency to avoid duplication - sed -i '/ "glide-rs":/d' ../../package.json - export pkg_name=valkey-glide-base - export package_version="${{ env.RELEASE_VERSION }}" - export scope=`if [ "$NPM_SCOPE" != '' ]; then echo "$NPM_SCOPE/"; fi` - mv package.json package.json.tmpl - envsubst < package.json.tmpl > "package.json" - cat package.json - - - name: Run ORT tools for Node - uses: ./.github/workflows/run-ort-tools - with: - folder_path: "${{ github.workspace }}/node" + - name: Create package.json file for the Node wrapper + uses: ./.github/workflows/node-create-package-file + with: + release_version: ${{ env.TARGET_COMMIT }} + os: "ubuntu-latest" - ### Python ### + - name: Fix Node base NPM package.json file for ORT + working-directory: ./node/npm/glide + run: | + # Remove the glide-rs dependency to avoid duplication + sed -i '/ "glide-rs":/d' ../../package.json + export pkg_name=valkey-glide-base + export package_version="${{ env.TARGET_COMMIT }}" + export scope=`if [ "$NPM_SCOPE" != '' ]; then echo "$NPM_SCOPE/"; fi` + mv package.json package.json.tmpl + envsubst < package.json.tmpl > "package.json" + cat package.json - - name: Set up Python 3.10 - uses: actions/setup-python@v5 - with: - python-version: "3.10" + - name: Run ORT tools for Node + uses: ./.github/workflows/run-ort-tools + with: + folder_path: "${{ github.workspace }}/node" - - name: Install python-inspector - working-directory: ./python - run: | - python -m pip install --upgrade pip - pip install git+https://github.com/nexB/python-inspector + ### Python ### + - name: Set up Python 3.10 + uses: actions/setup-python@v5 + with: + python-version: "3.10" - - name: Run ORT tools for Python - uses: ./.github/workflows/run-ort-tools - with: - folder_path: "${{ github.workspace }}/python" + - name: Install python-inspector + working-directory: ./python + run: | + python -m pip install --upgrade pip + pip install git+https://github.com/nexB/python-inspector - ### Rust ### + - name: Run ORT tools for Python + uses: ./.github/workflows/run-ort-tools + with: + folder_path: "${{ github.workspace }}/python" - - name: Run ORT tools for Rust - uses: ./.github/workflows/run-ort-tools - with: - folder_path: "${{ github.workspace }}/glide-core" + ### Rust glide-core ### + - name: Run ORT tools for glide-core + uses: ./.github/workflows/run-ort-tools + with: + folder_path: "${{ github.workspace }}/glide-core" ### Java ### + - name: Set up JDK 11 + uses: actions/setup-java@v4 + with: + distribution: "temurin" + java-version: 11 + + - name: Run ORT tools for Java + uses: ./.github/workflows/run-ort-tools + with: + folder_path: "${{ github.workspace }}/java" + + + ### Get licenses ### + - name: Retrieve licenses list + working-directory: ./utils + run: | + { + echo 'LICENSES_LIST<> "$GITHUB_ENV" + + ### Upload licenses ### + - name: Get current date + id: date + run: | + CURR_DATE=$(date +'%Y-%m-%d-%H') + echo "date=${CURR_DATE}" >> $GITHUB_OUTPUT + + - name: Upload the final package list + continue-on-error: true + uses: actions/upload-artifact@v4 + with: + name: final-package-list-${{ steps.date.outputs.date }} + path: | + utils/final_package_list.txt + retention-days: 30 + + - name: Upload the skipped package list + continue-on-error: true + uses: actions/upload-artifact@v4 + with: + name: skipped-package-list-${{ steps.date.outputs.date }} + path: | + utils/skipped_package_list.txt + retention-days: 30 - - name: Set up JDK 11 - uses: actions/setup-java@v4 - with: - distribution: "temurin" - java-version: 11 - - - name: Run ORT tools for Java - uses: ./.github/workflows/run-ort-tools - with: - folder_path: "${{ github.workspace }}/java" - - ### Process results ### - - - name: Check for diff - run: | - cp python/ort_results/NOTICE_DEFAULT $PYTHON_ATTRIBUTIONS - cp node/ort_results/NOTICE_DEFAULT $NODE_ATTRIBUTIONS - cp glide-core/ort_results/NOTICE_DEFAULT $RUST_ATTRIBUTIONS - cp java/ort_results/NOTICE_DEFAULT $JAVA_ATTRIBUTIONS - GIT_DIFF=`git diff $PYTHON_ATTRIBUTIONS $NODE_ATTRIBUTIONS $RUST_ATTRIBUTIONS $JAVA_ATTRIBUTIONS` - if [ -n "$GIT_DIFF" ]; then + - name: Upload the unknown/unapproved package list + continue-on-error: true + uses: actions/upload-artifact@v4 + with: + name: unapproved-package-list-${{ steps.date.outputs.date }} + path: | + utils/unapproved_package_list.txt + retention-days: 30 + + ### TODO: Fail if there are unapproved packages ### + + ### Check for attributions diff ### + - name: Check for diff + run: | + cp python/ort_results/NOTICE_DEFAULT $PYTHON_ATTRIBUTIONS + cp node/ort_results/NOTICE_DEFAULT $NODE_ATTRIBUTIONS + cp glide-core/ort_results/NOTICE_DEFAULT $RUST_ATTRIBUTIONS + cp java/ort_results/NOTICE_DEFAULT $JAVA_ATTRIBUTIONS + GIT_DIFF=`git diff $PYTHON_ATTRIBUTIONS $NODE_ATTRIBUTIONS $RUST_ATTRIBUTIONS $JAVA_ATTRIBUTIONS` + if [ -n "$GIT_DIFF" ]; then echo "FOUND_DIFF=true" >> $GITHUB_ENV - else + else echo "FOUND_DIFF=false" >> $GITHUB_ENV - fi - - - name: Retrieve licenses list - working-directory: ./utils - run: | - { - echo 'LICENSES_LIST<> "$GITHUB_ENV" - - ### Create PR ### - - - name: Create pull request - if: ${{ env.FOUND_DIFF == 'true' && github.event_name != 'pull_request' }} - run: | - export BRANCH_NAME=`if [ "$EVENT_NAME" == 'schedule' ] || [ "$EVENT_NAME" == 'pull_request' ]; then echo 'scheduled-ort'; else echo "ort-v$INPUT_VERSION"; fi` - echo "Creating pull request from branch ${BRANCH_NAME} to branch ${{ env.BASE_BRANCH }}" - git config --global user.email "valkey-glide@lists.valkey.io" - git config --global user.name "ort-bot" - git checkout -b ${BRANCH_NAME} - git add $PYTHON_ATTRIBUTIONS $NODE_ATTRIBUTIONS $RUST_ATTRIBUTIONS $JAVA_ATTRIBUTIONS - git commit -m "Updated attribution files" -s - git push --set-upstream origin ${BRANCH_NAME} -f - title="Updated attribution files for ${BRANCH_NAME}" - gh pr create -B ${{ env.BASE_BRANCH }} -H ${BRANCH_NAME} --title "${title}" --body 'Created by Github action.\n${{ env.LICENSES_LIST }}' - env: - GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} - EVENT_NAME: ${{ github.event_name }} - INPUT_VERSION: ${{ github.event.inputs.version }} - - - name: Get current date - id: date - run: | - CURR_DATE=$(date +'%Y-%m-%d-%H') - echo "date=${CURR_DATE}" >> $GITHUB_OUTPUT - - - name: Upload the final package list - continue-on-error: true - uses: actions/upload-artifact@v4 - with: - name: final-package-list-${{ steps.date.outputs.date }} - path: | - utils/final_package_list.txt - retention-days: 30 - - - name: Upload the skipped package list - continue-on-error: true - uses: actions/upload-artifact@v4 - with: - name: skipped-package-list-${{ steps.date.outputs.date }} - path: | - utils/skipped_package_list.txt - retention-days: 30 + fi + + ### Create PR, Note a potential race on the source branch ### + - name: Create pull request + if: ${{ env.FOUND_DIFF == 'true' && github.event_name != 'pull_request' }} + run: | + export ORT_DIFF_BRANCH_NAME="ort-diff-for-$TARGET_BRANCH" + echo "Creating pull request from branch $ORT_DIFF_BRANCH_NAME to branch $TARGET_BRANCH" + git config --global user.email "valkey-glide@lists.valkey.io" + git config --global user.name "ort-bot" + git checkout -b ${ORT_DIFF_BRANCH_NAME} + git add $PYTHON_ATTRIBUTIONS $NODE_ATTRIBUTIONS $RUST_ATTRIBUTIONS $JAVA_ATTRIBUTIONS + git commit -m "Updated attribution files" -s + git push --set-upstream origin ${ORT_DIFF_BRANCH_NAME} -f + + # Check if PR already exists + existing_pr=$(gh pr list --base ${TARGET_BRANCH} --head ${ORT_DIFF_BRANCH_NAME} --json number --jq '.[0].number') + + if [ -z "$existing_pr" ]; then + # Create a new PR if none exists + title="Updated attribution files for commit ${TARGET_COMMIT}" + gh pr create -B ${TARGET_BRANCH} -H ${ORT_DIFF_BRANCH_NAME} --title "${title}" --body "Created by Github action. ${{ env.LICENSES_LIST }}" + echo "Pull request created successfully." + else + # Update the existing PR + echo "Pull request #$existing_pr already exists. Updating branch." + gh pr edit $existing_pr --title "Updated attribution files for commit ${TARGET_COMMIT}" --body "Created by Github action. ${{ env.LICENSES_LIST }}" + echo "Pull request updated successfully." + fi + env: + GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} + INPUT_VERSION: ${{ github.event.inputs.version }} + + ### Warn of outdated attributions for PR ### + - name: Warn of outdated attributions due to the PR + if: ${{ env.FOUND_DIFF == 'true' && github.event_name == 'pull_request' }} + uses: actions/github-script@v6 + with: + script: | + core.warning('WARNING! Note the attribution files differ with this PR, make sure an updating PR is issued using scheduled or manual run of this workflow!'); diff --git a/glide-core/redis-rs/Cargo.toml b/glide-core/redis-rs/Cargo.toml index 2f4ebbcbbe..fa989b93cd 100644 --- a/glide-core/redis-rs/Cargo.toml +++ b/glide-core/redis-rs/Cargo.toml @@ -1,3 +1,9 @@ +# Dummy package so ORT tool will not fail on virtual workspace +[package] +name = "dummy-for-ort" +version = "0.1.0" +edition = "2021" + [workspace] -members = ["redis", "redis-test"] +members = ["redis", "redis-test", "afl/parser"] resolver = "2" diff --git a/glide-core/redis-rs/src/main.rs b/glide-core/redis-rs/src/main.rs new file mode 100644 index 0000000000..a5610f8be9 --- /dev/null +++ b/glide-core/redis-rs/src/main.rs @@ -0,0 +1,3 @@ +fn main() { + println!("Dummy source to bypass ORT OSS Tool virtual workspace restrictions."); +} diff --git a/utils/get_licenses_from_ort.py b/utils/get_licenses_from_ort.py index 3d9853ea53..9c7d7b62ba 100644 --- a/utils/get_licenses_from_ort.py +++ b/utils/get_licenses_from_ort.py @@ -124,6 +124,10 @@ def __str__(self): with open(skipped_list_file_path, mode="wt", encoding="utf-8") as f: f.writelines(f"{package}\n" for package in skipped_packages) +unapproved_list_file_path = f"{SCRIPT_PATH}/unapproved_package_list.txt" +with open(unapproved_list_file_path, mode="wt", encoding="utf-8") as f: + f.writelines(f"{package}\n" for package in unknown_licenses) + print("\n\n#### Found Licenses #####\n") all_licenses_set = set(sorted(all_licenses_set)) for license in all_licenses_set: From 86fa0d464819219360d2372ba086c27772dec923 Mon Sep 17 00:00:00 2001 From: BoazBD <50696333+BoazBD@users.noreply.github.com> Date: Tue, 19 Nov 2024 12:07:08 +0200 Subject: [PATCH 03/11] Python : Client API for retrieving internal statistics (#2707) * add get statistics support to python wrapper Signed-off-by: BoazBD * fmt rust Signed-off-by: BoazBD * fix deprecated funcs Signed-off-by: BoazBD * fmt again Signed-off-by: BoazBD * add to changelog Signed-off-by: BoazBD * refactor imports Signed-off-by: BoazBD --------- Signed-off-by: BoazBD --- CHANGELOG.md | 1 + python/python/glide/glide.pyi | 1 + python/python/glide/glide_client.py | 4 +++ python/python/tests/test_async_client.py | 9 +++++++ python/src/lib.rs | 31 +++++++++++++++++++++++- 5 files changed, 45 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 23e0dc32f4..0bdef111eb 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ #### Changes +* Python: Client API for retrieving internal statistics ([#2707](https://github.com/valkey-io/valkey-glide/pull/2707)) * Node, Python: Adding support for replacing connection configured password ([#2651](https://github.com/valkey-io/valkey-glide/pull/2651)) * Node: Add FT._ALIASLIST command([#2652](https://github.com/valkey-io/valkey-glide/pull/2652)) * Python: Python: `FT._ALIASLIST` command added([#2638](https://github.com/valkey-io/valkey-glide/pull/2638)) diff --git a/python/python/glide/glide.pyi b/python/python/glide/glide.pyi index b544a3948e..bbd5274770 100644 --- a/python/python/glide/glide.pyi +++ b/python/python/glide/glide.pyi @@ -31,5 +31,6 @@ def start_socket_listener_external(init_callback: Callable) -> None: ... def value_from_pointer(pointer: int) -> TResult: ... def create_leaked_value(message: str) -> int: ... def create_leaked_bytes_vec(args_vec: List[bytes]) -> int: ... +def get_statistics() -> dict: ... def py_init(level: Optional[Level], file_name: Optional[str]) -> Level: ... def py_log(log_level: Level, log_identifier: str, message: str) -> None: ... diff --git a/python/python/glide/glide_client.py b/python/python/glide/glide_client.py index 2838ae288e..1c1dc07ee7 100644 --- a/python/python/glide/glide_client.py +++ b/python/python/glide/glide_client.py @@ -33,6 +33,7 @@ MAX_REQUEST_ARGS_LEN, ClusterScanCursor, create_leaked_bytes_vec, + get_statistics, start_socket_listener_external, value_from_pointer, ) @@ -533,6 +534,9 @@ async def _reader_loop(self) -> None: else: await self._process_response(response=response) + async def get_statistics(self) -> dict: + return get_statistics() + async def _update_connection_password( self, password: Optional[str], re_auth: bool ) -> TResult: diff --git a/python/python/tests/test_async_client.py b/python/python/tests/test_async_client.py index 7560cc7b23..da812c07a3 100644 --- a/python/python/tests/test_async_client.py +++ b/python/python/tests/test_async_client.py @@ -286,6 +286,15 @@ async def test_closed_client_raises_error(self, glide_client: TGlideClient): await glide_client.set("foo", "bar") assert "the client is closed" in str(e) + @pytest.mark.parametrize("cluster_mode", [True, False]) + @pytest.mark.parametrize("protocol", [ProtocolVersion.RESP2, ProtocolVersion.RESP3]) + async def test_statistics(self, glide_client: TGlideClient): + stats = await glide_client.get_statistics() + assert isinstance(stats, dict) + assert "total_connections" in stats + assert "total_clients" in stats + assert len(stats) == 2 + @pytest.mark.asyncio class TestCommands: diff --git a/python/src/lib.rs b/python/src/lib.rs index 8a1a0d3444..6b41123dd3 100644 --- a/python/src/lib.rs +++ b/python/src/lib.rs @@ -4,12 +4,14 @@ use glide_core::client::FINISHED_SCAN_CURSOR; * Copyright Valkey GLIDE Project Contributors - SPDX Identifier: Apache-2.0 */ use glide_core::start_socket_listener; +use glide_core::Telemetry; use glide_core::MAX_REQUEST_ARGS_LENGTH; use pyo3::exceptions::PyTypeError; use pyo3::prelude::*; -use pyo3::types::{PyAny, PyBool, PyBytes, PyDict, PyFloat, PyList, PySet}; +use pyo3::types::{PyAny, PyBool, PyBytes, PyDict, PyFloat, PyList, PySet, PyString}; use pyo3::Python; use redis::Value; +use std::collections::HashMap; use std::sync::Arc; pub const DEFAULT_TIMEOUT_IN_MILLISECONDS: u32 = @@ -120,12 +122,39 @@ fn glide(_py: Python, m: &Bound) -> PyResult<()> { m.add_function(wrap_pyfunction!(value_from_pointer, m)?)?; m.add_function(wrap_pyfunction!(create_leaked_value, m)?)?; m.add_function(wrap_pyfunction!(create_leaked_bytes_vec, m)?)?; + m.add_function(wrap_pyfunction!(get_statistics, m)?)?; #[pyfunction] fn py_log(log_level: Level, log_identifier: String, message: String) { log(log_level, log_identifier, message); } + #[pyfunction] + fn get_statistics(_py: Python) -> PyResult { + let mut stats_map = HashMap::::new(); + stats_map.insert( + "total_connections".to_string(), + Telemetry::total_connections().to_string(), + ); + stats_map.insert( + "total_clients".to_string(), + Telemetry::total_clients().to_string(), + ); + + Python::with_gil(|py| { + let py_dict = PyDict::new_bound(py); + + for (key, value) in stats_map { + py_dict.set_item( + PyString::new_bound(py, &key), + PyString::new_bound(py, &value), + )?; + } + + Ok(py_dict.into_py(py)) + }) + } + #[pyfunction] #[pyo3(signature = (level=None, file_name=None))] fn py_init(level: Option, file_name: Option<&str>) -> Level { From 168069c14457832833d9a62225b8f8dca3586610 Mon Sep 17 00:00:00 2001 From: Avi Fenesh <55848801+avifenesh@users.noreply.github.com> Date: Tue, 19 Nov 2024 14:55:58 +0200 Subject: [PATCH 04/11] ORT-external-fork-fix (#2712) Python : Client API for retrieving internal statistics (#2707) * add get statistics support to python wrapper * fmt rust * fix deprecated funcs * fmt again * add to changelog * refactor imports --------- Signed-off-by: BoazBD Signed-off-by: avifenesh Co-authored-by: BoazBD <50696333+BoazBD@users.noreply.github.com> --- .github/workflows/ort.yml | 40 ++++++++++++++++++++++++++++++++------- 1 file changed, 33 insertions(+), 7 deletions(-) diff --git a/.github/workflows/ort.yml b/.github/workflows/ort.yml index 486229e733..a01659e4f5 100644 --- a/.github/workflows/ort.yml +++ b/.github/workflows/ort.yml @@ -43,7 +43,11 @@ jobs: - name: Checkout target branch uses: actions/checkout@v4 with: - ref: ${{ env.TARGET_BRANCH }} + ref: ${{ env.TARGET_BRANCH }} + repository: ${{ github.event.pull_request.head.repo.full_name }} + token: ${{ secrets.GITHUB_TOKEN }} + fetch-depth: 0 # Fetch all history for all branches and tags + - name: Setup target commit run: | @@ -251,10 +255,32 @@ jobs: GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} INPUT_VERSION: ${{ github.event.inputs.version }} - ### Warn of outdated attributions for PR ### + ### Warn of outdated attributions for PR ### - name: Warn of outdated attributions due to the PR - if: ${{ env.FOUND_DIFF == 'true' && github.event_name == 'pull_request' }} - uses: actions/github-script@v6 - with: - script: | - core.warning('WARNING! Note the attribution files differ with this PR, make sure an updating PR is issued using scheduled or manual run of this workflow!'); + if: ${{ env.FOUND_DIFF == 'true' && github.event_name == 'pull_request' }} + run: | + ATTRIBUTION_FILES=( + "${{ env.PYTHON_ATTRIBUTIONS }}" + "${{ env.NODE_ATTRIBUTIONS }}" + "${{ env.RUST_ATTRIBUTIONS }}" + "${{ env.JAVA_ATTRIBUTIONS }}" + ) + + MESSAGE="WARNING! The attribution files differ in this PR. Please ensure an updating PR is issued using a scheduled or manual run of this workflow!" + + # Echo the message to the console + echo "$MESSAGE" + + # Emit a general warning in the action log + echo "::warning::$MESSAGE" + + # Loop through the attribution files + for FILE in "${ATTRIBUTION_FILES[@]}"; do + if git diff --quiet "$FILE"; then + continue + else + # Emit a warning associated with the changed file + echo "::warning file=$FILE::WARNING! The attribution file '$FILE' differs in this PR." + fi + done + From 188ae70d5572a3f1a74372d1b45cbfd9091955ce Mon Sep 17 00:00:00 2001 From: tjzhang-BQ <111323543+tjzhang-BQ@users.noreply.github.com> Date: Tue, 19 Nov 2024 10:47:06 -0800 Subject: [PATCH 05/11] Node: adding binary test case for json module (#2644) * Adding binary test case for json.set Signed-off-by: TJ Zhang --- node/tests/ServerModules.test.ts | 21 +++++++++++++++++++-- 1 file changed, 19 insertions(+), 2 deletions(-) diff --git a/node/tests/ServerModules.test.ts b/node/tests/ServerModules.test.ts index f480718a6a..df16ce89e7 100644 --- a/node/tests/ServerModules.test.ts +++ b/node/tests/ServerModules.test.ts @@ -106,13 +106,29 @@ describe("Server Module Tests", () => { let result = await GlideJson.get(client, key, { path: "." }); expect(JSON.parse(result.toString())).toEqual(jsonValue); + // binary buffer test + result = await GlideJson.get(client, Buffer.from(key), { + path: Buffer.from("."), + decoder: Decoder.Bytes, + }); + expect(result).toEqual(Buffer.from(JSON.stringify(jsonValue))); + + expect( + await GlideJson.set( + client, + Buffer.from(key), + Buffer.from("$"), + Buffer.from(JSON.stringify({ a: 1.0, b: 3 })), + ), + ).toBe("OK"); + // JSON.get with array of paths result = await GlideJson.get(client, key, { path: ["$.a", "$.b"], }); expect(JSON.parse(result.toString())).toEqual({ "$.a": [1.0], - "$.b": [2], + "$.b": [3], }); // JSON.get with non-existing key @@ -1300,8 +1316,9 @@ describe("Server Module Tests", () => { expect( await GlideJson.resp(client, Buffer.from(key), { path: Buffer.from("..a"), + decoder: Decoder.Bytes, }), - ).toEqual(["[", 1, 2, 3]); + ).toEqual([Buffer.from("["), 1, 2, 3]); }); it("json.arrtrim tests", async () => { From bcab08c674eb68bfeefb679c98851ca9ab739474 Mon Sep 17 00:00:00 2001 From: Avi Fenesh <55848801+avifenesh@users.noreply.github.com> Date: Wed, 20 Nov 2024 00:18:56 +0200 Subject: [PATCH 06/11] Improve password update functionality and add tests (#2695) * Update Redis dependency and enhance password update functionality - Bump afl dependency version to 0.15. - Add UserOperationError to handle incorrect management operations. - Modify update_connection_password to support immediate authentication. - Refactor password update logic to include timeout handling. Signed-off-by: avifenesh * Rename re_auth to immediate_auth in UpdateConnectionPassword message Signed-off-by: avifenesh * Refactor updateConnectionPassword method to rename reAuth parameter to immediateAuth and enhance documentation Signed-off-by: avifenesh * Remove redundant tests for updateConnectionPassword and streamline test suite Signed-off-by: avifenesh * Refactor update_connection_password method to rename re_auth to immediate_auth and enhance documentation Signed-off-by: avifenesh --------- Signed-off-by: avifenesh --- CHANGELOG.md | 2 +- glide-core/redis-rs/afl/parser/Cargo.toml | 2 +- glide-core/redis-rs/redis/src/types.rs | 6 + glide-core/src/client/mod.rs | 67 +++- glide-core/src/protobuf/command_request.proto | 2 +- glide-core/src/socket_listener.rs | 2 +- node/src/BaseClient.ts | 16 +- node/tests/AuthTest.test.ts | 330 ++++++++++++++++++ node/tests/SharedTests.ts | 155 -------- python/python/glide/async_commands/core.py | 21 +- python/python/glide/glide_client.py | 9 +- python/python/tests/conftest.py | 97 ++++- python/python/tests/test_auth.py | 151 ++++---- 13 files changed, 593 insertions(+), 267 deletions(-) create mode 100644 node/tests/AuthTest.test.ts diff --git a/CHANGELOG.md b/CHANGELOG.md index 0bdef111eb..228aa7bdf2 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,6 +1,6 @@ #### Changes * Python: Client API for retrieving internal statistics ([#2707](https://github.com/valkey-io/valkey-glide/pull/2707)) -* Node, Python: Adding support for replacing connection configured password ([#2651](https://github.com/valkey-io/valkey-glide/pull/2651)) +* Node, Python: Adding support for replacing connection configured password ([#2651](https://github.com/valkey-io/valkey-glide/pull/2651))([#2659](https://github.com/valkey-io/valkey-glide/pull/2659)) * Node: Add FT._ALIASLIST command([#2652](https://github.com/valkey-io/valkey-glide/pull/2652)) * Python: Python: `FT._ALIASLIST` command added([#2638](https://github.com/valkey-io/valkey-glide/pull/2638)) * Node: alias commands added: FT.ALIASADD, FT.ALIADDEL, FT.ALIASUPDATE([#2596](https://github.com/valkey-io/valkey-glide/pull/2596)) diff --git a/glide-core/redis-rs/afl/parser/Cargo.toml b/glide-core/redis-rs/afl/parser/Cargo.toml index 9f5202d86a..ef356faaf9 100644 --- a/glide-core/redis-rs/afl/parser/Cargo.toml +++ b/glide-core/redis-rs/afl/parser/Cargo.toml @@ -13,5 +13,5 @@ name = "reproduce" path = "src/reproduce.rs" [dependencies] -afl = "0.4" +afl = "0.15" redis = { path = "../../redis" } diff --git a/glide-core/redis-rs/redis/src/types.rs b/glide-core/redis-rs/redis/src/types.rs index 4b6cdbb150..2d8035d697 100644 --- a/glide-core/redis-rs/redis/src/types.rs +++ b/glide-core/redis-rs/redis/src/types.rs @@ -153,6 +153,10 @@ pub enum ErrorKind { /// Not all slots are covered by the cluster NotAllSlotsCovered, + + /// Used when an error occurs on when user perform wrong usage of management operation. + /// E.g. not allowed configuration change. + UserOperationError, } #[derive(PartialEq, Debug)] @@ -900,6 +904,7 @@ impl RedisError { ErrorKind::RESP3NotSupported => "resp3 is not supported by server", ErrorKind::ParseError => "parse error", ErrorKind::NotAllSlotsCovered => "not all slots are covered", + ErrorKind::UserOperationError => "Wrong usage of management operation", } } @@ -1095,6 +1100,7 @@ impl RedisError { ErrorKind::NotAllSlotsCovered => RetryMethod::NoRetry, ErrorKind::FatalReceiveError => RetryMethod::Reconnect, ErrorKind::FatalSendError => RetryMethod::ReconnectAndRetry, + ErrorKind::UserOperationError => RetryMethod::NoRetry, } } } diff --git a/glide-core/src/client/mod.rs b/glide-core/src/client/mod.rs index 78558f8e28..cfe8d6dc05 100644 --- a/glide-core/src/client/mod.rs +++ b/glide-core/src/client/mod.rs @@ -480,28 +480,63 @@ impl Client { /// Update the password used to authenticate with the servers. /// If None is passed, the password will be removed. - /// If `re_auth` is true, the new password will be used to re-authenticate with all of the nodes. + /// If `immediate_auth` is true, the password will be used to authenticate with the servers immediately using the `AUTH` command. + /// The default behavior is to update the password without authenticating immediately. + /// If the password is empty or None, and `immediate_auth` is true, the password will be updated and an error will be returned. pub async fn update_connection_password( &mut self, password: Option, - re_auth: bool, + immediate_auth: bool, ) -> RedisResult { - if re_auth { - let routing = RoutingInfo::MultiNode(( - MultipleNodeRoutingInfo::AllNodes, - Some(ResponsePolicy::AllSucceeded), - )); - let mut cmd = redis::cmd("AUTH"); - cmd.arg(&password); - self.send_command(&cmd, Some(routing)).await?; + let timeout = self.request_timeout; + // The password update operation is wrapped in a timeout to prevent it from blocking indefinitely. + // If the operation times out, an error is returned. + // Since the password update operation is not a command that go through the regular command pipeline, + // it is not have the regular timeout handling, as such we need to handle it separately. + match tokio::time::timeout(timeout, async { + match self.internal_client { + ClientWrapper::Standalone(ref mut client) => { + client.update_connection_password(password.clone()).await + } + ClientWrapper::Cluster { ref mut client } => { + client.update_connection_password(password.clone()).await + } + } + }) + .await + { + Ok(result) => { + if immediate_auth { + self.send_immediate_auth(password).await + } else { + result + } + } + Err(_elapsed) => Err(RedisError::from(( + ErrorKind::IoError, + "Password update operation timed out, please check the connection", + ))), } + } - match self.internal_client { - ClientWrapper::Standalone(ref mut client) => { - client.update_connection_password(password).await - } - ClientWrapper::Cluster { ref mut client } => { - client.update_connection_password(password).await + async fn send_immediate_auth(&mut self, password: Option) -> RedisResult { + match &password { + Some(pw) if pw.is_empty() => Err(RedisError::from(( + ErrorKind::UserOperationError, + "Empty password provided for authentication", + ))), + None => Err(RedisError::from(( + ErrorKind::UserOperationError, + "No password provided for authentication", + ))), + Some(password) => { + let routing = RoutingInfo::MultiNode(( + MultipleNodeRoutingInfo::AllNodes, + Some(ResponsePolicy::AllSucceeded), + )); + let mut cmd = redis::cmd("AUTH"); + cmd.arg(password); + self.send_command(&cmd, Some(routing)).await } } } diff --git a/glide-core/src/protobuf/command_request.proto b/glide-core/src/protobuf/command_request.proto index e50cdc8b3c..30b33362af 100644 --- a/glide-core/src/protobuf/command_request.proto +++ b/glide-core/src/protobuf/command_request.proto @@ -510,7 +510,7 @@ message ClusterScan { message UpdateConnectionPassword { optional string password = 1; - bool re_auth = 2; + bool immediate_auth = 2; } message CommandRequest { diff --git a/glide-core/src/socket_listener.rs b/glide-core/src/socket_listener.rs index b7f967e0bd..b9db4e6d99 100644 --- a/glide-core/src/socket_listener.rs +++ b/glide-core/src/socket_listener.rs @@ -529,7 +529,7 @@ fn handle_request(request: CommandRequest, mut client: Client, writer: Rc( diff --git a/node/tests/AuthTest.test.ts b/node/tests/AuthTest.test.ts new file mode 100644 index 0000000000..3466199ba8 --- /dev/null +++ b/node/tests/AuthTest.test.ts @@ -0,0 +1,330 @@ +/** + * Copyright Valkey GLIDE Project Contributors - SPDX Identifier: Apache-2.0 + */ + +import { + afterAll, + afterEach, + beforeAll, + describe, + expect, + it, +} from "@jest/globals"; +import { + BaseClientConfiguration, + GlideClient, + GlideClusterClient, + ProtocolVersion, + RequestError, +} from ".."; +import { ValkeyCluster } from "../../utils/TestUtils"; +import { + flushAndCloseClient, + getServerVersion, + parseEndpoints, +} from "./TestUtilities"; + +type BaseClient = GlideClient | GlideClusterClient; + +const TIMEOUT = 50000; + +type AddressEntry = [string, number]; + +describe("Auth tests", () => { + let cmeCluster: ValkeyCluster; + let cmdCluster: ValkeyCluster; + let managementClient: BaseClient; + let client: BaseClient; + beforeAll(async () => { + const standaloneAddresses = global.STAND_ALONE_ENDPOINT; + const clusterAddresses = global.CLUSTER_ENDPOINTS; + + // Connect to cluster or create a new one based on the parsed addresses + cmdCluster = standaloneAddresses + ? await ValkeyCluster.initFromExistingCluster( + false, + parseEndpoints(standaloneAddresses), + getServerVersion, + ) + : await ValkeyCluster.createCluster(false, 1, 1, getServerVersion); + + cmeCluster = clusterAddresses + ? await ValkeyCluster.initFromExistingCluster( + true, + parseEndpoints(clusterAddresses), + getServerVersion, + ) + : await ValkeyCluster.createCluster(true, 3, 1, getServerVersion); + + // Create appropriate client based on mode + const isStandaloneMode = !!standaloneAddresses; + const activeCluster = isStandaloneMode ? cmdCluster : cmeCluster; + const ClientClass = isStandaloneMode ? GlideClient : GlideClusterClient; + + managementClient = await ClientClass.createClient({ + addresses: formatAddresses(activeCluster.getAddresses()), + }); + }, 40000); + + const formatAddresses = ( + addresses: AddressEntry[], + ): { host: string; port: number }[] => + addresses.map(([host, port]) => ({ host, port })); + + afterEach(async () => { + if (managementClient) { + try { + await managementClient.customCommand(["AUTH", "new_password"]); + await managementClient.configSet({ requirepass: "" }); + } catch { + // Ignore errors + } + + await managementClient.flushall(); + + try { + await client.updateConnectionPassword(""); + } catch { + // Ignore errors + } + } + + if (cmdCluster) { + await flushAndCloseClient(false, cmdCluster.getAddresses()); + } + + if (cmeCluster) { + await flushAndCloseClient(true, cmeCluster.getAddresses()); + } + }); + + afterAll(async () => { + await cmdCluster?.close(); + await cmeCluster?.close(); + managementClient?.close(); + }); + + const runTest = async ( + test: (client: BaseClient) => Promise, + protocol: ProtocolVersion, + configOverrides?: Partial, + ) => { + const isStandaloneMode = configOverrides?.addresses?.length === 1; + const activeCluster = isStandaloneMode ? cmdCluster : cmeCluster; + + if (!activeCluster) { + throw new Error( + `${isStandaloneMode ? "Standalone" : "Cluster"} mode not configured`, + ); + } + + const ClientClass = isStandaloneMode ? GlideClient : GlideClusterClient; + const addresses = formatAddresses(activeCluster.getAddresses()); + + client = await ClientClass.createClient({ + addresses, + protocol, + ...configOverrides, + }); + + try { + await test(client); + } finally { + client.close(); + } + }; + + describe.each([ProtocolVersion.RESP2, ProtocolVersion.RESP3])( + "update_connection_password_%p", + (protocol) => { + const NEW_PASSWORD = "new_password"; + const WRONG_PASSWORD = "wrong_password"; + + /** + * Test replacing connection password with immediate re-authentication using a non-valid password. + * Verifies that immediate re-authentication fails when the password is not valid. + */ + it("test_update_connection_password_auth_non_valid_pass", async () => { + await runTest(async (client: BaseClient) => { + await expect( + client.updateConnectionPassword(null, true), + ).rejects.toThrow(RequestError); + await expect( + client.updateConnectionPassword("", true), + ).rejects.toThrow(RequestError); + }, protocol); + }); + + /** + * Test replacing the connection password without immediate re-authentication. + * Verifies that: + * 1. The client can update its internal password + * 2. The client remains connected with current auth + * 3. The client can reconnect using the new password after server password change + * Currently, this test is only supported for cluster mode, + * since standalone mode dont have multiple connections to manage, + * and the client will try to reconnect and will not listen to new tasks. + */ + it( + "test_update_connection_password", + async () => { + await runTest(async (client: BaseClient) => { + if (client instanceof GlideClient) { + return; + } + + // Update password without re-authentication + const result = await client.updateConnectionPassword( + NEW_PASSWORD, + false, + ); + expect(result).toEqual("OK"); + + // Verify client still works with old auth + await client.set("test_key", "test_value"); + const value = await client.get("test_key"); + expect(value).toEqual("test_value"); + + // Update server password + await client.configSet({ requirepass: NEW_PASSWORD }); + + // Kill all other clients to force reconnection + await managementClient.customCommand([ + "CLIENT", + "KILL", + "TYPE", + "normal", + ]); + + // Verify client auto-reconnects with new password + await client.set("test_key2", "test_value2"); + const value2 = await client.get("test_key2"); + expect(value2).toEqual("test_value2"); + }, protocol); + }, + TIMEOUT, + ); + + /** + * Test that immediate re-authentication fails when no server password is set. + */ + it("test_update_connection_password_no_server_auth", async () => { + await runTest(async (client: BaseClient) => { + try { + await expect( + client.updateConnectionPassword(NEW_PASSWORD, true), + ).rejects.toThrow(RequestError); + } finally { + client?.close(); + } + }, protocol); + }); + + /** + * Test replacing connection password with a long password string. + */ + it("test_update_connection_password_long", async () => { + await runTest(async (client: BaseClient) => { + const longPassword = "p".repeat(1000); + expect( + await client.updateConnectionPassword( + longPassword, + false, + ), + ).toEqual("OK"); + await client.configSet({ + requirepass: "", + }); + }, protocol); + }); + + /** + * Test that re-authentication fails when using wrong password. + */ + it("test_replace_password_immediateAuth_wrong_password", async () => { + await runTest(async (client: BaseClient) => { + await client.configSet({ + requirepass: NEW_PASSWORD, + }); + await expect( + client.updateConnectionPassword(WRONG_PASSWORD, true), + ).rejects.toThrow(RequestError); + await expect( + client.updateConnectionPassword(NEW_PASSWORD, true), + ).resolves.toBe("OK"); + }, protocol); + }); + + /** + * Test replacing connection password with immediate re-authentication. + */ + it( + "test_update_connection_password_with_immediateAuth", + async () => { + await runTest(async (client: BaseClient) => { + // Set server password + await client.configSet({ requirepass: NEW_PASSWORD }); + + // Update client password with re-auth + expect( + await client.updateConnectionPassword( + NEW_PASSWORD, + true, + ), + ).toEqual("OK"); + + // Verify client works with new auth + await client.set("test_key", "test_value"); + const value = await client.get("test_key"); + expect(value).toEqual("test_value"); + }, protocol); + }, + TIMEOUT, + ); + + /** + * Test changing server password when connection is lost before password update. + * Verifies that the client will not be able to reach the connection under the abstraction and return an error. + * + * **Note: This test is only supported for standalone mode, bellow explanation why* + * + * Some explanation for the curious mind: + * Our library is abstracting a connection or connections, with a lot of mechanism around it, making it behave like what we call a "client". + * When using standalone mode, the client is a single connection, so on disconnection the first thing it planned to do is to reconnect. + * Theres no reason to get other commands and to take care of them since to serve commands we need to be connected. + * Hence, the client will try to reconnect and will not listen try to take care of new tasks, but will let them wait in line, + * so the update connection password will not be able to reach the connection and will return an error. + * For future versions, standalone will be considered as a different animal then it is now, since standalone is not necessarily one node. + * It can be replicated and have a lot of nodes, and to be what we like to call "one shard cluster". + * So, in the future, we will have many existing connection and request can be managed also when one connection is locked. + * + */ + it("test_update_connection_password_connection_lost_before_password_update", async () => { + await runTest(async (client: BaseClient) => { + if (client instanceof GlideClusterClient) { + return; + } + + // Set a key to ensure connection is established + await client.set("test_key", "test_value"); + // Update server password + await client.configSet({ requirepass: NEW_PASSWORD }); + // Kill client connections + await managementClient.customCommand([ + "CLIENT", + "KILL", + "TYPE", + "normal", + ]); + // Try updating client password without immediate re-auth and with, both should fail + await expect( + client.updateConnectionPassword(NEW_PASSWORD, false), + ).rejects.toThrow(RequestError); + await expect( + client.updateConnectionPassword(NEW_PASSWORD, true), + ).rejects.toThrow(RequestError); + }, protocol); + }); + }, + ); +}); diff --git a/node/tests/SharedTests.ts b/node/tests/SharedTests.ts index 1a6ddaa5cf..6cada7b66f 100644 --- a/node/tests/SharedTests.ts +++ b/node/tests/SharedTests.ts @@ -12240,161 +12240,6 @@ export function runBaseTests(config: { }, config.timeout, ); - - describe.each([ProtocolVersion.RESP2, ProtocolVersion.RESP3])( - "update_connection_password_%p", - (protocol) => { - const NEW_PASSWORD = "new_password"; - const WRONG_PASSWORD = "wrong_password"; - /** - * Test replacing the connection password without immediate re-authentication. - * Verifies that: - * 1. The client can update its internal password - * 2. The client remains connected with current auth - * 3. The client can reconnect using the new password after server password change - * Currently, this test is only supported for cluster mode, - * since standalone mode dont have retry mechanism. - */ - it("test_update_connection_password", async () => { - await runTest(async (client: BaseClient) => { - try { - if (client instanceof GlideClient) { - return; - } - - const result = await client.updateConnectionPassword( - NEW_PASSWORD, - false, - ); - expect(result).toEqual("OK"); - - await client.set("test_key", "test_value"); - const value = await client.get("test_key"); - expect(value).toEqual("test_value"); - await expect( - client.configSet({ - requirepass: NEW_PASSWORD, - }), - ).resolves.toBe("OK"); - await client.customCommand([ - "CLIENT", - "KILL", - "TYPE", - "normal", - "skipme", - "no", - ]); - await client.set("test_key2", "test_value2"); - const value2 = await client.get("test_key2"); - expect(value2).toEqual("test_value2"); - await client.configSet({ - requirepass: "", - }); - } finally { - client?.close(); - } - }, protocol); - }); - - /** - * Test that immediate re-authentication fails when no server password is set. - * This verifies proper error handling when trying to re-authenticate with a - * password when the server has no password set. - */ - it("test_update_connection_password_no_server_auth", async () => { - await runTest(async (client: BaseClient) => { - try { - await expect( - client.updateConnectionPassword(NEW_PASSWORD, true), - ).rejects.toThrow(RequestError); - } finally { - client?.close(); - } - }, protocol); - }); - - /** - * Test replacing connection password with a long password string. - * Verifies that the client can handle long passwords (1000 characters). - */ - it("test_update_connection_password_long", async () => { - await runTest(async (client: BaseClient) => { - try { - const longPassword = "p".repeat(1000); - await expect( - client.updateConnectionPassword( - longPassword, - false, - ), - ).resolves.toBe("OK"); - await client.configSet({ - requirepass: "", - }); - } finally { - client?.close(); - } - }, protocol); - }); - - /** - * Test that re-authentication fails when using wrong password. - * Verifies proper error handling when immediate re-authentication is attempted - * with a password that doesn't match the server's password. - */ - it("test_replace_password_reauth_wrong_password", async () => { - await runTest(async (client: BaseClient) => { - try { - await client.configSet({ - requirepass: NEW_PASSWORD, - }); - - await expect( - client.updateConnectionPassword( - WRONG_PASSWORD, - true, - ), - ).rejects.toThrow(RequestError); - await client.updateConnectionPassword( - NEW_PASSWORD, - true, - ); - await client.configSet({ - requirepass: "", - }); - } finally { - client?.close(); - } - }, protocol); - }); - /** - * Test replacing connection password with immediate re-authentication. - * Verifies that: - * 1. The client can update its password and re-authenticate immediately - * 2. The client remains operational after re-authentication - */ - it("test_update_connection_password_with_reauth", async () => { - await runTest(async (client: BaseClient) => { - try { - await client.configSet({ - requirepass: NEW_PASSWORD, - }); - - await expect( - client.updateConnectionPassword(NEW_PASSWORD, true), - ).resolves.toBe("OK"); - await client.set("test_key", "test_value"); - const value = await client.get("test_key"); - expect(value).toEqual("test_value"); - await client.configSet({ - requirepass: "", - }); - } finally { - client?.close(); - } - }, protocol); - }); - }, - ); } export function runCommonTests(config: { diff --git a/python/python/glide/async_commands/core.py b/python/python/glide/async_commands/core.py index 4c29fbc3c9..6ebc8d2ab6 100644 --- a/python/python/glide/async_commands/core.py +++ b/python/python/glide/async_commands/core.py @@ -393,11 +393,11 @@ async def _cluster_scan( ) -> TResult: ... async def _update_connection_password( - self, password: Optional[str], re_auth: bool + self, password: Optional[str], immediate_auth: bool ) -> TResult: ... async def update_connection_password( - self, password: Optional[str], re_auth: bool + self, password: Optional[str], immediate_auth=False ) -> TOK: """ Update the current connection password with a new password. @@ -412,20 +412,23 @@ async def update_connection_password( handle reconnection seamlessly, preventing the loss of in-flight commands. Args: - password (Optional[str]): The new password to use for the connection, + password (`Optional[str]`): The new password to use for the connection, if `None` the password will be removed. - re_auth (bool): - - `True`: The client will re-authenticate immediately with the new password. - - `False`: The new password will be used for the next connection attempt. + immediate_auth (`bool`): + - `True`: The client will authenticate immediately with the new password against all connections, Using `AUTH` command. + If password supplied is an empty string, auth will not be performed and warning will be returned. + The default is `False`. Returns: - TOK: A simple OK response. + TOK: A simple OK response. If `immediate_auth=True` returns OK if the reauthenticate succeed. Example: - >>> await client.update_connection_password("new_password", re_auth=True) + >>> await client.update_connection_password("new_password", immediate_auth=True) 'OK' """ - return cast(TOK, await self._update_connection_password(password, re_auth)) + return cast( + TOK, await self._update_connection_password(password, immediate_auth) + ) async def set( self, diff --git a/python/python/glide/glide_client.py b/python/python/glide/glide_client.py index 1c1dc07ee7..ea648c49ba 100644 --- a/python/python/glide/glide_client.py +++ b/python/python/glide/glide_client.py @@ -525,7 +525,7 @@ async def _reader_loop(self) -> None: read_bytes, read_bytes_view, offset, Response ) except PartialMessageException: - # Recieved only partial response, break the inner loop + # Received only partial response, break the inner loop remaining_read_bytes = read_bytes[offset:] break response = cast(Response, response) @@ -538,12 +538,13 @@ async def get_statistics(self) -> dict: return get_statistics() async def _update_connection_password( - self, password: Optional[str], re_auth: bool + self, password: Optional[str], immediate_auth: bool ) -> TResult: request = CommandRequest() request.callback_idx = self._get_callback_index() - request.update_connection_password.password = password - request.update_connection_password.re_auth = re_auth + if password is not None: + request.update_connection_password.password = password + request.update_connection_password.immediate_auth = immediate_auth response = await self._write_request_await_response(request) # Update the client binding side password if managed to change core configuration password if response is OK: diff --git a/python/python/tests/conftest.py b/python/python/tests/conftest.py index 526308743a..437fbd8fbb 100644 --- a/python/python/tests/conftest.py +++ b/python/python/tests/conftest.py @@ -11,9 +11,11 @@ ProtocolVersion, ServerCredentials, ) +from glide.exceptions import ClosingError, RequestError from glide.glide_client import GlideClient, GlideClusterClient, TGlideClient from glide.logger import Level as logLevel from glide.logger import Logger +from glide.routes import AllNodes from tests.utils.cluster import ValkeyCluster from tests.utils.utils import check_if_server_version_lt @@ -204,9 +206,11 @@ def pytest_collection_modifyitems(config, items): ) -@pytest.fixture() +@pytest.fixture(scope="function") async def glide_client( - request, cluster_mode: bool, protocol: ProtocolVersion + request, + cluster_mode: bool, + protocol: ProtocolVersion, ) -> AsyncGenerator[TGlideClient, None]: "Get async socket client for tests" client = await create_client(request, cluster_mode, protocol=protocol) @@ -215,6 +219,19 @@ async def glide_client( await client.close() +@pytest.fixture(scope="function") +async def management_client( + request, + cluster_mode: bool, + protocol: ProtocolVersion, +) -> AsyncGenerator[TGlideClient, None]: + "Get async socket client for tests, used to manage the state when tests are on the client ability to connect" + client = await create_client(request, cluster_mode, protocol=protocol) + yield client + await test_teardown(request, cluster_mode, protocol) + await client.close() + + async def create_client( request, cluster_mode: bool, @@ -268,17 +285,81 @@ async def create_client( return await GlideClient.create(config) +NEW_PASSWORD = "new_secure_password" +WRONG_PASSWORD = "wrong_password" + + +async def auth_client(client: TGlideClient, password): + """ + Authenticates the given TGlideClient server connected. + """ + if isinstance(client, GlideClient): + await client.custom_command(["AUTH", password]) + elif isinstance(client, GlideClusterClient): + await client.custom_command(["AUTH", password], route=AllNodes()) + + +async def config_set_new_password(client: TGlideClient, password): + """ + Sets a new password for the given TGlideClient server connected. + This function updates the server to require a new password. + """ + if isinstance(client, GlideClient): + await client.config_set({"requirepass": password}) + elif isinstance(client, GlideClusterClient): + await client.config_set({"requirepass": password}, route=AllNodes()) + + +async def kill_connections(client: TGlideClient): + """ + Kills all connections to the given TGlideClient server connected. + """ + if isinstance(client, GlideClient): + await client.custom_command(["CLIENT", "KILL", "TYPE", "normal"]) + elif isinstance(client, GlideClusterClient): + await client.custom_command( + ["CLIENT", "KILL", "TYPE", "normal"], route=AllNodes() + ) + + async def test_teardown(request, cluster_mode: bool, protocol: ProtocolVersion): """ Perform teardown tasks such as flushing all data from the cluster. - We create a new client here because some tests load lots of data to the cluster, - which might cause the client to time out during flushing. Therefore, we create - a client with a custom timeout to ensure the operation completes successfully. + If authentication is required, attempt to connect with the known password, + reset it back to empty, and proceed with teardown. """ - client = await create_client(request, cluster_mode, protocol=protocol, timeout=2000) - await client.custom_command(["FLUSHALL"]) - await client.close() + credentials = None + try: + # Try connecting without credentials + client = await create_client( + request, cluster_mode, protocol=protocol, timeout=2000 + ) + await client.custom_command(["FLUSHALL"]) + await client.close() + except ClosingError as e: + # Check if the error is due to authentication + if "NOAUTH" in str(e): + # Use the known password to authenticate + credentials = ServerCredentials(password=NEW_PASSWORD) + client = await create_client( + request, + cluster_mode, + protocol=protocol, + timeout=2000, + credentials=credentials, + ) + try: + await auth_client(client, NEW_PASSWORD) + # Reset the server password back to empty + await config_set_new_password(client, "") + await client.update_connection_password(None) + # Perform the teardown + await client.custom_command(["FLUSHALL"]) + finally: + await client.close() + else: + raise e @pytest.fixture(autouse=True) diff --git a/python/python/tests/test_auth.py b/python/python/tests/test_auth.py index 694c8c345b..7e3fc67851 100644 --- a/python/python/tests/test_auth.py +++ b/python/python/tests/test_auth.py @@ -1,84 +1,54 @@ +# Copyright Valkey GLIDE Project Contributors - SPDX Identifier: Apache-2.0 + +import asyncio + import pytest from glide.config import ProtocolVersion from glide.constants import OK from glide.exceptions import RequestError from glide.glide_client import GlideClient, GlideClusterClient, TGlideClient -from glide.routes import AllNodes - -NEW_PASSWORD = "new_secure_password" -WRONG_PASSWORD = "wrong_password" - - -async def auth_client(client: TGlideClient, password): - """ - Authenticates the given TGlideClient server connected. - """ - if isinstance(client, GlideClient): - await client.custom_command(["AUTH", password]) - if isinstance(client, GlideClusterClient): - await client.custom_command(["AUTH", password], route=AllNodes()) - - -async def config_set_new_password(client: TGlideClient, password): - """ - Sets a new password for the given TGlideClient server connected. - This function updates the server to require a new password. - """ - if isinstance(client, GlideClient): - await client.config_set({"requirepass": password}) - if isinstance(client, GlideClusterClient): - await client.config_set({"requirepass": password}, route=AllNodes()) - - -async def kill_connections(client: TGlideClient): - """ - Kills all connections to the given TGlideClient server connected. - """ - if isinstance(client, GlideClient): - await client.custom_command( - ["CLIENT", "KILL", "TYPE", "normal", "skipme", "no"] - ) - if isinstance(client, GlideClusterClient): - await client.custom_command( - ["CLIENT", "KILL", "TYPE", "normal", "skipme", "no"], route=AllNodes() - ) +from tests.conftest import ( + NEW_PASSWORD, + WRONG_PASSWORD, + auth_client, + config_set_new_password, + kill_connections, +) @pytest.mark.asyncio class TestAuthCommands: """Test cases for password authentication and management""" - @pytest.fixture(autouse=True) - async def setup(self, glide_client: TGlideClient): + @pytest.fixture(autouse=True, scope="function") + async def cleanup(self, request, management_client: TGlideClient): """ - Teardown the test environment, make sure that theres no password set on the server side + Ensure password is reset after each test, regardless of test outcome. + This fixture runs after each test. """ - try: - await auth_client(glide_client, NEW_PASSWORD) - await config_set_new_password(glide_client, "") - except RequestError: - pass yield try: - await auth_client(glide_client, NEW_PASSWORD) - await config_set_new_password(glide_client, "") + await auth_client(management_client, NEW_PASSWORD) + await config_set_new_password(management_client, "") + await management_client.update_connection_password(None) except RequestError: pass @pytest.mark.parametrize("cluster_mode", [True]) @pytest.mark.parametrize("protocol", [ProtocolVersion.RESP2, ProtocolVersion.RESP3]) - async def test_update_connection_password(self, glide_client: TGlideClient): + async def test_update_connection_password( + self, glide_client: TGlideClient, management_client: TGlideClient + ): """ Test replacing the connection password without immediate re-authentication. Verifies that: 1. The client can update its internal password 2. The client remains connected with current auth 3. The client can reconnect using the new password after server password change - Currently, this test is only supported for cluster mode, - since standalone mode dont have retry mechanism. + This test is only for cluster mode, as standalone mode does not have a connection available handler """ result = await glide_client.update_connection_password( - NEW_PASSWORD, re_auth=False + NEW_PASSWORD, immediate_auth=False ) assert result == OK # Verify that the client is still authenticated @@ -86,15 +56,46 @@ async def test_update_connection_password(self, glide_client: TGlideClient): value = await glide_client.get("test_key") assert value == b"test_value" await config_set_new_password(glide_client, NEW_PASSWORD) - await kill_connections(glide_client) - # Verify that the client is able to reconnect with the new password + await kill_connections(management_client) + # Add a short delay to allow the server to apply the new password + # without this delay, command may or may not time out while the client reconnect + # ending up with a flaky test + await asyncio.sleep(1) + # Verify that the client is able to reconnect with the new password, value = await glide_client.get("test_key") assert value == b"test_value" + await glide_client.update_connection_password(None) + await kill_connections(management_client) + # Verify that the client is able to immediateAuth with the new password after client is killed + result = await glide_client.update_connection_password( + NEW_PASSWORD, immediate_auth=True + ) + assert result == OK + # Verify that the client is still authenticated + assert await glide_client.set("test_key", "test_value") == OK + + @pytest.mark.parametrize("cluster_mode", [False]) + @pytest.mark.parametrize("protocol", [ProtocolVersion.RESP2, ProtocolVersion.RESP3]) + async def test_update_connection_password_connection_lost_before_password_update( + self, glide_client: TGlideClient, management_client: TGlideClient + ): + """ + Test changing server password when connection is lost before password update. + Verifies that the client will not be able to reach the inner core and return an error. + """ + await glide_client.set("test_key", "test_value") + await config_set_new_password(glide_client, NEW_PASSWORD) + await kill_connections(management_client) + await asyncio.sleep(1) + with pytest.raises(RequestError): + await glide_client.update_connection_password( + NEW_PASSWORD, immediate_auth=False + ) @pytest.mark.parametrize("cluster_mode", [True, False]) @pytest.mark.parametrize("protocol", [ProtocolVersion.RESP2, ProtocolVersion.RESP3]) async def test_update_connection_password_no_server_auth( - self, glide_client: TGlideClient + self, glide_client: TGlideClient, management_client: TGlideClient ): """ Test that immediate re-authentication fails when no server password is set. @@ -102,25 +103,29 @@ async def test_update_connection_password_no_server_auth( password when the server has no password set. """ with pytest.raises(RequestError): - await glide_client.update_connection_password(WRONG_PASSWORD, re_auth=True) + await glide_client.update_connection_password( + WRONG_PASSWORD, immediate_auth=True + ) @pytest.mark.parametrize("cluster_mode", [True, False]) @pytest.mark.parametrize("protocol", [ProtocolVersion.RESP2, ProtocolVersion.RESP3]) - async def test_update_connection_password_long(self, glide_client: TGlideClient): + async def test_update_connection_password_long( + self, glide_client: TGlideClient, management_client: TGlideClient + ): """ Test replacing connection password with a long password string. Verifies that the client can handle long passwords (1000 characters). """ long_password = "p" * 1000 result = await glide_client.update_connection_password( - long_password, re_auth=False + long_password, immediate_auth=False ) assert result == OK @pytest.mark.parametrize("cluster_mode", [True, False]) @pytest.mark.parametrize("protocol", [ProtocolVersion.RESP2, ProtocolVersion.RESP3]) - async def test_replace_password_reauth_wrong_password( - self, glide_client: TGlideClient + async def test_replace_password_immediate_auth_wrong_password( + self, glide_client: TGlideClient, management_client: TGlideClient ): """ Test that re-authentication fails when using wrong password. @@ -129,12 +134,14 @@ async def test_replace_password_reauth_wrong_password( """ await config_set_new_password(glide_client, NEW_PASSWORD) with pytest.raises(RequestError): - await glide_client.update_connection_password(WRONG_PASSWORD, re_auth=True) + await glide_client.update_connection_password( + WRONG_PASSWORD, immediate_auth=True + ) @pytest.mark.parametrize("cluster_mode", [True, False]) @pytest.mark.parametrize("protocol", [ProtocolVersion.RESP2, ProtocolVersion.RESP3]) - async def test_update_connection_password_with_reauth( - self, glide_client: TGlideClient + async def test_update_connection_password_with_immediate_auth( + self, glide_client: TGlideClient, management_client: TGlideClient ): """ Test replacing connection password with immediate re-authentication. @@ -144,10 +151,24 @@ async def test_update_connection_password_with_reauth( """ await config_set_new_password(glide_client, NEW_PASSWORD) result = await glide_client.update_connection_password( - NEW_PASSWORD, re_auth=True + NEW_PASSWORD, immediate_auth=True ) assert result == OK # Verify that the client is still authenticated assert await glide_client.set("test_key", "test_value") == OK value = await glide_client.get("test_key") assert value == b"test_value" + + @pytest.mark.parametrize("cluster_mode", [True, False]) + @pytest.mark.parametrize("protocol", [ProtocolVersion.RESP2, ProtocolVersion.RESP3]) + async def test_update_connection_password_auth_non_valid_pass( + self, glide_client: TGlideClient, management_client: TGlideClient + ): + """ + Test replacing connection password with immediate re-authentication using a non-valid password. + Verifies that immediate re-authentication fails when the password is not valid. + """ + with pytest.raises(RequestError): + await glide_client.update_connection_password(None, immediate_auth=True) + with pytest.raises(RequestError): + await glide_client.update_connection_password("", immediate_auth=True) From 9d1c26c8798f476d6abb6a7e089ba5d917684af6 Mon Sep 17 00:00:00 2001 From: ikolomi Date: Tue, 19 Nov 2024 16:54:04 +0200 Subject: [PATCH 07/11] Run prettier on github workflow files Signed-off-by: ikolomi --- .github/workflows/ort-sweeper.yml | 102 +++--- .github/workflows/ort.yml | 505 +++++++++++++++--------------- 2 files changed, 302 insertions(+), 305 deletions(-) diff --git a/.github/workflows/ort-sweeper.yml b/.github/workflows/ort-sweeper.yml index 91889c2239..25482f9737 100644 --- a/.github/workflows/ort-sweeper.yml +++ b/.github/workflows/ort-sweeper.yml @@ -2,56 +2,56 @@ name: ORT - Trigger periodic checks for relevant branches on: schedule: - - cron: '0 0 * * *' # Runs daily at 00:00 UTC + - cron: "0 0 * * *" # Runs daily at 00:00 UTC jobs: - trigger-ort-check: - runs-on: ubuntu-latest - - steps: - - name: Checkout repository - uses: actions/checkout@v4 - - - name: Fetch relevant branches - id: get-branches - run: | - # Get all branches matching 'release-*' and include 'main' - branches=$(git ls-remote --heads origin | awk -F'/' '/refs\/heads\/release-/ {print $NF}') - branches="main $branches" - echo "::set-output name=branches::$branches" - env: - GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} - - - name: Trigger ORT Check workflows - uses: actions/github-script@v6 - with: - script: | - const branches = "${{ steps.get-branches.outputs.branches }}".split(" "); - const workflowFile = "ort.yml"; - - const triggerWorkflow = async (branch) => { - try { - console.log(`Triggering workflow for branch: ${branch}`); - await github.rest.actions.createWorkflowDispatch({ - owner: context.repo.owner, - repo: context.repo.repo, - workflow_id: workflowFile, - ref: branch, // The branch where workflow_dispatch is triggered - inputs: { - branch_name: branch - } - }); - console.log(`Successfully triggered workflow for branch: ${branch}`); - } catch (error) { - core.setFailed(error.message); - } - }; - - // Fire all workflow dispatch requests concurrently - const promises = branches - .filter(branch => branch) // Skip empty branches - .map(branch => triggerWorkflow(branch)); - - await Promise.allSettled(promises); - env: - GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} + trigger-ort-check: + runs-on: ubuntu-latest + + steps: + - name: Checkout repository + uses: actions/checkout@v4 + + - name: Fetch relevant branches + id: get-branches + run: | + # Get all branches matching 'release-*' and include 'main' + branches=$(git ls-remote --heads origin | awk -F'/' '/refs\/heads\/release-/ {print $NF}') + branches="main $branches" + echo "::set-output name=branches::$branches" + env: + GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} + + - name: Trigger ORT Check workflows + uses: actions/github-script@v6 + with: + script: | + const branches = "${{ steps.get-branches.outputs.branches }}".split(" "); + const workflowFile = "ort.yml"; + + const triggerWorkflow = async (branch) => { + try { + console.log(`Triggering workflow for branch: ${branch}`); + await github.rest.actions.createWorkflowDispatch({ + owner: context.repo.owner, + repo: context.repo.repo, + workflow_id: workflowFile, + ref: branch, // The branch where workflow_dispatch is triggered + inputs: { + branch_name: branch + } + }); + console.log(`Successfully triggered workflow for branch: ${branch}`); + } catch (error) { + core.setFailed(error.message); + } + }; + + // Fire all workflow dispatch requests concurrently + const promises = branches + .filter(branch => branch) // Skip empty branches + .map(branch => triggerWorkflow(branch)); + + await Promise.allSettled(promises); + env: + GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} diff --git a/.github/workflows/ort.yml b/.github/workflows/ort.yml index a01659e4f5..2134f1f7a4 100644 --- a/.github/workflows/ort.yml +++ b/.github/workflows/ort.yml @@ -6,9 +6,9 @@ on: workflow_dispatch: inputs: branch_name: - description: 'The branch to run against the ORT tool' + description: "The branch to run against the ORT tool" required: true - default: 'main' + default: "main" jobs: run-ort: @@ -18,11 +18,11 @@ jobs: # 1. For workflow_dispatch, always allow # 2. For pull_request, run if branch is not autogenerated ort-diff-for- branches if: > - github.event_name == 'workflow_dispatch' || - (github.event_name == 'pull_request' && !startsWith(github.head_ref, 'ort-diff-for-')) + github.event_name == 'workflow_dispatch' || + (github.event_name == 'pull_request' && !startsWith(github.head_ref, 'ort-diff-for-')) strategy: fail-fast: false - env: + env: PYTHON_ATTRIBUTIONS: "python/THIRD_PARTY_LICENSES_PYTHON" NODE_ATTRIBUTIONS: "node/THIRD_PARTY_LICENSES_NODE" RUST_ATTRIBUTIONS: "glide-core/THIRD_PARTY_LICENSES_RUST" @@ -32,255 +32,252 @@ jobs: HEAD_REF: ${{ github.head_ref }} steps: - - name: Setup target branch and commit - run: | - if [ "$EVENT_NAME" == 'workflow_dispatch' ]; then - echo "TARGET_BRANCH=$INPUT_TARGET_BRANCH" >> $GITHUB_ENV - elif [ "$EVENT_NAME" == 'pull_request' ]; then - echo "TARGET_BRANCH=$HEAD_REF" >> $GITHUB_ENV - fi - - - name: Checkout target branch - uses: actions/checkout@v4 - with: - ref: ${{ env.TARGET_BRANCH }} - repository: ${{ github.event.pull_request.head.repo.full_name }} - token: ${{ secrets.GITHUB_TOKEN }} - fetch-depth: 0 # Fetch all history for all branches and tags - - - - name: Setup target commit - run: | - echo "TARGET_COMMIT=`git rev-parse HEAD`" >> $GITHUB_ENV - - - name: Set up JDK 11 for the ORT package - uses: actions/setup-java@v4 - with: - distribution: "temurin" - java-version: 11 - - - name: Cache ORT and Gradle packages - uses: actions/cache@v4 - id: cache-ort - with: - path: | - ./ort - ~/.gradle/caches - ~/.gradle/wrapper - key: ${{ runner.os }}-ort - - - name: Checkout ORT Repository - if: steps.cache-ort.outputs.cache-hit != 'true' - uses: actions/checkout@v4 - with: - repository: "oss-review-toolkit/ort" - path: "./ort" - ref: "26.0.0" - submodules: recursive - - - name: Install Rust toolchain - uses: dtolnay/rust-toolchain@1.78 - - - name: Build and install ORT - if: steps.cache-ort.outputs.cache-hit != 'true' - working-directory: ./ort/ - run: | - export JAVA_OPTS="$JAVA_OPTS -Xmx8g" - ./gradlew installDist - - - name: Create ORT config file - run: | - mkdir -p ~/.ort/config - cat << EOF > ~/.ort/config/config.yml - ort: - analyzer: - allowDynamicVersions: true - enabledPackageManagers: [Cargo, NPM, PIP, GradleInspector] - EOF - cat ~/.ort/config/config.yml - - ### NodeJS ### - - name: Set up Node.js 16.x - uses: actions/setup-node@v4 - with: - node-version: 16.x - - - name: Create package.json file for the Node wrapper - uses: ./.github/workflows/node-create-package-file - with: - release_version: ${{ env.TARGET_COMMIT }} - os: "ubuntu-latest" - - - name: Fix Node base NPM package.json file for ORT - working-directory: ./node/npm/glide - run: | - # Remove the glide-rs dependency to avoid duplication - sed -i '/ "glide-rs":/d' ../../package.json - export pkg_name=valkey-glide-base - export package_version="${{ env.TARGET_COMMIT }}" - export scope=`if [ "$NPM_SCOPE" != '' ]; then echo "$NPM_SCOPE/"; fi` - mv package.json package.json.tmpl - envsubst < package.json.tmpl > "package.json" - cat package.json - - - name: Run ORT tools for Node - uses: ./.github/workflows/run-ort-tools - with: - folder_path: "${{ github.workspace }}/node" - - ### Python ### - - name: Set up Python 3.10 - uses: actions/setup-python@v5 - with: - python-version: "3.10" - - - name: Install python-inspector - working-directory: ./python - run: | - python -m pip install --upgrade pip - pip install git+https://github.com/nexB/python-inspector - - - name: Run ORT tools for Python - uses: ./.github/workflows/run-ort-tools - with: - folder_path: "${{ github.workspace }}/python" - - ### Rust glide-core ### - - name: Run ORT tools for glide-core - uses: ./.github/workflows/run-ort-tools - with: - folder_path: "${{ github.workspace }}/glide-core" - - ### Java ### - - name: Set up JDK 11 - uses: actions/setup-java@v4 - with: - distribution: "temurin" - java-version: 11 - - - name: Run ORT tools for Java - uses: ./.github/workflows/run-ort-tools - with: - folder_path: "${{ github.workspace }}/java" - - - ### Get licenses ### - - name: Retrieve licenses list - working-directory: ./utils - run: | - { - echo 'LICENSES_LIST<> "$GITHUB_ENV" - - ### Upload licenses ### - - name: Get current date - id: date - run: | - CURR_DATE=$(date +'%Y-%m-%d-%H') - echo "date=${CURR_DATE}" >> $GITHUB_OUTPUT - - - name: Upload the final package list - continue-on-error: true - uses: actions/upload-artifact@v4 - with: - name: final-package-list-${{ steps.date.outputs.date }} - path: | - utils/final_package_list.txt - retention-days: 30 - - - name: Upload the skipped package list - continue-on-error: true - uses: actions/upload-artifact@v4 - with: - name: skipped-package-list-${{ steps.date.outputs.date }} - path: | - utils/skipped_package_list.txt - retention-days: 30 - - - name: Upload the unknown/unapproved package list - continue-on-error: true - uses: actions/upload-artifact@v4 - with: - name: unapproved-package-list-${{ steps.date.outputs.date }} - path: | - utils/unapproved_package_list.txt - retention-days: 30 - - ### TODO: Fail if there are unapproved packages ### - - ### Check for attributions diff ### - - name: Check for diff - run: | - cp python/ort_results/NOTICE_DEFAULT $PYTHON_ATTRIBUTIONS - cp node/ort_results/NOTICE_DEFAULT $NODE_ATTRIBUTIONS - cp glide-core/ort_results/NOTICE_DEFAULT $RUST_ATTRIBUTIONS - cp java/ort_results/NOTICE_DEFAULT $JAVA_ATTRIBUTIONS - GIT_DIFF=`git diff $PYTHON_ATTRIBUTIONS $NODE_ATTRIBUTIONS $RUST_ATTRIBUTIONS $JAVA_ATTRIBUTIONS` - if [ -n "$GIT_DIFF" ]; then - echo "FOUND_DIFF=true" >> $GITHUB_ENV - else - echo "FOUND_DIFF=false" >> $GITHUB_ENV - fi - - ### Create PR, Note a potential race on the source branch ### - - name: Create pull request - if: ${{ env.FOUND_DIFF == 'true' && github.event_name != 'pull_request' }} - run: | - export ORT_DIFF_BRANCH_NAME="ort-diff-for-$TARGET_BRANCH" - echo "Creating pull request from branch $ORT_DIFF_BRANCH_NAME to branch $TARGET_BRANCH" - git config --global user.email "valkey-glide@lists.valkey.io" - git config --global user.name "ort-bot" - git checkout -b ${ORT_DIFF_BRANCH_NAME} - git add $PYTHON_ATTRIBUTIONS $NODE_ATTRIBUTIONS $RUST_ATTRIBUTIONS $JAVA_ATTRIBUTIONS - git commit -m "Updated attribution files" -s - git push --set-upstream origin ${ORT_DIFF_BRANCH_NAME} -f - - # Check if PR already exists - existing_pr=$(gh pr list --base ${TARGET_BRANCH} --head ${ORT_DIFF_BRANCH_NAME} --json number --jq '.[0].number') - - if [ -z "$existing_pr" ]; then - # Create a new PR if none exists - title="Updated attribution files for commit ${TARGET_COMMIT}" - gh pr create -B ${TARGET_BRANCH} -H ${ORT_DIFF_BRANCH_NAME} --title "${title}" --body "Created by Github action. ${{ env.LICENSES_LIST }}" - echo "Pull request created successfully." - else - # Update the existing PR - echo "Pull request #$existing_pr already exists. Updating branch." - gh pr edit $existing_pr --title "Updated attribution files for commit ${TARGET_COMMIT}" --body "Created by Github action. ${{ env.LICENSES_LIST }}" - echo "Pull request updated successfully." - fi - env: - GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} - INPUT_VERSION: ${{ github.event.inputs.version }} - - ### Warn of outdated attributions for PR ### - - name: Warn of outdated attributions due to the PR - if: ${{ env.FOUND_DIFF == 'true' && github.event_name == 'pull_request' }} - run: | - ATTRIBUTION_FILES=( - "${{ env.PYTHON_ATTRIBUTIONS }}" - "${{ env.NODE_ATTRIBUTIONS }}" - "${{ env.RUST_ATTRIBUTIONS }}" - "${{ env.JAVA_ATTRIBUTIONS }}" - ) - - MESSAGE="WARNING! The attribution files differ in this PR. Please ensure an updating PR is issued using a scheduled or manual run of this workflow!" - - # Echo the message to the console - echo "$MESSAGE" - - # Emit a general warning in the action log - echo "::warning::$MESSAGE" - - # Loop through the attribution files - for FILE in "${ATTRIBUTION_FILES[@]}"; do - if git diff --quiet "$FILE"; then - continue + - name: Setup target branch and commit + run: | + if [ "$EVENT_NAME" == 'workflow_dispatch' ]; then + echo "TARGET_BRANCH=$INPUT_TARGET_BRANCH" >> $GITHUB_ENV + elif [ "$EVENT_NAME" == 'pull_request' ]; then + echo "TARGET_BRANCH=$HEAD_REF" >> $GITHUB_ENV + fi + + - name: Checkout target branch + uses: actions/checkout@v4 + with: + ref: ${{ env.TARGET_BRANCH }} + repository: ${{ github.event.pull_request.head.repo.full_name }} + token: ${{ secrets.GITHUB_TOKEN }} + fetch-depth: 0 # Fetch all history for all branches and tags + + - name: Setup target commit + run: | + echo "TARGET_COMMIT=`git rev-parse HEAD`" >> $GITHUB_ENV + + - name: Set up JDK 11 for the ORT package + uses: actions/setup-java@v4 + with: + distribution: "temurin" + java-version: 11 + + - name: Cache ORT and Gradle packages + uses: actions/cache@v4 + id: cache-ort + with: + path: | + ./ort + ~/.gradle/caches + ~/.gradle/wrapper + key: ${{ runner.os }}-ort + + - name: Checkout ORT Repository + if: steps.cache-ort.outputs.cache-hit != 'true' + uses: actions/checkout@v4 + with: + repository: "oss-review-toolkit/ort" + path: "./ort" + ref: "26.0.0" + submodules: recursive + + - name: Install Rust toolchain + uses: dtolnay/rust-toolchain@1.78 + + - name: Build and install ORT + if: steps.cache-ort.outputs.cache-hit != 'true' + working-directory: ./ort/ + run: | + export JAVA_OPTS="$JAVA_OPTS -Xmx8g" + ./gradlew installDist + + - name: Create ORT config file + run: | + mkdir -p ~/.ort/config + cat << EOF > ~/.ort/config/config.yml + ort: + analyzer: + allowDynamicVersions: true + enabledPackageManagers: [Cargo, NPM, PIP, GradleInspector] + EOF + cat ~/.ort/config/config.yml + + ### NodeJS ### + - name: Set up Node.js 16.x + uses: actions/setup-node@v4 + with: + node-version: 16.x + + - name: Create package.json file for the Node wrapper + uses: ./.github/workflows/node-create-package-file + with: + release_version: ${{ env.TARGET_COMMIT }} + os: "ubuntu-latest" + + - name: Fix Node base NPM package.json file for ORT + working-directory: ./node/npm/glide + run: | + # Remove the glide-rs dependency to avoid duplication + sed -i '/ "glide-rs":/d' ../../package.json + export pkg_name=valkey-glide-base + export package_version="${{ env.TARGET_COMMIT }}" + export scope=`if [ "$NPM_SCOPE" != '' ]; then echo "$NPM_SCOPE/"; fi` + mv package.json package.json.tmpl + envsubst < package.json.tmpl > "package.json" + cat package.json + + - name: Run ORT tools for Node + uses: ./.github/workflows/run-ort-tools + with: + folder_path: "${{ github.workspace }}/node" + + ### Python ### + - name: Set up Python 3.10 + uses: actions/setup-python@v5 + with: + python-version: "3.10" + + - name: Install python-inspector + working-directory: ./python + run: | + python -m pip install --upgrade pip + pip install git+https://github.com/nexB/python-inspector + + - name: Run ORT tools for Python + uses: ./.github/workflows/run-ort-tools + with: + folder_path: "${{ github.workspace }}/python" + + ### Rust glide-core ### + - name: Run ORT tools for glide-core + uses: ./.github/workflows/run-ort-tools + with: + folder_path: "${{ github.workspace }}/glide-core" + + ### Java ### + - name: Set up JDK 11 + uses: actions/setup-java@v4 + with: + distribution: "temurin" + java-version: 11 + + - name: Run ORT tools for Java + uses: ./.github/workflows/run-ort-tools + with: + folder_path: "${{ github.workspace }}/java" + + ### Get licenses ### + - name: Retrieve licenses list + working-directory: ./utils + run: | + { + echo 'LICENSES_LIST<> "$GITHUB_ENV" + + ### Upload licenses ### + - name: Get current date + id: date + run: | + CURR_DATE=$(date +'%Y-%m-%d-%H') + echo "date=${CURR_DATE}" >> $GITHUB_OUTPUT + + - name: Upload the final package list + continue-on-error: true + uses: actions/upload-artifact@v4 + with: + name: final-package-list-${{ steps.date.outputs.date }} + path: | + utils/final_package_list.txt + retention-days: 30 + + - name: Upload the skipped package list + continue-on-error: true + uses: actions/upload-artifact@v4 + with: + name: skipped-package-list-${{ steps.date.outputs.date }} + path: | + utils/skipped_package_list.txt + retention-days: 30 + + - name: Upload the unknown/unapproved package list + continue-on-error: true + uses: actions/upload-artifact@v4 + with: + name: unapproved-package-list-${{ steps.date.outputs.date }} + path: | + utils/unapproved_package_list.txt + retention-days: 30 + + ### TODO: Fail if there are unapproved packages ### + + ### Check for attributions diff ### + - name: Check for diff + run: | + cp python/ort_results/NOTICE_DEFAULT $PYTHON_ATTRIBUTIONS + cp node/ort_results/NOTICE_DEFAULT $NODE_ATTRIBUTIONS + cp glide-core/ort_results/NOTICE_DEFAULT $RUST_ATTRIBUTIONS + cp java/ort_results/NOTICE_DEFAULT $JAVA_ATTRIBUTIONS + GIT_DIFF=`git diff $PYTHON_ATTRIBUTIONS $NODE_ATTRIBUTIONS $RUST_ATTRIBUTIONS $JAVA_ATTRIBUTIONS` + if [ -n "$GIT_DIFF" ]; then + echo "FOUND_DIFF=true" >> $GITHUB_ENV else - # Emit a warning associated with the changed file - echo "::warning file=$FILE::WARNING! The attribution file '$FILE' differs in this PR." + echo "FOUND_DIFF=false" >> $GITHUB_ENV fi - done + ### Create PR, Note a potential race on the source branch ### + - name: Create pull request + if: ${{ env.FOUND_DIFF == 'true' && github.event_name != 'pull_request' }} + run: | + export ORT_DIFF_BRANCH_NAME="ort-diff-for-$TARGET_BRANCH" + echo "Creating pull request from branch $ORT_DIFF_BRANCH_NAME to branch $TARGET_BRANCH" + git config --global user.email "valkey-glide@lists.valkey.io" + git config --global user.name "ort-bot" + git checkout -b ${ORT_DIFF_BRANCH_NAME} + git add $PYTHON_ATTRIBUTIONS $NODE_ATTRIBUTIONS $RUST_ATTRIBUTIONS $JAVA_ATTRIBUTIONS + git commit -m "Updated attribution files" -s + git push --set-upstream origin ${ORT_DIFF_BRANCH_NAME} -f + + # Check if PR already exists + existing_pr=$(gh pr list --base ${TARGET_BRANCH} --head ${ORT_DIFF_BRANCH_NAME} --json number --jq '.[0].number') + + if [ -z "$existing_pr" ]; then + # Create a new PR if none exists + title="Updated attribution files for commit ${TARGET_COMMIT}" + gh pr create -B ${TARGET_BRANCH} -H ${ORT_DIFF_BRANCH_NAME} --title "${title}" --body "Created by Github action. ${{ env.LICENSES_LIST }}" + echo "Pull request created successfully." + else + # Update the existing PR + echo "Pull request #$existing_pr already exists. Updating branch." + gh pr edit $existing_pr --title "Updated attribution files for commit ${TARGET_COMMIT}" --body "Created by Github action. ${{ env.LICENSES_LIST }}" + echo "Pull request updated successfully." + fi + env: + GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} + INPUT_VERSION: ${{ github.event.inputs.version }} + + ### Warn of outdated attributions for PR ### + - name: Warn of outdated attributions due to the PR + if: ${{ env.FOUND_DIFF == 'true' && github.event_name == 'pull_request' }} + run: | + ATTRIBUTION_FILES=( + "${{ env.PYTHON_ATTRIBUTIONS }}" + "${{ env.NODE_ATTRIBUTIONS }}" + "${{ env.RUST_ATTRIBUTIONS }}" + "${{ env.JAVA_ATTRIBUTIONS }}" + ) + + MESSAGE="WARNING! The attribution files differ in this PR. Please ensure an updating PR is issued using a scheduled or manual run of this workflow!" + + # Echo the message to the console + echo "$MESSAGE" + + # Emit a general warning in the action log + echo "::warning::$MESSAGE" + + # Loop through the attribution files + for FILE in "${ATTRIBUTION_FILES[@]}"; do + if git diff --quiet "$FILE"; then + continue + else + # Emit a warning associated with the changed file + echo "::warning file=$FILE::WARNING! The attribution file '$FILE' differs in this PR." + fi + done From 4d33f95c344bb07f4b6d4b6bf37ce5fab55be00f Mon Sep 17 00:00:00 2001 From: BoazBD <50696333+BoazBD@users.noreply.github.com> Date: Wed, 20 Nov 2024 10:52:56 +0200 Subject: [PATCH 08/11] Python: add AZ Affinity ReadFrom strategy Support (#2676) Python: add AZ Affinity readFrom strategy --------- Signed-off-by: BoazBD Signed-off-by: BoazBD <50696333+BoazBD@users.noreply.github.com> --- CHANGELOG.md | 1 + python/python/glide/config.py | 18 ++ python/python/tests/conftest.py | 31 +++ python/python/tests/test_config.py | 15 ++ .../python/tests/test_read_from_strategy.py | 228 ++++++++++++++++++ 5 files changed, 293 insertions(+) create mode 100644 python/python/tests/test_read_from_strategy.py diff --git a/CHANGELOG.md b/CHANGELOG.md index 228aa7bdf2..80f7c054d4 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ #### Changes +* Python: AZ Affinity - Python Wrapper Support ([#2676](https://github.com/valkey-io/valkey-glide/pull/2676)) * Python: Client API for retrieving internal statistics ([#2707](https://github.com/valkey-io/valkey-glide/pull/2707)) * Node, Python: Adding support for replacing connection configured password ([#2651](https://github.com/valkey-io/valkey-glide/pull/2651))([#2659](https://github.com/valkey-io/valkey-glide/pull/2659)) * Node: Add FT._ALIASLIST command([#2652](https://github.com/valkey-io/valkey-glide/pull/2652)) diff --git a/python/python/glide/config.py b/python/python/glide/config.py index db85202876..b33c037cbf 100644 --- a/python/python/glide/config.py +++ b/python/python/glide/config.py @@ -41,6 +41,11 @@ class ReadFrom(Enum): Spread the requests between all replicas in a round robin manner. If no replica is available, route the requests to the primary. """ + AZ_AFFINITY = ProtobufReadFrom.AZAffinity + """ + Spread the read requests between replicas in the same client's AZ (Aviliablity zone) in a round robin manner, + falling back to other replicas or the primary if needed + """ class ProtocolVersion(Enum): @@ -135,6 +140,7 @@ def __init__( client_name: Optional[str] = None, protocol: ProtocolVersion = ProtocolVersion.RESP3, inflight_requests_limit: Optional[int] = None, + client_az: Optional[str] = None, ): """ Represents the configuration settings for a Glide client. @@ -172,6 +178,12 @@ def __init__( self.client_name = client_name self.protocol = protocol self.inflight_requests_limit = inflight_requests_limit + self.client_az = client_az + + if read_from == ReadFrom.AZ_AFFINITY and not client_az: + raise ValueError( + "client_az mus t be set when read_from is set to AZ_AFFINITY" + ) def _create_a_protobuf_conn_request( self, cluster_mode: bool = False @@ -204,6 +216,8 @@ def _create_a_protobuf_conn_request( request.protocol = self.protocol.value if self.inflight_requests_limit: request.inflight_requests_limit = self.inflight_requests_limit + if self.client_az: + request.client_az = self.client_az return request @@ -293,6 +307,7 @@ def __init__( protocol: ProtocolVersion = ProtocolVersion.RESP3, pubsub_subscriptions: Optional[PubSubSubscriptions] = None, inflight_requests_limit: Optional[int] = None, + client_az: Optional[str] = None, ): super().__init__( addresses=addresses, @@ -303,6 +318,7 @@ def __init__( client_name=client_name, protocol=protocol, inflight_requests_limit=inflight_requests_limit, + client_az=client_az, ) self.reconnect_strategy = reconnect_strategy self.database_id = database_id @@ -442,6 +458,7 @@ def __init__( ] = PeriodicChecksStatus.ENABLED_DEFAULT_CONFIGS, pubsub_subscriptions: Optional[PubSubSubscriptions] = None, inflight_requests_limit: Optional[int] = None, + client_az: Optional[str] = None, ): super().__init__( addresses=addresses, @@ -452,6 +469,7 @@ def __init__( client_name=client_name, protocol=protocol, inflight_requests_limit=inflight_requests_limit, + client_az=client_az, ) self.periodic_checks = periodic_checks self.pubsub_subscriptions = pubsub_subscriptions diff --git a/python/python/tests/conftest.py b/python/python/tests/conftest.py index 437fbd8fbb..85bc58c4b1 100644 --- a/python/python/tests/conftest.py +++ b/python/python/tests/conftest.py @@ -9,6 +9,7 @@ GlideClusterClientConfiguration, NodeAddress, ProtocolVersion, + ReadFrom, ServerCredentials, ) from glide.exceptions import ClosingError, RequestError @@ -132,6 +133,7 @@ def create_clusters(tls, load_module, cluster_endpoints, standalone_endpoints): cluster_mode=True, load_module=load_module, addresses=cluster_endpoints, + replica_count=1, ) pytest.standalone_cluster = ValkeyCluster( tls=tls, @@ -248,6 +250,8 @@ async def create_client( GlideClientConfiguration.PubSubSubscriptions ] = None, inflight_requests_limit: Optional[int] = None, + read_from: ReadFrom = ReadFrom.PRIMARY, + client_az: Optional[str] = None, ) -> Union[GlideClient, GlideClusterClient]: # Create async socket client use_tls = request.config.getoption("--tls") @@ -265,6 +269,8 @@ async def create_client( request_timeout=timeout, pubsub_subscriptions=cluster_mode_pubsub, inflight_requests_limit=inflight_requests_limit, + read_from=read_from, + client_az=client_az, ) return await GlideClusterClient.create(cluster_config) else: @@ -281,6 +287,8 @@ async def create_client( request_timeout=timeout, pubsub_subscriptions=standalone_mode_pubsub, inflight_requests_limit=inflight_requests_limit, + read_from=read_from, + client_az=client_az, ) return await GlideClient.create(config) @@ -381,3 +389,26 @@ async def test_meow_meow(...): reason=f"This feature added in version {min_version}", allow_module_level=True, ) + + +@pytest.fixture(scope="module") +def multiple_replicas_cluster(request): + """ + Fixture to create a special cluster with 4 replicas for specific tests. + """ + tls = request.config.getoption("--tls") + load_module = request.config.getoption("--load-module") + cluster_endpoints = request.config.getoption("--cluster-endpoints") + + if not cluster_endpoints: + multiple_replica_cluster = ValkeyCluster( + tls=tls, + cluster_mode=True, + load_module=load_module, + addresses=cluster_endpoints, + replica_count=4, + ) + yield multiple_replica_cluster + multiple_replica_cluster.__del__() + else: + yield None diff --git a/python/python/tests/test_config.py b/python/python/tests/test_config.py index 93c280245f..3b22adb09c 100644 --- a/python/python/tests/test_config.py +++ b/python/python/tests/test_config.py @@ -52,3 +52,18 @@ def test_periodic_checks_interval_to_protobuf(): config.periodic_checks = PeriodicChecksManualInterval(30) request = config._create_a_protobuf_conn_request(cluster_mode=True) assert request.periodic_checks_manual_interval.duration_in_sec == 30 + + +def test_convert_config_with_azaffinity_to_protobuf(): + az = "us-east-1a" + config = BaseClientConfiguration( + [NodeAddress("127.0.0.1")], + use_tls=True, + read_from=ReadFrom.AZ_AFFINITY, + client_az=az, + ) + request = config._create_a_protobuf_conn_request() + assert isinstance(request, ConnectionRequest) + assert request.tls_mode is TlsMode.SecureTls + assert request.read_from == ProtobufReadFrom.AZAffinity + assert request.client_az == az diff --git a/python/python/tests/test_read_from_strategy.py b/python/python/tests/test_read_from_strategy.py new file mode 100644 index 0000000000..fc15481a07 --- /dev/null +++ b/python/python/tests/test_read_from_strategy.py @@ -0,0 +1,228 @@ +# Copyright Valkey GLIDE Project Contributors - SPDX Identifier: Apache-2.0 + +import re + +import pytest +from glide.async_commands.core import InfoSection +from glide.config import ProtocolVersion, ReadFrom +from glide.constants import OK +from glide.glide_client import GlideClusterClient +from glide.routes import AllNodes, SlotIdRoute, SlotType +from tests.conftest import create_client +from tests.utils.utils import get_first_result + + +@pytest.mark.asyncio +@pytest.mark.usefixtures("multiple_replicas_cluster") +class TestAZAffinity: + async def _get_num_replicas(self, client: GlideClusterClient) -> int: + info_replicas = get_first_result( + await client.info([InfoSection.REPLICATION]) + ).decode() + match = re.search(r"connected_slaves:(\d+)", info_replicas) + if match: + return int(match.group(1)) + else: + raise ValueError( + "Could not find the number of replicas in the INFO REPLICATION response" + ) + + @pytest.mark.skip_if_version_below("8.0.0") + @pytest.mark.parametrize("cluster_mode", [True]) + @pytest.mark.parametrize("protocol", [ProtocolVersion.RESP2, ProtocolVersion.RESP3]) + async def test_routing_by_slot_to_replica_with_az_affinity_strategy_to_all_replicas( + self, + request, + cluster_mode: bool, + protocol: ProtocolVersion, + multiple_replicas_cluster, + ): + """Test that the client with AZ affinity strategy routes in a round-robin manner to all replicas within the specified AZ""" + + az = "us-east-1a" + client_for_config_set = await create_client( + request, + cluster_mode, + addresses=multiple_replicas_cluster.nodes_addr, + protocol=protocol, + timeout=2000, + ) + await client_for_config_set.config_resetstat() == OK + await client_for_config_set.custom_command( + ["CONFIG", "SET", "availability-zone", az], AllNodes() + ) + await client_for_config_set.close() + + client_for_testing_az = await create_client( + request, + cluster_mode, + addresses=multiple_replicas_cluster.nodes_addr, + protocol=protocol, + read_from=ReadFrom.AZ_AFFINITY, + timeout=2000, + client_az=az, + ) + azs = await client_for_testing_az.custom_command( + ["CONFIG", "GET", "availability-zone"], AllNodes() + ) + + # Check that all replicas have the availability zone set to the az + assert all( + ( + node[1].decode() == az + if isinstance(node, list) + else node[b"availability-zone"].decode() == az + ) + for node in azs.values() + ) + + n_replicas = await self._get_num_replicas(client_for_testing_az) + GET_CALLS = 3 * n_replicas + get_cmdstat = f"cmdstat_get:calls={GET_CALLS // n_replicas}" + + for _ in range(GET_CALLS): + await client_for_testing_az.get("foo") + + info_result = await client_for_testing_az.info( + [InfoSection.COMMAND_STATS, InfoSection.SERVER], AllNodes() + ) + + # Check that all replicas have the same number of GET calls + matching_entries_count = sum( + 1 + for value in info_result.values() + if get_cmdstat in value.decode() and az in value.decode() + ) + assert matching_entries_count == n_replicas + + await client_for_testing_az.close() + + @pytest.mark.skip_if_version_below("8.0.0") + @pytest.mark.parametrize("cluster_mode", [True]) + @pytest.mark.parametrize("protocol", [ProtocolVersion.RESP2, ProtocolVersion.RESP3]) + async def test_routing_with_az_affinity_strategy_to_1_replica( + self, + request, + cluster_mode: bool, + protocol: ProtocolVersion, + multiple_replicas_cluster, + ): + """Test that the client with az affinity strategy will only route to the 1 replica with the same az""" + az = "us-east-1a" + GET_CALLS = 3 + get_cmdstat = f"cmdstat_get:calls={GET_CALLS}" + + client_for_config_set = await create_client( + request, + cluster_mode, + addresses=multiple_replicas_cluster.nodes_addr, + protocol=protocol, + timeout=2000, + ) + + # Reset the availability zone for all nodes + await client_for_config_set.custom_command( + ["CONFIG", "SET", "availability-zone", ""], + route=AllNodes(), + ) + await client_for_config_set.config_resetstat() == OK + + # 12182 is the slot of "foo" + await client_for_config_set.custom_command( + ["CONFIG", "SET", "availability-zone", az], + route=SlotIdRoute(SlotType.REPLICA, 12182), + ) + + await client_for_config_set.close() + + client_for_testing_az = await create_client( + request, + cluster_mode, + addresses=multiple_replicas_cluster.nodes_addr, + protocol=protocol, + read_from=ReadFrom.AZ_AFFINITY, + timeout=2000, + client_az=az, + ) + + for _ in range(GET_CALLS): + await client_for_testing_az.get("foo") + + info_result = await client_for_testing_az.info( + [InfoSection.SERVER, InfoSection.COMMAND_STATS], AllNodes() + ) + + # Check that only the replica with az has all the GET calls + matching_entries_count = sum( + 1 + for value in info_result.values() + if get_cmdstat in value.decode() and az in value.decode() + ) + assert matching_entries_count == 1 + + # Check that the other replicas have no availability zone set + changed_az_count = sum( + 1 + for node in info_result.values() + if f"availability_zone:{az}" in node.decode() + ) + assert changed_az_count == 1 + + await client_for_testing_az.close() + + @pytest.mark.skip_if_version_below("8.0.0") + @pytest.mark.parametrize("cluster_mode", [True]) + @pytest.mark.parametrize("protocol", [ProtocolVersion.RESP2, ProtocolVersion.RESP3]) + async def test_az_affinity_non_existing_az( + self, + request, + cluster_mode: bool, + protocol: ProtocolVersion, + multiple_replicas_cluster, + ): + GET_CALLS = 4 + + client_for_testing_az = await create_client( + request, + cluster_mode, + addresses=multiple_replicas_cluster.nodes_addr, + protocol=protocol, + read_from=ReadFrom.AZ_AFFINITY, + timeout=2000, + client_az="non-existing-az", + ) + await client_for_testing_az.config_resetstat() == OK + + for _ in range(GET_CALLS): + await client_for_testing_az.get("foo") + + n_replicas = await self._get_num_replicas(client_for_testing_az) + # We expect the calls to be distributed evenly among the replicas + get_cmdstat = f"cmdstat_get:calls={GET_CALLS // n_replicas}" + + info_result = await client_for_testing_az.info( + [InfoSection.COMMAND_STATS, InfoSection.SERVER], AllNodes() + ) + + matching_entries_count = sum( + 1 for value in info_result.values() if get_cmdstat in value.decode() + ) + assert matching_entries_count == GET_CALLS + + await client_for_testing_az.close() + + @pytest.mark.skip_if_version_below("8.0.0") + @pytest.mark.parametrize("cluster_mode", [True]) + @pytest.mark.parametrize("protocol", [ProtocolVersion.RESP2, ProtocolVersion.RESP3]) + async def test_az_affinity_requires_client_az( + self, request, cluster_mode: bool, protocol: ProtocolVersion + ): + """Test that setting read_from to AZ_AFFINITY without client_az raises an error.""" + with pytest.raises(ValueError): + await create_client( + request, + cluster_mode=cluster_mode, + protocol=protocol, + read_from=ReadFrom.AZ_AFFINITY, + timeout=2000, + ) From 5aaa5f376d261bbc1bd44a1288352a97b7d76a84 Mon Sep 17 00:00:00 2001 From: eifrah-aws Date: Wed, 20 Nov 2024 12:32:15 +0200 Subject: [PATCH 09/11] =?UTF-8?q?When=20no=20keys=20are=20provided,=20rout?= =?UTF-8?q?e=20to=20a=20random=20node=20to=20achieve=20the=20same=E2=80=A6?= =?UTF-8?q?=20(#2715)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Fixes: https://github.com/valkey-io/valkey-glide/issues/2714 --- .../redis-rs/redis/src/cluster_async/mod.rs | 14 +++++++++++++- glide-core/redis-rs/redis/src/cluster_routing.rs | 6 +++++- glide-core/tests/test_client.rs | 15 +++++++++++++++ 3 files changed, 33 insertions(+), 2 deletions(-) diff --git a/glide-core/redis-rs/redis/src/cluster_async/mod.rs b/glide-core/redis-rs/redis/src/cluster_async/mod.rs index e9d432b6f2..3726d7a674 100644 --- a/glide-core/redis-rs/redis/src/cluster_async/mod.rs +++ b/glide-core/redis-rs/redis/src/cluster_async/mod.rs @@ -1516,12 +1516,24 @@ where convert_result(receiver.await) }; + // Sanity + if receivers.is_empty() { + return Err(RedisError::from(( + ErrorKind::ClientError, + "Client internal error", + "Failed to aggregate results for multi-slot command. Maybe a malformed command?" + .to_string(), + ))); + } + // TODO - once Value::Error will be merged, these will need to be updated to handle this new value. match response_policy { Some(ResponsePolicy::AllSucceeded) => { future::try_join_all(receivers.into_iter().map(get_receiver)) .await - .map(|mut results| results.pop().unwrap()) // unwrap is safe, since at least one function succeeded + .map(|mut results| { + results.pop().unwrap() // unwrap is safe, since at least one function succeeded + }) } Some(ResponsePolicy::OneSucceeded) => future::select_ok( receivers diff --git a/glide-core/redis-rs/redis/src/cluster_routing.rs b/glide-core/redis-rs/redis/src/cluster_routing.rs index 858e725408..eab3bf398a 100644 --- a/glide-core/redis-rs/redis/src/cluster_routing.rs +++ b/glide-core/redis-rs/redis/src/cluster_routing.rs @@ -26,7 +26,7 @@ pub enum LogicalAggregateOp { // Or, omitted due to dead code warnings. ATM this value isn't constructed anywhere } -/// Numerical aggreagting operators. +/// Numerical aggregating operators. #[derive(Debug, Clone, Copy, PartialEq)] pub enum AggregateOp { /// Choose minimal value @@ -512,6 +512,10 @@ where } let mut routes: Vec<(Route, Vec)> = routes.into_iter().collect(); + if routes.is_empty() { + return None; + } + Some(if routes.len() == 1 { RoutingInfo::SingleNode(SingleNodeRoutingInfo::SpecificNode(routes.pop().unwrap().0)) } else { diff --git a/glide-core/tests/test_client.rs b/glide-core/tests/test_client.rs index 4e197b5c66..ffc672fee6 100644 --- a/glide-core/tests/test_client.rs +++ b/glide-core/tests/test_client.rs @@ -638,6 +638,21 @@ pub(crate) mod shared_client_tests { }); } + #[test] + #[serial_test::serial] + fn test_multi_key_no_args_in_cluster() { + block_on_all(async { + let cluster = cluster::setup_default_cluster().await; + println!("Creating 1st cluster client..."); + let mut c1 = cluster::setup_default_client(&cluster).await; + let result = c1.send_command(&redis::cmd("MSET"), None).await; + assert!(result.is_err()); + let e = result.unwrap_err(); + assert!(e.kind().clone().eq(&redis::ErrorKind::ResponseError)); + assert!(e.to_string().contains("wrong number of arguments")); + }); + } + #[rstest] #[serial_test::serial] #[timeout(SHORT_CLUSTER_TEST_TIMEOUT)] From 157ef1670a5900145c7fe501ec0ead1c2227f555 Mon Sep 17 00:00:00 2001 From: adarovadya Date: Wed, 20 Nov 2024 16:23:01 +0200 Subject: [PATCH 10/11] Revert "Python: add AZ Affinity ReadFrom strategy Support" (#2719) * Revert "Python: add AZ Affinity ReadFrom strategy Support (#2676)" This reverts commit 4d33f95c344bb07f4b6d4b6bf37ce5fab55be00f. * removed unessesrry import * add Unicode-3.0 license --------- Signed-off-by: Adar Ovadia Co-authored-by: Adar Ovadia --- CHANGELOG.md | 1 - deny.toml | 3 +- python/python/glide/config.py | 18 -- python/python/glide/glide_client.py | 1 - python/python/tests/conftest.py | 31 --- python/python/tests/test_config.py | 15 -- .../python/tests/test_read_from_strategy.py | 228 ------------------ 7 files changed, 2 insertions(+), 295 deletions(-) delete mode 100644 python/python/tests/test_read_from_strategy.py diff --git a/CHANGELOG.md b/CHANGELOG.md index 80f7c054d4..228aa7bdf2 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,5 +1,4 @@ #### Changes -* Python: AZ Affinity - Python Wrapper Support ([#2676](https://github.com/valkey-io/valkey-glide/pull/2676)) * Python: Client API for retrieving internal statistics ([#2707](https://github.com/valkey-io/valkey-glide/pull/2707)) * Node, Python: Adding support for replacing connection configured password ([#2651](https://github.com/valkey-io/valkey-glide/pull/2651))([#2659](https://github.com/valkey-io/valkey-glide/pull/2659)) * Node: Add FT._ALIASLIST command([#2652](https://github.com/valkey-io/valkey-glide/pull/2652)) diff --git a/deny.toml b/deny.toml index 0a43bfa193..526ce9bd1e 100644 --- a/deny.toml +++ b/deny.toml @@ -57,7 +57,8 @@ allow = [ "Unicode-DFS-2016", "ISC", "OpenSSL", - "MPL-2.0" + "MPL-2.0", + "Unicode-3.0" ] # The confidence threshold for detecting a license from license text. # The higher the value, the more closely the license text must be to the diff --git a/python/python/glide/config.py b/python/python/glide/config.py index b33c037cbf..db85202876 100644 --- a/python/python/glide/config.py +++ b/python/python/glide/config.py @@ -41,11 +41,6 @@ class ReadFrom(Enum): Spread the requests between all replicas in a round robin manner. If no replica is available, route the requests to the primary. """ - AZ_AFFINITY = ProtobufReadFrom.AZAffinity - """ - Spread the read requests between replicas in the same client's AZ (Aviliablity zone) in a round robin manner, - falling back to other replicas or the primary if needed - """ class ProtocolVersion(Enum): @@ -140,7 +135,6 @@ def __init__( client_name: Optional[str] = None, protocol: ProtocolVersion = ProtocolVersion.RESP3, inflight_requests_limit: Optional[int] = None, - client_az: Optional[str] = None, ): """ Represents the configuration settings for a Glide client. @@ -178,12 +172,6 @@ def __init__( self.client_name = client_name self.protocol = protocol self.inflight_requests_limit = inflight_requests_limit - self.client_az = client_az - - if read_from == ReadFrom.AZ_AFFINITY and not client_az: - raise ValueError( - "client_az mus t be set when read_from is set to AZ_AFFINITY" - ) def _create_a_protobuf_conn_request( self, cluster_mode: bool = False @@ -216,8 +204,6 @@ def _create_a_protobuf_conn_request( request.protocol = self.protocol.value if self.inflight_requests_limit: request.inflight_requests_limit = self.inflight_requests_limit - if self.client_az: - request.client_az = self.client_az return request @@ -307,7 +293,6 @@ def __init__( protocol: ProtocolVersion = ProtocolVersion.RESP3, pubsub_subscriptions: Optional[PubSubSubscriptions] = None, inflight_requests_limit: Optional[int] = None, - client_az: Optional[str] = None, ): super().__init__( addresses=addresses, @@ -318,7 +303,6 @@ def __init__( client_name=client_name, protocol=protocol, inflight_requests_limit=inflight_requests_limit, - client_az=client_az, ) self.reconnect_strategy = reconnect_strategy self.database_id = database_id @@ -458,7 +442,6 @@ def __init__( ] = PeriodicChecksStatus.ENABLED_DEFAULT_CONFIGS, pubsub_subscriptions: Optional[PubSubSubscriptions] = None, inflight_requests_limit: Optional[int] = None, - client_az: Optional[str] = None, ): super().__init__( addresses=addresses, @@ -469,7 +452,6 @@ def __init__( client_name=client_name, protocol=protocol, inflight_requests_limit=inflight_requests_limit, - client_az=client_az, ) self.periodic_checks = periodic_checks self.pubsub_subscriptions = pubsub_subscriptions diff --git a/python/python/glide/glide_client.py b/python/python/glide/glide_client.py index ea648c49ba..6178b997a7 100644 --- a/python/python/glide/glide_client.py +++ b/python/python/glide/glide_client.py @@ -26,7 +26,6 @@ from glide.protobuf.response_pb2 import RequestErrorType, Response from glide.protobuf_codec import PartialMessageException, ProtobufCodec from glide.routes import Route, set_protobuf_route -from pyrsistent import optional from .glide import ( DEFAULT_TIMEOUT_IN_MILLISECONDS, diff --git a/python/python/tests/conftest.py b/python/python/tests/conftest.py index 85bc58c4b1..437fbd8fbb 100644 --- a/python/python/tests/conftest.py +++ b/python/python/tests/conftest.py @@ -9,7 +9,6 @@ GlideClusterClientConfiguration, NodeAddress, ProtocolVersion, - ReadFrom, ServerCredentials, ) from glide.exceptions import ClosingError, RequestError @@ -133,7 +132,6 @@ def create_clusters(tls, load_module, cluster_endpoints, standalone_endpoints): cluster_mode=True, load_module=load_module, addresses=cluster_endpoints, - replica_count=1, ) pytest.standalone_cluster = ValkeyCluster( tls=tls, @@ -250,8 +248,6 @@ async def create_client( GlideClientConfiguration.PubSubSubscriptions ] = None, inflight_requests_limit: Optional[int] = None, - read_from: ReadFrom = ReadFrom.PRIMARY, - client_az: Optional[str] = None, ) -> Union[GlideClient, GlideClusterClient]: # Create async socket client use_tls = request.config.getoption("--tls") @@ -269,8 +265,6 @@ async def create_client( request_timeout=timeout, pubsub_subscriptions=cluster_mode_pubsub, inflight_requests_limit=inflight_requests_limit, - read_from=read_from, - client_az=client_az, ) return await GlideClusterClient.create(cluster_config) else: @@ -287,8 +281,6 @@ async def create_client( request_timeout=timeout, pubsub_subscriptions=standalone_mode_pubsub, inflight_requests_limit=inflight_requests_limit, - read_from=read_from, - client_az=client_az, ) return await GlideClient.create(config) @@ -389,26 +381,3 @@ async def test_meow_meow(...): reason=f"This feature added in version {min_version}", allow_module_level=True, ) - - -@pytest.fixture(scope="module") -def multiple_replicas_cluster(request): - """ - Fixture to create a special cluster with 4 replicas for specific tests. - """ - tls = request.config.getoption("--tls") - load_module = request.config.getoption("--load-module") - cluster_endpoints = request.config.getoption("--cluster-endpoints") - - if not cluster_endpoints: - multiple_replica_cluster = ValkeyCluster( - tls=tls, - cluster_mode=True, - load_module=load_module, - addresses=cluster_endpoints, - replica_count=4, - ) - yield multiple_replica_cluster - multiple_replica_cluster.__del__() - else: - yield None diff --git a/python/python/tests/test_config.py b/python/python/tests/test_config.py index 3b22adb09c..93c280245f 100644 --- a/python/python/tests/test_config.py +++ b/python/python/tests/test_config.py @@ -52,18 +52,3 @@ def test_periodic_checks_interval_to_protobuf(): config.periodic_checks = PeriodicChecksManualInterval(30) request = config._create_a_protobuf_conn_request(cluster_mode=True) assert request.periodic_checks_manual_interval.duration_in_sec == 30 - - -def test_convert_config_with_azaffinity_to_protobuf(): - az = "us-east-1a" - config = BaseClientConfiguration( - [NodeAddress("127.0.0.1")], - use_tls=True, - read_from=ReadFrom.AZ_AFFINITY, - client_az=az, - ) - request = config._create_a_protobuf_conn_request() - assert isinstance(request, ConnectionRequest) - assert request.tls_mode is TlsMode.SecureTls - assert request.read_from == ProtobufReadFrom.AZAffinity - assert request.client_az == az diff --git a/python/python/tests/test_read_from_strategy.py b/python/python/tests/test_read_from_strategy.py deleted file mode 100644 index fc15481a07..0000000000 --- a/python/python/tests/test_read_from_strategy.py +++ /dev/null @@ -1,228 +0,0 @@ -# Copyright Valkey GLIDE Project Contributors - SPDX Identifier: Apache-2.0 - -import re - -import pytest -from glide.async_commands.core import InfoSection -from glide.config import ProtocolVersion, ReadFrom -from glide.constants import OK -from glide.glide_client import GlideClusterClient -from glide.routes import AllNodes, SlotIdRoute, SlotType -from tests.conftest import create_client -from tests.utils.utils import get_first_result - - -@pytest.mark.asyncio -@pytest.mark.usefixtures("multiple_replicas_cluster") -class TestAZAffinity: - async def _get_num_replicas(self, client: GlideClusterClient) -> int: - info_replicas = get_first_result( - await client.info([InfoSection.REPLICATION]) - ).decode() - match = re.search(r"connected_slaves:(\d+)", info_replicas) - if match: - return int(match.group(1)) - else: - raise ValueError( - "Could not find the number of replicas in the INFO REPLICATION response" - ) - - @pytest.mark.skip_if_version_below("8.0.0") - @pytest.mark.parametrize("cluster_mode", [True]) - @pytest.mark.parametrize("protocol", [ProtocolVersion.RESP2, ProtocolVersion.RESP3]) - async def test_routing_by_slot_to_replica_with_az_affinity_strategy_to_all_replicas( - self, - request, - cluster_mode: bool, - protocol: ProtocolVersion, - multiple_replicas_cluster, - ): - """Test that the client with AZ affinity strategy routes in a round-robin manner to all replicas within the specified AZ""" - - az = "us-east-1a" - client_for_config_set = await create_client( - request, - cluster_mode, - addresses=multiple_replicas_cluster.nodes_addr, - protocol=protocol, - timeout=2000, - ) - await client_for_config_set.config_resetstat() == OK - await client_for_config_set.custom_command( - ["CONFIG", "SET", "availability-zone", az], AllNodes() - ) - await client_for_config_set.close() - - client_for_testing_az = await create_client( - request, - cluster_mode, - addresses=multiple_replicas_cluster.nodes_addr, - protocol=protocol, - read_from=ReadFrom.AZ_AFFINITY, - timeout=2000, - client_az=az, - ) - azs = await client_for_testing_az.custom_command( - ["CONFIG", "GET", "availability-zone"], AllNodes() - ) - - # Check that all replicas have the availability zone set to the az - assert all( - ( - node[1].decode() == az - if isinstance(node, list) - else node[b"availability-zone"].decode() == az - ) - for node in azs.values() - ) - - n_replicas = await self._get_num_replicas(client_for_testing_az) - GET_CALLS = 3 * n_replicas - get_cmdstat = f"cmdstat_get:calls={GET_CALLS // n_replicas}" - - for _ in range(GET_CALLS): - await client_for_testing_az.get("foo") - - info_result = await client_for_testing_az.info( - [InfoSection.COMMAND_STATS, InfoSection.SERVER], AllNodes() - ) - - # Check that all replicas have the same number of GET calls - matching_entries_count = sum( - 1 - for value in info_result.values() - if get_cmdstat in value.decode() and az in value.decode() - ) - assert matching_entries_count == n_replicas - - await client_for_testing_az.close() - - @pytest.mark.skip_if_version_below("8.0.0") - @pytest.mark.parametrize("cluster_mode", [True]) - @pytest.mark.parametrize("protocol", [ProtocolVersion.RESP2, ProtocolVersion.RESP3]) - async def test_routing_with_az_affinity_strategy_to_1_replica( - self, - request, - cluster_mode: bool, - protocol: ProtocolVersion, - multiple_replicas_cluster, - ): - """Test that the client with az affinity strategy will only route to the 1 replica with the same az""" - az = "us-east-1a" - GET_CALLS = 3 - get_cmdstat = f"cmdstat_get:calls={GET_CALLS}" - - client_for_config_set = await create_client( - request, - cluster_mode, - addresses=multiple_replicas_cluster.nodes_addr, - protocol=protocol, - timeout=2000, - ) - - # Reset the availability zone for all nodes - await client_for_config_set.custom_command( - ["CONFIG", "SET", "availability-zone", ""], - route=AllNodes(), - ) - await client_for_config_set.config_resetstat() == OK - - # 12182 is the slot of "foo" - await client_for_config_set.custom_command( - ["CONFIG", "SET", "availability-zone", az], - route=SlotIdRoute(SlotType.REPLICA, 12182), - ) - - await client_for_config_set.close() - - client_for_testing_az = await create_client( - request, - cluster_mode, - addresses=multiple_replicas_cluster.nodes_addr, - protocol=protocol, - read_from=ReadFrom.AZ_AFFINITY, - timeout=2000, - client_az=az, - ) - - for _ in range(GET_CALLS): - await client_for_testing_az.get("foo") - - info_result = await client_for_testing_az.info( - [InfoSection.SERVER, InfoSection.COMMAND_STATS], AllNodes() - ) - - # Check that only the replica with az has all the GET calls - matching_entries_count = sum( - 1 - for value in info_result.values() - if get_cmdstat in value.decode() and az in value.decode() - ) - assert matching_entries_count == 1 - - # Check that the other replicas have no availability zone set - changed_az_count = sum( - 1 - for node in info_result.values() - if f"availability_zone:{az}" in node.decode() - ) - assert changed_az_count == 1 - - await client_for_testing_az.close() - - @pytest.mark.skip_if_version_below("8.0.0") - @pytest.mark.parametrize("cluster_mode", [True]) - @pytest.mark.parametrize("protocol", [ProtocolVersion.RESP2, ProtocolVersion.RESP3]) - async def test_az_affinity_non_existing_az( - self, - request, - cluster_mode: bool, - protocol: ProtocolVersion, - multiple_replicas_cluster, - ): - GET_CALLS = 4 - - client_for_testing_az = await create_client( - request, - cluster_mode, - addresses=multiple_replicas_cluster.nodes_addr, - protocol=protocol, - read_from=ReadFrom.AZ_AFFINITY, - timeout=2000, - client_az="non-existing-az", - ) - await client_for_testing_az.config_resetstat() == OK - - for _ in range(GET_CALLS): - await client_for_testing_az.get("foo") - - n_replicas = await self._get_num_replicas(client_for_testing_az) - # We expect the calls to be distributed evenly among the replicas - get_cmdstat = f"cmdstat_get:calls={GET_CALLS // n_replicas}" - - info_result = await client_for_testing_az.info( - [InfoSection.COMMAND_STATS, InfoSection.SERVER], AllNodes() - ) - - matching_entries_count = sum( - 1 for value in info_result.values() if get_cmdstat in value.decode() - ) - assert matching_entries_count == GET_CALLS - - await client_for_testing_az.close() - - @pytest.mark.skip_if_version_below("8.0.0") - @pytest.mark.parametrize("cluster_mode", [True]) - @pytest.mark.parametrize("protocol", [ProtocolVersion.RESP2, ProtocolVersion.RESP3]) - async def test_az_affinity_requires_client_az( - self, request, cluster_mode: bool, protocol: ProtocolVersion - ): - """Test that setting read_from to AZ_AFFINITY without client_az raises an error.""" - with pytest.raises(ValueError): - await create_client( - request, - cluster_mode=cluster_mode, - protocol=protocol, - read_from=ReadFrom.AZ_AFFINITY, - timeout=2000, - ) From ea5667b68626567fcc891d148db1d18b4a7fe38f Mon Sep 17 00:00:00 2001 From: Shoham Elias <116083498+shohamazon@users.noreply.github.com> Date: Wed, 20 Nov 2024 17:49:51 +0200 Subject: [PATCH 11/11] Python: rename glide json (#2702) * Python: rename glide json Signed-off-by: Shoham Elias --------- Signed-off-by: Shoham Elias --- python/python/glide/__init__.py | 6 +- .../server_modules/{json.py => glide_json.py} | 405 +++++++++--------- .../tests/tests_server_modules/test_ft.py | 2 +- .../tests/tests_server_modules/test_json.py | 4 +- 4 files changed, 214 insertions(+), 203 deletions(-) rename python/python/glide/async_commands/server_modules/{json.py => glide_json.py} (80%) diff --git a/python/python/glide/__init__.py b/python/python/glide/__init__.py index 6f77cac760..c08814ee7a 100644 --- a/python/python/glide/__init__.py +++ b/python/python/glide/__init__.py @@ -32,7 +32,7 @@ InsertPosition, UpdateOptions, ) -from glide.async_commands.server_modules import ft, json +from glide.async_commands.server_modules import ft, glide_json from glide.async_commands.server_modules.ft_options.ft_aggregate_options import ( FtAggregateApply, FtAggregateClause, @@ -67,7 +67,7 @@ FtSearchOptions, ReturnField, ) -from glide.async_commands.server_modules.json import ( +from glide.async_commands.server_modules.glide_json import ( JsonArrIndexOptions, JsonArrPopOptions, JsonGetOptions, @@ -264,7 +264,7 @@ # PubSub "PubSubMsg", # Json - "json", + "glide_json", "JsonGetOptions", "JsonArrIndexOptions", "JsonArrPopOptions", diff --git a/python/python/glide/async_commands/server_modules/json.py b/python/python/glide/async_commands/server_modules/glide_json.py similarity index 80% rename from python/python/glide/async_commands/server_modules/json.py rename to python/python/glide/async_commands/server_modules/glide_json.py index d4c3f8797b..ba9fe26e57 100644 --- a/python/python/glide/async_commands/server_modules/json.py +++ b/python/python/glide/async_commands/server_modules/glide_json.py @@ -3,16 +3,16 @@ Examples: - >>> from glide import json - >>> import json as jsonpy + >>> from glide import glide_json + >>> import json >>> value = {'a': 1.0, 'b': 2} - >>> json_str = jsonpy.dumps(value) # Convert Python dictionary to JSON string using json.dumps() + >>> json_str = json.dumps(value) # Convert Python dictionary to JSON string using json.dumps() >>> await json.set(client, "doc", "$", json_str) 'OK' # Indicates successful setting of the value at path '$' in the key stored at `doc`. - >>> json_get = await json.get(client, "doc", "$") # Returns the value at path '$' in the JSON document stored at `doc` as JSON string. + >>> json_get = await glide_json.get(client, "doc", "$") # Returns the value at path '$' in the JSON document stored at `doc` as JSON string. >>> print(json_get) b"[{\"a\":1.0,\"b\":2}]" - >>> jsonpy.loads(str(json_get)) + >>> json.loads(str(json_get)) [{"a": 1.0, "b" :2}] # JSON object retrieved from the key `doc` using json.loads() """ from typing import List, Optional, Union, cast @@ -100,7 +100,7 @@ def __init__(self, path: TEncodable, index: Optional[int] = None): def to_args(self) -> List[TEncodable]: """ - Get the options as a list of arguments for the JSON.ARRPOP command. + Get the options as a list of arguments for the `JSON.ARRPOP` command. Returns: List[TEncodable]: A list containing the path and, if specified, the index. @@ -135,11 +135,11 @@ async def set( If `value` isn't set because of `set_condition`, returns None. Examples: - >>> from glide import json - >>> import json as jsonpy + >>> from glide import glide_json + >>> import json >>> value = {'a': 1.0, 'b': 2} - >>> json_str = jsonpy.dumps(value) - >>> await json.set(client, "doc", "$", json_str) + >>> json_str = json.dumps(value) + >>> await glide_json.set(client, "doc", "$", json_str) 'OK' # Indicates successful setting of the value at path '$' in the key stored at `doc`. """ args = ["JSON.SET", key, path, value] @@ -181,16 +181,16 @@ async def get( For more information about the returned type, see `TJsonResponse`. Examples: - >>> from glide import json, JsonGetOptions - >>> import as jsonpy - >>> json_str = await json.get(client, "doc", "$") - >>> jsonpy.loads(str(json_str)) # Parse JSON string to Python data + >>> from glide import glide_json, JsonGetOptions + >>> import json + >>> json_str = await glide_json.get(client, "doc", "$") + >>> json.loads(str(json_str)) # Parse JSON string to Python data [{"a": 1.0, "b" :2}] # JSON object retrieved from the key `doc` using json.loads() - >>> await json.get(client, "doc", "$") + >>> await glide_json.get(client, "doc", "$") b"[{\"a\":1.0,\"b\":2}]" # Returns the value at path '$' in the JSON document stored at `doc`. - >>> await json.get(client, "doc", ["$.a", "$.b"], JsonGetOptions(indent=" ", newline="\n", space=" ")) + >>> await glide_json.get(client, "doc", ["$.a", "$.b"], JsonGetOptions(indent=" ", newline="\n", space=" ")) b"{\n \"$.a\": [\n 1.0\n ],\n \"$.b\": [\n 2\n ]\n}" # Returns the values at paths '$.a' and '$.b' in the JSON document stored at `doc`, with specified formatting options. - >>> await json.get(client, "doc", "$.non_existing_path") + >>> await glide_json.get(client, "doc", "$.non_existing_path") b"[]" # Returns an empty array since the path '$.non_existing_path' does not exist in the JSON document stored at `doc`. """ args = ["JSON.GET", key] @@ -204,53 +204,6 @@ async def get( return cast(TJsonResponse[Optional[bytes]], await client.custom_command(args)) -async def mget( - client: TGlideClient, - keys: List[TEncodable], - path: TEncodable, -) -> List[Optional[bytes]]: - """ - Retrieves the JSON values at the specified `path` stored at multiple `keys`. - - Note: - In cluster mode, if keys in `keys` map to different hash slots, the command - will be split across these slots and executed separately for each. This means the command - is atomic only at the slot level. If one or more slot-specific requests fail, the entire - call will return the first encountered error, even though some requests may have succeeded - while others did not. If this behavior impacts your application logic, consider splitting - the request into sub-requests per slot to ensure atomicity. - - Args: - client (TGlideClient): The client to execute the command. - keys (List[TEncodable]): A list of keys for the JSON documents. - path (TEncodable): The path within the JSON documents. - - Returns: - List[Optional[bytes]]: - For JSONPath (`path` starts with `$`): - Returns a list of byte representations of the values found at the given path for each key. - If `path` does not exist within the key, the entry will be an empty array. - For legacy path (`path` doesn't starts with `$`): - Returns a list of byte representations of the values found at the given path for each key. - If `path` does not exist within the key, the entry will be None. - If a key doesn't exist, the corresponding list element will be None. - - - Examples: - >>> from glide import json as glideJson - >>> import json - >>> json_strs = await glideJson.mget(client, ["doc1", "doc2"], "$") - >>> [json.loads(js) for js in json_strs] # Parse JSON strings to Python data - [[{"a": 1.0, "b": 2}], [{"a": 2.0, "b": {"a": 3.0, "b" : 4.0}}]] # JSON objects retrieved from keys `doc1` and `doc2` - >>> await glideJson.mget(client, ["doc1", "doc2"], "$.a") - [b"[1.0]", b"[2.0]"] # Returns values at path '$.a' for the JSON documents stored at `doc1` and `doc2`. - >>> await glideJson.mget(client, ["doc1"], "$.non_existing_path") - [None] # Returns an empty array since the path '$.non_existing_path' does not exist in the JSON document stored at `doc1`. - """ - args = ["JSON.MGET"] + keys + [path] - return cast(TJsonResponse[Optional[bytes]], await client.custom_command(args)) - - async def arrappend( client: TGlideClient, key: TEncodable, @@ -281,15 +234,15 @@ async def arrappend( For more information about the returned type, see `TJsonResponse`. Examples: - >>> from glide import json as valkeyJson + >>> from glide import glide_json >>> import json - >>> await valkeyJson.set(client, "doc", "$", '{"a": 1, "b": ["one", "two"]}') + >>> await glide_json.set(client, "doc", "$", '{"a": 1, "b": ["one", "two"]}') 'OK' # Indicates successful setting of the value at path '$' in the key stored at `doc`. - >>> await valkeyJson.arrappend(client, "doc", ["three"], "$.b") + >>> await glide_json.arrappend(client, "doc", ["three"], "$.b") [3] # Returns the new length of the array at path '$.b' after appending the value. - >>> await valkeyJson.arrappend(client, "doc", ["four"], ".b") + >>> await glide_json.arrappend(client, "doc", ["four"], ".b") 4 # Returns the new length of the array at path '.b' after appending the value. - >>> json.loads(await valkeyJson.get(client, "doc", ".")) + >>> json.loads(await glide_json.get(client, "doc", ".")) {"a": 1, "b": ["one", "two", "three", "four"]} # Returns the updated JSON document """ args = ["JSON.ARRAPPEND", key, path] + values @@ -324,7 +277,7 @@ async def arrindex( Defaults to the full array if not provided. See `JsonArrIndexOptions`. Returns: - Optional[Union[int, List[int]]]: + Optional[TJsonResponse[int]]: For JSONPath (`path` starts with `$`): Returns an array of integers for every possible path, indicating of the first occurrence of `value` within the array, or None for JSON values matching the path that are not an array. @@ -336,28 +289,29 @@ async def arrindex( If multiple paths match, the index of the value from the first matching array is returned. If the JSON value at the `path` is not an array or if `path` does not exist, an error is raised. If `key` does not exist, an error is raised. + For more information about the returned type, see `TJsonResponse`. Examples: - >>> from glide import json - >>> await json.set(client, "doc", "$", '[[], ["a"], ["a", "b"], ["a", "b", "c"]]') + >>> from glide import glide_json + >>> await glide_json.set(client, "doc", "$", '[[], ["a"], ["a", "b"], ["a", "b", "c"]]') 'OK' - >>> await json.arrindex(client, "doc", "$[*]", '"b"') + >>> await glide_json.arrindex(client, "doc", "$[*]", '"b"') [-1, -1, 1, 1] - >>> await json.set(client, "doc", ".", '{"children": ["John", "Jack", "Tom", "Bob", "Mike"]}') + >>> await glide_json.set(client, "doc", ".", '{"children": ["John", "Jack", "Tom", "Bob", "Mike"]}') 'OK' - >>> await json.arrindex(client, "doc", ".children", '"Tom"') + >>> await glide_json.arrindex(client, "doc", ".children", '"Tom"') 2 - >>> await json.set(client, "doc", "$", '{"fruits": ["apple", "banana", "cherry", "banana", "grape"]}') + >>> await glide_json.set(client, "doc", "$", '{"fruits": ["apple", "banana", "cherry", "banana", "grape"]}') 'OK' - >>> await json.arrindex(client, "doc", "$.fruits", '"banana"', JsonArrIndexOptions(start=2, end=4)) + >>> await glide_json.arrindex(client, "doc", "$.fruits", '"banana"', JsonArrIndexOptions(start=2, end=4)) 3 - >>> await json.set(client, "k", ".", '[1, 2, "a", 4, "a", 6, 7, "b"]') + >>> await glide_json.set(client, "k", ".", '[1, 2, "a", 4, "a", 6, 7, "b"]') 'OK' - >>> await json.arrindex(client, "k", ".", '"b"', JsonArrIndexOptions(start=4, end=0)) + >>> await glide_json.arrindex(client, "k", ".", '"b"', JsonArrIndexOptions(start=4, end=0)) 7 # "b" found at index 7 within the specified range, treating end=0 as the entire array's end. - >>> await json.arrindex(client, "k", ".", '"b"', JsonArrIndexOptions(start=4, end=-1)) + >>> await glide_json.arrindex(client, "k", ".", '"b"', JsonArrIndexOptions(start=4, end=-1)) 7 # "b" found at index 7, with end=-1 covering the full array to its last element. - >>> await json.arrindex(client, "k", ".", '"b"', JsonArrIndexOptions(start=4, end=7)) + >>> await glide_json.arrindex(client, "k", ".", '"b"', JsonArrIndexOptions(start=4, end=7)) -1 # "b" not found within the range from index 4 to exclusive end at index 7. """ args = ["JSON.ARRINDEX", key, path, value] @@ -398,21 +352,22 @@ async def arrinsert( If `path` doesn't exist or the value at `path` is not an array, an error is raised. If the index is out of bounds, an error is raised. If `key` doesn't exist, an error is raised. + For more information about the returned type, see `TJsonResponse`. Examples: - >>> from glide import json - >>> await json.set(client, "doc", "$", '[[], ["a"], ["a", "b"]]') + >>> from glide import glide_json + >>> await glide_json.set(client, "doc", "$", '[[], ["a"], ["a", "b"]]') 'OK' - >>> await json.arrinsert(client, "doc", "$[*]", 0, ['"c"', '{"key": "value"}', "true", "null", '["bar"]']) + >>> await glide_json.arrinsert(client, "doc", "$[*]", 0, ['"c"', '{"key": "value"}', "true", "null", '["bar"]']) [5, 6, 7] # New lengths of arrays after insertion - >>> await json.get(client, "doc") + >>> await glide_json.get(client, "doc") b'[["c",{"key":"value"},true,null,["bar"]],["c",{"key":"value"},true,null,["bar"],"a"],["c",{"key":"value"},true,null,["bar"],"a","b"]]' - >>> await json.set(client, "doc", "$", '[[], ["a"], ["a", "b"]]') + >>> await glide_json.set(client, "doc", "$", '[[], ["a"], ["a", "b"]]') 'OK' - >>> await json.arrinsert(client, "doc", ".", 0, ['"c"']) + >>> await glide_json.arrinsert(client, "doc", ".", 0, ['"c"']) 4 # New length of the root array after insertion - >>> await json.get(client, "doc") + >>> await glide_json.get(client, "doc") b'[\"c\",[],[\"a\"],[\"a\",\"b\"]]' """ args = ["JSON.ARRINSERT", key, path, str(index)] + values @@ -443,25 +398,26 @@ async def arrlen( If multiple paths match, the length of the first array match is returned. If the JSON value at `path` is not a array or if `path` doesn't exist, an error is raised. If `key` doesn't exist, None is returned. + For more information about the returned type, see `TJsonResponse`. Examples: - >>> from glide import json - >>> await json.set(client, "doc", "$", '{"a": [1, 2, 3], "b": {"a": [1, 2], "c": {"a": 42}}}') + >>> from glide import glide_json + >>> await glide_json.set(client, "doc", "$", '{"a": [1, 2, 3], "b": {"a": [1, 2], "c": {"a": 42}}}') 'OK' # JSON is successfully set for doc - >>> await json.arrlen(client, "doc", "$") + >>> await glide_json.arrlen(client, "doc", "$") [None] # No array at the root path. - >>> await json.arrlen(client, "doc", "$.a") + >>> await glide_json.arrlen(client, "doc", "$.a") [3] # Retrieves the length of the array at path $.a. - >>> await json.arrlen(client, "doc", "$..a") + >>> await glide_json.arrlen(client, "doc", "$..a") [3, 2, None] # Retrieves lengths of arrays found at all levels of the path `$..a`. - >>> await json.arrlen(client, "doc", "..a") + >>> await glide_json.arrlen(client, "doc", "..a") 3 # Legacy path retrieves the first array match at path `..a`. - >>> await json.arrlen(client, "non_existing_key", "$.a") + >>> await glide_json.arrlen(client, "non_existing_key", "$.a") None # Returns None because the key does not exist. - >>> await json.set(client, "doc", "$", '[1, 2, 3, 4]') + >>> await glide_json.set(client, "doc", "$", '[1, 2, 3, 4]') 'OK' # JSON is successfully set for doc - >>> await json.arrlen(client, "doc") + >>> await glide_json.arrlen(client, "doc") 4 # Retrieves lengths of array in root. """ args = ["JSON.ARRLEN", key] @@ -500,29 +456,30 @@ async def arrpop( If multiple paths match, the value from the first matching array that is not empty is returned. If the JSON value at `options.path` is not a array or if `options.path` doesn't exist, an error is raised. If `key` doesn't exist, an error is raised. + For more information about the returned type, see `TJsonResponse`. Examples: - >>> from glide import json - >>> await json.set(client, "doc", "$", '{"a": [1, 2, true], "b": {"a": [3, 4, ["value", 3, false], 5], "c": {"a": 42}}}') + >>> from glide import glide_json + >>> await glide_json.set(client, "doc", "$", '{"a": [1, 2, true], "b": {"a": [3, 4, ["value", 3, false], 5], "c": {"a": 42}}}') b'OK' - >>> await json.arrpop(client, "doc", JsonArrPopOptions(path="$.a", index=1)) + >>> await glide_json.arrpop(client, "doc", JsonArrPopOptions(path="$.a", index=1)) [b'2'] # Pop second element from array at path $.a - >>> await json.arrpop(client, "doc", JsonArrPopOptions(path="$..a")) + >>> await glide_json.arrpop(client, "doc", JsonArrPopOptions(path="$..a")) [b'true', b'5', None] # Pop last elements from all arrays matching path `$..a` #### Using a legacy path (..) to pop the first matching array - >>> await json.arrpop(client, "doc", JsonArrPopOptions(path="..a")) + >>> await glide_json.arrpop(client, "doc", JsonArrPopOptions(path="..a")) b"1" # First match popped (from array at path ..a) #### Even though only one value is returned from `..a`, subsequent arrays are also affected - >>> await json.get(client, "doc", "$..a") + >>> await glide_json.get(client, "doc", "$..a") b"[[], [3, 4], 42]" # Remaining elements after pop show the changes - >>> await json.set(client, "doc", "$", '[[], ["a"], ["a", "b", "c"]]') + >>> await glide_json.set(client, "doc", "$", '[[], ["a"], ["a", "b", "c"]]') b'OK' # JSON is successfully set - >>> await json.arrpop(client, "doc", JsonArrPopOptions(path=".", index=-1)) + >>> await glide_json.arrpop(client, "doc", JsonArrPopOptions(path=".", index=-1)) b'["a","b","c"]' # Pop last elements at path `.` - >>> await json.arrpop(client, "doc") + >>> await glide_json.arrpop(client, "doc") b'["a"]' # Pop last elements at path `.` """ args = ["JSON.ARRPOP", key] @@ -567,21 +524,22 @@ async def arrtrim( If multiple paths match, the length of the first trimmed array match is returned. If `path` doesn't exist, or the value at `path` is not an array, an error is raised. If `key` doesn't exist, an error is raised. + For more information about the returned type, see `TJsonResponse`. Examples: - >>> from glide import json - >>> await json.set(client, "doc", "$", '[[], ["a"], ["a", "b"], ["a", "b", "c"]]') + >>> from glide import glide_json + >>> await glide_json.set(client, "doc", "$", '[[], ["a"], ["a", "b"], ["a", "b", "c"]]') 'OK' - >>> await json.arrtrim(client, "doc", "$[*]", 0, 1) + >>> await glide_json.arrtrim(client, "doc", "$[*]", 0, 1) [0, 1, 2, 2] - >>> await json.get(client, "doc") + >>> await glide_json.get(client, "doc") b'[[],[\"a\"],[\"a\",\"b\"],[\"a\",\"b\"]]' - >>> await json.set(client, "doc", "$", '{"children": ["John", "Jack", "Tom", "Bob", "Mike"]}') + >>> await glide_json.set(client, "doc", "$", '{"children": ["John", "Jack", "Tom", "Bob", "Mike"]}') 'OK' - >>> await json.arrtrim(client, "doc", ".children", 0, 1) + >>> await glide_json.arrtrim(client, "doc", ".children", 0, 1) 2 - >>> await json.get(client, "doc", ".children") + >>> await glide_json.get(client, "doc", ".children") b'["John","Jack"]' """ return cast( @@ -611,28 +569,28 @@ async def clear( If `key doesn't exist, an error is raised. Examples: - >>> from glide import json - >>> await json.set(client, "doc", "$", '{"obj":{"a":1, "b":2}, "arr":[1,2,3], "str": "foo", "bool": true, "int": 42, "float": 3.14, "nullVal": null}') + >>> from glide import glide_json + >>> await glide_json.set(client, "doc", "$", '{"obj":{"a":1, "b":2}, "arr":[1,2,3], "str": "foo", "bool": true, "int": 42, "float": 3.14, "nullVal": null}') 'OK' # JSON document is successfully set. - >>> await json.clear(client, "doc", "$.*") + >>> await glide_json.clear(client, "doc", "$.*") 6 # 6 values are cleared (arrays/objects/strings/numbers/booleans), but `null` remains as is. - >>> await json.get(client, "doc", "$") + >>> await glide_json.get(client, "doc", "$") b'[{"obj":{},"arr":[],"str":"","bool":false,"int":0,"float":0.0,"nullVal":null}]' - >>> await json.clear(client, "doc", "$.*") + >>> await glide_json.clear(client, "doc", "$.*") 0 # No further clearing needed since the containers are already empty and the values are defaults. - >>> await json.set(client, "doc", "$", '{"a": 1, "b": {"a": [5, 6, 7], "b": {"a": true}}, "c": {"a": "value", "b": {"a": 3.5}}, "d": {"a": {"foo": "foo"}}, "nullVal": null}') + >>> await glide_json.set(client, "doc", "$", '{"a": 1, "b": {"a": [5, 6, 7], "b": {"a": true}}, "c": {"a": "value", "b": {"a": 3.5}}, "d": {"a": {"foo": "foo"}}, "nullVal": null}') 'OK' - >>> await json.clear(client, "doc", "b.a[1:3]") + >>> await glide_json.clear(client, "doc", "b.a[1:3]") 2 # 2 elements (`6` and `7`) are cleared. - >>> await json.clear(client, "doc", "b.a[1:3]") + >>> await glide_json.clear(client, "doc", "b.a[1:3]") 0 # No elements cleared since specified slice has already been cleared. - >>> await json.get(client, "doc", "$..a") + >>> await glide_json.get(client, "doc", "$..a") b'[1,[5,0,0],true,"value",3.5,{"foo":"foo"}]' - >>> await json.clear(client, "doc", "$..a") + >>> await glide_json.clear(client, "doc", "$..a") 6 # All numeric, boolean, and string values across paths are cleared. - >>> await json.get(client, "doc", "$..a") + >>> await glide_json.get(client, "doc", "$..a") b'[0,[],false,"",0.0,{}]' """ args = ["JSON.CLEAR", key] @@ -673,21 +631,22 @@ async def debug_fields( If `path` doesn't exist, an error is raised. If `path` is not provided, it reports the total number of fields in the entire JSON document. If `key` doesn't exist, None is returned. + For more information about the returned type, see `TJsonUniversalResponse`. Examples: - >>> from glide import json - >>> await json.set(client, "k1", "$", '[1, 2.3, "foo", true, null, {}, [], {"a":1, "b":2}, [1,2,3]]') + >>> from glide import glide_json + >>> await glide_json.set(client, "k1", "$", '[1, 2.3, "foo", true, null, {}, [], {"a":1, "b":2}, [1,2,3]]') 'OK' - >>> await json.debug_fields(client, "k1", "$[*]") + >>> await glide_json.debug_fields(client, "k1", "$[*]") [1, 1, 1, 1, 1, 0, 0, 2, 3] - >>> await json.debug_fields(client, "k1", ".") + >>> await glide_json.debug_fields(client, "k1", ".") 14 # 9 top-level fields + 5 nested address fields - >>> await json.set(client, "k1", "$", '{"firstName":"John","lastName":"Smith","age":27,"weight":135.25,"isAlive":true,"address":{"street":"21 2nd Street","city":"New York","state":"NY","zipcode":"10021-3100"},"phoneNumbers":[{"type":"home","number":"212 555-1234"},{"type":"office","number":"646 555-4567"}],"children":[],"spouse":null}') + >>> await glide_json.set(client, "k1", "$", '{"firstName":"John","lastName":"Smith","age":27,"weight":135.25,"isAlive":true,"address":{"street":"21 2nd Street","city":"New York","state":"NY","zipcode":"10021-3100"},"phoneNumbers":[{"type":"home","number":"212 555-1234"},{"type":"office","number":"646 555-4567"}],"children":[],"spouse":null}') 'OK' - >>> await json.debug_fields(client, "k1") + >>> await glide_json.debug_fields(client, "k1") 19 - >>> await json.debug_fields(client, "k1", ".address") + >>> await glide_json.debug_fields(client, "k1", ".address") 4 """ args = ["JSON.DEBUG", "FIELDS", key] @@ -723,19 +682,20 @@ async def debug_memory( If `path` doesn't exist, an error is raised. If `path` is not provided, it reports the total memory usage in bytes in the entire JSON document. If `key` doesn't exist, None is returned. + For more information about the returned type, see `TJsonUniversalResponse`. Examples: - >>> from glide import json - >>> await json.set(client, "k1", "$", '[1, 2.3, "foo", true, null, {}, [], {"a":1, "b":2}, [1,2,3]]') + >>> from glide import glide_json + >>> await glide_json.set(client, "k1", "$", '[1, 2.3, "foo", true, null, {}, [], {"a":1, "b":2}, [1,2,3]]') 'OK' - >>> await json.debug_memory(client, "k1", "$[*]") + >>> await glide_json.debug_memory(client, "k1", "$[*]") [16, 16, 19, 16, 16, 16, 16, 66, 64] - >>> await json.set(client, "k1", "$", '{"firstName":"John","lastName":"Smith","age":27,"weight":135.25,"isAlive":true,"address":{"street":"21 2nd Street","city":"New York","state":"NY","zipcode":"10021-3100"},"phoneNumbers":[{"type":"home","number":"212 555-1234"},{"type":"office","number":"646 555-4567"}],"children":[],"spouse":null}') + >>> await glide_json.set(client, "k1", "$", '{"firstName":"John","lastName":"Smith","age":27,"weight":135.25,"isAlive":true,"address":{"street":"21 2nd Street","city":"New York","state":"NY","zipcode":"10021-3100"},"phoneNumbers":[{"type":"home","number":"212 555-1234"},{"type":"office","number":"646 555-4567"}],"children":[],"spouse":null}') 'OK' - >>> await json.debug_memory(client, "k1") + >>> await glide_json.debug_memory(client, "k1") 472 - >>> await json.debug_memory(client, "k1", ".phoneNumbers") + >>> await glide_json.debug_memory(client, "k1", ".phoneNumbers") 164 """ args = ["JSON.DEBUG", "MEMORY", key] @@ -766,14 +726,14 @@ async def delete( If `key` or `path` doesn't exist, returns 0. Examples: - >>> from glide import json - >>> await json.set(client, "doc", "$", '{"a": 1, "nested": {"a": 2, "b": 3}}') + >>> from glide import glide_json + >>> await glide_json.set(client, "doc", "$", '{"a": 1, "nested": {"a": 2, "b": 3}}') 'OK' # Indicates successful setting of the value at path '$' in the key stored at `doc`. - >>> await json.delete(client, "doc", "$..a") + >>> await glide_json.delete(client, "doc", "$..a") 2 # Indicates successful deletion of the specific values in the key stored at `doc`. - >>> await json.get(client, "doc", "$") + >>> await glide_json.get(client, "doc", "$") "[{\"nested\":{\"b\":3}}]" # Returns the value at path '$' in the JSON document stored at `doc`. - >>> await json.delete(client, "doc") + >>> await glide_json.delete(client, "doc") 1 # Deletes the entire JSON document stored at `doc`. """ @@ -801,14 +761,14 @@ async def forget( If `key` or `path` doesn't exist, returns 0. Examples: - >>> from glide import json - >>> await json.set(client, "doc", "$", '{"a": 1, "nested": {"a": 2, "b": 3}}') + >>> from glide import glide_json + >>> await glide_json.set(client, "doc", "$", '{"a": 1, "nested": {"a": 2, "b": 3}}') 'OK' # Indicates successful setting of the value at path '$' in the key stored at `doc`. - >>> await json.forget(client, "doc", "$..a") + >>> await glide_json.forget(client, "doc", "$..a") 2 # Indicates successful deletion of the specific values in the key stored at `doc`. - >>> await json.get(client, "doc", "$") + >>> await glide_json.get(client, "doc", "$") "[{\"nested\":{\"b\":3}}]" # Returns the value at path '$' in the JSON document stored at `doc`. - >>> await json.forget(client, "doc") + >>> await glide_json.forget(client, "doc") 1 # Deletes the entire JSON document stored at `doc`. """ @@ -818,6 +778,53 @@ async def forget( ) +async def mget( + client: TGlideClient, + keys: List[TEncodable], + path: TEncodable, +) -> List[Optional[bytes]]: + """ + Retrieves the JSON values at the specified `path` stored at multiple `keys`. + + Note: + In cluster mode, if keys in `keys` map to different hash slots, the command + will be split across these slots and executed separately for each. This means the command + is atomic only at the slot level. If one or more slot-specific requests fail, the entire + call will return the first encountered error, even though some requests may have succeeded + while others did not. If this behavior impacts your application logic, consider splitting + the request into sub-requests per slot to ensure atomicity. + + Args: + client (TGlideClient): The client to execute the command. + keys (List[TEncodable]): A list of keys for the JSON documents. + path (TEncodable): The path within the JSON documents. + + Returns: + List[Optional[bytes]]: + For JSONPath (`path` starts with `$`): + Returns a list of byte representations of the values found at the given path for each key. + If `path` does not exist within the key, the entry will be an empty array. + For legacy path (`path` doesn't starts with `$`): + Returns a list of byte representations of the values found at the given path for each key. + If `path` does not exist within the key, the entry will be None. + If a key doesn't exist, the corresponding list element will be None. + + + Examples: + >>> from glide import glide_json + >>> import json + >>> json_strs = await glide_json.mget(client, ["doc1", "doc2"], "$") + >>> [json.loads(js) for js in json_strs] # Parse JSON strings to Python data + [[{"a": 1.0, "b": 2}], [{"a": 2.0, "b": {"a": 3.0, "b" : 4.0}}]] # JSON objects retrieved from keys `doc1` and `doc2` + >>> await glide_json.mget(client, ["doc1", "doc2"], "$.a") + [b"[1.0]", b"[2.0]"] # Returns values at path '$.a' for the JSON documents stored at `doc1` and `doc2`. + >>> await glide_json.mget(client, ["doc1"], "$.non_existing_path") + [None] # Returns an empty array since the path '$.non_existing_path' does not exist in the JSON document stored at `doc1`. + """ + args = ["JSON.MGET"] + keys + [path] + return cast(TJsonResponse[Optional[bytes]], await client.custom_command(args)) + + async def numincrby( client: TGlideClient, key: TEncodable, @@ -847,12 +854,12 @@ async def numincrby( If the result is out of the range of 64-bit IEEE double, an error is raised. Examples: - >>> from glide import json - >>> await json.set(client, "doc", "$", '{"a": [], "b": [1], "c": [1, 2], "d": [1, 2, 3]}') + >>> from glide import glide_json + >>> await glide_json.set(client, "doc", "$", '{"a": [], "b": [1], "c": [1, 2], "d": [1, 2, 3]}') 'OK' - >>> await json.numincrby(client, "doc", "$.d[*]", 10) + >>> await glide_json.numincrby(client, "doc", "$.d[*]", 10) b'[11,12,13]' # Increment each element in `d` array by 10. - >>> await json.numincrby(client, "doc", ".c[1]", 10) + >>> await glide_json.numincrby(client, "doc", ".c[1]", 10) b'12' # Increment the second element in the `c` array by 10. """ args = ["JSON.NUMINCRBY", key, path, str(number)] @@ -889,12 +896,12 @@ async def nummultby( If the result is out of the range of 64-bit IEEE double, an error is raised. Examples: - >>> from glide import json - >>> await json.set(client, "doc", "$", '{"a": [], "b": [1], "c": [1, 2], "d": [1, 2, 3]}') + >>> from glide import glide_json + >>> await glide_json.set(client, "doc", "$", '{"a": [], "b": [1], "c": [1, 2], "d": [1, 2, 3]}') 'OK' - >>> await json.nummultby(client, "doc", "$.d[*]", 2) + >>> await glide_json.nummultby(client, "doc", "$.d[*]", 2) b'[2,4,6]' # Multiplies each element in the `d` array by 2. - >>> await json.nummultby(client, "doc", ".c[1]", 2) + >>> await glide_json.nummultby(client, "doc", ".c[1]", 2) b'4' # Multiplies the second element in the `c` array by 2. """ args = ["JSON.NUMMULTBY", key, path, str(number)] @@ -926,23 +933,24 @@ async def objlen( If multiple paths match, the length of the first object match is returned. If the JSON value at `path` is not an object or if `path` doesn't exist, an error is raised. If `key` doesn't exist, None is returned. + For more information about the returned type, see `TJsonResponse`. Examples: - >>> from glide import json - >>> await json.set(client, "doc", "$", '{"a": 1.0, "b": {"a": {"x": 1, "y": 2}, "b": 2.5, "c": true}}') + >>> from glide import glide_json + >>> await glide_json.set(client, "doc", "$", '{"a": 1.0, "b": {"a": {"x": 1, "y": 2}, "b": 2.5, "c": true}}') b'OK' # Indicates successful setting of the value at the root path '$' in the key `doc`. - >>> await json.objlen(client, "doc", "$") + >>> await glide_json.objlen(client, "doc", "$") [2] # Returns the number of key-value pairs at the root object, which has 2 keys: 'a' and 'b'. - >>> await json.objlen(client, "doc", ".") + >>> await glide_json.objlen(client, "doc", ".") 2 # Returns the number of key-value pairs for the object matching the path '.', which has 2 keys: 'a' and 'b'. - >>> await json.objlen(client, "doc", "$.b") + >>> await glide_json.objlen(client, "doc", "$.b") [3] # Returns the length of the object at path '$.b', which has 3 keys: 'a', 'b', and 'c'. - >>> await json.objlen(client, "doc", ".b") + >>> await glide_json.objlen(client, "doc", ".b") 3 # Returns the length of the nested object at path '.b', which has 3 keys. - >>> await json.objlen(client, "doc", "$..a") + >>> await glide_json.objlen(client, "doc", "$..a") [None, 2] - >>> await json.objlen(client, "doc") + >>> await glide_json.objlen(client, "doc") 2 # Returns the number of key-value pairs for the object matching the path '.', which has 2 keys: 'a' and 'b'. """ args = ["JSON.OBJLEN", key] @@ -980,18 +988,19 @@ async def objkeys( If a value matching the path is not an object, an error is raised. If `path` doesn't exist, None is returned. If `key` doesn't exist, None is returned. + For more information about the returned type, see `TJsonUniversalResponse`. Examples: - >>> from glide import json - >>> await json.set(client, "doc", "$", '{"a": 1.0, "b": {"a": {"x": 1, "y": 2}, "b": 2.5, "c": true}}') + >>> from glide import glide_json + >>> await glide_json.set(client, "doc", "$", '{"a": 1.0, "b": {"a": {"x": 1, "y": 2}, "b": 2.5, "c": true}}') b'OK' # Indicates successful setting of the value at the root path '$' in the key `doc`. - >>> await json.objkeys(client, "doc", "$") + >>> await glide_json.objkeys(client, "doc", "$") [[b"a", b"b"]] # Returns a list of arrays containing the key names for objects matching the path '$'. - >>> await json.objkeys(client, "doc", ".") + >>> await glide_json.objkeys(client, "doc", ".") [b"a", b"b"] # Returns key names for the object matching the path '.' as it is the only match. - >>> await json.objkeys(client, "doc", "$.b") + >>> await glide_json.objkeys(client, "doc", "$.b") [[b"a", b"b", b"c"]] # Returns key names as a nested list for objects matching the JSONPath '$.b'. - >>> await json.objkeys(client, "doc", ".b") + >>> await glide_json.objkeys(client, "doc", ".b") [b"a", b"b", b"c"] # Returns key names for the nested object at path '.b'. """ args = ["JSON.OBJKEYS", key] @@ -1036,14 +1045,15 @@ async def resp( If multiple paths match, the value of the first JSON value match is returned. If `path` doesn't exist, an error is raised. If `key` doesn't exist, an None is returned. + For more information about the returned type, see `TJsonUniversalResponse`. Examples: - >>> from glide import json - >>> await json.set(client, "doc", "$", '{"a": [1, 2, 3], "b": {"a": [1, 2], "c": {"a": 42}}}') + >>> from glide import glide_json + >>> await glide_json.set(client, "doc", "$", '{"a": [1, 2, 3], "b": {"a": [1, 2], "c": {"a": 42}}}') 'OK' - >>> await json.resp(client, "doc", "$..a") + >>> await glide_json.resp(client, "doc", "$..a") [[b"[", 1, 2, 3],[b"[", 1, 2],42] - >>> await json.resp(client, "doc", "..a") + >>> await glide_json.resp(client, "doc", "..a") [b"[", 1, 2, 3] """ args = ["JSON.RESP", key] @@ -1087,15 +1097,15 @@ async def strappend( For more information about the returned type, see `TJsonResponse`. Examples: - >>> from glide import json - >>> import json as jsonpy - >>> await json.set(client, "doc", "$", jsonpy.dumps({"a":"foo", "nested": {"a": "hello"}, "nested2": {"a": 31}})) + >>> from glide import glide_json + >>> import json + >>> await glide_json.set(client, "doc", "$", json.dumps({"a":"foo", "nested": {"a": "hello"}, "nested2": {"a": 31}})) 'OK' - >>> await json.strappend(client, "doc", jsonpy.dumps("baz"), "$..a") + >>> await glide_json.strappend(client, "doc", json.dumps("baz"), "$..a") [6, 8, None] # The new length of the string values at path '$..a' in the key stored at `doc` after the append operation. - >>> await json.strappend(client, "doc", '"foo"', "nested.a") + >>> await glide_json.strappend(client, "doc", '"foo"', "nested.a") 11 # The length of the string value after appending "foo" to the string at path 'nested.array' in the key stored at `doc`. - >>> jsonpy.loads(await json.get(client, jsonpy.dumps("doc"), "$")) + >>> json.loads(await glide_json.get(client, json.dumps("doc"), "$")) [{"a":"foobaz", "nested": {"a": "hellobazfoo"}, "nested2": {"a": 31}}] # The updated JSON value in the key stored at `doc`. """ @@ -1133,17 +1143,17 @@ async def strlen( For more information about the returned type, see `TJsonResponse`. Examples: - >>> from glide import json - >>> import jsonpy - >>> await json.set(client, "doc", "$", jsonpy.dumps({"a":"foo", "nested": {"a": "hello"}, "nested2": {"a": 31}})) + >>> from glide import glide_json + >>> import json + >>> await glide_json.set(client, "doc", "$", json.dumps({"a":"foo", "nested": {"a": "hello"}, "nested2": {"a": 31}})) 'OK' - >>> await json.strlen(client, "doc", "$..a") + >>> await glide_json.strlen(client, "doc", "$..a") [3, 5, None] # The length of the string values at path '$..a' in the key stored at `doc`. - >>> await json.strlen(client, "doc", "nested.a") + >>> await glide_json.strlen(client, "doc", "nested.a") 5 # The length of the JSON value at path 'nested.a' in the key stored at `doc`. - >>> await json.strlen(client, "doc", "$") + >>> await glide_json.strlen(client, "doc", "$") [None] # Returns an array with None since the value at root path does in the JSON document stored at `doc` is not a string. - >>> await json.strlen(client, "non_existing_key", ".") + >>> await glide_json.strlen(client, "non_existing_key", ".") None # `key` doesn't exist. """ @@ -1181,15 +1191,15 @@ async def toggle( For more information about the returned type, see `TJsonResponse`. Examples: - >>> from glide import json - >>> import json as jsonpy - >>> await json.set(client, "doc", "$", jsonpy.dumps({"bool": True, "nested": {"bool": False, "nested": {"bool": 10}}})) + >>> from glide import glide_json + >>> import json + >>> await glide_json.set(client, "doc", "$", json.dumps({"bool": True, "nested": {"bool": False, "nested": {"bool": 10}}})) 'OK' - >>> await json.toggle(client, "doc", "$.bool") + >>> await glide_json.toggle(client, "doc", "$.bool") [False, True, None] # Indicates successful toggling of the Boolean values at path '$.bool' in the key stored at `doc`. - >>> await json.toggle(client, "doc", "bool") + >>> await glide_json.toggle(client, "doc", "bool") True # Indicates successful toggling of the Boolean value at path 'bool' in the key stored at `doc`. - >>> jsonpy.loads(await json.get(client, "doc", "$")) + >>> json.loads(await glide_json.get(client, "doc", "$")) [{"bool": True, "nested": {"bool": True, "nested": {"bool": 10}}}] # The updated JSON value in the key stored at `doc`. """ @@ -1222,16 +1232,17 @@ async def type( If multiple paths match, the type of the first JSON value match is returned. If `path` doesn't exist, None will be returned. If `key` doesn't exist, None is returned. + For more information about the returned type, see `TJsonUniversalResponse`. Examples: - >>> from glide import json - >>> await json.set(client, "doc", "$", '{"a": 1, "nested": {"a": 2, "b": 3}}') + >>> from glide import glide_json + >>> await glide_json.set(client, "doc", "$", '{"a": 1, "nested": {"a": 2, "b": 3}}') 'OK' - >>> await json.type(client, "doc", "$.nested") + >>> await glide_json.type(client, "doc", "$.nested") [b'object'] # Indicates the type of the value at path '$.nested' in the key stored at `doc`. - >>> await json.type(client, "doc", "$.nested.a") + >>> await glide_json.type(client, "doc", "$.nested.a") [b'integer'] # Indicates the type of the value at path '$.nested.a' in the key stored at `doc`. - >>> await json.type(client, "doc", "$[*]") + >>> await glide_json.type(client, "doc", "$[*]") [b'integer', b'object'] # Array of types in all top level elements. """ args = ["JSON.TYPE", key] diff --git a/python/python/tests/tests_server_modules/test_ft.py b/python/python/tests/tests_server_modules/test_ft.py index ebacb209f0..ee2b9416ee 100644 --- a/python/python/tests/tests_server_modules/test_ft.py +++ b/python/python/tests/tests_server_modules/test_ft.py @@ -8,7 +8,7 @@ import pytest from glide.async_commands.command_args import OrderBy from glide.async_commands.server_modules import ft -from glide.async_commands.server_modules import json as GlideJson +from glide.async_commands.server_modules import glide_json as GlideJson from glide.async_commands.server_modules.ft_options.ft_aggregate_options import ( FtAggregateApply, FtAggregateGroupBy, diff --git a/python/python/tests/tests_server_modules/test_json.py b/python/python/tests/tests_server_modules/test_json.py index eba62faa17..85657914de 100644 --- a/python/python/tests/tests_server_modules/test_json.py +++ b/python/python/tests/tests_server_modules/test_json.py @@ -7,8 +7,8 @@ import pytest from glide.async_commands.core import ConditionalChange, InfoSection -from glide.async_commands.server_modules import json -from glide.async_commands.server_modules.json import ( +from glide.async_commands.server_modules import glide_json as json +from glide.async_commands.server_modules.glide_json import ( JsonArrIndexOptions, JsonArrPopOptions, JsonGetOptions,