[jira] [Commented] (FLINK-9070) Improve performance of RocksDBMapState.clear()
[ https://issues.apache.org/jira/browse/FLINK-9070?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16480382#comment-16480382 ] ASF GitHub Bot commented on FLINK-9070: --- Github user asfgit closed the pull request at: https://github.com/apache/flink/pull/5979 > 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)
[jira] [Commented] (FLINK-9070) Improve performance of RocksDBMapState.clear()
[ https://issues.apache.org/jira/browse/FLINK-9070?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16479152#comment-16479152 ] ASF GitHub Bot commented on FLINK-9070: --- Github user sihuazhou commented on the issue: https://github.com/apache/flink/pull/5979 Thank you all @StefanRRichter @StephanEwen @bowenli86 > 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)
[jira] [Commented] (FLINK-9070) Improve performance of RocksDBMapState.clear()
[ https://issues.apache.org/jira/browse/FLINK-9070?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16479135#comment-16479135 ] ASF GitHub Bot commented on FLINK-9070: --- Github user StefanRRichter commented on the issue: https://github.com/apache/flink/pull/5979 LGTM, will merge. > 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)
[jira] [Commented] (FLINK-9070) Improve performance of RocksDBMapState.clear()
[ https://issues.apache.org/jira/browse/FLINK-9070?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16479112#comment-16479112 ] ASF GitHub Bot commented on FLINK-9070: --- Github user sihuazhou commented on the issue: https://github.com/apache/flink/pull/5979 Hi @StefanRRichter I rebased the PR, could you please have a look? > 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)
[jira] [Commented] (FLINK-9070) Improve performance of RocksDBMapState.clear()
[ https://issues.apache.org/jira/browse/FLINK-9070?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16479105#comment-16479105 ] ASF GitHub Bot commented on FLINK-9070: --- Github user sihuazhou commented on the issue: https://github.com/apache/flink/pull/5979 Hi @StefanRRichter If I'm not sure whether we can do that without `seek()`, because the `key bytes` is length is not fixed which may lead to delete wrongly, What do you think? Sure, rebasing... > 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)
[jira] [Commented] (FLINK-9070) Improve performance of RocksDBMapState.clear()
[ https://issues.apache.org/jira/browse/FLINK-9070?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16479079#comment-16479079 ] ASF GitHub Bot commented on FLINK-9070: --- Github user StefanRRichter commented on the issue: https://github.com/apache/flink/pull/5979 With the approach I outlined, we would not require any `seek()` to the last key, we can simply create the exclusive end key. Nevertheless, you are right about the comment that is only in the C++ code but not in the Java API. So I am fine with the current implementation. Could you please rebase on the latest master so that your code can use `RocksIteratorWrapper`? > 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)
[jira] [Commented] (FLINK-9070) Improve performance of RocksDBMapState.clear()
[ https://issues.apache.org/jira/browse/FLINK-9070?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16479073#comment-16479073 ] ASF GitHub Bot commented on FLINK-9070: --- Github user sihuazhou commented on the issue: https://github.com/apache/flink/pull/5979 Hmm...there is another reason, indeed the mainly performance overhead is the `seek()`. Even though we use the `deleteRange()` to implement this, we also need to get the last key of the entries which means we also need to iterate all the entries. So, the `deleteRange()` approach seems won't make a obvious lift then this approach. > 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)
[jira] [Commented] (FLINK-9070) Improve performance of RocksDBMapState.clear()
[ https://issues.apache.org/jira/browse/FLINK-9070?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16479058#comment-16479058 ] ASF GitHub Bot commented on FLINK-9070: --- Github user sihuazhou commented on the issue: https://github.com/apache/flink/pull/5979 @StefanRRichter , the reason I prefer this approach is that: - From the comment in RocksDB's source we can find that deleteRange() should be used for deleting big range, what if the entries num of the map is not that big. - From the comments we can also find that deleteRange() would hurt the read performance, so we should consider to set ReadOptions::ignore_range_deletions = true to avoid the negative effect by deleteRange(), but if we use it for MapState.clear(), it seems that we can't set ReadOptions::ignore_range_deletions = true. And current approach should not bring any downside, what do you think? > 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)
[jira] [Commented] (FLINK-9070) Improve performance of RocksDBMapState.clear()
[ https://issues.apache.org/jira/browse/FLINK-9070?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16479049#comment-16479049 ] ASF GitHub Bot commented on FLINK-9070: --- Github user StefanRRichter commented on the issue: https://github.com/apache/flink/pull/5979 @sihuazhou I wonder why you would chose iterator + batched write over simply calling `db.deleteRange(...)` where start key is `serializeCurrentKeyAndNamespace()` and end key is increasing the last byte of the start key by one. That seems like an even better idea to me, what do you think? > 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)
[jira] [Commented] (FLINK-9070) Improve performance of RocksDBMapState.clear()
[ https://issues.apache.org/jira/browse/FLINK-9070?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16479029#comment-16479029 ] ASF GitHub Bot commented on FLINK-9070: --- Github user StephanEwen commented on the issue: https://github.com/apache/flink/pull/5979 Okay, looks really good from my side. Would be good if @StefanRRichter or @azagrebin to double check the change, otherwise good to go. > 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)
[jira] [Commented] (FLINK-9070) Improve performance of RocksDBMapState.clear()
[ 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)
[jira] [Commented] (FLINK-9070) Improve performance of RocksDBMapState.clear()
[ https://issues.apache.org/jira/browse/FLINK-9070?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16477867#comment-16477867 ] ASF GitHub Bot commented on FLINK-9070: --- Github user StephanEwen commented on the issue: https://github.com/apache/flink/pull/5979 Could you share some micro-benchmark numbers? When we change something that we know works well to something new, would be good to understand what benefits we are talking about. > 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)
[jira] [Commented] (FLINK-9070) Improve performance of RocksDBMapState.clear()
[ https://issues.apache.org/jira/browse/FLINK-9070?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16470880#comment-16470880 ] ASF GitHub Bot commented on FLINK-9070: --- Github user bowenli86 commented on the issue: https://github.com/apache/flink/pull/5979 LGTM +1 > 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)
[jira] [Commented] (FLINK-9070) Improve performance of RocksDBMapState.clear()
[ https://issues.apache.org/jira/browse/FLINK-9070?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16470129#comment-16470129 ] ASF GitHub Bot commented on FLINK-9070: --- Github user sihuazhou commented on a diff in the pull request: https://github.com/apache/flink/pull/5979#discussion_r187278034 --- Diff: flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBMapState.java --- @@ -356,6 +369,20 @@ private UV deserializeUserValue(byte[] rawValueBytes, TypeSerializer valueSe return isNull ? null : valueSerializer.deserialize(in); } + private boolean startWithKeyPrefix(byte[] keyPrefixBytes, byte[] rawKeyBytes) { + if (rawKeyBytes.length < keyPrefixBytes.length) { + return false; + } + + for (int i = keyPrefixBytes.length; --i >= backend.getKeyGroupPrefixBytes(); ) { --- End diff -- Ah, this is a function that simply polled out from the `RocksDBMapState#RocksDBMapIterator`, in`RocksDBMapState#RocksDBMapIterator` it's name was `underSameKey()` , I didn't change any code related to it's implementation. But concern this loop, yes, I was the person that written this loop for this method(`underSameKey()`), and your suggestion was the first version that I implemented it, but during the reviewing by @StefanRRichter , he suggest that the current style, and I feel that looks more simpler, so I made it into the current shape finally, personally I would't against the current version.Ah, this is a function that simply polled out from the `RocksDBMapState#RocksDBMapIterator`, in`RocksDBMapState#RocksDBMapIterator` it's name was `underSameKey()` , I didn't change any code related to it's implementation. But concern this loop, yes, I was the person that written this loop for this method(`underSameKey()`), and your suggestion was the first version that I implemented it, but during the reviewing by @StefanRRichter , he suggest that the current style, and I feel that looks more simpler, so I made it into the current shape finally, personally I would't against the current version. > 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)
[jira] [Commented] (FLINK-9070) Improve performance of RocksDBMapState.clear()
[ https://issues.apache.org/jira/browse/FLINK-9070?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16470004#comment-16470004 ] ASF GitHub Bot commented on FLINK-9070: --- Github user bowenli86 commented on a diff in the pull request: https://github.com/apache/flink/pull/5979#discussion_r187255609 --- Diff: flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBMapState.java --- @@ -356,6 +369,20 @@ private UV deserializeUserValue(byte[] rawValueBytes, TypeSerializer valueSe return isNull ? null : valueSerializer.deserialize(in); } + private boolean startWithKeyPrefix(byte[] keyPrefixBytes, byte[] rawKeyBytes) { + if (rawKeyBytes.length < keyPrefixBytes.length) { + return false; + } + + for (int i = keyPrefixBytes.length; --i >= backend.getKeyGroupPrefixBytes(); ) { --- End diff -- I recommend moving `--i` to the increment part of the `for` loop, instead of keeping it in the termination part > 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)
[jira] [Commented] (FLINK-9070) Improve performance of RocksDBMapState.clear()
[ https://issues.apache.org/jira/browse/FLINK-9070?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16469054#comment-16469054 ] ASF GitHub Bot commented on FLINK-9070: --- Github user sihuazhou commented on the issue: https://github.com/apache/flink/pull/5979 cc @StefanRRichter (This is for 1.6, I just complete it when I have time currently) > 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)
[jira] [Commented] (FLINK-9070) Improve performance of RocksDBMapState.clear()
[ 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 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)
[jira] [Commented] (FLINK-9070) Improve performance of RocksDBMapState.clear()
[ https://issues.apache.org/jira/browse/FLINK-9070?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16432548#comment-16432548 ] Sihua Zhou commented on FLINK-9070: --- Thanks, I will make a PR for this ticket for flink 1.6 once flink 1.5 is released out. > 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 >Priority: Minor > 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)
[jira] [Commented] (FLINK-9070) Improve performance of RocksDBMapState.clear()
[ https://issues.apache.org/jira/browse/FLINK-9070?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16432213#comment-16432213 ] Truong Duc Kien commented on FLINK-9070: Sure. Go ahead. > 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 >Priority: Minor > > 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)
[jira] [Commented] (FLINK-9070) Improve performance of RocksDBMapState.clear()
[ https://issues.apache.org/jira/browse/FLINK-9070?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16430550#comment-16430550 ] Sihua Zhou commented on FLINK-9070: --- Hi [~kien_truong] nice catch! Thanks for your reviews, if you don't mind, I'd like to make a PR for this issue base on the code I posted above? > 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 >Priority: Minor > > 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)
[jira] [Commented] (FLINK-9070) Improve performance of RocksDBMapState.clear()
[ https://issues.apache.org/jira/browse/FLINK-9070?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16430525#comment-16430525 ] Truong Duc Kien commented on FLINK-9070: This is definitely a safer way than my idea and should not bring any downside. It makes sense to skip the RocksDBMapIterator's cache for deletion, and using WriteBatch is also a nice touch. Still, I think you made a mistake of breaking immediately after the removal operation in line 232, shouldn't it only break when the condition is false ? > 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 >Priority: Minor > > 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)
[jira] [Commented] (FLINK-9070) Improve performance of RocksDBMapState.clear()
[ https://issues.apache.org/jira/browse/FLINK-9070?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16429383#comment-16429383 ] Sihua Zhou commented on FLINK-9070: --- Hi [~kien_truong] I also share my idea for improving this here [https://github.com/sihuazhou/flink/commit/6ff501416487608633e19aea7ff32c181e6e09b7|https://github.com/sihuazhou/flink/commit/6ff501416487608633e19aea7ff32c181e6e09b7] It 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. What do you think? > 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 >Priority: Minor > > 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)
[jira] [Commented] (FLINK-9070) Improve performance of RocksDBMapState.clear()
[ https://issues.apache.org/jira/browse/FLINK-9070?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16417588#comment-16417588 ] Sihua Zhou commented on FLINK-9070: --- Hi [~kien_truong] Thanks for sharing the code and now I have a good understand the optimization solution you proposed using {{deleteRange()}}. I only have two question: - From the [comment|https://github.com/facebook/rocksdb/blob/master/include/rocksdb/db.h#L284-L299] in RocksDB's source we can find that deleteRange() should be used for deleting big range, what if the entries num of the map is not that big? - From the comments we can also find that deleteRange() would hurt the read performance, so we should consider to set ReadOptions::ignore_range_deletions = true to avoid the negative effect by {{deleteRange()}}, but if we use it for MapState.clear(), it seems that we can't set ReadOptions::ignore_range_deletions = true? > 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 >Priority: Minor > > 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)
[jira] [Commented] (FLINK-9070) Improve performance of RocksDBMapState.clear()
[ https://issues.apache.org/jira/browse/FLINK-9070?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16415609#comment-16415609 ] Truong Duc Kien commented on FLINK-9070: So here's my idea for RocksDBMapState.clear() with DeleteRange. [https://github.com/dikei/flink/commit/dc887af5b48e8f9d2f1e1f5b612efd2ebae5a801] It bases on the assumtion that for all key Ki with prefix P we have P < Ki < Q, where Q is a byte array with the same length as P, and there's no byte array with the same length between them lexicographically. Therefore, we can safely call deleteRange(P, Q) to delete all Ki and not touching anything else. > 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 >Priority: Minor > > 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)
[jira] [Commented] (FLINK-9070) Improve performance of RocksDBMapState.clear()
[ https://issues.apache.org/jira/browse/FLINK-9070?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16413567#comment-16413567 ] Truong Duc Kien commented on FLINK-9070: Yeah, that bug with DeleteRange was fixed, I'm just concerned about the overall stability of that method, since the bug's discovery was quite recent. Also there's this [comment|https://github.com/facebook/rocksdb/blob/master/include/rocksdb/db.h#L284-L299] in RocksDB's source. I'll put an implementation for RocksDBMapState.clear() using DeleteRange up for review soon. > 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 >Priority: Minor > > 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)
[jira] [Commented] (FLINK-9070) Improve performance of RocksDBMapState.clear()
[ https://issues.apache.org/jira/browse/FLINK-9070?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16413431#comment-16413431 ] Sihua Zhou commented on FLINK-9070: --- Hi [~kien_truong], this is interesting. I agree with you that the {{RocksDBMapState.clear()}} can be improved, but I don't know how to improve it via {{DeleteRange}} or {{DeleteFilesInRange}} (I only know how to improve this via reduce seek times), do you have a detail suggestion? By the way, AFAIK the bug of {{DeleteRange}} has been fixed since 5.7.3, am I mistake? I ask this because there is another improvement issue will use it. [FLINK-8790|https://issues.apache.org/jira/browse/FLINK-8790]. > 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 >Priority: Minor > > 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)