[
https://issues.apache.org/jira/browse/FLINK-17769?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17126427#comment-17126427
]
Yuan Mei commented on FLINK-17769:
----------------------------------
Originally, I was thinking this problem is introduced by FLINK-17663, but the
problem seems to be there before the change.
The problem is caused in Exception handling of
StreamTask.disposeAllOperators(true);
|-StreamTask.cleanUpInvoke
|-StreamTask.invoke
In ``disposeAllOperators(true)'', "true" means logging error only without
throwing disposal exceptions. That's why the error during disposal is printed
before the real root cause.
I agree these are confusing, here are a couple of ways I am considering to
solve this problem:
1. Print the invoke error before ``cleanUpInvoke()'', as shown below; The
problem is it is likely the same error can be printed twice.
catch (Exception invokeException) {
{color:#FF0000}LOG.error("Invoke Error", invokeException);{color}
try {
cleanUpInvoke();
}
catch (Throwable cleanUpException) {
throw (Exception) ExceptionUtils.firstOrSuppressed(cleanUpException,
invokeException);
}
throw invokeException;
}
2. use disposeAllOperators(false) instead, that is throwing the disposal
exception as well. Is there any specific reason why disposal error is expected
to be swallowed in this case?
3. Be specific in the log of disposeAllOperators(true) that "this is not the
real root cause, is a caused by some other errors".
> Wrong order of log events on a task failure
> -------------------------------------------
>
> Key: FLINK-17769
> URL: https://issues.apache.org/jira/browse/FLINK-17769
> Project: Flink
> Issue Type: Bug
> Components: Runtime / Task
> Reporter: Robert Metzger
> Priority: Critical
> Fix For: 1.11.0
>
>
> In this example, errors from the {{close()}} method call are logged before
> the {{switched from RUNNING to FAILED}} log line with the actual exception
> (which is confusing, because the exceptions coming from {{close()}} could be
> considered as the failure root cause, because they are first in the log)
> {code}
> 2020-05-14 10:12:42,660 INFO
> org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer [] -
> Started Kinesis producer instance for region 'eu-central-1'
> 2020-05-14 10:12:42,660 DEBUG
> org.apache.flink.streaming.api.operators.BackendRestorerProcedure [] -
> Creating operator state backend for
> StreamSource_cbc357ccb763df2852fee8c4fc7d55f2_(1/1) with empty state.
> 2020-05-14 10:12:42,823 INFO
> org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer [] -
> Closing producer
> 2020-05-14 10:12:42,823 INFO
> org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer [] -
> Flushing outstanding 2 records
> 2020-05-14 10:12:42,826 ERROR
> org.apache.flink.streaming.runtime.tasks.StreamTask [] - Error
> during disposal of stream operator.
> org.apache.flink.kinesis.shaded.com.amazonaws.services.kinesis.producer.DaemonException:
> The child process has been shutdown and can no longer accept messages.
> 2020-05-14 10:12:42,834 WARN org.apache.flink.runtime.taskmanager.Task
> [] - Source: Custom Source -> Sink: Unnamed (1/1)
> (4a49aea047aeb3e67cf79c788df0e558) switched from RUNNING to FAILED.
> {code}
--
This message was sent by Atlassian Jira
(v8.3.4#803005)