The maintainer of librdkafka was able to reproduce the latency. He thinks it may be some sort of batching algorithm similar to Nagle inside OpenSSL.
Status of the issue is maintained at: https://github.com/edenhill/librdkafka/issues/920 Thanks to all on this mailing list for your help in diagnosing. - Aaron On Fri, Nov 18, 2016 at 5:06 PM, Aaron Wilkinson <aa...@modopayments.com> wrote: > So the kafka performance tools seem to indicate that the problem is not in > the broker, but rather somewhere in librdkafka/OpenSSL. I'm not completely > sure I got the configs right to try and eliminate any batching > considerations in the latency calculation (it seems like encrypting / > decrypting a batch of 1000 messages 1 time would be more efficient than > encrypting / decrypting 1 message 1000 times and I am interested in the > latter). BUT doing a relative test of plaintext vs ssl seemed to show the > promised 20% - 50% overhead rather than the thousands of percent I am > seeing with librdkafka + OpenSSL. > > Plaintext connection > /usr/local/kafka/kafka_2.11-0.10.1.0$ bin/kafka-run-class.sh > kafka.tools.EndToEndLatency <broker hostname>:9092 latency.test 10000 1 128 > /usr/local/kafka/kafka_2.11-0.10.1.0/config/client.properties > ... > Avg latency: 1.8739 ms > > SSL Connection > bin/kafka-run-class.sh kafka.tools.EndToEndLatency <broker hostname>:9093 > latency.test 10000 1 128 /usr/local/kafka/kafka_2.11-0. > 10.1.0/config/client-ssl.properties > ... > Avg latency: 2.4234 ms > > 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. > > Seems to show decent latency after the initial SSL handshake as well. > > So I will try to look harder at how librdkafka + OpenSSL are doing SSL. > If I figure anything out, I'll do one last follow up email to save someone > else with this stack a similar headache. > > Thanks for teaching me about the command line tools, guys! > - Aaron > > > > On Fri, Nov 18, 2016 at 2:59 PM, Aaron Wilkinson <aa...@modopayments.com> > wrote: > >> Thank you both, Hans and Rajini. >> >> I will try out all the methods you suggested and report back. >> >> As an aside my investigation into the known, slow software implementation >> of the GCM class of cipher algorithms in java 8 was a bust. I tried all of >> the default cipher suites common to OpenSSL (on the client) and java (on >> the broker) and they all gave consistent (slow) results of about 40 ms per >> hop. >> >> For posterity at the time of this writing those were (OpenSSL format): >> 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 >> >> I can't guarantee that I'm not looking at a problem where the java crypto >> module is not using hardware acceleration. (I've verified that OpenSSL has >> access to the aesni hardware instructions, but I have no idea how to tell >> if the java crypto module is making use of them.) However, it would appear >> that it is at least not a problem specific to the GCM algorithm. >> >> - Aaron >> >> >> On Fri, Nov 18, 2016 at 2:37 AM, Rajini Sivaram < >> rajinisiva...@googlemail.com> wrote: >> >>> You can use the tools shipped with Kafka to measure latency. >>> >>> For latency at low load, run: >>> >>> >>> - bin/kafka-run-class.sh kafka.tools.EndToEndLatency >>> >>> >>> You may also find it useful to run producer performance test at different >>> throughputs. The tool prints out latency as well: >>> >>> >>> - bin/kafka-producer-perf-test.sh >>> >>> >>> On Fri, Nov 18, 2016 at 1:25 AM, Hans Jespersen <h...@confluent.io> >>> wrote: >>> >>> > Publish lots of messages and measure in seconds or minutes. Otherwise >>> you >>> > are just benchmarking the initial SSL handshake setup time which should >>> > normally be a one time overhead, not a per message overhead. If you >>> just >>> > send one message then of course SSL is much slower. >>> > >>> > -hans >>> > >>> > > On Nov 18, 2016, at 1:07 AM, Aaron Wilkinson <aa...@modopayments.com >>> > >>> > wrote: >>> > > >>> > > Hi, Hans. I was able to get the command line producer / consumer >>> working >>> > > with SSL but I'm not sure how to measure millisecond resolution >>> latency >>> > > with them. I thought maybe the '--property print.timestamp=true' >>> > argument >>> > > would help, but only has second resolution. Do you know of any way >>> to >>> > get >>> > > the consumer to print out a receipt time-stamp with millisecond >>> > > resolution? Or of any extended documentation on the command line >>> tools >>> > in >>> > > general? >>> > > >>> > > Oh also, a couple other tidbits that may help: >>> > > Ubuntu 16.04 >>> > > Kafka 10.1.0 >>> > > openjdk version "1.8.0_111" >>> > > TLS 1.2 >>> > > >>> > > I was wondering if maybe this could be my problem: >>> > > http://stackoverflow.com/questions/25992131/slow-aes- >>> > gcm-encryption-and-decryption-with-java-8u20 >>> > > >>> > > I didn't specify any cipher suites in either the broker or the client >>> > > config which I gather leaves it up to the broker/client to decide >>> during >>> > > TLS handshaking. I'm not sure if there is an easy way to figure out >>> > which >>> > > one they ended up with... I'll work on specifying which cipher >>> suite I >>> > > want and try to pick something with which java is simpatico. >>> > > >>> > > >>> > >> On Thu, Nov 17, 2016 at 4:04 PM, Hans Jespersen <h...@confluent.io> >>> > wrote: >>> > >> >>> > >> What is the difference using the bin/kafka-console-producer and >>> > >> kafka-console-consumer as pub/sub clients? >>> > >> >>> > >> see http://docs.confluent.io/3.1.0/kafka/ssl.html >>> > >> >>> > >> -hans >>> > >> >>> > >> /** >>> > >> * Hans Jespersen, Principal Systems Engineer, Confluent Inc. >>> > >> * h...@confluent.io (650)924-2670 >>> > >> */ >>> > >> >>> > >> On Thu, Nov 17, 2016 at 11:56 PM, Aaron Wilkinson < >>> > aa...@modopayments.com> >>> > >> wrote: >>> > >> >>> > >>> Pardon if this is a oft repeated issue, but all the information I >>> could >>> > >>> find said I should expect a 20-50% performance hit when using SSL >>> with >>> > >>> kafka, and I am seeing closer to 2000-3000% >>> > >>> >>> > >>> I'm trying to get kafka to behave like a fast, secured message bus. >>> > So I >>> > >>> am sending small messages, one at a time. I have set up a simple, >>> 2 >>> > >>> machine experiment in AWS with 1 client machine and 1 >>> zookeeper/broker >>> > >>> machine and I'm an running a very linear test. >>> > >>> >>> > >>> There are 2 topics: "request" and "response" and 2 threads on the >>> > client >>> > >>> machine each of which connects to those 2 topics. Thread 1 >>> produces a >>> > >>> "request", thread 2 consumes it and then produces a "response" >>> which >>> > >> thread >>> > >>> 1 then consumes. At that point thread 1 proceeds to send the next >>> > >>> "request" and the process repeats. >>> > >>> >>> > >>> So there are a total of 4 connections to the broker. >>> > >>> >>> > >>> I can run a sustained test without SSL and see 1 to 1.5 ms per >>> message >>> > >> hop >>> > >>> (where a "hop" means the message has traveled across 1 of the 4 >>> > >>> connections- either a production or a consumption of either the >>> request >>> > >> or >>> > >>> the response). >>> > >>> >>> > >>> Each connection for which I turn on SSL increases the hop time 35 >>> to 45 >>> > >> ms. >>> > >>> >>> > >>> Now, the problem could be with the stack I'm using (PHP 7 talking >>> to >>> > the >>> > >>> broker via the librdkafka C library). But before I go about >>> trying to >>> > >>> reproduce this with a java client (which is not my forte) I was >>> > wondering >>> > >>> if anyone else has run into a similar issue either with PHP or any >>> > other >>> > >>> language / library. Or does anyone know a direct way to figure out >>> > >> whether >>> > >>> this slow down is at the broker or at the client? >>> > >>> >>> > >>> Thanks in advance for your help! >>> > >>> Aaron >>> > >>> >>> > >> >>> > >>> >>> >>> >>> -- >>> Regards, >>> >>> Rajini >>> >> >> >