The exceptions we got were mostly about the connection with some task
managers being lost and the only way in which we could solve this was by
increasing memory. The reason why we increased the managed fraction was
mostly for improving performance (by giving more memory to rocksdb for its
cache). We have also tried giving lower values, such as 0.25, 0.1 and 0.01.
As for the memory consumption, in all cases the job demands all the
available memory in the long run...performance degrades, though, the lower
the value. I guess what we are trying to achieve is to operate with some
visible margin since seeing the job taking all the memory seems dangerous.
When this happens, eventually snapshots start to fail and eventually the
job itself becomes unstable (maybe after some days).

On 2022/02/07 08:10:37 Chesnay Schepler wrote:
> What where the errors you received that caused you to increase the
> amount of memory you provided to RocksDB?
>
> On 05/02/2022 07:12, Salva Alcántara wrote:
> > I have a job running on Flink 1.14.3 (Java 11) that uses rocksdb as
> > the state backend. The problem is that the job requires an amount of
> > memory pretty similar to the overall state size.
> >
> > Indeed, for making it stable (and capable of taking snapshots) this is
> > what I'm using:
> >
> > - 4 TMs with 30 GB of RAM and 7 CPUs
> > - Everything is run on top of Kubernetes on AWS using nodes with 32 GB
> > of RAM and locally attached SSD disks (M5ad instances for what it's
worth)
> >
> > I have these settings in place:
> >
> > ```
> > state.backend: rocksdb
> > state.backend.incremental: 'true'
> > state.backend.rocksdb.localdir: /opt/flink/rocksdb <-- SSD volume (see
> > below)
> > state.backend.rocksdb.memory.managed: 'true'
> > state.backend.rocksdb.predefined-options: FLASH_SSD_OPTIMIZED
> > taskmanager.memory.managed.fraction: '0.9'
> > taskmanager.memory.framework.off-heap.size: 512mb
> > taskmanager.numberOfTaskSlots: '4' (parallelism: 16)
> > ```
> >
> > Also this:
> >
> > ```
> >   - name: rocksdb-volume
> >     volume:
> >       emptyDir:
> >         sizeLimit: 100Gi
> >       name: rocksdb-volume
> >     volumeMount:
> >       mountPath: /opt/flink/rocksdb
> > ```
> >
> > Which provides plenty of disk space for each task manager. With those
> > settings, the job runs smoothly and in particular there is a
> > relatively big memory margin. Problem is that memory consumption
> > slowly increases and also that with less memory margin snapshots fail.
> > I have tried reducing the number of taskmanagers but I need 4. Same
> > with the amount of RAM, I have tried giving e.g. 16 GB instead of 30
> > GB but same problem. Another setting that has worked for us is using 8
> > TMs each with 16 GB of RAM, but again, this leads to the same amount
> > of memory overall as the current settings. Even with that amount of
> > memory, I can see that memory keeps growing and will probably lead to
> > a bad end...
> >
> > Also, the latest snapshot took around 120 GBs, so as you can see I am
> > using an amount of RAM similar to the size of the total state, which
> > defeats the whole purpose of using a disk-based state backend
> > (rocksdb) plus local SSDs.
> >
> > Is there an effective way of limiting the memory that rocksdb takes
> > (to that available on the running pods)? Nothing I have found/tried
> > out so far has worked. Theoretically, the images I am using have
> > `jemalloc` in place for memory allocation, which should avoid memory
> > fragmentation issues observed with malloc in the past.
> >
> > PS: I posted the issue in SO too:
> >
https://stackoverflow.com/questions/70986020/flink-job-requiring-a-lot-of-memory-despite-using-rocksdb-state-backend
>
>
>

Reply via email to