[ 
https://issues.apache.org/jira/browse/FLINK-9506?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16500165#comment-16500165
 ] 

Stefan Richter commented on FLINK-9506:
---------------------------------------

[~yow] I had another look at your code and can point out a number of 
inefficiencies that also sum up to a bigger difference. I also suggest you 
update to a newer Flink version >= 1.4.
First, when using the {{FSStateBackend}}, try to set asynchronous checkpoints 
to true. This will change the implementation to something that is often a bit 
more efficient. Next, your comparison does not consider that in case that you 
are using the reducing state, {{out.collect(output);}} in {{onTimer}} produces 
an output and not just forwards {{null}}. Furthermore, you can think about the 
object reuse setting {{env.getConfig().enableObjectReuse();}}. And you can also 
make your {{AggregationKey}} much more efficient, e.g. storing a single 
{{char[]}} in which you concatenate all 3 input strings and a cached hashcode, 
instead of 3 strings.

> Flink ReducingState.add causing more than 100% performance drop
> ---------------------------------------------------------------
>
>                 Key: FLINK-9506
>                 URL: https://issues.apache.org/jira/browse/FLINK-9506
>             Project: Flink
>          Issue Type: Improvement
>    Affects Versions: 1.4.2
>            Reporter: swy
>            Priority: Major
>         Attachments: KeyNoHash_VS_KeyHash.png, flink.png
>
>
> Hi, we found out application performance drop more than 100% when 
> ReducingState.add is used in the source code. In the test checkpoint is 
> disable. And filesystem(hdfs) as statebackend.
> It could be easyly reproduce with a simple app, without checkpoint, just 
> simply keep storing record, also with simple reduction function(in fact with 
> empty function would see the same result). Any idea would be appreciated. 
> What an unbelievable obvious issue.
> Basically the app just keep storing record into the state, and we measure how 
> many record per second in "JsonTranslator", which is shown in the graph. The 
> difference between is just 1 line, comment/un-comment "recStore.add(r)".
> {code}
> DataStream<String> stream = env.addSource(new GeneratorSource(loop);
> DataStream<JSONObject> convert = stream.map(new JsonTranslator())
>                                        .keyBy()
>                                        .process(new ProcessAggregation())
>                                        .map(new PassthruFunction());  
> public class ProcessAggregation extends ProcessFunction {
>     private ReducingState<Record> recStore;
>     public void processElement(Recordr, Context ctx, Collector<Record> out) {
>         recStore.add(r); //this line make the difference
> }
> {code}
> Record is POJO class contain 50 String private member.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to