Skip to content

Commit 1779dcc

Browse files
shanickyMrCroxx
andauthored
refactor: refactor test function in isolation.rs to enhance result validation (#21102)
Signed-off-by: Shanicky Chen <[email protected]> Signed-off-by: MrCroxx <[email protected]> Co-authored-by: MrCroxx <[email protected]>
1 parent 324f1d8 commit 1779dcc

File tree

1 file changed

+26
-30
lines changed
  • src/tests/simulation/tests/integration_tests/scale

1 file changed

+26
-30
lines changed

src/tests/simulation/tests/integration_tests/scale/isolation.rs

+26-30
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,6 @@ use std::time::Duration;
1919
use anyhow::Result;
2020
use itertools::Itertools;
2121
use risingwave_simulation::cluster::{Cluster, Configuration, Session};
22-
use risingwave_simulation::utils::AssertResult;
2322
use tokio::time::sleep;
2423

2524
const DATABASE_RECOVERY_START: &str = "DATABASE_RECOVERY_START";
@@ -182,19 +181,18 @@ async fn test_isolation_simple_two_databases_join() -> Result<()> {
182181
.run("insert into t2 select * from generate_series(1, 50);")
183182
.await?;
184183

185-
session
186-
.run("select count(*) from group1.public.t1;")
187-
.await?
188-
.assert_result_eq("100");
184+
wait_until(
185+
&mut session,
186+
"select count(*) from group1.public.t1;",
187+
"100",
188+
)
189+
.await?;
189190

190191
session
191192
.run("create materialized view mv_join as select t2.v as v from group1.public.t1 join t2 on t1.v = t2.v;")
192193
.await?;
193194

194-
session
195-
.run("select count(*) from mv_join;")
196-
.await?
197-
.assert_result_eq("50");
195+
wait_until(&mut session, "select count(*) from mv_join;", "50").await?;
198196

199197
cluster.simple_kill_nodes(["compute-1"]).await;
200198

@@ -208,15 +206,8 @@ async fn test_isolation_simple_two_databases_join() -> Result<()> {
208206
.run("insert into t2 select * from generate_series(51, 120)")
209207
.await?;
210208

211-
session
212-
.run("select max(v) from t2")
213-
.await?
214-
.assert_result_eq("120");
215-
216-
session
217-
.run("select count(*) from mv_join;")
218-
.await?
219-
.assert_result_eq("100");
209+
wait_until(&mut session, "select max(v) from t2", "120").await?;
210+
wait_until(&mut session, "select count(*) from mv_join;", "100").await?;
220211

221212
cluster.simple_restart_nodes(["compute-1"]).await;
222213

@@ -232,10 +223,7 @@ async fn test_isolation_simple_two_databases_join() -> Result<()> {
232223
// flush is only oriented to the current database, so flush is required here
233224
session.run("flush").await?;
234225

235-
session
236-
.run("select count(*) from mv_join;")
237-
.await?
238-
.assert_result_eq("110");
226+
wait_until(&mut session, "select count(*) from mv_join;", "110").await?;
239227

240228
let mut database_recovery_events = database_recovery_events(&mut session).await?;
241229

@@ -297,10 +285,7 @@ async fn test_isolation_simple_two_databases_join_in_other() -> Result<()> {
297285
.run("create materialized view mv_join as select t2.v as v from group1.public.t1 join group2.public.t2 on t1.v = t2.v;")
298286
.await?;
299287

300-
session
301-
.run("select count(*) from mv_join;")
302-
.await?
303-
.assert_result_eq("50");
288+
wait_until(&mut session, "select count(*) from mv_join;", "50").await?;
304289

305290
cluster.simple_kill_nodes(["compute-1", "compute-3"]).await;
306291

@@ -335,10 +320,7 @@ async fn test_isolation_simple_two_databases_join_in_other() -> Result<()> {
335320

336321
session.run("use group3").await?;
337322

338-
session
339-
.run("select count(*) from mv_join;")
340-
.await?
341-
.assert_result_eq("110");
323+
wait_until(&mut session, "select count(*) from mv_join;", "110").await?;
342324

343325
let mut database_recovery_events = database_recovery_events(&mut session).await?;
344326

@@ -371,6 +353,20 @@ async fn test_isolation_simple_two_databases_join_in_other() -> Result<()> {
371353
Ok(())
372354
}
373355

356+
async fn wait_until(session: &mut Session, sql: &str, target: &str) -> Result<()> {
357+
tokio::time::timeout(Duration::from_secs(100), async {
358+
loop {
359+
if session.run(sql).await.unwrap() == target {
360+
return;
361+
}
362+
sleep(Duration::from_secs(1)).await;
363+
}
364+
})
365+
.await?;
366+
367+
Ok(())
368+
}
369+
374370
async fn prepare_isolation_env() -> Result<(Cluster, Session)> {
375371
let mut config = Configuration::for_auto_parallelism(MAX_HEARTBEAT_INTERVAL_SEC, true);
376372

0 commit comments

Comments
 (0)