[ https://issues.apache.org/jira/browse/FLINK-2491?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14699299#comment-14699299 ]
Robert Metzger commented on FLINK-2491: --------------------------------------- The issue still persists {code} 1750 [Checkpoint Timer] INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Triggering checkpoint 9 @ 1439805353030 1750 [Checkpoint Timer] INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Checkpoint triggering task Custom Source (2/4) is not being executed at the moment. Aborting checkpoint. 1800 [Checkpoint Timer] INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Triggering checkpoint 10 @ 1439805353080 1800 [Checkpoint Timer] INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Checkpoint triggering task Custom Source (2/4) is not being executed at the moment. Aborting checkpoint. 1850 [Checkpoint Timer] INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Triggering checkpoint 11 @ 1439805353130 1851 [Checkpoint Timer] INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Checkpoint triggering task Custom Source (2/4) is not being executed at the moment. Aborting checkpoint. 1900 [Checkpoint Timer] INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Triggering checkpoint 12 @ 1439805353180 1900 [Checkpoint Timer] INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Checkpoint triggering task Custom Source (2/4) is not being executed at the moment. Aborting checkpoint. 1950 [Checkpoint Timer] INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Triggering checkpoint 13 @ 1439805353230 1951 [Checkpoint Timer] INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Checkpoint triggering task Custom Source (2/4) is not being executed at the moment. Aborting checkpoint. 2000 [Checkpoint Timer] INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Triggering checkpoint 14 @ 1439805353280 2000 [Checkpoint Timer] INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Checkpoint triggering task Custom Source (2/4) is not being executed at the moment. Aborting checkpoint. 2050 [Checkpoint Timer] INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Triggering checkpoint 15 @ 1439805353330 2051 [Checkpoint Timer] INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Checkpoint triggering task Custom Source (2/4) is not being executed at the moment. Aborting checkpoint. 2100 [Checkpoint Timer] INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Triggering checkpoint 16 @ 1439805353380 2100 [Checkpoint Timer] INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Checkpoint triggering task Custom Source (2/4) is not being executed at the moment. Aborting checkpoint. 2150 [Checkpoint Timer] INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Triggering checkpoint 17 @ 1439805353430 2150 [Checkpoint Timer] INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Checkpoint triggering task Custom Source (2/4) is not being executed at the moment. Aborting checkpoint. {code} > Operators are not participating in state checkpointing in some cases > -------------------------------------------------------------------- > > Key: FLINK-2491 > URL: https://issues.apache.org/jira/browse/FLINK-2491 > Project: Flink > Issue Type: Bug > Components: Streaming > Affects Versions: 0.10 > Reporter: Robert Metzger > Assignee: Márton Balassi > Priority: Critical > Fix For: 0.10 > > > While implementing a test case for the Kafka Consumer, I came across the > following bug: > Consider the following topology, with the operator parallelism in parentheses: > Source (2) --> Sink (1). > In this setup, the {{snapshotState()}} method is called on the source, but > not on the Sink. > The sink receives the generated data. > The only one of the two sources is generating data. > I've implemented a test case for this, you can find it here: > https://github.com/rmetzger/flink/blob/para_checkpoint_bug/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ParallelismChangeCheckpoinedITCase.java -- This message was sent by Atlassian JIRA (v6.3.4#6332)