Hello Johan,
Thanks for the information. Could you apply the patch as in
https://github.com/apache/kafka/pull/4085 from a source release of Streams
and see if it fixed your problem?
Guozhang
On Fri, Oct 20, 2017 at 12:25 PM, Johan Genberg
wrote:
> Thanks. I've changed the config settings and
Thanks. I've changed the config settings and gotten a little better
results, but the issue still remains, and doesn't seem only related to this.
There seems to be an excessive use of bandwidth when initializing state
stores in 0.11.0.1 compared to previous version. I realized that at least
one of
Hi John,
It looks like this is an issue. Guozhang has provided a fix for it here:
https://github.com/apache/kafka/pull/4085
You can try it by cloning his repository and build the streams jar.
Thanks,
Damian
On Tue, 17 Oct 2017 at 23:07 Johan Genberg wrote:
> Yes, it's noticeably faster in my t
Yes, it's noticeably faster in my test environment, under load. I'm going
to apply the default value (100) and see how it behaves in production.
Thanks,
Johan
On Tue, Oct 17, 2017 at 1:58 PM Guozhang Wang wrote:
> This is only for my diagnosing your issue.
>
> I suspect that due to your very sm
This is only for my diagnosing your issue.
I suspect that due to your very small traffic (5 records per sec) and long
polling parameter, the restoration process was blocked on waiting for data.
More specifically, in KAFKA-5152 which is merged in 0.11.0.1, a thread may
be restoring for some tasks w
Ok, I will try that. Is there any documentation as to what exactly this
does, in general? Just want to safe guard against side effects. Also, do
you recommend 0 as a production setting (default is 100ms, I think), or is
this just to test this out/diagnose?
The only description I can find is this:
Johan,
Could you please try change the following configs and try again?
properties.put(StreamsConfig.POLL_MS_CONFIG, 0);
Guozhang
On Tue, Oct 17, 2017 at 12:34 PM, Johan Genberg
wrote:
> Hi,
>
> The trace log is here (tried to attach to email but it was rejected):
>
> https://gist.github.co
Hi,
The trace log is here (tried to attach to email but it was rejected):
https://gist.github.com/trickleup/fe0c095df65b8ae10906ea6774804b54
The logs are from a test environment, but I was able to somewhat reproduce
the issue, and with 5 or so messages per second on average. The production
syste
Hello Johan,
Could you list the following information in addition to your topology?
1. Your config settings if you have any overwrites, especially consumer's "
poll.ms" and "max.poll.records", and "num.threads".
2. Your expected incoming data rate (messages per sec) at normal processing
phase.
Just to clarify, in 0.11.0.1 we have optimized the restoration process so
it is actually expected to be faster than 0.10.2.1, so what you observed is
indeed very weird. We'd need some more information like log entries to see
what's happening there.
Guozhang
On Tue, Oct 17, 2017 at 2:50 AM, Damia
Hi,
Thank you for responding so quickly. This is the topology. I've simplified
it a bit, but these are the steps it goes through, not sure if that is
helpful. I'll try to get some logs too in a bit.
*KStream eventStream = builder.stream(*
* topic.keySerde(),*
* topic.valueSerde(),*
* topic.nam
Hi Johan,
Do you have any logs? The state store restoration changed significantly in
0.11.0.1. If you could get some logs at trace level, that would be useful.
Also if you could provide your topology (removing anything
proprietary/sensitive).
Thanks,
Damian
On Tue, 17 Oct 2017 at 05:55 Johan Gen
Hi,
I'm upgrading a kafka streams application from 0.10.2.1 to 0.11.0.1,
running against a kafka cluster with version 0.10.2.1. The application uses
a couple of state stores.
When stopping/starting the application prior to the upgrade (with 0.10.2.1
client) on 2 instances, it was up and running i
13 matches
Mail list logo