[
https://issues.apache.org/jira/browse/FLINK-13969?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16962064#comment-16962064
]
Congxian Qiu(klion26) commented on FLINK-13969:
-----------------------------------------------
After thinking it a little deeper, I think the root cause here is that we
triggered a new checkpoint after the job status is not running, so we should
change the logical in {{CheckpointCoordinator to not trigger checkpoint anymore
when job status is not running, what do you think about this [~trohrmann]. If
this is ok, could you please assign this ticket to me?}}
{code:java}
2019-09-05 02:26:19,126 INFO
org.apache.flink.runtime.executiongraph.ExecutionGraph - Job General
purpose test job (5614f2d16ac6f58966f738f0c93a2f63)
switched from state RUNNING to CANCELLING.
2019-09-05 02:26:19,154 INFO
org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Completed
checkpoint 15 for job 5614f2d16ac6f58966f738f0c93a2f63
(628221 bytes in 1824 ms).
2019-09-05 02:26:19,189 INFO
org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Triggering
checkpoint 16 @ 1567650379153 for job 5614f2d16ac6f58
966f738f0c93a2f63.
......
2019-09-05 02:26:19,788 INFO
org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Discarding
checkpoint 16 of job 5614f2d16ac6f58966f738f0c93a2f63.
org.apache.flink.runtime.checkpoint.CheckpointException: Could not complete
snapshot 16 for operator ArtificalKeyedStateMapper_Avro ->
ArtificalOperatorStateMapper (2/4). Failure reason: Checkpoint was declined.
at
org.apache.flink.streaming.api.operators.AbstractStreamOperator.snapshotState(AbstractStreamOperator.java:431)
at
org.apache.flink.streaming.runtime.tasks.StreamTask$CheckpointingOperation.checkpointStreamOperator(StreamTask.java:1302)
at
org.apache.flink.streaming.runtime.tasks.StreamTask$CheckpointingOperation.executeCheckpointing(StreamTask.java:1236)
at
org.apache.flink.streaming.runtime.tasks.StreamTask.checkpointState(StreamTask.java:892)
at
org.apache.flink.streaming.runtime.tasks.StreamTask.performCheckpoint(StreamTask.java:797)
at
org.apache.flink.streaming.runtime.tasks.StreamTask.triggerCheckpointOnBarrier(StreamTask.java:728)
at
org.apache.flink.streaming.runtime.io.CheckpointBarrierHandler.notifyCheckpoint(CheckpointBarrierHandler.java:88)
at
org.apache.flink.streaming.runtime.io.CheckpointBarrierAligner.processBarrier(CheckpointBarrierAligner.java:177)
at
org.apache.flink.streaming.runtime.io.CheckpointedInputGate.pollNext(CheckpointedInputGate.java:155)
at
org.apache.flink.streaming.runtime.io.StreamTaskNetworkInput.pollNextNullable(StreamTaskNetworkInput.java:118)
at
org.apache.flink.streaming.runtime.io.StreamTaskNetworkInput.pollNextNullable(StreamTaskNetworkInput.java:48)
at
org.apache.flink.streaming.runtime.io.StreamOneInputProcessor.processInput(StreamOneInputProcessor.java:144)
at
org.apache.flink.streaming.runtime.tasks.StreamTask.performDefaultAction(StreamTask.java:277)
at
org.apache.flink.streaming.runtime.tasks.mailbox.execution.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:147)
at
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:404)
at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:705)
at org.apache.flink.runtime.taskmanager.Task.run(Task.java:530)
at java.lang.Thread.run(Thread.java:748)
Caused by: java.io.IOException: Cannot register Closeable, registry is already
closed. Closing argument.
at
org.apache.flink.util.AbstractCloseableRegistry.registerCloseable(AbstractCloseableRegistry.java:85)
at
org.apache.flink.runtime.state.AsyncSnapshotCallable$AsyncSnapshotTask.<init>(AsyncSnapshotCallable.java:122)
at
org.apache.flink.runtime.state.AsyncSnapshotCallable$AsyncSnapshotTask.<init>(AsyncSnapshotCallable.java:110)
at
org.apache.flink.runtime.state.AsyncSnapshotCallable.toAsyncSnapshotFutureTask(AsyncSnapshotCallable.java:104)
at
org.apache.flink.contrib.streaming.state.snapshot.RocksIncrementalSnapshotStrategy.doSnapshot(RocksIncrementalSnapshotStrategy.java:170)
at
org.apache.flink.contrib.streaming.state.snapshot.RocksDBSnapshotStrategyBase.snapshot(RocksDBSnapshotStrategyBase.java:126)
at
org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend.snapshot(RocksDBKeyedStateBackend.java:439)
at
org.apache.flink.streaming.api.operators.AbstractStreamOperator.snapshotState(AbstractStreamOperator.java:411)
... 17 more
{code}
> Resuming Externalized Checkpoint (rocks, incremental, scale down) end-to-end
> test fails on Travis
> -------------------------------------------------------------------------------------------------
>
> Key: FLINK-13969
> URL: https://issues.apache.org/jira/browse/FLINK-13969
> Project: Flink
> Issue Type: Bug
> Components: Runtime / Checkpointing
> Affects Versions: 1.10.0
> Reporter: Till Rohrmann
> Priority: Critical
> Labels: test-stability
> Fix For: 1.10.0
>
>
> The {{Resuming Externalized Checkpoint (rocks, incremental, scale down)}}
> end-to-end test fails on Travis because its log contains an exception
> {code}
> org.apache.flink.runtime.checkpoint.CheckpointException: Could not complete
> snapshot 16 for operator ArtificalKeyedStateMapper_Avro ->
> ArtificalOperatorStateMapper (2/4). Failure reason: Checkpoint was declined.
> at
> org.apache.flink.streaming.api.operators.AbstractStreamOperator.snapshotState(AbstractStreamOperator.java:431)
> at
> org.apache.flink.streaming.runtime.tasks.StreamTask$CheckpointingOperation.checkpointStreamOperator(StreamTask.java:1302)
> at
> org.apache.flink.streaming.runtime.tasks.StreamTask$CheckpointingOperation.executeCheckpointing(StreamTask.java:1236)
> at
> org.apache.flink.streaming.runtime.tasks.StreamTask.checkpointState(StreamTask.java:892)
> at
> org.apache.flink.streaming.runtime.tasks.StreamTask.performCheckpoint(StreamTask.java:797)
> at
> org.apache.flink.streaming.runtime.tasks.StreamTask.triggerCheckpointOnBarrier(StreamTask.java:728)
> at
> org.apache.flink.streaming.runtime.io.CheckpointBarrierHandler.notifyCheckpoint(CheckpointBarrierHandler.java:88)
> at
> org.apache.flink.streaming.runtime.io.CheckpointBarrierAligner.processBarrier(CheckpointBarrierAligner.java:177)
> at
> org.apache.flink.streaming.runtime.io.CheckpointedInputGate.pollNext(CheckpointedInputGate.java:155)
> at
> org.apache.flink.streaming.runtime.io.StreamTaskNetworkInput.pollNextNullable(StreamTaskNetworkInput.java:118)
> at
> org.apache.flink.streaming.runtime.io.StreamTaskNetworkInput.pollNextNullable(StreamTaskNetworkInput.java:48)
> at
> org.apache.flink.streaming.runtime.io.StreamOneInputProcessor.processInput(StreamOneInputProcessor.java:144)
> at
> org.apache.flink.streaming.runtime.tasks.StreamTask.performDefaultAction(StreamTask.java:277)
> at
> org.apache.flink.streaming.runtime.tasks.mailbox.execution.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:147)
> at
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:404)
> at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:705)
> at org.apache.flink.runtime.taskmanager.Task.run(Task.java:530)
> at java.lang.Thread.run(Thread.java:748)
> Caused by: java.io.IOException: Cannot register Closeable, registry is
> already closed. Closing argument.
> at
> org.apache.flink.util.AbstractCloseableRegistry.registerCloseable(AbstractCloseableRegistry.java:85)
> at
> org.apache.flink.runtime.state.AsyncSnapshotCallable$AsyncSnapshotTask.<init>(AsyncSnapshotCallable.java:122)
> at
> org.apache.flink.runtime.state.AsyncSnapshotCallable$AsyncSnapshotTask.<init>(AsyncSnapshotCallable.java:110)
> at
> org.apache.flink.runtime.state.AsyncSnapshotCallable.toAsyncSnapshotFutureTask(AsyncSnapshotCallable.java:104)
> at
> org.apache.flink.contrib.streaming.state.snapshot.RocksIncrementalSnapshotStrategy.doSnapshot(RocksIncrementalSnapshotStrategy.java:170)
> at
> org.apache.flink.contrib.streaming.state.snapshot.RocksDBSnapshotStrategyBase.snapshot(RocksDBSnapshotStrategyBase.java:126)
> at
> org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend.snapshot(RocksDBKeyedStateBackend.java:439)
> at
> org.apache.flink.streaming.api.operators.AbstractStreamOperator.snapshotState(AbstractStreamOperator.java:411)
> ... 17 more
> {code}
> https://api.travis-ci.org/v3/job/580915660/log.txt
--
This message was sent by Atlassian Jira
(v8.3.4#803005)