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

Reply via email to