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

Aljoscha Krettek reassigned FLINK-2697:
---------------------------------------

    Assignee: Aljoscha Krettek

> Deadlock in StreamDiscretizer
> -----------------------------
>
>                 Key: FLINK-2697
>                 URL: https://issues.apache.org/jira/browse/FLINK-2697
>             Project: Flink
>          Issue Type: Bug
>          Components: Streaming
>            Reporter: Till Rohrmann
>            Assignee: Aljoscha Krettek
>
> Encountered a deadlock in the {{StreamDiscretizer}}
> {code}
> Found one Java-level deadlock:
> =============================
> "Thread-11":
>   waiting to lock monitor 0x00007f9d081e1ab8 (object 0x00000000ff6b4590, a 
> org.apache.flink.streaming.api.operators.windowing.StreamDiscretizer),
>   which is held by "StreamDiscretizer -> TumblingGroupedPreReducer -> 
> (Filter, ExtractParts) (3/4)"
> "StreamDiscretizer -> TumblingGroupedPreReducer -> (Filter, ExtractParts) 
> (3/4)":
>   waiting to lock monitor 0x00007f9d081e20e8 (object 0x00000000ff75fd88, a 
> org.apache.flink.streaming.api.windowing.policy.TimeTriggerPolicy),
>   which is held by "Thread-11"
> Java stack information for the threads listed above:
> ===================================================
> "Thread-11":
>       at 
> org.apache.flink.streaming.api.operators.windowing.StreamDiscretizer.triggerOnFakeElement(StreamDiscretizer.java:121)
>       - waiting to lock <0x00000000ff6b4590> (a 
> org.apache.flink.streaming.api.operators.windowing.StreamDiscretizer)
>       at 
> org.apache.flink.streaming.api.operators.windowing.StreamDiscretizer$WindowingCallback.sendFakeElement(StreamDiscretizer.java:203)
>       at 
> org.apache.flink.streaming.api.windowing.policy.TimeTriggerPolicy.activeFakeElementEmission(TimeTriggerPolicy.java:117)
>       - locked <0x00000000ff75fd88> (a 
> org.apache.flink.streaming.api.windowing.policy.TimeTriggerPolicy)
>       at 
> org.apache.flink.streaming.api.windowing.policy.TimeTriggerPolicy$TimeCheck.run(TimeTriggerPolicy.java:144)
>       at java.lang.Thread.run(Thread.java:745)
> "StreamDiscretizer -> TumblingGroupedPreReducer -> (Filter, ExtractParts) 
> (3/4)":
>       at 
> org.apache.flink.streaming.api.windowing.policy.TimeTriggerPolicy.preNotifyTrigger(TimeTriggerPolicy.java:74)
>       - waiting to lock <0x00000000ff75fd88> (a 
> org.apache.flink.streaming.api.windowing.policy.TimeTriggerPolicy)
>       at 
> org.apache.flink.streaming.api.operators.windowing.StreamDiscretizer.processRealElement(StreamDiscretizer.java:91)
>       - locked <0x00000000ff6b4590> (a 
> org.apache.flink.streaming.api.operators.windowing.StreamDiscretizer)
>       at 
> org.apache.flink.streaming.api.operators.windowing.StreamDiscretizer.processElement(StreamDiscretizer.java:73)
>       at 
> org.apache.flink.streaming.runtime.io.StreamInputProcessor.processInput(StreamInputProcessor.java:162)
>       at 
> org.apache.flink.streaming.runtime.tasks.OneInputStreamTask.run(OneInputStreamTask.java:56)
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:171)
>       at org.apache.flink.runtime.taskmanager.Task.run(Task.java:581)
>       at java.lang.Thread.run(Thread.java:745)
> Found 1 deadlock.
> {code}
> https://s3.amazonaws.com/archive.travis-ci.org/jobs/80770719/log.txt



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to