[ https://issues.apache.org/jira/browse/FLINK-18473?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17786602#comment-17786602 ]
Hangxiang Yu commented on FLINK-18473: -------------------------------------- Thanks [~fanrui] for pinging me here. [~yigress] It should work when multiple slots within single TM. But Could It go worser for multiple slots within multiple TM ? If true, I'd not suggest to make it as default. BTW, We didn't see this problem in our production environment. So could you also share your scenarios about using this ? Do your jobs have IO bottleneck in one disk ? then you'd like to use multiple HDD disks or multiple cloud disks like EBS ? If we have strong request about sharing multiple disks, I'd also suggest to consider making them become SharedResources just like Memory as you could see in FLINK-29928. It should be better than introducing other component like Zookeeper, but still a bit complex. So let us see whether it worth to do. > Optimize RocksDB disk load balancing strategy > --------------------------------------------- > > Key: FLINK-18473 > URL: https://issues.apache.org/jira/browse/FLINK-18473 > Project: Flink > Issue Type: Improvement > Components: Runtime / State Backends > Affects Versions: 1.12.0 > Reporter: Rui Fan > Priority: Minor > Labels: auto-deprioritized-major, stale-minor > > In general, bigdata servers have many disks. For large-state jobs, if > multiple slots are running on a TM, then each slot will create a RocksDB > instance. We hope that multiple RocksDB instances use different disks to > achieve load balancing. > h3. The problem of current load balancing strategy: > When the current RocksDB is initialized, a random value nextDirectory is > generated according to the number of RocksDB dir: [code > link|https://github.com/apache/flink/blob/2d371eb5ac9a3e485d3665cb9a740c65e2ba2ac6/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java#L441] > {code:java} > nextDirectory = new Random().nextInt(initializedDbBasePaths.length); > {code} > Different slots generate different RocksDBStateBackend objects, so each slot > will generate its own *nextDirectory*. The random algorithm used here, so the > random value generated by different slots may be the same. For example: the > current RocksDB dir is configured with 10 disks, the *nextDirectory* > generated by slot0 and slot1 are both 5, then slot0 and slot1 will use the > same disk. This disk will be under a lot of pressure, other disks will not be > under pressure. > h3. Optimization ideas: > *{{nextDirectory}}* should belong to slot sharing, the initial value of > *{{nextDirectory}}* cannot be 0, it is still generated by random. But define > *nextDirectory* as +_{{static AtomicInteger()}}_+ and execute > +_{{nextDirectory.incrementAndGet()}}_+ every time RocksDBKeyedStateBackend > is applied for. > {{nextDirectory}} takes the remainder of {{initializedDbBasePaths.length}} to > decide which disk to use. > Is there any problem with the above ideas? > -- This message was sent by Atlassian Jira (v8.20.10#820010)