flink
flink copied to clipboard
[FLINK-28010][state] Use deleteRange to optimize the clear method of RocksDBMapState.
What is the purpose of the change
This pull request will use deleteRange to optimize the clear method of RocksDBMapState.
Brief change log
Replace RocksDB's Iterator with deleteRange.
Verifying this change
This change is already covered by existing tests, such as
- org.apache.flink.contrib.streaming.state.EmbeddedRocksDBStateBackendTest#testMapStateClear
- org.apache.flink.runtime.state.StateBackendTestBase#testMapState
Does this pull request potentially affect one of the following parts:
- Dependencies (does it add or upgrade a dependency): (yes / no)
- The public API, i.e., is any changed class annotated with
@Public(Evolving): (yes / no) - The serializers: (yes / no / don't know)
- The runtime per-record code paths (performance sensitive): (yes / no / don't know)
- Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn, ZooKeeper: (yes / no / don't know)
- The S3 file system connector: (yes / no / don't know)
Documentation
- Does this pull request introduce a new feature? (yes / no)
- If yes, how is the feature documented? (not applicable / docs / JavaDocs / not documented)
CI report:
- 74bc3fa090dd66df0fea161f57da74273c5f0268 Azure: SUCCESS
Bot commands
The @flinkbot bot supports the following commands:@flinkbot run azurere-run the last Azure build
@liming30 Thanks for your PR. Would you like to share some improvements if this PR merged via flink-benchmarks.
@liming30 Thanks for your PR. Would you like to share some improvements if this PR merged via flink-benchmarks.
Hi, @fredia, I wrote the following benchmark on my local Mac and ran the JMH test. deleteRange is almost 10 times faster than Iterator.
@Benchmark
public void mapClear(KeyValue keyValue) throws Exception {
keyedStateBackend.setCurrentKey(keyValue.setUpKey);
mapState.clear();
}
Performance test results using Iterator:
Benchmark (backendType) Mode Cnt Score Error Units
MapStateBenchmark.mapClear ROCKSDB thrpt 30 13.165 ± 2.088 ops/ms
Performance test results using deleteRange:
Benchmark (backendType) Mode Cnt Score Error Units
MapStateBenchmark.mapClear ROCKSDB thrpt 30 121.860 ± 16.605 ops/ms
@liming30 This optimization may increase the number of tombstones. As the blog says, deleteRange would bring some regression on Point Lookups and Range Scans, could you please share the performance of other operations after this optimization?
@fredia If we want to test the performance of RocksDB after deleteRange, the conclusion should have been given in the last part of this blog.
Do you mean that the performance of Point Lookups and Range Scans should be verified again in Flink's benchmark?
Do you mean that the performance of Point Lookups and Range Scans should be verified again in Flink's benchmark?
Not Flink's benchmark, the tests in Flink bechmark are fine-grained. If we have some jobs that use MapState, we can observe the overall TPS of these jobs before and after optimization. BTW, I'm just curious, I'm absolutely fine with this PR.