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

Arvid Heise resolved FLINK-20418.
---------------------------------
    Resolution: Fixed

> NPE in IteratorSourceReader
> ---------------------------
>
>                 Key: FLINK-20418
>                 URL: https://issues.apache.org/jira/browse/FLINK-20418
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Task
>    Affects Versions: 1.12.0
>            Reporter: Roman Khachatryan
>            Assignee: Arvid Heise
>            Priority: Blocker
>              Labels: pull-request-available
>             Fix For: 1.12.0
>
>
> With the following job
> {code}
>       @Test
>       public void testNpe() throws Exception {
>               StreamExecutionEnvironment env = 
> StreamExecutionEnvironment.getExecutionEnvironment();
>               env.setParallelism(1);
>               env.setRestartStrategy(new NoRestartStrategyConfiguration());
>               env.enableCheckpointing(50, CheckpointingMode.EXACTLY_ONCE);
>               env
>                       .fromSequence(0, 100)
>                       .map(x -> {
>                               Thread.sleep(10);
>                               return x;
>                       })
>                       .addSink(new DiscardingSink<>());
>               env.execute();
>       }
> {code}
> I (always) get  an exception like this:
> {code}
> ...
> Caused by: java.lang.Exception: Could not perform checkpoint 1 for operator 
> Source: Sequence Source -> Map -> Sink: Unnamed (1/1)#0.
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.triggerCheckpoint(StreamTask.java:866)
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$triggerCheckpointAsync$8(StreamTask.java:831)
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$1.runThrowing(StreamTaskActionExecutor.java:47)
>       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:283)
>       at 
> org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:184)
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:575)
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:539)
>       at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:722)
>       at org.apache.flink.runtime.taskmanager.Task.run(Task.java:547)
>       at java.lang.Thread.run(Thread.java:748)
> Caused by: org.apache.flink.runtime.checkpoint.CheckpointException: Could not 
> complete snapshot 1 for operator Source: Sequence Source -> Map -> Sink: 
> Unnamed (1/1)#0. Failure reason: Checkpoint was declined.
>       at 
> org.apache.flink.streaming.api.operators.StreamOperatorStateHandler.snapshotState(StreamOperatorStateHandler.java:226)
>       at 
> org.apache.flink.streaming.api.operators.StreamOperatorStateHandler.snapshotState(StreamOperatorStateHandler.java:158)
>       at 
> org.apache.flink.streaming.api.operators.AbstractStreamOperator.snapshotState(AbstractStreamOperator.java:343)
>       at 
> org.apache.flink.streaming.runtime.tasks.SubtaskCheckpointCoordinatorImpl.checkpointStreamOperator(SubtaskCheckpointCoordinatorImpl.java:603)
>       at 
> org.apache.flink.streaming.runtime.tasks.SubtaskCheckpointCoordinatorImpl.buildOperatorSnapshotFutures(SubtaskCheckpointCoordinatorImpl.java:529)
>       at 
> org.apache.flink.streaming.runtime.tasks.SubtaskCheckpointCoordinatorImpl.takeSnapshotSync(SubtaskCheckpointCoordinatorImpl.java:496)
>       at 
> org.apache.flink.streaming.runtime.tasks.SubtaskCheckpointCoordinatorImpl.checkpointState(SubtaskCheckpointCoordinatorImpl.java:266)
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$performCheckpoint$9(StreamTask.java:924)
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$1.runThrowing(StreamTaskActionExecutor.java:47)
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.performCheckpoint(StreamTask.java:914)
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.triggerCheckpoint(StreamTask.java:857)
>       ... 10 more
> Caused by: java.lang.NullPointerException
>       at 
> org.apache.flink.api.connector.source.lib.util.IteratorSourceReader.snapshotState(IteratorSourceReader.java:132)
>       at 
> org.apache.flink.streaming.api.operators.SourceOperator.snapshotState(SourceOperator.java:264)
>       at 
> org.apache.flink.streaming.api.operators.StreamOperatorStateHandler.snapshotState(StreamOperatorStateHandler.java:197)
>       ... 20 more
> {code}
> Adding a null check solves the issue. But if I then change sleep time from 10 
> to 50 I get
> {code}
> Caused by: java.lang.IllegalArgumentException: 'from' must be <= 'to'
>       at 
> org.apache.flink.util.Preconditions.checkArgument(Preconditions.java:142)
>       at 
> org.apache.flink.api.connector.source.lib.NumberSequenceSource$NumberSequenceSplit.<init>(NumberSequenceSource.java:142)
>       at 
> org.apache.flink.api.connector.source.lib.NumberSequenceSource$NumberSequenceSplit.getUpdatedSplitForIterator(NumberSequenceSource.java:169)
>       at 
> org.apache.flink.api.connector.source.lib.NumberSequenceSource$NumberSequenceSplit.getUpdatedSplitForIterator(NumberSequenceSource.java:135)
>       at 
> org.apache.flink.api.connector.source.lib.util.IteratorSourceReader.snapshotState(IteratorSourceReader.java:135)
>       at 
> org.apache.flink.streaming.api.operators.SourceOperator.snapshotState(SourceOperator.java:264)
>       at 
> org.apache.flink.streaming.api.operators.StreamOperatorStateHandler.snapshotState(StreamOperatorStateHandler.java:197)
>       ... 20 more
> {code}



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

Reply via email to