pnowojski commented on a change in pull request #15375:
URL: https://github.com/apache/flink/pull/15375#discussion_r604867957
##########
File path:
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
##########
@@ -212,6 +213,9 @@
*/
private volatile boolean isRunning;
+ /** Flag to mark that the state of current task was restored successfully.
*/
+ private volatile boolean restored;
Review comment:
remove `volatile`? Either it doesn't need to be thread safe or we are
doing something wrong in some test. Note it doesn't look like it's thread safe
right now even with the `volatile`.
##########
File path:
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxProcessor.java
##########
@@ -183,27 +198,34 @@ public void runMailboxLoop() throws Exception {
final MailboxController defaultActionContext = new
MailboxController(this);
- while (isMailboxLoopRunning()) {
+ while (isNotSuspended()) {
// The blocking `processMail` call will not return until default
action is available.
processMail(localMailbox, false);
- if (isMailboxLoopRunning()) {
+ if (isNotSuspended()) {
Review comment:
nit: I think code would be a bit more self-explanatory if it was
```
while (isMailboxLoopRunning()) {
...
if (isMailboxLoopRunning()) {
```
as it was before, but with modified `isMailboxLoopRunning()` definition to:
`mailboxLoopRunning && !suspended`?
Maybe if changing the semantic of `isMailboxLoopRunning()` is causing some
issues, we could introduce `isMailboxLoopRunningAndNotSuspended()`? Because
saying that "mailbox is suspended" when it's actually closed is confusing.
##########
File path:
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
##########
@@ -528,49 +537,79 @@ protected void beforeInvoke() throws Exception {
init();
// save the work of reloading state, etc, if the task is already
canceled
- if (canceled) {
- throw new CancelTaskException();
- }
+ ensureNotCanceled();
// -------- Invoke --------
LOG.debug("Invoking {}", getName());
// 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(this::restoreGates);
+
+ // Run mailbox until all gates will be recovered.
+ do {
+ mailboxProcessor.runMailboxLoop();
+ } while (isMailboxLoopRunning() && !allGatesRecoveredFuture.isDone());
+
+ ensureNotCanceled();
+
+ if (!allGatesRecoveredFuture.isDone()) {
+ throw new Exception("Mailbox loop interrupted before recovery was
finished.");
+ }
Review comment:
Is this just a safe guard against bugs? If so:
```
checkState(allGatesRecoveredFuture.isDone(), "Mailbox loop interrupted
before recovery was finished.");
```
--
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]