Re: Optimize exact deduplication for tens of billions data per day

2024-04-15 Thread Alex Cruise
It may not be completely relevant to this conversation in this year, but I find myself sharing this article once or twice a year when opining about how hard deduplication at scale can be. 😅 -0xe1a On Thu, Apr 11, 2024 at 10:22 PM Péter Váry wrote

Re: Optimize exact deduplication for tens of billions data per day

2024-04-11 Thread Péter Váry
Hi Lei, There is an additional overhead when adding new keys to an operator, since Flink needs to maintain the state, timers etc for the individual keys. If you are interested in more details, I suggest to use the FlinkUI and compare the flamegraph for the stages. There you can see the difference

Re: Optimize exact deduplication for tens of billions data per day

2024-04-10 Thread Lei Wang
Hi Peter, I tried,this improved performance significantly,but i don't know exactly why. According to what i know, the number of keys in RocksDB doesn't decrease. Any specific technical material about this? Thanks, Lei On Fri, Mar 29, 2024 at 9:49 PM Lei Wang wrote: > Perhaps I can keyBy(Has

Re: Re: Optimize exact deduplication for tens of billions data per day

2024-04-01 Thread Jeyhun Karimov
Hi Lei, In addition to the valuable suggested options above, maybe you can try to optimize your partitioning function (since you know your data). Maybe sample [subset of] your data if possible and/or check the key distribution, before re-defining your partitioning function. Regards, Jeyhun On Mo

Re:Re: Optimize exact deduplication for tens of billions data per day

2024-03-31 Thread Xuyang
Hi, Wang. What about just increasing the parallemism to reduce the number of keys processed per parallelism? Is the distribution of keys uneven? If so, you can use the datastream api to manually implement some optimization methods of flink sql.[1] [1] https://nightlies.apache.org/flink/flin

Re: Optimize exact deduplication for tens of billions data per day

2024-03-29 Thread Lei Wang
Perhaps I can keyBy(Hash(originalKey) % 10) Then in the KeyProcessOperator using MapState instead of ValueState MapState mapState There's about 10 OriginalKey for each mapState Hope this will help On Fri, Mar 29, 2024 at 9:24 PM Péter Váry wrote: > Hi Lei, > > Have you tried to ma

Re: Optimize exact deduplication for tens of billions data per day

2024-03-29 Thread Péter Váry
Hi Lei, Have you tried to make the key smaller, and store a list of found keys as a value? Let's make the operator key a hash of your original key, and store a list of the full keys in the state. You can play with your hash length to achieve the optimal number of keys. I hope this helps, Peter

Optimize exact deduplication for tens of billions data per day

2024-03-28 Thread Lei Wang
Use RocksDBBackend to store whether the element appeared within the last one day, here is the code: *public class DedupFunction extends KeyedProcessFunction {* *private ValueState isExist;* *public void open(Configuration parameters) throws Exception {* *ValueStateDescriptor de