Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2629#discussion_r86340921
  
    --- Diff: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
 ---
    @@ -540,15 +540,12 @@ private boolean performCheckpoint(CheckpointMetaData 
checkpointMetaData) throws
     
                synchronized (lock) {
                        if (isRunning) {
    +                           checkpointState(checkpointMetaData);
     
    -                           // Since both state checkpointing and 
downstream barrier emission occurs in this
    -                           // lock scope, they are an atomic operation 
regardless of the order in which they occur.
    -                           // Given this, we immediately emit the 
checkpoint barriers, so the downstream operators
    -                           // can start their checkpoint work as soon as 
possible
    +                           // broadcast barriers after snapshot operators' 
states.
                                operatorChain.broadcastCheckpointBarrier(
    -                                           
checkpointMetaData.getCheckpointId(), checkpointMetaData.getTimestamp());
    -
    -                           checkpointState(checkpointMetaData);
    +                                           
checkpointMetaData.getCheckpointId(), checkpointMetaData.getTimestamp()
    +                           );
    --- End diff --
    
    Changing the order of broadcasting the checkpoint barrier to downstream 
operators and performing the actual checkpoint is very problematic. First 
drawing the checkpoint and then sending the checkpoint barrier downstream will 
add the latency of creating the checkpoint to the barrier for each operator. 
This will considerably influence the overall checkpointing time, because the 
individual checkpoints are effectively executed sequentially. Thus, I fear that 
this change is not possible. 
    
    Is the problem that the `AsyncWaitOperator#Emitter` can emit results while 
we're drawing the checkpoints? I think the proper way to solve this problem is 
to let the output operation happen under the checkpoint lock. That way, you are 
sure that no element will be outputted while the checkpointing is active.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---

Reply via email to