Skip to content

Commit c1f8e73

Browse files
committed
[FLINK-22419] Wait unlimited in RpcEndpointTest.testCallAsyncTimeout
1 parent 9025168 commit c1f8e73

File tree

1 file changed

+3
-1
lines changed

1 file changed

+3
-1
lines changed

flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcEndpointTest.java

+3-1
Original file line numberDiff line numberDiff line change
@@ -376,13 +376,14 @@ public void testCallAsyncTimeout()
376376
throws InterruptedException, ExecutionException, TimeoutException {
377377
final RpcEndpoint endpoint = new BaseEndpoint(rpcService);
378378
final Time timeout = Time.milliseconds(100);
379+
CountDownLatch latch = new CountDownLatch(1);
379380
try {
380381
endpoint.start();
381382
final CompletableFuture<Throwable> throwableFuture =
382383
endpoint.callAsync(
383384
() -> {
384385
endpoint.validateRunsInMainThread();
385-
TimeUnit.MILLISECONDS.sleep(timeout.toMilliseconds() * 2);
386+
latch.await();
386387
return 12345;
387388
},
388389
timeout)
@@ -392,6 +393,7 @@ public void testCallAsyncTimeout()
392393
assertNotNull(throwable);
393394
assertThat(throwable, instanceOf(TimeoutException.class));
394395
} finally {
396+
latch.countDown();
395397
RpcUtils.terminateRpcEndpoint(endpoint, TIMEOUT);
396398
}
397399
}

0 commit comments

Comments
 (0)