Skip to content

Commit 333f591

Browse files
jaymellsvix-james
authored andcommitted
Refactor redis helper functions into methods
This is slightly less error-prone, since the caller no longer has to specify whether to create a pooled or unpooled manager, and it's also less repetitive.
1 parent d89d063 commit 333f591

File tree

6 files changed

+93
-145
lines changed

6 files changed

+93
-145
lines changed

server/svix-server/src/core/cache/redis.rs

Lines changed: 6 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -81,7 +81,7 @@ mod tests {
8181
super::{kv_def, string_kv_def, CacheValue},
8282
*,
8383
};
84-
use crate::cfg::CacheType;
84+
use crate::cfg::Configuration;
8585

8686
// Test structures
8787

@@ -112,14 +112,8 @@ mod tests {
112112
}
113113
}
114114

115-
async fn get_pool(redis_dsn: &str, cfg: &crate::cfg::Configuration) -> RedisManager {
116-
match cfg.cache_type {
117-
CacheType::RedisCluster => crate::redis::new_redis_clustered_unpooled(redis_dsn).await,
118-
CacheType::Redis => crate::redis::new_redis_unpooled(redis_dsn).await,
119-
_ => panic!(
120-
"This test should only be run when redis is configured as the cache provider"
121-
),
122-
}
115+
async fn get_pool(cfg: &Configuration) -> RedisManager {
116+
RedisManager::from_cache_backend(&cfg.cache_backend()).await
123117
}
124118

125119
#[tokio::test]
@@ -129,7 +123,7 @@ mod tests {
129123
dotenvy::dotenv().ok();
130124
let cfg = crate::cfg::load().unwrap();
131125

132-
let redis_pool = get_pool(cfg.redis_dsn.as_ref().unwrap().as_str(), &cfg).await;
126+
let redis_pool = get_pool(&cfg).await;
133127
let cache = super::new(redis_pool);
134128

135129
let (first_key, first_val_a, first_val_b) =
@@ -206,7 +200,7 @@ mod tests {
206200
dotenvy::dotenv().ok();
207201
let cfg = crate::cfg::load().unwrap();
208202

209-
let redis_pool = get_pool(cfg.redis_dsn.as_ref().unwrap().as_str(), &cfg).await;
203+
let redis_pool = get_pool(&cfg).await;
210204
let cache = super::new(redis_pool);
211205

212206
let key = TestKeyA::new("key".to_owned());
@@ -225,7 +219,7 @@ mod tests {
225219
dotenvy::dotenv().ok();
226220
let cfg = crate::cfg::load().unwrap();
227221

228-
let redis_pool = get_pool(cfg.redis_dsn.as_ref().unwrap().as_str(), &cfg).await;
222+
let redis_pool = get_pool(&cfg).await;
229223
let cache = super::new(redis_pool);
230224

231225
let key = TestKeyA::new("nx_status_test_key".to_owned());

server/svix-server/src/lib.rs

Lines changed: 5 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -16,6 +16,7 @@ use cfg::ConfigurationInner;
1616
use opentelemetry_otlp::WithExportConfig;
1717
use opentelemetry_sdk::runtime::Tokio;
1818
use queue::TaskQueueProducer;
19+
use redis::RedisManager;
1920
use sea_orm::DatabaseConnection;
2021
use sentry::integrations::tracing::EventFilter;
2122
use tower::layer::layer_fn;
@@ -106,15 +107,12 @@ pub async fn run_with_prefix(
106107
tracing::debug!("DB: Started");
107108

108109
tracing::debug!("Cache: Initializing {:?}", cfg.cache_type);
109-
let cache = match cfg.cache_backend() {
110+
let cache_backend = cfg.cache_backend();
111+
let cache = match &cache_backend {
110112
CacheBackend::None => cache::none::new(),
111113
CacheBackend::Memory => cache::memory::new(),
112-
CacheBackend::Redis(dsn) => {
113-
let mgr = crate::redis::new_redis_unpooled(dsn).await;
114-
cache::redis::new(mgr)
115-
}
116-
CacheBackend::RedisCluster(dsn) => {
117-
let mgr = crate::redis::new_redis_clustered_unpooled(dsn).await;
114+
CacheBackend::Redis(_) | CacheBackend::RedisCluster(_) => {
115+
let mgr = RedisManager::from_cache_backend(&cache_backend).await;
118116
cache::redis::new(mgr)
119117
}
120118
};

server/svix-server/src/queue/redis.rs

Lines changed: 4 additions & 17 deletions
Original file line numberDiff line numberDiff line change
@@ -133,10 +133,8 @@ async fn new_pair_inner(
133133
// - `queue::new_pair` if the queue type is redis and a DSN is set
134134
// - redis tests that only makes sense to run with the DSN set
135135
let dsn = cfg.redis_dsn.as_deref().unwrap();
136-
let pool = match &cfg.queue_type {
137-
QueueType::RedisCluster => crate::redis::new_redis_clustered_pooled(dsn, cfg).await,
138-
_ => crate::redis::new_redis_pooled(dsn, cfg).await,
139-
};
136+
let pool =
137+
RedisManager::from_queue_backend(&cfg.queue_backend(), cfg.redis_pool_max_size).await;
140138

141139
// Create the stream and consumer group for the MAIN queue should it not already exist. The
142140
// consumer is created automatically upon use so it does not have to be created here.
@@ -355,25 +353,14 @@ pub mod tests {
355353

356354
use super::{migrate_list, migrate_list_to_stream, migrate_sset, new_pair_inner};
357355
use crate::{
358-
cfg::{Configuration, QueueType},
356+
cfg::Configuration,
359357
core::types::{ApplicationId, EndpointId, MessageAttemptTriggerType, MessageId},
360358
queue::{MessageTask, QueueTask, TaskQueueConsumer, TaskQueueProducer},
361359
redis::RedisManager,
362360
};
363361

364362
async fn get_pool(cfg: &Configuration) -> RedisManager {
365-
match cfg.queue_type {
366-
QueueType::RedisCluster => {
367-
crate::redis::new_redis_clustered_pooled(cfg.redis_dsn.as_deref().unwrap(), cfg)
368-
.await
369-
}
370-
QueueType::Redis => {
371-
crate::redis::new_redis_pooled(cfg.redis_dsn.as_deref().unwrap(), cfg).await
372-
}
373-
_ => {
374-
panic!("This test should only be run when redis is configured as the queue backend")
375-
}
376-
}
363+
RedisManager::from_queue_backend(&cfg.queue_backend(), cfg.redis_pool_max_size).await
377364
}
378365

379366
#[tokio::test]

server/svix-server/src/redis/mod.rs

Lines changed: 70 additions & 85 deletions
Original file line numberDiff line numberDiff line change
@@ -7,7 +7,7 @@ use bb8_redis::RedisConnectionManager;
77
use redis::{FromRedisValue, RedisError, RedisResult};
88

99
pub use self::cluster::RedisClusterConnectionManager;
10-
use crate::cfg::Configuration;
10+
use crate::cfg::{CacheBackend, QueueBackend};
1111

1212
pub const REDIS_CONN_TIMEOUT: Duration = Duration::from_secs(2);
1313

@@ -20,6 +20,73 @@ pub enum RedisManager {
2020
}
2121

2222
impl RedisManager {
23+
async fn new_pooled(dsn: &str, clustered: bool, max_conns: u16) -> Self {
24+
if clustered {
25+
let mgr = RedisClusterConnectionManager::new(dsn)
26+
.expect("Error initializing redis cluster client");
27+
let pool = bb8::Pool::builder()
28+
.max_size(max_conns.into())
29+
.build(mgr)
30+
.await
31+
.expect("Error initializing redis cluster connection pool");
32+
let pool = ClusteredRedisPool { pool };
33+
RedisManager::Clustered(pool)
34+
} else {
35+
let mgr = RedisConnectionManager::new(dsn).expect("Error initializing redis client");
36+
let pool = bb8::Pool::builder()
37+
.max_size(max_conns.into())
38+
.build(mgr)
39+
.await
40+
.expect("Error initializing redis connection pool");
41+
let pool = NonClusteredRedisPool { pool };
42+
RedisManager::NonClustered(pool)
43+
}
44+
}
45+
46+
async fn new_unpooled(dsn: &str, clustered: bool) -> Self {
47+
if clustered {
48+
let cli = redis::cluster::ClusterClient::builder(vec![dsn])
49+
.retries(1)
50+
.connection_timeout(REDIS_CONN_TIMEOUT)
51+
.build()
52+
.expect("Error initializing redis-unpooled cluster client");
53+
let con = cli
54+
.get_async_connection()
55+
.await
56+
.expect("Failed to get redis-cluster-unpooled connection");
57+
RedisManager::ClusteredUnpooled(ClusteredRedisUnpooled { con })
58+
} else {
59+
let cli = redis::Client::open(dsn).expect("Error initializing redis unpooled client");
60+
let con = redis::aio::ConnectionManager::new_with_backoff_and_timeouts(
61+
cli,
62+
2,
63+
100,
64+
1,
65+
Duration::MAX,
66+
REDIS_CONN_TIMEOUT,
67+
)
68+
.await
69+
.expect("Failed to get redis-unpooled connection manager");
70+
RedisManager::NonClusteredUnpooled(NonClusteredRedisUnpooled { con })
71+
}
72+
}
73+
74+
pub async fn from_cache_backend(cache_backend: &CacheBackend<'_>) -> Self {
75+
match cache_backend {
76+
CacheBackend::Redis(dsn) => Self::new_unpooled(dsn, false).await,
77+
CacheBackend::RedisCluster(dsn) => Self::new_unpooled(dsn, true).await,
78+
_ => panic!("Queue type not supported with redis"),
79+
}
80+
}
81+
82+
pub async fn from_queue_backend(queue_backend: &QueueBackend<'_>, max_conns: u16) -> Self {
83+
match queue_backend {
84+
QueueBackend::Redis(dsn) => Self::new_pooled(dsn, false, max_conns).await,
85+
QueueBackend::RedisCluster(dsn) => Self::new_pooled(dsn, true, max_conns).await,
86+
_ => panic!("Queue type not supported with redis"),
87+
}
88+
}
89+
2390
pub async fn get(&self) -> Result<PooledConnection<'_>, RunError<RedisError>> {
2491
match self {
2592
Self::Clustered(pool) => pool.get().await,
@@ -230,93 +297,11 @@ impl ClusteredUnpooledConnection {
230297
}
231298
}
232299

233-
async fn new_redis_pool_helper(
234-
redis_dsn: &str,
235-
clustered: bool,
236-
max_connections: u16,
237-
) -> RedisManager {
238-
if clustered {
239-
let mgr = RedisClusterConnectionManager::new(redis_dsn)
240-
.expect("Error initializing redis cluster client");
241-
let pool = bb8::Pool::builder()
242-
.max_size(max_connections.into())
243-
.build(mgr)
244-
.await
245-
.expect("Error initializing redis cluster connection pool");
246-
let pool = ClusteredRedisPool { pool };
247-
RedisManager::Clustered(pool)
248-
} else {
249-
let mgr = RedisConnectionManager::new(redis_dsn).expect("Error initializing redis client");
250-
let pool = bb8::Pool::builder()
251-
.max_size(max_connections.into())
252-
.build(mgr)
253-
.await
254-
.expect("Error initializing redis connection pool");
255-
let pool = NonClusteredRedisPool { pool };
256-
RedisManager::NonClustered(pool)
257-
}
258-
}
259-
260-
async fn new_redis_unpooled_helper(redis_dsn: &str, clustered: bool) -> RedisManager {
261-
if clustered {
262-
let cli = redis::cluster::ClusterClient::builder(vec![redis_dsn])
263-
.retries(1)
264-
.connection_timeout(REDIS_CONN_TIMEOUT)
265-
.build()
266-
.expect("Error initializing redis-unpooled cluster client");
267-
let con = cli
268-
.get_async_connection()
269-
.await
270-
.expect("Failed to get redis-cluster-unpooled connection");
271-
RedisManager::ClusteredUnpooled(ClusteredRedisUnpooled { con })
272-
} else {
273-
let cli = redis::Client::open(redis_dsn).expect("Error initializing redis unpooled client");
274-
let con = redis::aio::ConnectionManager::new_with_backoff_and_timeouts(
275-
cli,
276-
2,
277-
100,
278-
1,
279-
Duration::MAX,
280-
REDIS_CONN_TIMEOUT,
281-
)
282-
.await
283-
.expect("Failed to get redis-unpooled connection manager");
284-
RedisManager::NonClusteredUnpooled(NonClusteredRedisUnpooled { con })
285-
}
286-
}
287-
288-
pub async fn new_redis_clustered_pooled(redis_dsn: &str, cfg: &Configuration) -> RedisManager {
289-
new_redis_pool_helper(redis_dsn, true, cfg.redis_pool_max_size).await
290-
}
291-
292-
pub async fn new_redis_clustered_unpooled(redis_dsn: &str) -> RedisManager {
293-
new_redis_unpooled_helper(redis_dsn, true).await
294-
}
295-
296-
pub async fn new_redis_pooled(redis_dsn: &str, cfg: &Configuration) -> RedisManager {
297-
new_redis_pool_helper(redis_dsn, false, cfg.redis_pool_max_size).await
298-
}
299-
300-
pub async fn new_redis_unpooled(redis_dsn: &str) -> RedisManager {
301-
new_redis_unpooled_helper(redis_dsn, false).await
302-
}
303-
304300
#[cfg(test)]
305301
mod tests {
306302
use redis::AsyncCommands;
307303

308304
use super::RedisManager;
309-
use crate::cfg::{CacheType, Configuration};
310-
311-
async fn get_pool(redis_dsn: &str, cfg: &Configuration) -> RedisManager {
312-
match cfg.cache_type {
313-
CacheType::RedisCluster => super::new_redis_clustered_unpooled(redis_dsn).await,
314-
CacheType::Redis => super::new_redis_unpooled(redis_dsn).await,
315-
_ => panic!(
316-
"This test should only be run when redis is configured as the cache provider"
317-
),
318-
}
319-
}
320305

321306
// Ensure basic set/get works -- should test sharding as well:
322307
#[tokio::test]
@@ -326,8 +311,8 @@ mod tests {
326311
dotenvy::dotenv().ok();
327312
let cfg = crate::cfg::load().unwrap();
328313

329-
let pool = get_pool(cfg.redis_dsn.as_ref().unwrap().as_str(), &cfg).await;
330-
let mut conn = pool.get().await.unwrap();
314+
let mgr = RedisManager::from_cache_backend(&cfg.cache_backend()).await;
315+
let mut conn = mgr.get().await.unwrap();
331316

332317
for (val, key) in "abcdefghijklmnopqrstuvwxyz".chars().enumerate() {
333318
let key = key.to_string();

server/svix-server/tests/it/message_app.rs

Lines changed: 5 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -12,7 +12,7 @@ use svix_server::{
1212
message_app::AppEndpointKey,
1313
types::{BaseId, OrganizationId},
1414
},
15-
redis::{new_redis_clustered_unpooled, new_redis_unpooled},
15+
redis::RedisManager,
1616
};
1717

1818
use crate::utils::{
@@ -65,12 +65,8 @@ async fn test_app_deletion() {
6565
// Delete the cached [`CreateMessageApp`] here instead of waiting 30s for it to expire
6666
let cache = match cfg.cache_backend() {
6767
CacheBackend::None => cache::none::new(),
68-
CacheBackend::Redis(dsn) => {
69-
let mgr = new_redis_unpooled(dsn).await;
70-
cache::redis::new(mgr)
71-
}
72-
CacheBackend::RedisCluster(dsn) => {
73-
let mgr = new_redis_clustered_unpooled(dsn).await;
68+
CacheBackend::Redis(_) | CacheBackend::RedisCluster(_) => {
69+
let mgr = RedisManager::from_cache_backend(&cfg.cache_backend()).await;
7470
cache::redis::new(mgr)
7571
}
7672

@@ -149,12 +145,8 @@ async fn test_endp_deletion() {
149145
// Delete the cached [`CreateMessageApp`] here instead of waiting 30s for it to expire
150146
let cache = match cfg.cache_backend() {
151147
CacheBackend::None => cache::none::new(),
152-
CacheBackend::Redis(dsn) => {
153-
let mgr = new_redis_unpooled(dsn).await;
154-
cache::redis::new(mgr)
155-
}
156-
CacheBackend::RedisCluster(dsn) => {
157-
let mgr = new_redis_clustered_unpooled(dsn).await;
148+
CacheBackend::Redis(_) | CacheBackend::RedisCluster(_) => {
149+
let mgr = RedisManager::from_cache_backend(&cfg.cache_backend()).await;
158150
cache::redis::new(mgr)
159151
}
160152

server/svix-server/tests/it/redis_queue.rs

Lines changed: 3 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -8,25 +8,17 @@ use std::{str::FromStr, time::Duration};
88

99
use redis::AsyncCommands as _;
1010
use svix_server::{
11-
cfg::{Configuration, QueueType},
11+
cfg::Configuration,
1212
core::types::{ApplicationId, EndpointId, MessageAttemptTriggerType, MessageId},
1313
queue::{
1414
new_pair, MessageTask, QueueTask, TaskQueueConsumer, TaskQueueDelivery, TaskQueueProducer,
1515
},
16-
redis::{new_redis_clustered_pooled, new_redis_pooled, RedisManager},
16+
redis::RedisManager,
1717
};
1818

1919
// TODO: Don't copy this from the Redis queue test directly, place the fn somewhere both can access
2020
async fn get_pool(cfg: &Configuration) -> RedisManager {
21-
match cfg.queue_type {
22-
QueueType::RedisCluster => {
23-
new_redis_clustered_pooled(cfg.redis_dsn.as_deref().unwrap(), cfg).await
24-
}
25-
QueueType::Redis => new_redis_pooled(cfg.redis_dsn.as_deref().unwrap(), cfg).await,
26-
_ => {
27-
panic!("This test should only be run when redis is configured as the queue backend")
28-
}
29-
}
21+
RedisManager::from_queue_backend(&cfg.queue_backend(), cfg.redis_pool_max_size).await
3022
}
3123

3224
fn task_queue_delivery_to_u16(tqd: &TaskQueueDelivery) -> u16 {

0 commit comments

Comments
 (0)