Divij Vaidya created KAFKA-14541:
------------------------------------
Summary: Profile produce workload for Apache Kafka
Key: KAFKA-14541
URL: https://issues.apache.org/jira/browse/KAFKA-14541
Project: Kafka
Issue Type: Improvement
Reporter: Divij Vaidya
Assignee: Divij Vaidya
Attachments:
flamegraph-openjdk11nodebug-cpu-withoutobjectserializer.html
I have been profiling Kafka (3.4.0 / trunk right now) for a produce only
workload and the [OpenMessaging|https://openmessaging.cloud/docs/benchmarks/]
workloads. The goal is to get a better understanding of CPU usage profile for
Kafka and eliminate potential overheads to reduce CPU consumption.
h2. *Setup*
R6i.16xl (64 cores)
OS: Amazon Linux 2
Single broker, One topic, One partition
Plaintext
Prometheus Java agent attached
{code:java}
-XX:+PreserveFramePointer
-XX:+UnlockDiagnosticVMOptions
-XX:+DebugNonSafepoints{code}
{code:java}
queued.max.requests=10000
num.network.threads=32
num.io.threads=128
socket.request.max.bytes=104857600{code}
h3. Producer setup:
{code:java}
batch.size=9000
buffer.memory=33554432
enable.idempotence=false
linger.ms=0
receive.buffer.bytes=-1
send.buffer.bytes=-1
max.in.flight.requests.per.connection=100000{code}
h3. Profiler setup:
[async-profiler|https://github.com/jvm-profiling-tools/async-profiler] (this
profiler + -XX:+DebugNonSafepoints ensure that profiling doesn't suffer from
safepoint bias). Note that flamegraph can be generated in "cpu" mode or "wall"
mode (wall clock time) or "cycles" mode (used for better kernel call stack)
h2. Observations
# Processor.run > Processor#updateRequestMetrics() is a very expensive call.
We should revisit whether we want to pre-compute histograms or not. Maybe
upgrading to latest dropwizard will improve this?
# (Processor > Selector.poll), (Processor > Selector.write) and many other
places - Accumulative cost of Sensor#recordInternal is high.
# Processor threads are consuming more CPU than Handler threads?!! Perhaps
because handler threads spend a lot of time waiting for partition lock at
UnifiedLock.scala
# HandleProduceRequest > RequestSizeInBytes - Unnecessary call to calculate
size in bytes here. Low hanging opportunity to improve CPU utilisation for a
request heavy workload.
# UnifiedLog#append > HeapByteBuffer.duplicate() - Why do we duplicate the
buffer here? Ideally we shouldn't be making copies of buffer during the produce
workflow. We should be using the same buffer after reading from socket to
writing in a file.
# Processor > Selector.select - Why is epoll consuming CPU cycles? It should
have the thread in a timed_waiting state and hence, shouldn't consume CPU at
all.
# In a produce workload writing to socket is more CPU intensive than reading
from the socket. This is surprising because reading would read more data from
the socket (produce records) whereas writing would only write the response back
which doesn't contain record data.
# RequestChannel#sendResponse > wakeup - This is the call which wakes up the
selector by writing to a file descriptor. Why is this so expensive?
I am still analysing the flamegraph (cpu mode attached here). Please feel free
to comment on any of the observations or add your own observations here.
--
This message was sent by Atlassian Jira
(v8.20.10#820010)