You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@flink.apache.org by "Roman Khachatryan (Jira)" <ji...@apache.org> on 2020/11/30 09:12:00 UTC

[jira] [Created] (FLINK-20418) NPE in IteratorSourceReader

Roman Khachatryan created FLINK-20418:
-----------------------------------------

             Summary: 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


{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}

{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}

cc: [~sewen]



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