[
https://issues.apache.org/jira/browse/FLINK-14346?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16952780#comment-16952780
]
Piotr Nowojski commented on FLINK-14346:
----------------------------------------
Thanks [~rgrebennikov] for reporting the findings and proposed solution. Yes,
we would be interested with such change.
To rephrase it and to make sure that I understand this correctly.
{{writeUTF()}} is 2x faster than current Flink's serializer, but it's binary
incompatible. However you have successfully experimented with adopting some of
it's ideas and incorporated them into a PoC, that's almost as efficient as
{{writeUTF}} but also binary compatible with the current
{{StringValue#writeString}}. The trick is to use the {{byte[] buffer = new
byte[buflen]}} to cache single byte writes and batch write whole buffer to the
{{DataOutput out}}?
If I haven't missed anything, I like the idea and will be happy to review the
PR. Couple of meta remarks:
# could you check if the [existing
benchmarks|https://github.com/dataArtisans/flink-benchmarks/] (like
{{SerializationFrameworkMiniBenchmarks}}) are already covering this scenario
and whether the performance improvement is visible there? Or should we add a
new benchmark for that?
# should we worry about {{buflen}} exceeding CPU's cache friendly size? 1MB?
4MB? 10MB? In that case maybe splitting the loop into a multiple 100KB chunks
steps is a better idea? (this could be an optional follow up, as long as there
should be no performance regression for larger strings compared to the current
version.)
> Performance issue with StringSerializer
> ---------------------------------------
>
> Key: FLINK-14346
> URL: https://issues.apache.org/jira/browse/FLINK-14346
> Project: Flink
> Issue Type: Improvement
> Components: API / Type Serialization System, Benchmarks
> Affects Versions: 1.9.0
> Environment: Tested on Flink 1.9.0, adoptopenjdk 8u222.
> Reporter: Roman Grebennikov
> Priority: Major
> Labels: performance
>
> While doing a performance profiling for our Flink state-heavy streaming job,
> we found that quite a significant amount of CPU time is spent inside
> StringSerializer writing data to the underlying byte buffer. The hottest part
> of the code is the StringValue.writeString function. And replacing the
> default StringSerializer with the custom one (to just play with a baseline),
> which is just calling DataOutput.writeUTF/readUTF surprisingly yielded to
> almost 2x speedup for string serialization.
> As writeUTF and writeString have incompatible wire formats, replacing latter
> with former is not a good idea in general as it may break
> checkpoint/savepoint compatibility.
> We also did an early performance analysis of the root cause of this
> performance issue, and the main reason of JDK's writeUTF being faster is that
> it's code is not writing directly to output stream byte-by-byte, but instead
> creating an underlying temporary byte buffer. This yields to a HotSpot almost
> perfectly unrolling the main loop, which results in much better data
> parallelism.
> I've tried to port the ideas from the JVM's implementation of writeUTF back
> to StringValue.writeString, and my current result is nice, having quite
> significant speedup compared to the current implementation:
> {{[info] Benchmark Mode Cnt Score Error Units}}
> {{[info] StringSerializerBenchmark.measureJDK avgt 30 82.871 ± 1.293 ns/op}}
> {{[info] StringSerializerBenchmark.measureNew avgt 30 94.004 ± 1.491 ns/op}}
> {{[info] StringSerializerBenchmark.measureOld avgt 30 156.905 ± 3.596 ns/op}}
>
> {{Where measureJDK is the JDK's writeUTF asa baseline, measureOld is the
> current upstream implementation in Flink, and the measureNew is the improved
> one. }}
>
> {{The code for the benchmark (and the improved version of the serializer) is
> here: [https://github.com/shuttie/flink-string-serializer]}}
>
> {{Next steps:}}
> # {{More benchmarks for non-ascii strings.}}
> # {{Benchmarks for long strings.}}
> # {{Benchmarks for deserialization.}}
> # {{Tests for old-new wire format compatibility.}}
> # {{PR to the Flink codebase.}}
> {{Is there an interest for this kind of performance improvement?}}
--
This message was sent by Atlassian Jira
(v8.3.4#803005)