sarama: Extremely high memory usage for zstd decompression

Versions
Sarama Kafka Go
1.26.4 2.6.0 1.14.1 (CGO)

When I ran with same config / same cluster / same data on sarama 1.21.0 / Golang 1.12 (CGO) I did NOT encounter this issue.

(I also tested briefly with sarama 1.27.0 / Go 1.15 (CGO) and saw the same issue)

Configuration

What configuration values are you using for Sarama and Kafka?

Using defaults from here, except for setting ChannelBufferSize to a variety of values including 256, 16, 4, 1, 0 - see same result regardless.

Logs

Unable to grab logs easily since using sarama in an existing dockerized open source application. Depending on the need may be able to delve further to get them.

logs: CLICK ME

Problem Description

We are using sarama within the benthos (v3.22) stream processing tool for over a year. We have run it for an extended period in our environment with compression = None and Snappy extensively and never had any issues.

Recently we switched over to using zstd compression in our main data pipeline. Under normal load (5-10MB/s on the topic) we see no issue, our application runs under 200MB memory usage.

When running under a consumer backlog, the exact same config will suddenly start using 2.3GB+ memory. We run it on nodes with only 4GB memory and kubernetes rapidly kills the pod under this state.

Looking at a memory trace (again this is within a few seconds of launch) we see all the memory used by zstd decoder.

Memory tracesScreen Shot 2020-11-13 at 1 38 52 PM Screen Shot 2020-11-13 at 1 39 18 PM

When we switch to older sarama version (via benthos v1.20), the issue goes away. When we switch to messages being written to the topic with snappy instead of zstd, the issue goes away.

About this issue

  • Original URL
  • State: closed
  • Created 4 years ago
  • Reactions: 2
  • Comments: 20 (10 by maintainers)

Most upvoted comments

more followup.

this issue appears to be a GC one. Memory is not leaking, but the GC cannot keep up in normal circumstances.

Some testing specs

  • ~1000/msg second all zstd compress on a topic of 64 partitions
  • messages size is in the 1-10kb range

some results

@ sarama’s default settings (1Mb fetch size) w/ GOGC=100

  • at any given time RAM consumption according to pprof is 512Mb - 1Gb for items in Zstd,
  • System ram explodes to the Many GB range (ran out at 32Gb)
  • pprof also indicated that github.com/Shopify/sarama.(*Broker).responseReceiver also had 512Mb - 1Gb in ram usage (i assume indicating things are getting backedup in internal buffers)

@ sarama’s default settings (1Mb fetch size) w/ GOGC=10

  • at any given time RAM consumption according to pprof is 512Mb - 1Gb for items in Zstd,
  • System ram explodes to the Many GB range (ran out at 32Gb), but was slower to reach OOM.
  • PProf indicated ~80% of the CPU time was in GC.

@ sarama’s default settings but with 8kb Fetch Size w/ GOGC=100

  • RAM consumption was much reduced over all, at any given time RAM consumption according to
  • pprof is 128Mb - 512Mb for items in Zstd
  • a rough steady state was reached where system assumed things were consuming about 2-3Gb/Ram but proff only had live ram at around 512 in total
  • much reduced github.com/Shopify/sarama.(*Broker).responseReceiver usage (pprof didn’t register it)

@ sarama’s default settings but with 8kb Fetch Size w/ GOGC=10

  • RAM consumption was much reduced over all, at any given time RAM consumption according to pprof is 128Mb - 512Mb for items in Zstd
  • much reduced github.com/Shopify/sarama.(*Broker).responseReceiver usage (pprof didn’t register it)
  • a rough steady state was reached where system assumed things were consuming about 1-2Gb/Ram.
  • most CPU time was all in GC

just in case this is relevent the “producer” of these messages using the jvm kafka lib 2.6.0 using zstd the `“com.github.luben:zstd-jni:1.4.4-7.zstd” lib

@BradLugo The PR above has been released as v1.11.3.

@klauspost that’s a good thought. I see the rfkafka producer has a conditional around whether to include uncompressed size based on the linked zstd version https://github.com/edenhill/librdkafka/blob/3b73ceb2049777331f3e544270bd8d0cdcd41de5/src/rdkafka_zstd.c#L149-L156

It could be that the compressed data doesn’t contain the uncompressed size and that sarama gives a slice with 0 byte capacity, so 1MB is allocated for each output.

If sarama want to limit that they can allocate a different size.