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

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_r115719033
  
    --- Diff: 
flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
 ---
    @@ -911,9 +915,11 @@ void releaseResources(boolean canceled) {
                        if (canceled) {
                                List<StateObject> statesToDiscard = new 
ArrayList<>();
     
    -                           statesToDiscard.add(metaStateHandle);
    -                           statesToDiscard.addAll(miscFiles.values());
    -                           statesToDiscard.addAll(newSstFiles.values());
    +                           synchronized (this) {
    --- End diff --
    
    Having a second look at this, the suggestion from the jira is not going far 
enough. 
    
    We actually must ensure that all checkpointing efforts (e.g. writing to 
HDFS) have already come to an end, so that no new state handles can be created 
anymore at this point. Otherwise, even if we synchronize, the parallel thread 
could still create a new state handle after the getting back the lock.
    
    However, once this is ensured, also the need for synchronization goes away. 
So I suggest that we rather make sure that cleanup is only performed after all 
checkpointing ended, and that no more checkpointing through this object can run 
after the cleanup.


> 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