understand, thanks for all the help
On Mon, Aug 27, 2018 at 2:16 PM Guozhang Wang wrote:
> Hello Nan,
>
> What you described seems to be a broker-side latency spike, not a client
> (either producer, or consumer, or streams)-side latency spike. There's a
> number of possible reasons for such spik
Hello Nan,
What you described seems to be a broker-side latency spike, not a client
(either producer, or consumer, or streams)-side latency spike. There's a
number of possible reasons for such spikes: disk flushing (though async, it
can also cause the processing thread to halt), GC, page faults (i
maybe easier to use github.
https://github.com/angelfox123/kperf
On Sat, Aug 25, 2018 at 8:43 PM Nan Xu wrote:
> so I did upgrade to 2.0.0 and still seeing the same result. below is the
> program I am using. I am running everything on a single server. (centos 7,
> 24 core, 32G ram , 1 broker
so I did upgrade to 2.0.0 and still seeing the same result. below is the
program I am using. I am running everything on a single server. (centos 7,
24 core, 32G ram , 1 broker, 1 zookeeper, single harddrive), I understand
the single hard drive is less ideal. but still don't expect it can over 3
se
Message-
From: Nan Xu [mailto:nanxu1...@gmail.com]
Sent: Friday, August 24, 2018 3:37 PM
To: users@kafka.apache.org
Subject: Re: kafka stream latency
Looks really promising but after upgrade, still show the same result. I
will post the program soon. Maybe you can see where the problem could be
Looks really promising but after upgrade, still show the same result. I
will post the program soon. Maybe you can see where the problem could be.
Nan
On Thu, Aug 23, 2018, 7:34 PM Guozhang Wang wrote:
> Hello Nan,
>
> Kafka does not tie up the processing thread to do disk flushing. However,
> s
Hello Nan,
Kafka does not tie up the processing thread to do disk flushing. However,
since you are on an older version of Kafka I suspect you're bumping into
some old issues that have been resolved in later versions. e.g.
https://issues.apache.org/jira/browse/KAFKA-4614
I'd suggest you upgrading
I will wait for the expert’s opinion:
Did the Transparent Huge Pages(THP) disabled on the broker machine? it’s a
Linux kernel parameter.
-Sudhir
> On Aug 23, 2018, at 4:46 PM, Nan Xu wrote:
>
> I think I found where the problem is, how to solve and why, still not sure.
>
> it related to disk
I think I found where the problem is, how to solve and why, still not sure.
it related to disk (maybe flushing?). I did a single machine, single node,
single topic and single partition setup. producer pub as 2000 message/s,
10K size message size. and single key.
when I save kafka log to the mem
Given your application code:
final KStream localDeltaStream = builder.stream(
localDeltaTopic,
Consumed.with(
new Serdes.StringSerde(),
new NodeMutationSerde<>()
)
);
KStream localHi
I was suspecting that too, but I also noticed the spike is not spaced
around 10s. to further prove it. I put kafka data directory in a memory
based directory. it still has such latency spikes. I am going to test it
on a single broker, single partition env. will report back soon.
On Wed, Aug 22,
Hello Nan,
Thanks for the detailed information you shared. When Kafka Streams is
normally running, no rebalances should be triggered unless some of the
instances (in your case, docker containers) have soft failures.
I suspect the latency spike is due to the commit intervals: streams will
try to c
did more test and and make the test case simple.
all the setup now is a single physical machine. running 3 docker instance.
a1, a2, a3
kafka + zookeeper running on all of those docker containers.
producer running on a1, send a single key, update speed 2000 message/s,
each message is 10K size.
3 c
Okay, so you're measuring end-to-end time from producer -> broker ->
streams' consumer client, there are multiple phases that can contribute to
the 100ms latency, and I cannot tell if stream's consumer phase is the
major contributor. For example, if the topic was not created before, then
when the b
right, so my kafka cluster is already up and running for a while, and I can
see from the log all broker instance already change from rebalance to
running.
I did a another test.
from producer, right before the message get send to the broker, I put a
timestamp in the message. and from the consumer s
Hello Nan,
Note that Streams may need some time to rebalance and assign tasks even if
you only starts with one instance.
I'd suggest you register your state listener in Kafka Streams via
KafkaStreams#setStateListener, and your customized StateListener should
record when the state transits from RE
thanks, which JMX properties indicate "processing latency spikes" /
"throughput"
On Sat, Aug 18, 2018 at 5:45 PM Matthias J. Sax
wrote:
> I cannot spot any obvious reasons.
>
> As you consume from the result topic for verification, we should verify
> that the latency spikes original on write
I cannot spot any obvious reasons.
As you consume from the result topic for verification, we should verify
that the latency spikes original on write and not on read: you might
want to have a look into Kafka Streams JMX metric to see if processing
latency spikes or throughput drops.
Also watch for
btw, I am using version 0.10.2.0
On Fri, Aug 17, 2018 at 2:04 PM Nan Xu wrote:
> I am working on a kafka stream app, and see huge latency variance,
> wondering what can cause this?
>
> the processing is very simple and don't have state, linger.ms already
> change to 5ms. the message size is arou
19 matches
Mail list logo