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

Kai Chen commented on FLINK-18196:
----------------------------------

hi [~pnowojski] [~roman_khachatryan] ,it's true that upgrading JDK version can 
fix this NPE and i agree that this must be some weird JDK bug. However 
upgrating JDK version is a big work in production cluster. So i offered this 
patch for the ones who meet the same problem with me.

I didn't dig into this JDK bug. I was confused with this NPE at first .Then i 
debugged and found the ‘checkpointMetaData’ probably be swept by gc(which 
should not happen). I tried to create a new CheckpointMetaData in the MailBox 
lambda expression to avoid it and it worked! I‘m looking forward to learning 
the root cause.

> flink throws `NullPointerException` when executeCheckpointing
> -------------------------------------------------------------
>
>                 Key: FLINK-18196
>                 URL: https://issues.apache.org/jira/browse/FLINK-18196
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Checkpointing
>    Affects Versions: 1.10.0, 1.10.1
>         Environment: flink version: flink-1.10.0 flink-1.10.1
> jdk version: jdk1.8.0_40
>            Reporter: Kai Chen
>            Priority: Major
>             Fix For: 1.12.0, 1.10.3, 1.11.3
>
>
> I meet checkpoint NPE when executing wordcount example:
> java.lang.Exception: Could not perform checkpoint 5505 for operator Source: 
> KafkaTableSource(xxx) > SourceConversion(table=[xxx, source: 
> [KafkaTableSource(xxx)]], fields=[xxx]) > Calc(select=[xxx) AS xxx]) > 
> SinkConversionToTuple2  --  > Sink: Elasticsearch6UpsertTableSink(xxx) (1/1).
>     at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.triggerCheckpoint(StreamTask.java:802)
>     at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$triggerCheckpointAsync$3(StreamTask.java:777)
>     at 
> org.apache.flink.streaming.runtime.tasks.StreamTask$$Lambda$228/1024478318.call(Unknown
>  Source)
>     at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>     at 
> org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$SynchronizedStreamTaskActionExecutor.run(StreamTaskActionExecutor.java:87)
>     at org.apache.flink.streaming.runtime.tasks.mailbox.Mail.run(Mail.java:78)
>     at 
> org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.processMail(MailboxProcessor.java:261)
>     at 
> org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:186)
>     at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:487)
>     at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:470)
>     at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:707)
>     at org.apache.flink.runtime.taskmanager.Task.run(Task.java:532)
>     at java.lang.Thread.run(Thread.java:745)
> Caused by: java.lang.NullPointerException
>     at 
> org.apache.flink.streaming.runtime.tasks.StreamTask$CheckpointingOperation.executeCheckpointing(StreamTask.java:1411)
>     at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.checkpointState(StreamTask.java:991)
>     at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$performCheckpoint$5(StreamTask.java:887)
>     at 
> org.apache.flink.streaming.runtime.tasks.StreamTask$$Lambda$229/1010499540.run(Unknown
>  Source)
>     at 
> org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$SynchronizedStreamTaskActionExecutor.runThrowing(StreamTaskActionExecutor.java:94)
>     at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.performCheckpoint(StreamTask.java:860)
>     at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.triggerCheckpoint(StreamTask.java:793)
>     ... 12 more
>  
> checkpoint configuration
> |Checkpointing Mode|Exactly Once|
> |Interval|5s|
> |Timeout|10m 0s|
> |Minimum Pause Between Checkpoints|0ms|
> |Maximum Concurrent Checkpoints|1|
> With debug enabled, I found `checkpointMetaData`  is null at
>  
> [https://github.com/apache/flink/blob/release-1.10.0/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java#L1421]
>  
> I fixed this with this patch: 
> [https://github.com/yuchuanchen/flink/commit/e5122d9787be1fee9bce141887e0d70c9b0a4f19]



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

Reply via email to