Hi Jing, Matt and Jark, Thanks for reviewing this FLIP and sorry for the late reply. I tried to benchmark the POC code of this FLIP on the TPC-DS benchmark, but it seems that the performance improvement is smaller than the benchmark framework's fluctuations, so that the improvement is barely observable. I have not tested the code on Nexmark but similar phenomena might be expected.
Instead, I added a benchmark similar to Flink's WordCount example and posted the results of this benchmark to the FLIP. Briefly speaking, my POC code achieved about a 2% performance increase in this scenario. The average length of the lines in the WordCount example is currently 42.3B, and if I repeat the lines so that the average length increases to 216B, performance increase is not observable. Besides, object reuse has been disabled this time, so I hope the above description has provided additional information about the performance of this FLIP in more common scenarios. Best, Yunfeng On Mon, Jul 31, 2023 at 10:32 PM Jark Wu <imj...@gmail.com> wrote: > > Hi Yunfeng, > > I think this is a great idea to improve the serialization performance, > especially for batch jobs. > I'm not sure whether you have considered or tested this optimization for > batch jobs. > IMO, this optimization can be enabled by default for batch jobs, because > they don't have watermarks > and don't need latency marker (batch job doesn't care about latency but > throughput). > I'm also very much looking forward to the benchmark result of TPC-DS (batch > mode)! > > On the other hand, I'm also very curious about performance improvement. > According to your analysis, > the performance improvement mainly comes from the 1-byte serialization > reduction. > The POC shows a 20% improvement which is amazing. However, I noticed this > POC is > not representative enough, because the record type is a simplest "boolean" > type which > means half serialization can be reduced. However, a real row of data won't > be a simple > boolean type, but with different types of 100B or 1KB. That means the 20% > is a max > theoretical improvement. I'd rather see some benchmark results of actual > workloads, > for example, TPC-DS, Nexmark[1], or even a WordCount job[2]. Could you help > to verify those > workloads? > > Best, > Jark > > [1]: https://github.com/nexmark/nexmark > [2]: > https://github.com/apache/flink/blob/master/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java > > > > On Mon, 17 Jul 2023 at 23:59, Jing Ge <j...@ververica.com.invalid> wrote: > > > Hi Yunfeng, > > > > Thanks for your clarification. It might make sense to add one more > > performance test with object-reuse disabled alongside to let us know how > > big the improvement will be. > > > > Best regards, > > Jing > > > > On Thu, Jul 13, 2023 at 11:51 AM Matt Wang <wang...@163.com> wrote: > > > > > Hi Yunfeng, > > > > > > Thanks for the proposal. The POC showed a performance improvement of 20%, > > > which is very exciting. But I have some questions: > > > 1. Is the performance improvement here mainly due to the reduction of > > > serialization, or is it due to the judgment consumption caused by tags? > > > 2. Watermark is not needed in some scenarios, but the latency maker is a > > > useful function. If the latency maker cannot be used, it will greatly > > limit > > > the usage scenarios. Whether the solution design can retain the > > capability > > > of the latency marker; > > > 3. The data of the POC test is of long type. Here I want to see how much > > > profit it will have if it is a string with a length of 100B or 1KB. > > > > > > > > > -- > > > > > > Best, > > > Matt Wang > > > > > > > > > ---- Replied Message ---- > > > | From | Yunfeng Zhou<flink.zhouyunf...@gmail.com> | > > > | Date | 07/13/2023 14:52 | > > > | To | <dev@flink.apache.org> | > > > | Subject | Re: [DISCUSS] FLIP-330: Support specifying record timestamp > > > requirement | > > > Hi Jing, > > > > > > Thanks for reviewing this FLIP. > > > > > > 1. I did change the names of some APIs in the FLIP compared with the > > > original version according to which I implemented the POC. As the core > > > optimization logic remains the same and the POC's performance can > > > still reflect the current FLIP's expected improvement, I have not > > > updated the POC code after that. I'll add a note on the benchmark > > > section of the FLIP saying that the namings in the POC code might be > > > outdated, and FLIP is still the source of truth for our proposed > > > design. > > > > > > 2. This FLIP could bring a fixed reduction on the workload of the > > > per-record serialization path in Flink, so if the absolute time cost > > > by non-optimized components could be lower, the performance > > > improvement of this FLIP would be more obvious. That's why I chose to > > > enable object-reuse and to transmit Boolean values in serialization. > > > If it would be more widely regarded as acceptable for a benchmark to > > > adopt more commonly-applied behavior(for object reuse, I believe > > > disable is more common), I would be glad to update the benchmark > > > result to disable object reuse. > > > > > > Best regards, > > > Yunfeng > > > > > > > > > On Thu, Jul 13, 2023 at 6:37 AM Jing Ge <j...@ververica.com.invalid> > > > wrote: > > > > > > Hi Yunfeng, > > > > > > Thanks for the proposal. It makes sense to offer the optimization. I got > > > some NIT questions. > > > > > > 1. I guess you changed your thoughts while coding the POC, I found > > > pipeline.enable-operator-timestamp in the code but is > > > pipeline.force-timestamp-support defined in the FLIP > > > 2. about the benchmark example, why did you enable object reuse? Since It > > > is an optimization of serde, will the benchmark be better if it is > > > disabled? > > > > > > Best regards, > > > Jing > > > > > > On Mon, Jul 10, 2023 at 11:54 AM Yunfeng Zhou < > > flink.zhouyunf...@gmail.com > > > > > > > wrote: > > > > > > Hi all, > > > > > > Dong(cc'ed) and I are opening this thread to discuss our proposal to > > > support optimizing StreamRecord's serialization performance. > > > > > > Currently, a StreamRecord would be converted into a 1-byte tag (+ > > > 8-byte timestamp) + N-byte serialized value during the serialization > > > process. In scenarios where timestamps and watermarks are not needed, > > > and latency tracking is enabled, this process would include > > > unnecessary information in the serialized byte array. This FLIP aims > > > to avoid such overhead and increases Flink job's performance during > > > serialization. > > > > > > Please refer to the FLIP document for more details about the proposed > > > design and implementation. We welcome any feedback and opinions on > > > this proposal. > > > > > > Best regards, Dong and Yunfeng > > > > > > [1] > > > > > > > > https://cwiki.apache.org/confluence/display/FLINK/FLIP-330%3A+Support+specifying+record+timestamp+requirement > > > > > > > >