Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-18473

Optimize RocksDB disk load balancing strategy

    XMLWordPrintableJSON

Details

    Description

      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.

      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

      nextDirectory = new Random().nextInt(initializedDbBasePaths.length);
      

      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.

      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?

       

      Attachments

        Activity

          People

            Unassigned Unassigned
            fanrui Rui Fan
            Votes:
            0 Vote for this issue
            Watchers:
            10 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: