/
RpcEndpointTest.java
516 lines (428 loc) · 19.4 KB
/
RpcEndpointTest.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.runtime.rpc;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.testutils.ManuallyTriggeredScheduledExecutorService;
import org.apache.flink.util.TestLoggerExtension;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.ExtendWith;
import java.time.Duration;
import java.util.concurrent.Callable;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.function.BiConsumer;
import java.util.function.BiFunction;
import java.util.function.Consumer;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail;
/** Tests for the RpcEndpoint, its self gateways and MainThreadExecutor scheduling command. */
@ExtendWith(TestLoggerExtension.class)
public class RpcEndpointTest {
private static RpcService rpcService = null;
@BeforeAll
public static void setup() throws Exception {
rpcService = RpcSystem.load().localServiceBuilder(new Configuration()).createAndStart();
}
@AfterAll
public static void teardown() throws Exception {
rpcService.closeAsync().get();
}
/**
* Tests that we can obtain the self gateway from a RpcEndpoint and can interact with it via the
* self gateway.
*/
@Test
public void testSelfGateway() throws Exception {
int expectedValue = 1337;
BaseEndpoint baseEndpoint = new BaseEndpoint(rpcService, expectedValue);
try {
baseEndpoint.start();
BaseGateway baseGateway = baseEndpoint.getSelfGateway(BaseGateway.class);
CompletableFuture<Integer> foobar = baseGateway.foobar();
assertEquals(Integer.valueOf(expectedValue), foobar.get());
} finally {
RpcUtils.terminateRpcEndpoint(baseEndpoint);
baseEndpoint.validateResourceClosed();
}
}
/**
* Tests that we cannot accidentally obtain a wrong self gateway type which is not implemented
* by the RpcEndpoint.
*/
@Test
public void testWrongSelfGateway() {
assertThrows(
RuntimeException.class,
() -> {
int expectedValue = 1337;
BaseEndpoint baseEndpoint = new BaseEndpoint(rpcService, expectedValue);
try {
baseEndpoint.start();
DifferentGateway differentGateway =
baseEndpoint.getSelfGateway(DifferentGateway.class);
fail(
"Expected to fail with a RuntimeException since we requested the wrong gateway type.");
} finally {
RpcUtils.terminateRpcEndpoint(baseEndpoint);
baseEndpoint.validateResourceClosed();
}
});
}
/**
* Tests that we can extend existing RpcEndpoints and can communicate with them via the self
* gateways.
*/
@Test
public void testEndpointInheritance() throws Exception {
int foobar = 1;
int barfoo = 2;
String foo = "foobar";
ExtendedEndpoint endpoint = new ExtendedEndpoint(rpcService, foobar, barfoo, foo);
try {
endpoint.start();
BaseGateway baseGateway = endpoint.getSelfGateway(BaseGateway.class);
ExtendedGateway extendedGateway = endpoint.getSelfGateway(ExtendedGateway.class);
DifferentGateway differentGateway = endpoint.getSelfGateway(DifferentGateway.class);
assertEquals(Integer.valueOf(foobar), baseGateway.foobar().get());
assertEquals(Integer.valueOf(foobar), extendedGateway.foobar().get());
assertEquals(Integer.valueOf(barfoo), extendedGateway.barfoo().get());
assertEquals(foo, differentGateway.foo().get());
} finally {
RpcUtils.terminateRpcEndpoint(endpoint);
endpoint.validateResourceClosed();
}
}
/** Tests that the RPC is running after it has been started. */
@Test
public void testRunningState()
throws InterruptedException, ExecutionException, TimeoutException {
RunningStateTestingEndpoint endpoint =
new RunningStateTestingEndpoint(
rpcService, CompletableFuture.completedFuture(null));
RunningStateTestingEndpointGateway gateway =
endpoint.getSelfGateway(RunningStateTestingEndpointGateway.class);
try {
endpoint.start();
assertTrue(gateway.queryIsRunningFlag().get());
} finally {
RpcUtils.terminateRpcEndpoint(endpoint);
endpoint.validateResourceClosed();
}
}
/** Tests that the RPC is not running if it is being stopped. */
@Test
public void testNotRunningState()
throws InterruptedException, ExecutionException, TimeoutException {
CompletableFuture<Void> stopFuture = new CompletableFuture<>();
RunningStateTestingEndpoint endpoint =
new RunningStateTestingEndpoint(rpcService, stopFuture);
RunningStateTestingEndpointGateway gateway =
endpoint.getSelfGateway(RunningStateTestingEndpointGateway.class);
endpoint.start();
CompletableFuture<Void> terminationFuture = endpoint.closeAndWaitUntilOnStopCalled();
assertFalse(gateway.queryIsRunningFlag().get());
stopFuture.complete(null);
terminationFuture.get();
endpoint.validateResourceClosed();
}
public interface BaseGateway extends RpcGateway {
CompletableFuture<Integer> foobar();
}
public interface ExtendedGateway extends BaseGateway {
CompletableFuture<Integer> barfoo();
}
public interface DifferentGateway extends RpcGateway {
CompletableFuture<String> foo();
}
public static class BaseEndpoint extends RpcEndpoint implements BaseGateway {
private final int foobarValue;
protected BaseEndpoint(RpcService rpcService) {
super(rpcService);
this.foobarValue = Integer.MAX_VALUE;
}
protected BaseEndpoint(RpcService rpcService, int foobarValue) {
super(rpcService);
this.foobarValue = foobarValue;
}
@Override
public CompletableFuture<Integer> foobar() {
return CompletableFuture.completedFuture(foobarValue);
}
}
public static class ExtendedEndpoint extends BaseEndpoint
implements ExtendedGateway, DifferentGateway {
private final int barfooValue;
private final String fooString;
protected ExtendedEndpoint(
RpcService rpcService, int foobarValue, int barfooValue, String fooString) {
super(rpcService, foobarValue);
this.barfooValue = barfooValue;
this.fooString = fooString;
}
@Override
public CompletableFuture<Integer> barfoo() {
return CompletableFuture.completedFuture(barfooValue);
}
@Override
public CompletableFuture<String> foo() {
return CompletableFuture.completedFuture(fooString);
}
}
public interface RunningStateTestingEndpointGateway extends RpcGateway {
CompletableFuture<Boolean> queryIsRunningFlag();
}
private static final class RunningStateTestingEndpoint extends RpcEndpoint
implements RunningStateTestingEndpointGateway {
private final CountDownLatch onStopCalled;
private final CompletableFuture<Void> stopFuture;
RunningStateTestingEndpoint(RpcService rpcService, CompletableFuture<Void> stopFuture) {
super(rpcService);
this.stopFuture = stopFuture;
this.onStopCalled = new CountDownLatch(1);
}
@Override
public CompletableFuture<Void> onStop() {
onStopCalled.countDown();
return stopFuture;
}
CompletableFuture<Void> closeAndWaitUntilOnStopCalled() throws InterruptedException {
CompletableFuture<Void> terminationFuture = closeAsync();
onStopCalled.await();
return terminationFuture;
}
public CompletableFuture<Boolean> queryIsRunningFlag() {
return CompletableFuture.completedFuture(isRunning());
}
}
/** Tests executing the runnable in the main thread of the underlying RPC endpoint. */
@Test
public void testExecute() throws InterruptedException, ExecutionException, TimeoutException {
final RpcEndpoint endpoint = new BaseEndpoint(rpcService);
final CompletableFuture<Void> asyncExecutionFuture = new CompletableFuture<>();
try {
endpoint.start();
endpoint.getMainThreadExecutor()
.execute(
() -> {
endpoint.validateRunsInMainThread();
asyncExecutionFuture.complete(null);
});
asyncExecutionFuture.get();
} finally {
RpcUtils.terminateRpcEndpoint(endpoint);
endpoint.validateResourceClosed();
}
}
@Test
public void testScheduleRunnableWithDelayInMilliseconds() throws Exception {
testScheduleWithDelay(
(mainThreadExecutor, expectedDelay) ->
mainThreadExecutor.schedule(
() -> {}, expectedDelay.toMillis(), TimeUnit.MILLISECONDS));
}
@Test
public void testScheduleRunnableWithDelayInSeconds() throws Exception {
testScheduleWithDelay(
(mainThreadExecutor, expectedDelay) ->
mainThreadExecutor.schedule(
() -> {}, expectedDelay.toMillis() / 1000, TimeUnit.SECONDS));
}
@Test
public void testScheduleRunnableAfterClose() throws Exception {
testScheduleAfterClose(
(mainThreadExecutor, expectedDelay) ->
mainThreadExecutor.schedule(
() -> {}, expectedDelay.toMillis() / 1000, TimeUnit.SECONDS));
}
@Test
public void testCancelScheduledRunnable() throws Exception {
testCancelScheduledTask(
(mainThreadExecutor, future) -> {
final Duration delayDuration = Duration.ofMillis(2);
return mainThreadExecutor.schedule(
() -> {
future.complete(null);
},
delayDuration.toMillis(),
TimeUnit.MILLISECONDS);
});
}
@Test
public void testScheduleCallableWithDelayInMilliseconds() throws Exception {
testScheduleWithDelay(
(mainThreadExecutor, expectedDelay) ->
mainThreadExecutor.schedule(
() -> 1, expectedDelay.toMillis(), TimeUnit.MILLISECONDS));
}
@Test
public void testScheduleCallableWithDelayInSeconds() throws Exception {
testScheduleWithDelay(
(mainThreadExecutor, expectedDelay) ->
mainThreadExecutor.schedule(
() -> 1, expectedDelay.toMillis() / 1000, TimeUnit.SECONDS));
}
@Test
public void testScheduleCallableAfterClose() throws Exception {
testScheduleAfterClose(
(mainThreadExecutor, expectedDelay) ->
mainThreadExecutor.schedule(
() -> 1, expectedDelay.toMillis() / 1000, TimeUnit.SECONDS));
}
@Test
public void testCancelScheduledCallable() {
testCancelScheduledTask(
(mainThreadExecutor, future) -> {
final Duration delayDuration = Duration.ofMillis(2);
return mainThreadExecutor.schedule(
() -> {
future.complete(null);
return null;
},
delayDuration.toMillis(),
TimeUnit.MILLISECONDS);
});
}
private static void testScheduleWithDelay(
BiConsumer<RpcEndpoint.MainThreadExecutor, Duration> scheduler) throws Exception {
final CompletableFuture<Void> taskCompletedFuture = new CompletableFuture<>();
final String endpointId = "foobar";
final MainThreadExecutable mainThreadExecutable =
new TestMainThreadExecutable((runnable) -> taskCompletedFuture.complete(null));
final RpcEndpoint.MainThreadExecutor mainThreadExecutor =
new RpcEndpoint.MainThreadExecutor(mainThreadExecutable, () -> {}, endpointId);
final Duration expectedDelay = Duration.ofSeconds(1);
scheduler.accept(mainThreadExecutor, expectedDelay);
taskCompletedFuture.get();
mainThreadExecutor.close();
}
private static void testScheduleAfterClose(
BiFunction<RpcEndpoint.MainThreadExecutor, Duration, ScheduledFuture<?>> scheduler) {
final CompletableFuture<Void> taskCompletedFuture = new CompletableFuture<>();
final String endpointId = "foobar";
final MainThreadExecutable mainThreadExecutable =
new TestMainThreadExecutable((runnable) -> taskCompletedFuture.complete(null));
final RpcEndpoint.MainThreadExecutor mainThreadExecutor =
new RpcEndpoint.MainThreadExecutor(mainThreadExecutable, () -> {}, endpointId);
mainThreadExecutor.close();
final Duration expectedDelay = Duration.ofSeconds(0);
ScheduledFuture<?> future = scheduler.apply(mainThreadExecutor, expectedDelay);
assertFalse(taskCompletedFuture.isDone());
assertFalse(future.isDone());
}
private static void testCancelScheduledTask(
BiFunction<RpcEndpoint.MainThreadExecutor, CompletableFuture<Void>, ScheduledFuture<?>>
scheduler) {
final MainThreadExecutable mainThreadExecutable =
new TestMainThreadExecutable(Runnable::run);
final ManuallyTriggeredScheduledExecutorService manuallyTriggeredScheduledExecutorService =
new ManuallyTriggeredScheduledExecutorService();
final RpcEndpoint.MainThreadExecutor mainThreadExecutor =
new RpcEndpoint.MainThreadExecutor(
mainThreadExecutable, () -> {}, manuallyTriggeredScheduledExecutorService);
final CompletableFuture<Void> actionFuture = new CompletableFuture<>();
ScheduledFuture<?> scheduledFuture = scheduler.apply(mainThreadExecutor, actionFuture);
scheduledFuture.cancel(true);
manuallyTriggeredScheduledExecutorService.triggerAllNonPeriodicTasks();
assertTrue(scheduledFuture.isCancelled());
assertFalse(actionFuture.isDone());
mainThreadExecutor.close();
}
/**
* Tests executing the callable in the main thread of the underlying RPC service, returning a
* future for the result of the callable. If the callable is not completed within the given
* timeout, then the future will be failed with a TimeoutException. This schedule method is
* called directly from RpcEndpoint, MainThreadExecutor do not support this method.
*/
@Test
public void testCallAsync() throws InterruptedException, ExecutionException, TimeoutException {
final RpcEndpoint endpoint = new BaseEndpoint(rpcService);
final Integer expectedInteger = 12345;
try {
endpoint.start();
final CompletableFuture<Integer> integerFuture =
endpoint.callAsync(
() -> {
endpoint.validateRunsInMainThread();
return expectedInteger;
},
Duration.ofSeconds(10L));
assertEquals(expectedInteger, integerFuture.get());
} finally {
RpcUtils.terminateRpcEndpoint(endpoint);
endpoint.validateResourceClosed();
}
}
/**
* Make the callable sleep some time more than specified timeout, so TimeoutException is
* expected.
*/
@Test
public void testCallAsyncTimeout()
throws InterruptedException, ExecutionException, TimeoutException {
final RpcEndpoint endpoint = new BaseEndpoint(rpcService);
final Duration timeout = Duration.ofMillis(100);
CountDownLatch latch = new CountDownLatch(1);
try {
endpoint.start();
final CompletableFuture<Throwable> throwableFuture =
endpoint.callAsync(
() -> {
endpoint.validateRunsInMainThread();
latch.await();
return 12345;
},
timeout)
.handle((ignore, throwable) -> throwable);
final Throwable throwable = throwableFuture.get();
assertNotNull(throwable);
assertTrue(throwable instanceof TimeoutException);
} finally {
latch.countDown();
RpcUtils.terminateRpcEndpoint(endpoint);
endpoint.validateResourceClosed();
}
}
private static class TestMainThreadExecutable implements MainThreadExecutable {
private final Consumer<Runnable> scheduleRunAsyncConsumer;
private TestMainThreadExecutable(Consumer<Runnable> scheduleRunAsyncConsumer) {
this.scheduleRunAsyncConsumer = scheduleRunAsyncConsumer;
}
@Override
public void runAsync(Runnable runnable) {
scheduleRunAsyncConsumer.accept(runnable);
}
@Override
public <V> CompletableFuture<V> callAsync(Callable<V> callable, Duration callTimeout) {
throw new UnsupportedOperationException();
}
@Override
public void scheduleRunAsync(Runnable runnable, long delay) {
scheduleRunAsyncConsumer.accept(runnable);
}
}
}