Thanks Ted, will read about it.

While we are on throughput.
Do you guys have any suggestion on how to optimise kafka reading from
flink?
In my current setup:
Flink is on 15 machines on yarn
Kafka on 9 brokers with 40 partitions. Source parallelism is 40 for flink,
And just for testing I left only filters there without sink to see max
throughput.
I am getting max 800-900k per sec. And definitely not utilising 1gb/s
network. Im more or less utilising only 20-30% of network bandwith.

I'm wondering what I can tweak further to increase this. I was reading in
this blog:
https://data-artisans.com/extending-the-yahoo-streaming-benchmark/
about 3 millions per sec with only 20 partitions. So i'm sure I should be
able to squeeze out more out of it.

On Thu, Mar 30, 2017 at 11:51 AM, Ted Yu <yuzhih...@gmail.com> wrote:

> Kamil:
> In the upcoming hbase 2.0 release, there are more write path optimizations
> which would boost write performance further.
>
> FYI
>
> On Mar 30, 2017, at 1:07 AM, Kamil Dziublinski <
> kamil.dziublin...@gmail.com> wrote:
>
> Hey guys,
>
> Sorry for confusion it turned out that I had a bug in my code, when I was
> not clearing this list in my batch object on each apply call. Forgot it has
> to be added since its different than fold.
> Which led to so high throughput. When I fixed this I was back to 160k per
> sec. I'm still investigating how I can speed it up.
>
> As a side note its quite interesting that hbase was able to do 2millions
> puts per second. But most of them were already stored with previous call so
> perhaps internally he is able to distinguish in memory if a put was stored
> or not. Not sure.
>
> Anyway my claim about window vs fold performance difference was wrong. So
> forget about it ;)
>
> On Wed, Mar 29, 2017 at 12:21 PM, Timo Walther <twal...@apache.org> wrote:
>
>> Hi Kamil,
>>
>> the performance implications might be the result of which state the
>> underlying functions are using internally. WindowFunctions use ListState or
>> ReducingState, fold() uses FoldingState. It also depends on the size of
>> your state and the state backend you are using. I recommend the following
>> documentation page. The FoldingState might be deprecated soon, once a
>> better alternative is available: https://ci.apache.org/projects
>> /flink/flink-docs-release-1.2/dev/stream/state.html#using-
>> managed-keyed-state
>>
>> I hope that helps.
>>
>> Regards,
>> Timo
>>
>> Am 29/03/17 um 11:27 schrieb Kamil Dziublinski:
>>
>> Hi guys,
>>
>> I’m using flink on production in Mapp. We recently swapped from storm.
>> Before I have put this live I was doing performance tests and I found
>> something that “feels” a bit off.
>> I have a simple streaming job reading from kafka, doing window for 3
>> seconds and then storing into hbase.
>>
>> Initially we had this second step written with a fold function, since I
>> thought performance and resource wise it’s a better idea.
>> But I couldn’t reach more than 120k writes per second to HBase and I
>> thought hbase sink is a bottlenck here. But then I tried doing the same
>> with window function and my performance jumped to 2 millions writes per
>> second. Just wow :) Comparing to storm where I had max 320k per second it
>> is amazing.
>>
>> Both fold and window functions were doing the same thing, taking together
>> all the records for the same tenant and user (key by is used for that) and
>> putting it in one batched object with arraylists for the mutations on user
>> profile. After that passing this object to the sink. I can post the code if
>> its needed.
>>
>> In case of fold I was just adding profile mutation to the list and in
>> case of window function iterating over all of it and returning this batched
>> entity in one go.
>>
>> I’m wondering if this is expected to have 20 times slower performance
>> just by using fold function. I would like to know what is so costly about
>> this, as intuitively I would expect fold function being a better choice
>> here since I assume that window function is using more memory for buffering.
>>
>> Also my colleagues when they were doing PoC on flink evaluation they were
>> seeing very similar results to what I am seeing now. But they were still
>> using fold function. This was on flink version 1.0.3 and now I am using
>> 1.2.0. So perhaps there is some regression?
>>
>> Please let me know what you think.
>>
>> Cheers,
>> Kamil.
>>
>>
>>
>

Reply via email to