Replies: 3 comments 5 replies
-
@dotnwat can this be some threading issue inside librdkafka? I wonder if we write our own benchmarking tool now that we have a client. I wrote one that takes into account coordinated omission (kinda jenky, but worked https://github.com/smfrpc/smf/blob/master/src/include/smf/load_channel.h) so we can also ship it with 28ms on some client side stuff seems insane, so the theory is that maybe could be the clock source in librdkafka if it has some low-res clock? just some ideas. |
Beta Was this translation helpful? Give feedback.
-
A separate tool, the kafka produce performance test, reports a metric for a similar workload
|
Beta Was this translation helpful? Give feedback.
-
We finally tracked down the source of the difference between rtt latency reported by librdkafka and the processing time for a request in redpanda. Effectively we were seeing queueing within the network stack: librdkafka will dispatch the requests onto the network and the rtt includes the time spend sitting in networking buffers... this can be controlled with Here is the rtt in milliseconds for a couple queue depth settings. 1024 is roughly one 1mb batch outstanding and 8192 would be roughly a queue depth of 8 batches outstanding. So for example at qd=2 we get 3ms rtt.
Here is the corresponding throughput numbers. For instance at qd=2 we can push 365 mb/s.
Keep in mind that this is single node, single partition, and queue depth = 1 inside the kafka layer. This means that throughput is very sensitive to latency. This serialization in the kafka layer is also why our throughput maxes out fairly quickly. What we'll be looking at next is some optimizations that allow us to keep mostly qd=1 on the kafka side, while improving throughput for produce handling. This should generally give us better latencies at deeper queue depths. |
Beta Was this translation helpful? Give feedback.
-
We'll be moving to queue depth 1 at the kafka layer so that we can properly enforce ordering constraints for kafka clients. Scale throughput with more partitions. We have a batching solution for small batches that should still provide good throughput with queue depth 1 by splitting raft replication into a synchronous component (ordering) and an asynchronous component (waiting on flushes) where only the first ordering phase is subject to the queue depth 1 constraint.
effectively the same behavior and performance. which is not surprising with the current queue depth 1 implementation: @mmaslankaprv I think I heard you mention that librdkafka only creates a single connection? we should remember this when implementing one-at-a-time processing so that we enforce at a finer granularity (e.g. per partition to deal with this sort of single connection client).
notice above that the rtt is about 30 milliseconds. where does that time go? we can start looking around. in this particular example with acks=all there is effectively 1 mb fsync'd writes split into 4 256kb writes that run in parallel. this is the first part of the trace below
segment_appender::flush sem_wait 1011 flush_wait 100 (total 1112)
this line says that we waited 1ms for the 4 writes to complete and then another 100us for the flush.finally,
commit index update wait time 1125
is in the raft layer waiting on the flush to finish before acking back to the higher levels that issued the replicate.if we look at a summary of a 2 minute run we see
so at the 99.9th percentile we aren't even close to accounting for the roughly 30ms rtt reported by the performance tool.
here is the e2e time for processing a request. this measure the start of reading the header off the input stream, to the time that writing the response to the output stream completes
so at the 99.9th percentile of 5.4 ms we still aren't accounting for the reported rtt of 30ms.
here is a franz-go client courtesy of @twmb that shows e2e latency of around 2ms! what we expect
so the next step for single node latency is go figure out what's going on in librdkafka that's leading to the higher reported rtt and if that's real or some artifact of the way the timestamps are taken.
Beta Was this translation helpful? Give feedback.
All reactions