confluentinc / librdkafka

The Apache Kafka C/C++ library
Other
199 stars 3.14k forks source link

SSL possibly causing major latency increase (2000-3000%). #920

Closed Airwise closed 5 years ago

Airwise commented 7 years ago

Description

(Note I originally posted this on issue: https://github.com/edenhill/librdkafka/issues/890, but moved here on the advice of Magnus.)

I'm trying to send 1000 messages, a single message at a time (i.e. no batching) so producer config vals looks something like:

batch.num.messages = 1
queue.buffering.max.messages = 10000
queue.buffering.max.ms = 1
message.send.max.retries =  0
socket.blocking.max.ms = 1

That gets me about 2 - 3 ms of latency through the broker (different machine) all the way to the consumer (same machine).

Turning on SSL on just the producer connection however gets ~42 ms of latency for the same path. (Turning on SSL on for the consumer as well gets me to over 80 ms.)

The commandline performance tools that kafka ships with seem to show that if I use the java vm (as opposed to librdkafka + OpenSSL) then turning on SSL should give me about a 30% increase in latency (which acceptable for my application).

I've verified that OpenSSL has access to the aesni hardware instructions. I'm only collecting latency data after a warm up period (to remove SSL handshaking from the latency calculations). And I've verified that all message sends are going over the same TCP socket (i.e. SSL handshaking is only happening once).

Currently using DHE-DSS-AES256-GCM-SHA384 cipher suite for TLS 1.2, but I've tried all of the ones that OpenSSL and default (i.e. non-regulated) kafka/java crypto have in common:

DHE-DSS-AES256-GCM-SHA384
DHE-DSS-AES256-SHA256
DHE-DSS-AES256-SHA
DHE-DSS-AES128-GCM-SHA256
DHE-DSS-AES128-SHA256
DHE-DSS-AES128-SHA
EDH-DSS-DES-CBC3-SHA

Any ideas on how to chase this down further? (Oh by the way I'm accessing librdkafka via the rdkafka PHP extension. I don't see how that would play into the SSL performance, but maybe?)

How to reproduce

Create a producer and consumer with the config properties above. Run a 1000 messages without SSL, then run 1000 with SSL enabled. I can provide test PHP code if it comes to that, but this is for my day job so I'm happy to run tests, collect data, or whatever I can do on my time. I'll wait to put together test code until after I've done every thing I can to rule out other parts of my system.

Checklist

Please provide the following information:

Airwise commented 7 years ago

Note that on the old issue Magnus posted:

Can you run with debug=protocol and compare ProduceResponse rtt (latency) between plaintext and ssl?

So the ball is in my court to provide that.

edenhill commented 7 years ago

It'd be interesting to see some latency statistics. Can you enable stats (https://github.com/edenhill/librdkafka/wiki/Statistics) and look for the broker's rtt metrics? Alternative enable debug=protocol and look at the "rtt" value)

Airwise commented 7 years ago

I've run with debug=protocoland and I can see the latency in the fetch request (which since I'm doing a low latency message bus are sent as fast as possible).

A small sampling of fetch requests in PLAINTEXT:

%7|1479766400.869|RECV|rdkafka#consumer-1| ip-172-31-60-3.ec2.internal:9092/1: Received FetchResponse (v1, 46 bytes, CorrId 6669, rtt 0.50ms)
%7|1479766400.869|SEND|rdkafka#consumer-1| ip-172-31-60-3.ec2.internal:9092/1: Sent FetchRequest (v1, 73 bytes @ 0, CorrId 6670)
%7|1479766400.870|RECV|rdkafka#consumer-1| ip-172-31-60-3.ec2.internal:9092/1: Received FetchResponse (v1, 46 bytes, CorrId 6670, rtt 0.52ms)
%7|1479766400.870|SEND|rdkafka#consumer-1| ip-172-31-60-3.ec2.internal:9092/1: Sent FetchRequest (v1, 73 bytes @ 0, CorrId 6671)
%7|1479766400.871|RECV|rdkafka#consumer-1| ip-172-31-60-3.ec2.internal:9092/1: Received FetchResponse (v1, 46 bytes, CorrId 6671, rtt 1.31ms)
%7|1479766400.872|SEND|rdkafka#consumer-1| ip-172-31-60-3.ec2.internal:9092/1: Sent FetchRequest (v1, 73 bytes @ 0, CorrId 6672)
%7|1479766400.873|RECV|rdkafka#consumer-1| ip-172-31-60-3.ec2.internal:9092/1: Received FetchResponse (v1, 46 bytes, CorrId 6672, rtt 1.34ms)
%7|1479766400.873|SEND|rdkafka#consumer-1| ip-172-31-60-3.ec2.internal:9092/1: Sent FetchRequest (v1, 73 bytes @ 0, CorrId 6673)
%7|1479766400.874|RECV|rdkafka#consumer-1| ip-172-31-60-3.ec2.internal:9092/1: Received FetchResponse (v1, 46 bytes, CorrId 6673, rtt 0.50ms)

A small sampling of fetch requests in SSL:

%7|1479766658.556|RECV|rdkafka#consumer-1| ssl://ip-172-31-60-3.ec2.internal:9093/1: Received FetchResponse (v1, 46 bytes, CorrId 387, rtt 39.92ms)
%7|1479766658.556|SEND|rdkafka#consumer-1| ssl://ip-172-31-60-3.ec2.internal:9093/1: Sent FetchRequest (v1, 73 bytes @ 0, CorrId 388)
%7|1479766658.596|RECV|rdkafka#consumer-1| ssl://ip-172-31-60-3.ec2.internal:9093/1: Received FetchResponse (v1, 46 bytes, CorrId 388, rtt 40.08ms)
%7|1479766658.597|SEND|rdkafka#consumer-1| ssl://ip-172-31-60-3.ec2.internal:9093/1: Sent FetchRequest (v1, 73 bytes @ 0, CorrId 389)
%7|1479766658.636|RECV|rdkafka#consumer-1| ssl://ip-172-31-60-3.ec2.internal:9093/1: Received FetchResponse (v1, 46 bytes, CorrId 389, rtt 39.45ms)
%7|1479766658.637|SEND|rdkafka#consumer-1| ssl://ip-172-31-60-3.ec2.internal:9093/1: Sent FetchRequest (v1, 73 bytes @ 0, CorrId 390)
%7|1479766658.676|RECV|rdkafka#consumer-1| ssl://ip-172-31-60-3.ec2.internal:9093/1: Received FetchResponse (v1, 46 bytes, CorrId 390, rtt 39.60ms)
%7|1479766658.677|SEND|rdkafka#consumer-1| ssl://ip-172-31-60-3.ec2.internal:9093/1: Sent FetchRequest (v1, 73 bytes @ 0, CorrId 391)
%7|1479766658.716|RECV|rdkafka#consumer-1| ssl://ip-172-31-60-3.ec2.internal:9093/1: Received FetchResponse (v1, 46 bytes, CorrId 391, rtt 39.56ms)
%7|1479766658.716|SEND|rdkafka#consumer-1| ssl://ip-172-31-60-3.ec2.internal:9093/1: Sent FetchRequest (v1, 73 bytes @ 0, CorrId 392)
%7|1479766658.756|RECV|rdkafka#consumer-1| ssl://ip-172-31-60-3.ec2.internal:9093/1: Received FetchResponse (v1, 46 bytes, CorrId 392, rtt 39.54ms)

Does this suggest anything in particular? Is there another avenue of discovery to which it points?

Airwise commented 7 years ago

For the record here are the results of an "EndToEndLatency" test using the kafka command line tools. It looks to me like the test is batching 1000 messages (even though I tried to set the batch size to 1), in which case it could be masking the actual latency of an SSL produce/fetch by dividing it by 1000.

PLAINTEXT results:

ubuntu@ip-172-31-57-116:/usr/local/kafka/kafka_2.11-0.10.1.0$ bin/kafka-run-class.sh kafka.tools.EndToEndLatency ip-172-31-60-3.ec2.internal:9092 latency.test 10000 1 128 /uonfig/client-ssl.properties
[2016-11-19 00:03:03,459] WARN The configuration 'batch.size' was supplied but isn't a known config. (org.apache.kafka.clients.consumer.ConsumerConfig)
0       41.932272
1000    1.796583
2000    1.534213
3000    2.178782
4000    1.9785730000000001
5000    2.183221
6000    1.074092
7000    1.68927
8000    1.346963
9000    1.088545
Avg latency: 1.8739 ms

SSL results:

ubuntu@ip-172-31-57-116:/usr/local/kafka/kafka_2.11-0.10.1.0$ bin/kafka-run-class.sh kafka.tools.EndToEndLatency ip-172-31-60-3.ec2.internal:9093 latency.test 10000 1 128 /uonfig/client-ssl.properties
[2016-11-19 00:06:14,496] WARN The configuration 'batch.size' was supplied but isn't a known config. (org.apache.kafka.clients.consumer.ConsumerConfig)
0       150.05167300000002
1000    1.953523
2000    2.6856970000000002
3000    2.650836
4000    2.52436
5000    2.3982170000000003
6000    2.256949
7000    1.487088
8000    1.644407
9000    2.977237
Avg latency: 2.4509 ms
edenhill commented 7 years ago

If you set the librdkafka batch.num.messages=1000 do you get the comparable rtts?

Airwise commented 7 years ago

No. If I set librdkafka to batch 1000 and the queue to 100,000 and then send 10,000 messages I end up with a per-message latency of around 0.000017 seconds (i.e. 17 micro seconds). So if the kafka command line tools were using batching / volume to calculate latency I would expect to see numbers in a similar order of magnitude.

So maybe the kafka tools aren't using batches to mask latency after all. I ran a different kafka command line tool called kafka-producer-perf-test and it seems to imply that with no batching (or very minimal batching) the latency for produce requests (after initial TCP socket + SSL handshaking setup) is still down around 3 ms with SSL on.

Also bin/kafka-producer-perf-test.sh --topic producer.latency.test --num-records 20 --record-size 128 --throughput 1 --producer.config /usr/local/kafka/kafka_2.11-0.10.1.0/config/client-ssl.properties
7 records sent, 1.3 records/sec (0.00 MB/sec), 94.7 ms avg latency, 591.0 max latency.
6 records sent, 1.0 records/sec (0.00 MB/sec), 1.8 ms avg latency, 2.0 max latency.
5 records sent, 1.0 records/sec (0.00 MB/sec), 2.8 ms avg latency, 3.0 max latency.
20 records sent, 1.022809 records/sec (0.00 MB/sec), 34.75 ms avg latency, 591.00 ms max latency, 3 ms 50th, 591 ms 95th, 591 ms 99th, 591 ms 99.9th.
Airwise commented 7 years ago

So I just found examples/rdkafka_performance. I thought maybe it would be better to use that to test instead of my homebrewed PHP test. But I can't seem to get it to send messages. (Note I'm just trying to get it to send plaintext messages to start with.)

ubuntu@ip-172-31-57-116:~/librdkafka/examples$ ./rdkafka_performance -P -t test.ssl.1 -p 1 -b ip-172-31-60-3.ec2.internal:9092 -c 100 -m "Hi I am a request" -a 1 -X batch.num.messages=1 -X socket.blocking.max.ms=1 -X queue.buffering.max.ms=1 -l
% Sending 100 messages of size 30 bytes
% 100 messages produced (3000 bytes), 0 delivered (offset 0, 100 failed) in 32ms: 0 msgs/s and 0.00 Mb/s, 0 produce failures, 0 in queue, no compression

Am I missing something obvious?

EDIT Never mind. My partition arg was messing things up.

Airwise commented 7 years ago

Well, even using the exactly config values from my test application, I can't get examples/rdkafka_performance to reproduce the 2-3 ms latency of my test application even without SSL. It reports and average of 50ms or so.

However, I found this interesting. Using examples/rdkafka_performance with -d protocol and SSL on, I saw these two messages:

%7|1479862977.616|RECV|rdkafka#consumer-1| ssl://ip-172-31-60-3.ec2.internal:9093/1: Received FetchResponse (v1, 42 bytes, CorrId 63, rtt 39.58ms)
%7|1479862977.617|RECV|rdkafka#consumer-1| ssl://ip-172-31-60-3.ec2.internal:9093/1: Received LeaveGroupResponse (v0, 2 bytes, CorrId 64, rtt 3.83ms)

And what I am wondering is why is FetchResponse rtt so much slower than LeaveGroupResponse rtt? Presumably they both are being SSL encrypted/decrypted. Granted the FetchResponse is only 2 bytes, but my basic experience with AES encryption is that a data length of 2 vs 42 bytes does not significantly affect the encryption time (a few micro seconds at worst).

edenhill commented 7 years ago

I have reproduced this and looking into what the issue may be. It points to some nagle-like bufferfing in OpenSSL that awaits more data before transmitting to remote server because increasing the thruput datarate decreases the latency.

edenhill commented 7 years ago

Disabling Nagle doesn't help, so it is not in the kernel.

shafqatevo commented 6 years ago

Hi, any update on this issue?

multiple1 commented 6 years ago

I also experience this issue.

With protocol debug on with ssl i measure a rtt of 44ms (local network). Producerequest Size = 1085 Bytes

Increasing queue.buffering.max.ms and so increasing the packet size reduces the rtt to 4ms Producerequest Size = 2072 Bytes

Is there any fix yet?

chun-wang commented 6 years ago

Disabling Nagle helps in my case. Latency was decreased from 30~40ms to 2~3ms with SSL.

With an old version of librdkafka, the macro TCP_NODELAY may not been defined because header file "netinet/tcp.h" was not included in src/rdkafka_transport.c. Add configuration of "socket.nagle.disable=true" will take no effect.

Try: Add "#include <netinet/tcp.h>" to src/rdkafka_transport.c and recompile the library.

multiple1 commented 6 years ago

I disabled Nagle already, no change.

I tried Version v0.11.5 and the current master branch. The #include is already in v0.11.5, so i think it will no help

edenhill commented 6 years ago

@mhowlett You spent quite a bit of time with latency measurements, do you have any instructions?