[ https://issues.apache.org/jira/browse/KAFKA-3565?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15246765#comment-15246765 ]
Ismael Juma commented on KAFKA-3565: ------------------------------------ Interesting about the tuning that may be required due to the lower broker latency. To add to what Jun said, another difference in the results I posted is that they were executed over a network (not localhost) with 3 brokers. For the gzip result, it involved 3 producers as well. These details can sometimes be important, for example KAFKA-2950 was only visible if there were multiple brokers. Given what you have found so far, the questions I have are: 1. Is the slowdown in each producer user thread solely explained by the additional 8 bytes or is there something else as well (as we saw in KAFKA-2950, some of the producer code is performance sensitive)? 2. Will existing users with compressed topics suffer a slowdown when they move to 0.10.0 unless they tune their producers or are the producer settings in `benchmark_test.py` unrealistic? If the former, do we need to add something to the upgrade notes? > Producer's throughput lower with compressed data after KIP-31/32 > ---------------------------------------------------------------- > > Key: KAFKA-3565 > URL: https://issues.apache.org/jira/browse/KAFKA-3565 > Project: Kafka > Issue Type: Bug > Reporter: Ismael Juma > Priority: Critical > Fix For: 0.10.0.0 > > > Relative offsets were introduced by KIP-31 so that the broker does not have > to recompress data (this was previously required after offsets were > assigned). The implicit assumption is that reducing CPU usage required by > recompression would mean that producer throughput for compressed data would > increase. > However, this doesn't seem to be the case: > {code} > Commit: eee95228fabe1643baa016a2d49fb0a9fe2c66bd (one before KIP-31/32) > test_id: > 2016-04-15--012.kafkatest.tests.benchmark_test.Benchmark.test_producer_throughput.topic=topic-replication-factor-three.security_protocol=PLAINTEXT.acks=1.message_size=100.compression_type=snappy > status: PASS > run time: 59.030 seconds > {"records_per_sec": 519418.343653, "mb_per_sec": 49.54} > {code} > Full results: https://gist.github.com/ijuma/0afada4ff51ad6a5ac2125714d748292 > {code} > Commit: fa594c811e4e329b6e7b897bce910c6772c46c0f (KIP-31/32) > test_id: > 2016-04-15--013.kafkatest.tests.benchmark_test.Benchmark.test_producer_throughput.topic=topic-replication-factor-three.security_protocol=PLAINTEXT.acks=1.message_size=100.compression_type=snappy > status: PASS > run time: 1 minute 0.243 seconds > {"records_per_sec": 427308.818848, "mb_per_sec": 40.75} > {code} > Full results: https://gist.github.com/ijuma/e49430f0548c4de5691ad47696f5c87d > The difference for the uncompressed case is smaller (and within what one > would expect given the additional size overhead caused by the timestamp > field): > {code} > Commit: eee95228fabe1643baa016a2d49fb0a9fe2c66bd (one before KIP-31/32) > test_id: > 2016-04-15--010.kafkatest.tests.benchmark_test.Benchmark.test_producer_throughput.topic=topic-replication-factor-three.security_protocol=PLAINTEXT.acks=1.message_size=100 > status: PASS > run time: 1 minute 4.176 seconds > {"records_per_sec": 321018.17747, "mb_per_sec": 30.61} > {code} > Full results: https://gist.github.com/ijuma/5fec369d686751a2d84debae8f324d4f > {code} > Commit: fa594c811e4e329b6e7b897bce910c6772c46c0f (KIP-31/32) > test_id: > 2016-04-15--014.kafkatest.tests.benchmark_test.Benchmark.test_producer_throughput.topic=topic-replication-factor-three.security_protocol=PLAINTEXT.acks=1.message_size=100 > status: PASS > run time: 1 minute 5.079 seconds > {"records_per_sec": 291777.608696, "mb_per_sec": 27.83} > {code} > Full results: https://gist.github.com/ijuma/1d35bd831ff9931448b0294bd9b787ed -- This message was sent by Atlassian JIRA (v6.3.4#6332)