pnowojski commented on a change in pull request #15375:
URL: https://github.com/apache/flink/pull/15375#discussion_r604425004
##########
File path:
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
##########
@@ -1528,6 +1529,66 @@ public void testProcessWithUnAvailableInput() throws
Exception {
}
}
+ @Test
+ public void testRestorePerformedOnlyOnce() throws Exception {
+ // given: the operator with empty snapshot result (all state handles
are null)
+ OneInputStreamOperator<String, String> statelessOperator =
+ streamOperatorWithSnapshot(new OperatorSnapshotFutures());
+
+ try (MockEnvironment mockEnvironment = new
MockEnvironmentBuilder().build()) {
+
+ // when: Invoke the restore explicitly before launching the task.
+ RunningTask<StreamTask<?, ?>> task =
+ runTask(
+ () -> {
+ StreamTask mockStreamTask =
+ spy(
+ new
MockStreamTaskBuilder(mockEnvironment)
+
.setStreamInputProcessor(
+ new
AvailabilityTestInputProcessor(
+ 10))
+ .build());
+
+ mockStreamTask.restore();
+
+ return mockStreamTask;
+ });
+ waitMailBoxIsRunning(task.streamTask);
+
+ task.streamTask.cancel();
+
+ // then: 'restore' was called only once.
+ verify(task.streamTask).restore();
Review comment:
We shouldn't be using mockito. Section "Avoid Mockito - Use reusable
test implementations" from:
https://flink.apache.org/contributing/code-style-and-quality-common.html#design-for-testability
##########
File path:
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskMailboxTestHarnessBuilder.java
##########
@@ -163,6 +163,7 @@ public StreamTaskMailboxTestHarnessBuilder(
streamMockEnvironment.setTaskMetricGroup(taskMetricGroup);
StreamTask<OUT, ?> task = taskFactory.apply(streamMockEnvironment);
+ task.restore();
task.beforeInvoke();
Review comment:
I'm a bit confused. Do we have both `restore()` and `beforeInvoke()`
now? Why do we need both?
##########
File path:
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
##########
@@ -535,41 +545,82 @@ protected void beforeInvoke() throws Exception {
// -------- Invoke --------
LOG.debug("Invoking {}", getName());
+ SuspendableMailboxLoop suspendableLoop =
mailboxProcessor.getSuspendableLoop();
+
// we need to make sure that any triggers scheduled in open() cannot be
// executed before all operators are opened
- actionExecutor.runThrowing(
- () -> {
- SequentialChannelStateReader reader =
- getEnvironment()
- .getTaskStateManager()
- .getSequentialChannelStateReader();
- reader.readOutputData(
- getEnvironment().getAllWriters(),
- !configuration.isGraphContainingLoops());
-
- operatorChain.initializeStateAndOpenOperators(
- createStreamTaskStateInitializer());
-
- channelIOExecutor.execute(
- () -> {
- try {
-
reader.readInputData(getEnvironment().getAllInputGates());
- } catch (Exception e) {
- asyncExceptionHandler.handleAsyncException(
- "Unable to read channel state", e);
- }
- });
+ CompletableFuture<Void> allGatesRecoveredFuture =
+ actionExecutor.call(
+ () -> {
+ SequentialChannelStateReader reader =
+ getEnvironment()
+ .getTaskStateManager()
+ .getSequentialChannelStateReader();
+ reader.readOutputData(
+ getEnvironment().getAllWriters(),
+ !configuration.isGraphContainingLoops());
+
+ operatorChain.initializeStateAndOpenOperators(
+ createStreamTaskStateInitializer());
+
+ IndexedInputGate[] inputGates =
getEnvironment().getAllInputGates();
+ channelIOExecutor.execute(
+ () -> {
+ try {
+ reader.readInputData(inputGates);
+ } catch (Exception e) {
+
asyncExceptionHandler.handleAsyncException(
+ "Unable to read channel
state", e);
+ }
+ });
+
+ List<CompletableFuture<?>> recoveredFutures =
+ new ArrayList<>(inputGates.length);
+ for (InputGate inputGate : inputGates) {
+
recoveredFutures.add(inputGate.getStateConsumedFuture());
+
+ inputGate
+ .getStateConsumedFuture()
+ .thenRun(
+ () ->
+
mainMailboxExecutor.execute(
+
inputGate::requestPartitions,
+ "Input gate
request partitions"));
+ }
- for (InputGate inputGate :
getEnvironment().getAllInputGates()) {
- inputGate
- .getStateConsumedFuture()
- .thenRun(
- () ->
- mainMailboxExecutor.execute(
-
inputGate::requestPartitions,
- "Input gate request
partitions"));
- }
- });
+ return gatesRecoveredFuture(suspendableLoop,
recoveredFutures);
+ });
+
+ // Run mailbox until all gates will be recovered.
+ do {
+ suspendableLoop.run();
+ } while (isMailboxLoopRunning() && !allGatesRecoveredFuture.isDone());
+
+ if (canceled) {
+ throw new CancelTaskException();
+ }
+
+ if (!allGatesRecoveredFuture.isDone()) {
+ throw new Exception("Mailbox loop interrupted before recovery was
finished.");
+ }
+
+ restored = true;
+ }
+
+ @VisibleForTesting
+ CompletableFuture<Void> gatesRecoveredFuture(
+ SuspendableMailboxLoop suspendableLoop, List<CompletableFuture<?>>
recoveredFutures) {
+ return CompletableFuture.allOf(recoveredFutures.toArray(new
CompletableFuture[0]))
+ .thenRun(suspendableLoop::suspend);
+ }
Review comment:
inline? I don't think it's being used anywhere else?
##########
File path:
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
##########
@@ -535,41 +545,82 @@ protected void beforeInvoke() throws Exception {
// -------- Invoke --------
LOG.debug("Invoking {}", getName());
+ SuspendableMailboxLoop suspendableLoop =
mailboxProcessor.getSuspendableLoop();
+
// we need to make sure that any triggers scheduled in open() cannot be
// executed before all operators are opened
- actionExecutor.runThrowing(
- () -> {
- SequentialChannelStateReader reader =
- getEnvironment()
- .getTaskStateManager()
- .getSequentialChannelStateReader();
- reader.readOutputData(
- getEnvironment().getAllWriters(),
- !configuration.isGraphContainingLoops());
-
- operatorChain.initializeStateAndOpenOperators(
- createStreamTaskStateInitializer());
-
- channelIOExecutor.execute(
- () -> {
- try {
-
reader.readInputData(getEnvironment().getAllInputGates());
- } catch (Exception e) {
- asyncExceptionHandler.handleAsyncException(
- "Unable to read channel state", e);
- }
- });
+ CompletableFuture<Void> allGatesRecoveredFuture =
+ actionExecutor.call(
+ () -> {
+ SequentialChannelStateReader reader =
+ getEnvironment()
+ .getTaskStateManager()
+ .getSequentialChannelStateReader();
+ reader.readOutputData(
+ getEnvironment().getAllWriters(),
+ !configuration.isGraphContainingLoops());
+
+ operatorChain.initializeStateAndOpenOperators(
+ createStreamTaskStateInitializer());
+
+ IndexedInputGate[] inputGates =
getEnvironment().getAllInputGates();
+ channelIOExecutor.execute(
+ () -> {
+ try {
+ reader.readInputData(inputGates);
+ } catch (Exception e) {
+
asyncExceptionHandler.handleAsyncException(
+ "Unable to read channel
state", e);
+ }
+ });
+
+ List<CompletableFuture<?>> recoveredFutures =
+ new ArrayList<>(inputGates.length);
+ for (InputGate inputGate : inputGates) {
+
recoveredFutures.add(inputGate.getStateConsumedFuture());
+
+ inputGate
+ .getStateConsumedFuture()
+ .thenRun(
+ () ->
+
mainMailboxExecutor.execute(
+
inputGate::requestPartitions,
+ "Input gate
request partitions"));
+ }
- for (InputGate inputGate :
getEnvironment().getAllInputGates()) {
- inputGate
- .getStateConsumedFuture()
- .thenRun(
- () ->
- mainMailboxExecutor.execute(
-
inputGate::requestPartitions,
- "Input gate request
partitions"));
- }
- });
+ return gatesRecoveredFuture(suspendableLoop,
recoveredFutures);
+ });
+
+ // Run mailbox until all gates will be recovered.
+ do {
+ suspendableLoop.run();
+ } while (isMailboxLoopRunning() && !allGatesRecoveredFuture.isDone());
Review comment:
Wouldn't it be simpler to just use
```
do {
mailboxProcessor.runMailboxStep();
} while (isMailboxLoopRunning() && !allGatesRecoveredFuture.isDone())
```
?
##########
File path:
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
##########
@@ -535,41 +545,82 @@ protected void beforeInvoke() throws Exception {
// -------- Invoke --------
LOG.debug("Invoking {}", getName());
+ SuspendableMailboxLoop suspendableLoop =
mailboxProcessor.getSuspendableLoop();
+
// we need to make sure that any triggers scheduled in open() cannot be
// executed before all operators are opened
- actionExecutor.runThrowing(
- () -> {
- SequentialChannelStateReader reader =
- getEnvironment()
- .getTaskStateManager()
- .getSequentialChannelStateReader();
- reader.readOutputData(
- getEnvironment().getAllWriters(),
- !configuration.isGraphContainingLoops());
-
- operatorChain.initializeStateAndOpenOperators(
- createStreamTaskStateInitializer());
-
- channelIOExecutor.execute(
- () -> {
- try {
-
reader.readInputData(getEnvironment().getAllInputGates());
- } catch (Exception e) {
- asyncExceptionHandler.handleAsyncException(
- "Unable to read channel state", e);
- }
- });
+ CompletableFuture<Void> allGatesRecoveredFuture =
+ actionExecutor.call(
+ () -> {
Review comment:
nit: is it feasible to extract this lambda function to a separate
method? `restore()` has grown a bit too much. Or maybe portions of it?
--
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]