[ 
https://issues.apache.org/jira/browse/FLINK-6504?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16004556#comment-16004556
 ] 

ASF GitHub Bot commented on FLINK-6504:
---------------------------------------

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

    https://github.com/apache/flink/pull/3859#discussion_r115721158
  
    --- Diff: 
flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
 ---
    @@ -880,7 +882,9 @@ KeyedStateHandle materializeSnapshot() throws Exception 
{
                        sstFiles.putAll(newSstFiles);
                        sstFiles.putAll(oldSstFiles);
     
    -                   stateBackend.materializedSstFiles.put(checkpointId, 
sstFiles);
    +                   synchronized (stateBackend.asyncSnapshotLock) {
    --- End diff --
    
    I wonder if this is enough. As soon as we assume that there can be parallel 
incremental snapshots (let's say s1 and s2) what could happen is the following 
race:
    
    - s1 completes and acknowledges to checkpoint coordinator, but the 
notifyCheckpointComplete did not yet arrive.
    - s2 runs and checks in `materializedSstFiles`, where it can not find 
anything from s1, yet.
    - s2 will perceive some files as new, which s1 has already registered in 
the shared state.
    - notifyCheckpointComplete from s1 arrives.
    - s2 acknowledges. Now, in the `SharedStateRegistry`, the handles from s2 
will try to register some sst files as new, which have been registered by s1 
before (without s1 noticing). On the checkpoint coordinator, the 
`RocksDBIncrementalStateHandle` will fail int the precondition check 
`Preconditions.checkState(referenceCount == 1);` as soon as it tries to 
register it's "new" sst files with `SharedStateRegistry`.


> Lack of synchronization on materializedSstFiles in RocksDBKEyedStateBackend
> ---------------------------------------------------------------------------
>
>                 Key: FLINK-6504
>                 URL: https://issues.apache.org/jira/browse/FLINK-6504
>             Project: Flink
>          Issue Type: Bug
>          Components: State Backends, Checkpointing
>    Affects Versions: 1.3.0
>            Reporter: Stefan Richter
>            Assignee: Xiaogang Shi
>
> Concurrent checkpoints could access `materializedSstFiles` in the 
> `RocksDBStateBackend` concurrently. This should be avoided.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Reply via email to