Hi, I suppose there are two things to clarify: 1. the function object will only be serialized once when deploying and deserialized once per task while initializing. 2. the ValueState itself is only a handle. It is set/setup with the key of each incoming record. When serializing, it is only a null field.
In conclusion, the cost of serialization should be constant, relatively ignorable, and there should be no impact on the performance no matter whether it is a transient. By the way, preparing, accessing and updating the value state can indeed be costly if the number of partitions is really large, but it has nothing to do with the `transient` qualifier. On Fri, Feb 24, 2023 at 7:49 PM simple <1028108...@qq.com> wrote: > 稍等,我看一下您的反馈 > > ------------------------------ > 发自我的iPhone > > > ------------------ Original ------------------ > *From:* Zhongpu Chen <chenlov...@gmail.com> > *Date:* Fri,Feb 24,2023 7:47 PM > *To:* user <user@flink.apache.org> > *Subject:* Re: RE: Re: Re: Should we always mark ValueState as > "transient" forRichFunctions > > Hi Shammon, > > Sorry for the inaccurate description of my last reply. Let me restate my > question again: > > Fact 1: we know that ValueState here should not > serialized/de-serialized, so it is a good practice to mark it with > "transient". > > Fact 2: on the other hand, if we don't mark it with "transient", it will > be initialized to null, and this null value will be > serialized/de-serialized. I think it will occur some overhead if the > number of partitions is very large. > > Given the two facts above, the program works well in both cases in terms > of accuracy. And my question is: is there any performance benchmark in > real (large) applications to compare two cases? > > Feel free to point out if I've misunderstood. > > On 2023/02/24 11:01:51 Shammon FY wrote: > > Hi > > > > Sorry that I don't quite understand your question. I think the above > > functions will only be deserialized when the job is submitted, do you > want > > to test the impact of this on submission throughput? > > > > Best, > > Shammon > > > > > > On Fri, Feb 24, 2023 at 3:04 PM Zhongpu Chen wrote: > > > > > Hi Gen, > > > > > > Thanks for your explanation. > > > > > > Back to this code snippet, since they are not marked with "transient" > > > now, I suppose Flink will use avro to serialize them (null values). Is > > > there any benchmark to show the performance test between null values > > > serialization and "transient"? I mean, it is indeed not good to write > > > them with "transient", but it works. So is there any performance > lose here? > > > > > > > > > On 2023/02/24 06:47:21 Gen Luo wrote: > > > > Hi, > > > > > > > > ValueState is a handle rather than an actual value. So it should > never > > > be > > > > serialized. In fact, ValueState itself is not a Serializable. It > > > should be > > > > ok to always mark it as transient. > > > > > > > > In this case, I suppose it works because the ValueState is not set > > > (which > > > > happens during the runtime) when the function is serialized (while > > > > deploying). But it's not good. > > > > > > > > On Fri, Feb 24, 2023 at 10:29 AM Zhongpu Chen > wrote: > > > > > > > > > Hi, > > > > > > > > > > When I am reading the code from flink-training-repo [1], I > noticed the > > > > > following code: > > > > > > > > > > ```java > > > > > > > > > > public static class EnrichmentFunction > > > > > extends RichCoFlatMapFunction { > > > > > > > > > > private ValueState rideState; private > > > ValueState fareState; > > > > > ... > > > > > } > > > > > > > > > > ``` > > > > > > > > > > From my understanding, since ValueState variables here are scoped > > > to each > > > > > instance, they should not be serialized for the performance sake. > > > Thus, we > > > > > should always mark them with "transient". Similar discussion can be > > > found > > > > > here [2]. > > > > > > > > > > Should we always mark ValueState as "transient", and why? Please > > > help me > > > > > to figure it out. > > > > > > > > > > [1] > > > > > > > > > > > > > https://github.com/apache/flink-training/blob/master/rides-and-fares/src/solution/java/org/apache/flink/training/solutions/ridesandfares/RidesAndFaresSolution.java > > > > > > > > > > [2] > > > > > > > > > > > > > https://stackoverflow.com/questions/72556202/flink-managed-state-as-transient > > > > > > > > > > > > > > >