[ 
https://issues.apache.org/jira/browse/FLINK-15427?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17004461#comment-17004461
 ] 

Congxian Qiu(klion26) commented on FLINK-15427:
-----------------------------------------------

This test itself passed, but failed when {{checking exceptions}}
{code:java}
Checking for errors...^M
Found error in log files:^M
{code}
after executing the command to find the exception we'll get
{code:java}
2019-12-27 05:18:34,743 ERROR 
org.apache.flink.streaming.runtime.tasks.StreamTask           - Received 
CancelTaskException while we are not canceled. This is a bug and should be 
reported^M
org.apache.flink.runtime.execution.CancelTaskException: Consumed partition 
PipelinedSubpartitionView(index: 2) of ResultPartition 
745fd76b3c0327b1b0732bb14045de1c@2e06db5ab07dfc5dabc32576a9a40a0f has been 
released.^M
        at 
org.apache.flink.runtime.io.network.partition.consumer.LocalInputChannel.getNextBuffer(LocalInputChannel.java:190)^M
        at 
org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate.waitAndGetNextData(SingleInputGate.java:509)^M
        at 
org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate.getNextBufferOrEvent(SingleInputGate.java:487)^M
        at 
org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate.pollNext(SingleInputGate.java:475)^M
        at 
org.apache.flink.runtime.taskmanager.InputGateWithMetrics.pollNext(InputGateWithMetrics.java:75)^M
        at 
org.apache.flink.streaming.runtime.io.CheckpointedInputGate.pollNext(CheckpointedInputGate.java:125)^M
        at 
org.apache.flink.streaming.runtime.io.StreamTaskNetworkInput.emitNext(StreamTaskNetworkInput.java:133)^M
        at 
org.apache.flink.streaming.runtime.io.StreamOneInputProcessor.processInput(StreamOneInputProcessor.java:69)^M
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.processInput(StreamTask.java:311)^M
        at 
org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:187)^M
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:488)^M
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:470)^M
        at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:702)^M

{code}
This {{CancelException}} was introduced by [~AHeise] in FLINK-15317, and I 
think FLINK-15403 has already been tracking it.

For this issue, it wants to verify state ttl, if the verify failed, it will 
print something in the stdout. we used the {{check exceptions(which will check 
whether the `.out` file is empty or not)}} to do this thing.

So, I propose to check in the {{test_stream_state__ttl}}.sh other than 
{{delegating to the exceptions check}}, the reason is something like 
FLINK-15105.

we'll add some checking logic in the end of {{test_stream_state_ttl.sh}}  such 
as below and skip exception check for this test. [~azagrebin]
{code:java}
if grep "TtlVerificationContext{" $FLINK_DIR/log/*.out > /dev/null; then
   exit 1; # contains the output
fi
{code}

> State TTL RocksDb backend end-to-end test stalls on travis
> ----------------------------------------------------------
>
>                 Key: FLINK-15427
>                 URL: https://issues.apache.org/jira/browse/FLINK-15427
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / State Backends
>    Affects Versions: 1.10.0
>            Reporter: Yu Li
>            Assignee: Congxian Qiu(klion26)
>            Priority: Blocker
>              Labels: test-stability
>             Fix For: 1.10.0
>
>
> The 'State TTL RocksDb backend end-to-end test' case stalls and finally 
> timedout with error message:
> {noformat}
> The job exceeded the maximum log length, and has been terminated.
> {noformat}
> https://api.travis-ci.org/v3/job/629699416/log.txt



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to