[ 
https://issues.apache.org/jira/browse/FLINK-20145?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Robert Metzger updated FLINK-20145:
-----------------------------------
    Description: 
While testing the 1.12 release, I came across the following failure cause:

{code}
2020-11-13 09:41:52,110 WARN  org.apache.flink.runtime.taskmanager.Task         
           [] - dynamic filter (3/4)#0 (b977944851531f96e5324e786f055eb7) 
switched from RUNNING to FAILED.
java.lang.IllegalStateException: Should only poll priority events
        at 
org.apache.flink.util.Preconditions.checkState(Preconditions.java:198) 
~[flink-dist_2.11-1.12.0.jar:1.12.0]
        at 
org.apache.flink.streaming.runtime.io.CheckpointedInputGate.processPriorityEvents(CheckpointedInputGate.java:116)
 ~[flink-dist_2.11-1.12.0.jar:1.12.0]
        at 
org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$1.runThrowing(StreamTaskActionExecutor.java:47)
 ~[flink-dist_2.11-1.12.0.jar:1.12.0]
        at 
org.apache.flink.streaming.runtime.tasks.mailbox.Mail.run(Mail.java:78) 
~[flink-dist_2.11-1.12.0.jar:1.12.0]
        at 
org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.processMail(MailboxProcessor.java:283)
 ~[flink-dist_2.11-1.12.0.jar:1.12.0]
        at 
org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:184)
 ~[flink-dist_2.11-1.12.0.jar:1.12.0]
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:577)
 ~[flink-dist_2.11-1.12.0.jar:1.12.0]
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:541) 
~[flink-dist_2.11-1.12.0.jar:1.12.0]
        at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:722) 
[flink-dist_2.11-1.12.0.jar:1.12.0]
        at org.apache.flink.runtime.taskmanager.Task.run(Task.java:547) 
[flink-dist_2.11-1.12.0.jar:1.12.0]
        at java.lang.Thread.run(Thread.java:748) [?:1.8.0_222]
{code}

I have unaligned checkpointing enabled, the failing operator is a 
CoFlatMapFunction. The error happend on all four TaskManagers, very soon after 
job submission. The error doesn't happen when unaligned checkpointing is 
disabled.

  was:
While testing the 1.12 release, I came across the following failure cause:

{code}
2020-11-13 09:41:52,110 WARN  org.apache.flink.runtime.taskmanager.Task         
           [] - dynamic filter (3/4)#0 (b977944851531f96e5324e786f055eb7) 
switched from RUNNING to FAILED.
java.lang.IllegalStateException: Should only poll priority events
        at 
org.apache.flink.util.Preconditions.checkState(Preconditions.java:198) 
~[flink-dist_2.11-1.12.0.jar:1.12.0]
        at 
org.apache.flink.streaming.runtime.io.CheckpointedInputGate.processPriorityEvents(CheckpointedInputGate.java:116)
 ~[flink-dist_2.11-1.12.0.jar:1.12.0]
        at 
org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$1.runThrowing(StreamTaskActionExecutor.java:47)
 ~[flink-dist_2.11-1.12.0.jar:1.12.0]
        at 
org.apache.flink.streaming.runtime.tasks.mailbox.Mail.run(Mail.java:78) 
~[flink-dist_2.11-1.12.0.jar:1.12.0]
        at 
org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.processMail(MailboxProcessor.java:283)
 ~[flink-dist_2.11-1.12.0.jar:1.12.0]
        at 
org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:184)
 ~[flink-dist_2.11-1.12.0.jar:1.12.0]
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:577)
 ~[flink-dist_2.11-1.12.0.jar:1.12.0]
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:541) 
~[flink-dist_2.11-1.12.0.jar:1.12.0]
        at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:722) 
[flink-dist_2.11-1.12.0.jar:1.12.0]
        at org.apache.flink.runtime.taskmanager.Task.run(Task.java:547) 
[flink-dist_2.11-1.12.0.jar:1.12.0]
        at java.lang.Thread.run(Thread.java:748) [?:1.8.0_222]
{code}

I have unaligned checkpointing enabled, the failing operator is a 
CoFlatMapFunction. The error happend on all four TaskManagers, very soon after 
job submission.


> Streaming job fails with IllegalStateException: Should only poll priority 
> events
> --------------------------------------------------------------------------------
>
>                 Key: FLINK-20145
>                 URL: https://issues.apache.org/jira/browse/FLINK-20145
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Checkpointing, Runtime / Network
>    Affects Versions: 1.12.0
>            Reporter: Robert Metzger
>            Priority: Critical
>             Fix For: 1.12.0
>
>
> While testing the 1.12 release, I came across the following failure cause:
> {code}
> 2020-11-13 09:41:52,110 WARN  org.apache.flink.runtime.taskmanager.Task       
>              [] - dynamic filter (3/4)#0 (b977944851531f96e5324e786f055eb7) 
> switched from RUNNING to FAILED.
> java.lang.IllegalStateException: Should only poll priority events
>       at 
> org.apache.flink.util.Preconditions.checkState(Preconditions.java:198) 
> ~[flink-dist_2.11-1.12.0.jar:1.12.0]
>       at 
> org.apache.flink.streaming.runtime.io.CheckpointedInputGate.processPriorityEvents(CheckpointedInputGate.java:116)
>  ~[flink-dist_2.11-1.12.0.jar:1.12.0]
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$1.runThrowing(StreamTaskActionExecutor.java:47)
>  ~[flink-dist_2.11-1.12.0.jar:1.12.0]
>       at 
> org.apache.flink.streaming.runtime.tasks.mailbox.Mail.run(Mail.java:78) 
> ~[flink-dist_2.11-1.12.0.jar:1.12.0]
>       at 
> org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.processMail(MailboxProcessor.java:283)
>  ~[flink-dist_2.11-1.12.0.jar:1.12.0]
>       at 
> org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:184)
>  ~[flink-dist_2.11-1.12.0.jar:1.12.0]
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:577)
>  ~[flink-dist_2.11-1.12.0.jar:1.12.0]
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:541)
>  ~[flink-dist_2.11-1.12.0.jar:1.12.0]
>       at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:722) 
> [flink-dist_2.11-1.12.0.jar:1.12.0]
>       at org.apache.flink.runtime.taskmanager.Task.run(Task.java:547) 
> [flink-dist_2.11-1.12.0.jar:1.12.0]
>       at java.lang.Thread.run(Thread.java:748) [?:1.8.0_222]
> {code}
> I have unaligned checkpointing enabled, the failing operator is a 
> CoFlatMapFunction. The error happend on all four TaskManagers, very soon 
> after job submission. The error doesn't happen when unaligned checkpointing 
> is disabled.



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

Reply via email to