pnowojski commented on a change in pull request #8858: [hotfix][tests] Change
some StreamTask tests to create a test task in the task's thread
URL: https://github.com/apache/flink/pull/8858#discussion_r297151246
##########
File path:
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
##########
@@ -805,29 +723,102 @@ public void
testSetsUserCodeClassLoaderForTimerThreadFactory() throws Throwable
new MockEnvironmentBuilder()
.setUserCodeClassLoader(new
TestUserCodeClassLoader())
.build()) {
- TimeServiceTask timerServiceTask = new
TimeServiceTask(mockEnvironment);
+ RunningTask<TimeServiceTask> task = runTask(() -> new
TimeServiceTask(mockEnvironment));
+ task.waitForTaskCompletion(false);
- CompletableFuture<Void> invokeFuture =
CompletableFuture.runAsync(
- () -> {
- try {
- timerServiceTask.invoke();
- } catch (Exception e) {
- throw new
CompletionException(e);
- }
- },
- TestingUtils.defaultExecutor());
-
- invokeFuture.get();
-
- assertThat(timerServiceTask.getClassLoaders(),
hasSize(greaterThanOrEqualTo(1)));
- assertThat(timerServiceTask.getClassLoaders(),
everyItem(instanceOf(TestUserCodeClassLoader.class)));
+ assertThat(task.streamTask.getClassLoaders(),
hasSize(greaterThanOrEqualTo(1)));
+ assertThat(task.streamTask.getClassLoaders(),
everyItem(instanceOf(TestUserCodeClassLoader.class)));
}
}
//
------------------------------------------------------------------------
// Test Utilities
//
------------------------------------------------------------------------
+ private static StreamOperator<?>
streamOperatorWithSnapshot(OperatorSnapshotFutures operatorSnapshotResult)
throws Exception {
+ StreamOperator<?> operator = mock(StreamOperator.class);
+ when(operator.getOperatorID()).thenReturn(new OperatorID());
+
+ when(operator.snapshotState(anyLong(), anyLong(),
any(CheckpointOptions.class), any(CheckpointStreamFactory.class)))
+ .thenReturn(operatorSnapshotResult);
+
+ return operator;
+ }
+
+ private static StreamOperator<?>
streamOperatorWithSnapshotException(Exception exception) throws Exception {
+ StreamOperator<?> operator = mock(StreamOperator.class);
+ when(operator.getOperatorID()).thenReturn(new OperatorID());
+
+ when(operator.snapshotState(anyLong(), anyLong(),
any(CheckpointOptions.class), any(CheckpointStreamFactory.class)))
+ .thenThrow(exception);
+
+ return operator;
+ }
+
+ private static <T> OperatorChain<T, AbstractStreamOperator<T>>
operatorChain(StreamOperator<?>... streamOperators) {
+ OperatorChain<T, AbstractStreamOperator<T>> operatorChain =
mock(OperatorChain.class);
+
when(operatorChain.getAllOperators()).thenReturn(streamOperators);
+ return operatorChain;
+ }
+
+ private static class RunningTask<T> {
+ final T streamTask;
+ final CompletableFuture<Void> invocationFuture;
+
+ RunningTask(T streamTask, CompletableFuture<Void>
invocationFuture) {
+ this.streamTask = streamTask;
+ this.invocationFuture = invocationFuture;
+ }
+
+ void waitForTaskCompletion(boolean cancelled) throws Exception {
+ if (!cancelled) {
+ invocationFuture.get();
+ return;
+ }
+ try {
+ invocationFuture.get();
+ } catch (Exception e) {
+ assertThat(e.getCause(),
is(instanceOf(CancelTaskException.class)));
+ }
+ }
+ }
+
+ private static <T extends StreamTask<?, ?>> RunningTask<T>
runTask(SupplierWithException<T, Exception> taskFactory) throws Exception {
+ CompletableFuture<T> taskFuture = new CompletableFuture<>();
+ CompletableFuture<Void> invocationFuture =
CompletableFuture.runAsync(
+ () -> {
+ T task;
+ try {
+ task = taskFactory.get();
+ taskFuture.complete(task);
+ } catch (Exception e) {
+ taskFuture.completeExceptionally(e);
+ return;
+ }
+ try {
+ task.invoke();
+ } catch (RuntimeException e) {
+ throw e;
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ }, Executors.newSingleThreadExecutor());
+
+ return new RunningTask<>(taskFuture.get(), invocationFuture);
+ }
+
+ private static void waitForTaskCompletion(CompletableFuture<Void>
taskInvocation, boolean cancelled) throws Exception {
Review comment:
Is this method used anywhere?
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
With regards,
Apache Git Services