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

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

GitHub user sihuazhou opened a pull request:

    https://github.com/apache/flink/pull/5979

    [FLINK-9070][state]improve the performance of RocksDBMapState.clear() 

    ## What is the purpose of the change
    
    This PR intend to improve the performance of `RocksDBMapState.clear()` base 
on the follow things:
    
    - Using RocksIterator to iterate the records directly (currently we use the 
RocksDBMapIterator, witch will buffer the records and may also need to perform 
seeking multi times.)
    - Using WriteBatch to perform deleting in bulk.
    
    
    ## Brief change log
    
      - *improve the `RocksDBMapState.clear()` via iterating the records 
directly and deleting records using WriteBatch.*
    
    ## Verifying this change
    
    This change is already covered by existing tests,.
    
    ## Does this pull request potentially affect one of the following parts:
    
      - Dependencies (does it add or upgrade a dependency): (no)
      - The public API, i.e., is any changed class annotated with 
`@Public(Evolving)`: (no)
      - The serializers: (no)
      - The runtime per-record code paths (performance sensitive): (no)
      - Anything that affects deployment or recovery: JobManager (and its 
components), Checkpointing, Yarn/Mesos, ZooKeeper: (no)
      - The S3 file system connector: (no)
    
    ## Documentation
    
    no

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/sihuazhou/flink improveMapStateClear

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/5979.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #5979
    
----
commit 969036a32b49b32a4168afb747dff54fe9ada6e6
Author: sihuazhou <summerleafs@...>
Date:   2018-04-07T13:52:05Z

    improve the RocksDBMapState.clear() via iterate the records directly and 
delete records using WriteBatch.

----


> Improve performance of RocksDBMapState.clear()
> ----------------------------------------------
>
>                 Key: FLINK-9070
>                 URL: https://issues.apache.org/jira/browse/FLINK-9070
>             Project: Flink
>          Issue Type: Improvement
>          Components: State Backends, Checkpointing
>    Affects Versions: 1.6.0
>            Reporter: Truong Duc Kien
>            Assignee: Sihua Zhou
>            Priority: Major
>             Fix For: 1.6.0
>
>
> Currently, RocksDBMapState.clear() is implemented by iterating over all the 
> keys and drop them one by one. This iteration can be quite slow with: 
>  * Large maps
>  * High-churn maps with a lot of tombstones
> There are a few methods to speed-up deletion for a range of keys, each with 
> their own caveats:
>  * DeleteRange: still experimental, likely buggy
>  * DeleteFilesInRange + CompactRange: only good for large ranges
>  
> Flink can also keep a list of inserted keys in-memory, then directly delete 
> them without having to iterate over the Rocksdb database again. 
>  
> Reference:
>  * [RocksDB article about range 
> deletion|https://github.com/facebook/rocksdb/wiki/Delete-A-Range-Of-Keys]
>  * [Bug in DeleteRange|https://pingcap.com/blog/2017-09-08-rocksdbbug]
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to