Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Extremely high memory usage for zstd decompression #1831

Closed
JoshuaC215 opened this issue Nov 13, 2020 · 20 comments
Closed

Extremely high memory usage for zstd decompression #1831

JoshuaC215 opened this issue Nov 13, 2020 · 20 comments
Labels
stale Issues and pull requests without any recent activity

Comments

@JoshuaC215
Copy link

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.

@JoshuaC215
Copy link
Author

Also tagging @klauspost in case you have any insight about this. It seems like sarama switched from using github.com/DataDog/zstd 1.3.5 to github.com/klauspost/compress v1.9.8 between 1.21.0 and 1.26.4, so I'm wondering if this is related.

Also I forgot to mention, we also have applications that interact with kafka through github.com/confluentinc/confluent-kafka-go more or less running with the defaults, and they have not encountered this issue.

@klauspost
Copy link
Contributor

It will allocate here if the dst slice is capacity too small. If the compression was done by EncodeAll it will contain the exact output size, so that will be allocated.

If the encode doesn't contain frame content size and a 0 capacity slice is passed, it will allocate WindowSize * 2 or at most 1 MB.

Either way it will hand back control if the dst slice when done, so it seems like it is kept referenced internally.

@JoshKCarroll Could you add -lines to the trace? At least then we know which case is being hit.

@JoshuaC215
Copy link
Author

JoshuaC215 commented Nov 14, 2020

Thank you for this information! I will see if we can get this trace and report back.

For context, I believe in all our cases where we see this behavior, the compression would have been done by confluent-kafka-go which uses https://github.com/edenhill/librdkafka (v1.4.2 I believe) under the hood. So it seems possible there is some mismatch between the two libraries.

In our particular case the impact of the issue is not huge since we only have this interaction on a couple of topics, and so we just switched those to snappy. But, still interesting and happy to see whether we can help debug.

@klauspost
Copy link
Contributor

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.

@dnwe
Copy link
Collaborator

dnwe commented Nov 15, 2020

@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

@klauspost
Copy link
Contributor

klauspost commented Nov 15, 2020

Actually it could be tweaked to allocate "compressed size * 2" or at most 1MiB up front. That should make this case behave more predictable.

It may have an unreasonably large window size for the content.

@klauspost
Copy link
Contributor

Added klauspost/compress#295

@BradLugo
Copy link

BradLugo commented Nov 16, 2020

Hey, y'all. I'm helping Josh investigate this. Looks like y'all already made some progress but here's the trace with -lines:

Memory trace

@klauspost
Copy link
Contributor

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

@wyndhblb
Copy link
Contributor

Hi just following up. The v1.11.3 version did not stem this issue. Things still have the same basic ram explosion all in

 927.53MB 99.39% 99.39%   929.17MB 99.56%  github.com/klauspost/compress/zstd.(*Decoder).DecodeAll

this issue is effected by the sarama setting of Consumer.Fetch.Default the smaller this is the slower the memory explosion occurs (it still does occur, just slower)

If the the value Consumer.Fetch.Default is in the 4Mb range and the message volume in the 1000s/second one goes from 0 -> 30Gb is a matter of seconds.

If the the value Consumer.Fetch.Default is in the 8kb range and the message volume in the 1000s/second one goes from 0 -> 30Gb is a matter of 15 min.

This issue does seem to effect the Producer side of things as well, not as violently, but we have shown at a message rate of ~100/second that the ram requirements have increased dramatically (from 200Mb to 2-4Gb) and constantly increases.

Using the confluent go lib (libkafkard) does not have this issue under the same conditions (100k/msg/sec consuming zstd compressed messages takes ~500-1Gb or ram total)

@klauspost
Copy link
Contributor

@wyndhblb Then I think you are looking in the wrong place. Something else is holding on to the references of the output.

@wyndhblb
Copy link
Contributor

@klauspost I'll clarify a bit

The issue really appears when consuming a "full backlog" (say many millions of messages, like a full topic read).

if the consumed messages are simply /dev/null'ed the issue still appears in full force

This issue is still very present and only under zstd

gzip, snappy, lz4, none do not suffer this same grand increase in memory usage and all pprof investigations point to

github.com/klauspost/compress/zstd.(*Decoder).DecodeAll

I'm still unclear if it's the zstd decoder or sarama holding on to slices somewhere, just pointing out this "issue" is not resolved by simply using the v1.11.3 release of the the compression lib.

@wyndhblb
Copy link
Contributor

wyndhblb commented Jan 3, 2021

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

@hackery
Copy link

hackery commented Apr 14, 2021

This has given me some interesting times today. We use Kafka to transport systems metrics and consume them with Telegraf into InfluxDB; We changed one of our metrics producers to use zstd compression, and Telegraf has exploded. We'll revert the compression, and maybe change to lz4, but in the meantime I have a topic full of compressed messages that I can't consume.

The only direct control that we have in Telegraf to change Sarama's behaviour is to select the Kafka version support. Is there any non-programmatic way to change settings such as the above (Fetch Size, GOGC)?

@dnwe
Copy link
Collaborator

dnwe commented Sep 22, 2021

@hackery so @wyndhblb saw an improvement (I believe) after @klauspost releases v1.11.5 of compress containing klauspost/compress#306 — which I believe we've been using since Sarama v1.28.0

Can you confirm which version of Sarama you were testing with? It looked like from your linked issue that Telegraf was at v1.27.1 at the time, and looking at the main branch it is currently at v1.27.2 https://github.com/influxdata/telegraf/blob/3eebfd2f0fd4d1768936ae98f601c827f6a271a2/go.mod#L35

@hackery
Copy link

hackery commented Mar 4, 2022

@dnwe sorry, missed seeing your update; thanks for the pointer, we'll take note in our upgrade plans.

I had to do a bit of code and update spelunking to find what version we were actually using at the time, which was 1.13.3, using Sarama 1.24.0. Looks like the first version of Telegraf using >=1.28.0 is 1.20.3. We're currently on 1.20.2, which is the latest in our repo.

@vincentbernat
Copy link
Contributor

The memory usage using zstd is still quite high (with Sarama 1.32 and compress 1.15.9). In my application, I see 1GB allocated during zstd's saveHist. Maybe this could be solved with #2375?

@github-actions

This comment was marked as outdated.

@github-actions github-actions bot added the stale Issues and pull requests without any recent activity label Aug 25, 2023
@dnwe dnwe removed the stale Issues and pull requests without any recent activity label Aug 25, 2023
@dnwe
Copy link
Collaborator

dnwe commented Aug 25, 2023

@JoshuaC215 where are we with this issue today?

Copy link

Thank you for taking the time to raise this issue. However, it has not had any activity on it in the past 90 days and will be closed in 30 days if no updates occur.
Please check if the main branch has already resolved the issue since it was raised. If you believe the issue is still valid and you would like input from the maintainers then please comment to ask for it to be reviewed.

@github-actions github-actions bot added the stale Issues and pull requests without any recent activity label Nov 26, 2023
@github-actions github-actions bot closed this as not planned Won't fix, can't repro, duplicate, stale Dec 27, 2023
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
stale Issues and pull requests without any recent activity
Projects
None yet
Development

No branches or pull requests

7 participants