You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by "Ismael Juma (JIRA)" <ji...@apache.org> on 2016/04/18 02:30:25 UTC

[jira] [Comment Edited] (KAFKA-3565) Producer's throughput lower with compressed data after KIP-31/32

    [ https://issues.apache.org/jira/browse/KAFKA-3565?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15245007#comment-15245007 ] 

Ismael Juma edited comment on KAFKA-3565 at 4/18/16 12:30 AM:
--------------------------------------------------------------

Becket, 

1. Yes, the non-compressed throughput change is expected, I just included it for reference (as mentioned in the issue description).

2. The additional 8 bytes don't explain the magnitude of the slowdown for the compressed case though (which is quite a bit higher than the non-compressed case).

3. Yes, I tested gzip internally first for the reason you mention and the results were similar (Jun suggested to try snappy too and to keep things simple I only mentioned the snappy results).

I paste the results for gzip with 3 producers that I computed earlier (note that this is 0.9 versus trunk, but KIP-31/32 is the main difference):

{code}
Test name                      version    parameters                 records/s         MB/s       Difference 
test_producer_throughput	0.9	  acks=1, message_size=1000, num_producers=3	124653.624	118.87	- 
test_producer_throughput	trunk     acks=1, message_size=1000, num_producers=3	102308.689	97.57	-17.93
{code}


was (Author: ijuma):
Becket, 

1. Yes, the non-compressed throughput change is expected, I just included it for reference (as mentioned in the issue description).

2. The additional 8 bytes don't explain the magnitude of the slowdown for the compressed case though (which is quite a bit higher than the non-compressed case).

3. Yes, I tested gzip internally first for the reason you mention and the results were similar (Jun suggested to try snappy too and to keep things simple I only mentioned the snappy results).

I paste the results for gzip with 3 producers that I computed earlier (note that this is 0.9 versus trunk, but KIP-31/32 is the main difference):

{code}
Test name                      version    parameters                                                    records/s         MB/s       Difference 
test_producer_throughput	0.9	  acks=1, message_size=1000, num_producers=3	124653.624	118.87	- 
test_producer_throughput	trunk     acks=1, message_size=1000, num_producers=3	102308.689	97.57	-17.93
{code}

> 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)