Hi Zakelly, Yeah that makes sense to me, I was just curious about whether reading could be a bottleneck or not, but I imagine user-specific logic would be better than a generic cache from Flink that might habe a low hit rate.
Thanks again, Alexis. On Mon, 19 Feb 2024, 07:29 Zakelly Lan, <zakelly....@gmail.com> wrote: > Hi Alexis, > > Assuming the bulk load for a batch of sequential keys performs better than > accessing them one by one, the main problem comes to do we really need to > access all the keys that were bulk-loaded to cache before. In other words, > cache hit rate is the key issue. If the rate is high, even though a single > key-value is large and loading them is slow, it is still worth it to load > them in advance. In case of timer and iteration (which I missed in last > mail), the cache is almost guaranteed to hit. Thus a cache is introduced to > enhance the performance here. > > > Best, > Zakelly > > On Sun, Feb 18, 2024 at 7:42 PM Alexis Sarda-Espinosa < > sarda.espin...@gmail.com> wrote: > >> Hi Zakelly, >> >> thanks for the information, that's interesting. Would you say that >> reading a subset from RocksDB is fast enough to be pretty much negligible, >> or could it be a bottleneck if the state of each key is "large"? Again >> assuming the number of distinct partition keys is large. >> >> Regards, >> Alexis. >> >> On Sun, 18 Feb 2024, 05:02 Zakelly Lan, <zakelly....@gmail.com> wrote: >> >>> Hi Alexis, >>> >>> Flink does need some heap memory to bridge requests to rocksdb and >>> gather the results. In most cases, the memory is discarded immediately >>> (eventually collected by GC). In case of timers, flink do cache a limited >>> subset of key-values in heap to improve performance. >>> >>> In general you don't need to consider its heap consumption since it is >>> minor. >>> >>> >>> Best, >>> Zakelly >>> >>> On Fri, Feb 16, 2024 at 4:43 AM Asimansu Bera <asimansu.b...@gmail.com> >>> wrote: >>> >>>> Hello Alexis, >>>> >>>> I don't think data in RocksDB resides in JVM even with function calls. >>>> >>>> For more details, check the link below: >>>> >>>> https://github.com/facebook/rocksdb/wiki/RocksDB-Overview#3-high-level-architecture >>>> >>>> RocksDB has three main components - memtable, sstfile and WAL(not used >>>> in Flink as Flink uses checkpointing). When TM starts with statebackend as >>>> RocksDB,TM has its own RocksDB instance and the state is managed as column >>>> Family by that TM. Any changes of state go into memtable --> sst--> >>>> persistent store. When read, data goes to the buffers and cache of RocksDB. >>>> >>>> In the case of RocksDB as state backend, JVM still holds threads stack >>>> as for high degree of parallelism, there are many >>>> stacks maintaining separate thread information. >>>> >>>> Hope this helps!! >>>> >>>> >>>> >>>> >>>> >>>> On Thu, Feb 15, 2024 at 11:21 AM Alexis Sarda-Espinosa < >>>> sarda.espin...@gmail.com> wrote: >>>> >>>>> Hi Asimansu >>>>> >>>>> The memory RocksDB manages is outside the JVM, yes, but the mentioned >>>>> subsets must be bridged to the JVM somehow so that the data can be exposed >>>>> to the functions running inside Flink, no? >>>>> >>>>> Regards, >>>>> Alexis. >>>>> >>>>> >>>>> On Thu, 15 Feb 2024, 14:06 Asimansu Bera, <asimansu.b...@gmail.com> >>>>> wrote: >>>>> >>>>>> Hello Alexis, >>>>>> >>>>>> RocksDB resides off-heap and outside of JVM. The small subset of data >>>>>> ends up on the off-heap in the memory. >>>>>> >>>>>> For more details, check the following link: >>>>>> >>>>>> https://nightlies.apache.org/flink/flink-docs-release-1.18/docs/deployment/memory/mem_setup_tm/#managed-memory >>>>>> >>>>>> I hope this addresses your inquiry. >>>>>> >>>>>> >>>>>> >>>>>> >>>>>> On Thu, Feb 15, 2024 at 12:52 AM Alexis Sarda-Espinosa < >>>>>> sarda.espin...@gmail.com> wrote: >>>>>> >>>>>>> Hello, >>>>>>> >>>>>>> Most info regarding RocksDB memory for Flink focuses on what's >>>>>>> needed independently of the JVM (although the Flink process configures >>>>>>> its >>>>>>> limits and so on). I'm wondering if there are additional special >>>>>>> considerations with regards to the JVM heap in the following scenario. >>>>>>> >>>>>>> Assuming a key used to partition a Flink stream and its state has a >>>>>>> high cardinality, but that the state of each key is small, when Flink >>>>>>> prepares the state to expose to a user function during a call (with a >>>>>>> given >>>>>>> partition key), I guess it loads only the required subset from RocksDB, >>>>>>> but >>>>>>> does this small subset end (temporarily) up on the JVM heap? And if it >>>>>>> does, does it stay "cached" in the JVM for some time or is it >>>>>>> immediately >>>>>>> discarded after the user function completes? >>>>>>> >>>>>>> Maybe this isn't even under Flink's control, but I'm curious. >>>>>>> >>>>>>> Regards, >>>>>>> Alexis. >>>>>>> >>>>>>