Hi Shailesh,

Are you sure you are using version 1.4.2? Do you run a vanilla flink, or
have you introduced some changes? I am asking cause the lines in
stacktrace does not align with the source code for 1.4.2.

Also it is a different exception than the one in the issue you've
linked, so if it is a problem than it is definitely a different one.
Last thing I would recommend upgrading to the newest version, as we
rewritten the SharedBuffer implementation in 1.6.0.

Best,

Dawid


On 26/09/18 13:50, Shailesh Jain wrote:
> Hi,
>
> I think I've hit this same issue on a 3 node standalone cluster
> (1.4.2) using HDFS (2.8.4) as state backend.
>
> 2018-09-26 17:07:39,370 INFO 
> org.apache.flink.runtime.taskmanager.Task                     -
> Attempting to fail task externally SelectCepOperator (1/1)
> (3bec4aa1ef2226c4e0c5ff7b3860d340).
> 2018-09-26 17:07:39,370 INFO 
> org.apache.flink.runtime.taskmanager.Task                     -
> SelectCepOperator (1/1) (3bec4aa1ef2226c4e0c5ff7b3860d340) switched
> from RUNNING to FAILED.
> AsynchronousException{java.lang.Exception: Could not materialize
> checkpoint 6 for operator SelectCepOperator (1/1).}
>     at
> org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.run(StreamTask.java:948)
>     at
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>     at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>     at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>     at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>     at java.lang.Thread.run(Thread.java:748)
> Caused by: java.lang.Exception: Could not materialize checkpoint 6 for
> operator SelectCepOperator (1/1).
>     ... 6 more
> Caused by: java.util.concurrent.ExecutionException:
> java.lang.NullPointerException
>     at java.util.concurrent.FutureTask.report(FutureTask.java:122)
>     at java.util.concurrent.FutureTask.get(FutureTask.java:192)
>     at
> org.apache.flink.util.FutureUtil.runIfNotDoneAndGet(FutureUtil.java:43)
>     at
> org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.run(StreamTask.java:894)
>     ... 5 more
>     Suppressed: java.lang.Exception: Could not properly cancel managed
> keyed state future.
>         at
> org.apache.flink.streaming.api.operators.OperatorSnapshotResult.cancel(OperatorSnapshotResult.java:91)
>         at
> org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.cleanup(StreamTask.java:976)
>         at
> org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.run(StreamTask.java:939)
>         ... 5 more
>     Caused by: java.util.concurrent.ExecutionException:
> java.lang.NullPointerException
>         at java.util.concurrent.FutureTask.report(FutureTask.java:122)
>         at java.util.concurrent.FutureTask.get(FutureTask.java:192)
>         at
> org.apache.flink.util.FutureUtil.runIfNotDoneAndGet(FutureUtil.java:43)
>         at
> org.apache.flink.runtime.state.StateUtil.discardStateFuture(StateUtil.java:66)
>         at
> org.apache.flink.streaming.api.operators.OperatorSnapshotResult.cancel(OperatorSnapshotResult.java:89)
>         ... 7 more
>     Caused by: java.lang.NullPointerException
>         at
> org.apache.flink.cep.nfa.SharedBuffer$SharedBufferSerializer.serialize(SharedBuffer.java:954)
>         at
> org.apache.flink.cep.nfa.SharedBuffer$SharedBufferSerializer.serialize(SharedBuffer.java:825)
>         at
> org.apache.flink.cep.nfa.NFA$NFASerializer.serialize(NFA.java:888)
>         at
> org.apache.flink.cep.nfa.NFA$NFASerializer.serialize(NFA.java:820)
>         at
> org.apache.flink.runtime.state.heap.CopyOnWriteStateTableSnapshot.writeMappingsInKeyGroup(CopyOnWriteStateTableSnapshot.java:196)
>         at
> org.apache.flink.runtime.state.heap.HeapKeyedStateBackend$1.performOperation(HeapKeyedStateBackend.java:390)
>         at
> org.apache.flink.runtime.state.heap.HeapKeyedStateBackend$1.performOperation(HeapKeyedStateBackend.java:339)
>         at
> org.apache.flink.runtime.io.async.AbstractAsyncCallableWithResources.call(AbstractAsyncCallableWithResources.java:75)
>         at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>         at
> org.apache.flink.util.FutureUtil.runIfNotDoneAndGet(FutureUtil.java:40)
>         at
> org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.run(StreamTask.java:894)
>         ... 5 more
>     [CIRCULAR REFERENCE:java.lang.NullPointerException]
>
> Any ideas on why I'm hitting this especially when this
> (https://issues.apache.org/jira/browse/FLINK-7756) says it has been
> fixed in 1.4.2 ?
>
> On Sat, Nov 4, 2017 at 12:34 AM Federico D'Ambrosio
> <federico.dambro...@smartlab.ws
> <mailto:federico.dambro...@smartlab.ws>> wrote:
>
>     Thank you very much for your steady response, Kostas!
>
>     Cheers,
>     Federico
>
>     2017-11-03 16:26 GMT+01:00 Kostas Kloudas
>     <k.klou...@data-artisans.com <mailto:k.klou...@data-artisans.com>>:
>
>         Hi Federico,
>
>         Thanks for trying it out! 
>         Great to hear that your problem was fixed!
>
>         The feature freeze for the release is going to be next week,
>         and I would expect 1 or 2 more weeks testing.
>         So I would say in 2.5 weeks. But this is of course subject to
>         potential issues we may find during testing.
>
>         Cheers,
>         Kostas
>
>>         On Nov 3, 2017, at 4:22 PM, Federico D'Ambrosio
>>         <federico.dambro...@smartlab.ws
>>         <mailto:federico.dambro...@smartlab.ws>> wrote:
>>
>>         Hi Kostas,
>>
>>         I just tried running the same job with 1.4-SNAPSHOT for 10
>>         minutes and it didn't crash, so that was the same underlying
>>         issue of the JIRA you linked.
>>
>>         Do you happen to know when it's expected the 1.4 stable release?
>>
>>         Thank you very much,
>>         Federico
>>
>>         2017-11-03 15:25 GMT+01:00 Kostas Kloudas
>>         <k.klou...@data-artisans.com
>>         <mailto:k.klou...@data-artisans.com>>:
>>
>>             Perfect! thanks a lot!
>>
>>             Kostas
>>
>>>             On Nov 3, 2017, at 3:23 PM, Federico D'Ambrosio
>>>             <federico.dambro...@smartlab.ws
>>>             <mailto:federico.dambro...@smartlab.ws>> wrote:
>>>
>>>             Hi Kostas,
>>>
>>>             yes, I'm using 1.3.2. I'll try the current master and
>>>             I'll get back to you.
>>>
>>>             2017-11-03 15:21 GMT+01:00 Kostas Kloudas
>>>             <k.klou...@data-artisans.com
>>>             <mailto:k.klou...@data-artisans.com>>:
>>>
>>>                 Hi Federico,
>>>
>>>                 I assume that you are using Flink 1.3, right?
>>>
>>>                 In this case, in 1.4 we have fixed a bug that seems
>>>                 similar to your case:
>>>                 https://issues.apache.org/jira/browse/FLINK-7756
>>>
>>>                 Could you try the current master to see if it fixes
>>>                 your problem?
>>>
>>>                 Thanks,
>>>                 Kostas
>>>
>>>>                 On Nov 3, 2017, at 3:12 PM, Federico D'Ambrosio
>>>>                 <federico.dambro...@smartlab.ws
>>>>                 <mailto:federico.dambro...@smartlab.ws>> wrote:
>>>>
>>>>                  Could not find id for
>>>>                 entry:                                                     
>>>>    
>>>
>>>
>>>
>>>
>>>             -- 
>>>             Federico D'Ambrosio
>>
>>
>>
>>
>>         -- 
>>         Federico D'Ambrosio
>
>
>
>
>     -- 
>     Federico D'Ambrosio
>

Attachment: signature.asc
Description: OpenPGP digital signature

Reply via email to