[
https://issues.apache.org/jira/browse/FLINK-9070?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16478522#comment-16478522
]
ASF GitHub Bot commented on FLINK-9070:
---------------------------------------
Github user sihuazhou commented on the issue:
https://github.com/apache/flink/pull/5979
@StephanEwen , I had a micro-benchmark, here is the result
```
---------> Batch VS Put <------------
BATCH: end insert - duration:255
PUT: end insert - duration:545
---------> MapState#Clear New VS Old <------------
---->
NEW: end delete 50 records - duration:1077719
OLD: end delete 50 records - duration:10949887
---->
NEW: end delete 100 records - duration:809182
OLD: end delete 100 records - duration:1617317
---->
NEW: end delete 200 records - duration:1970156
OLD: end delete 200 records - duration:2731749
---->
NEW: end delete 400 records - duration:2492822
OLD: end delete 400 records - duration:13894767
---->
NEW: end delete 800 records - duration:5816919
OLD: end delete 800 records - duration:13017591
```
I tested two things:
- To compare the performance between `WriteBatch()` VS `Put()`
- To compare the performance of the `RocsDBMapState#Clear()`: new version
vs old version
In general, the more records there, the more lift we would get from the new
version, here is my code for the test:
https://github.com/sihuazhou/flink/commit/75504ad6fdb33755cccef43935e007bd5804ea9d
> 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)