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

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

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

    https://github.com/apache/flink/pull/4798#discussion_r146161007
  
    --- Diff: 
flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
 ---
    @@ -313,10 +317,16 @@ public void dispose() {
                IOUtils.closeQuietly(dbOptions);
                IOUtils.closeQuietly(columnOptions);
     
    +           cleanInstanceBasePath();
    --- End diff --
    
    This again would not need the existence check that runs inside the method, 
because at this point the directory should always exist.


> Proactively cleanup local FS for RocksDBKeyedStateBackend on startup
> --------------------------------------------------------------------
>
>                 Key: FLINK-6505
>                 URL: https://issues.apache.org/jira/browse/FLINK-6505
>             Project: Flink
>          Issue Type: Sub-task
>          Components: State Backends, Checkpointing
>    Affects Versions: 1.3.0
>            Reporter: Stefan Richter
>            Assignee: Bowen Li
>             Fix For: 1.4.0
>
>
> In {{RocksDBKeyedStateBackend}}, the {{instanceBasePath}} is cleared on 
> {{dispose()}}. I think it might make sense to also clear this directory when 
> the backend is created, in case something crashed and the backend never 
> reached {{dispose()}}. At least for previous runs of the same job, we can 
> know what to delete on restart. 
> In general, it is very important for this backend to clean up the local FS, 
> because the local quota might be very limited compared to the DFS. And a node 
> that runs out of local disk space can bring down the whole job, with no way 
> to recover (it might always get rescheduled to that node).



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to