Out of memory on DefaultKafkaReceiver with high batch size, returned by poll
ivyazmitinov opened this issue · comments
During testing on the prod with the amount of data in Kafka about 250Gb and configs resulting in 50-100K records in one poll batch we faced the OOM on 2GB heap in about 30 minutes of usage. After an investigation we found out that the root cause in the default prefetch
parameters of the publishOn
and default queues in ConsumerHandler#sink
.
Expected Behavior
With the following configs:
max.poll.records: 100000
fetch.max.wait.ms: 1000
max.partition.fetch.bytes: 5242880 # 50Mb
fetch.max.bytes: 104857600 # 100Mb
And the following usage:
KafkaReceiver.create(receiverOptions)
.receiveExactlyOnce(transactionManager)
.concatMap(recordsBatch -> actualizeBatchTransactional(recordsBatch, transactionManager), 1)
It is expected to only one batch at the time to be requested from the KafkaReceiver
due to the concatMap
semantics.
Actual Behavior
256 batches are requested asynchronously due to implementation of DefaultKafkaReceiver
. It is caused by usage of publishOn
with the default prefetch
parameter in all receive*
methods, which is propagated to the ConsumerEventLoop#onRequest
instead of concatMap
's 1, effectively causing it to make 256 polls.
Also, as turned out, default size of ConsumerHandler#sink
also causes memory issues, probably becase incoming ConsumerRecords
are double buffered and can't be effectively garbage collected: in the underlying UnicastProcessor
queue and the publishOn
queue
Steps to Reproduce
It is tricky to reproduce locally, but, possibly, could be done with mocking of the kafka.clients.consumer.Consumer
Possible Solution
We fixed it by passing the prefetch value to the underlying publishOn
and reducing the ConsumerHandler#sink
's queue size to 1. See the following commit.
Also, on the 1.2.4.RELEASE
only publishOn
fix was required, EmitterProcessor
worked fine.
Your Environment
- Reactor version(s) used: 3.4.0
- Other relevant libraries versions (eg.
netty
, ...): io.projectreactor.kafka:reactor-kafka:1.3.0 - JVM version (
javar -version
): openjdk version "1.8.0_265" - OS and version (eg
uname -a
): Linux analytics-release-1 5.8.8-gentoo #1 SMP PREEMPT Mon Sep 28 10:22:33 +03 2020 x86_64 AMD Ryzen 5 3600 6-Core Processor AuthenticAMD GNU/Linux
Same problem here.
Reproductible with this : https://github.com/glandais-sparklane/reactor-kafka-issue-183
Xmx set to 64m, with heap dump and GC log in /tmp/
Hope it helps
@glandais-sparklane
Great, thank you! I've patched other receive*
methods and fixed the issue, it seems.
You could verify yourself, just assemble reactor-kafka
from my fork. Be sure to use exactly memory-usage-patch
branch.
It works better with your patch, but also with 1.2.4.RELEASE, so I'll stick to this version for the moment.
Same problem with 2020.0.4 release train :( Updated example : https://github.com/glandais-sparklane/reactor-kafka-issue-183
Same problem here I'm pretty sure:
Between this issue and #190 it seems that reactor-kafka is not ready for production use with any significant volume, wish I knew this 6 months ago 😞
@peterfigure
Well, with my fix we've successfully processed about 560gb of data on 2gb heap, so I'd say it is "near ready" :)
thanks @ivyazmitinov - have you tried rebasing 1.3.x or applying your fix to 1.3x?
@peterfigure
Well, it is based on 1.3.0, so there should be no issues with rebasing. I'll rebase later
@peterfigure
Done, fix is now on 1.3.2
@ivyazmitinov Feel free to submit your fix as a Pull Request; some tests would be useful, too.
@ivyazmitinov your branch did not go well for me, at 6 minutes into a nightly job:
Member xxx-176adbc3-3943-4aa5-b742-9e8be350a0fc sending LeaveGroup request to coordinator kafka-4.broker.kafka.svc.cluster.local:9092 (id: 2147483643 rack: null) due to consumer poll timeout has expired. This means the time between subsequent calls to poll() was longer than the configured max.poll.interval.ms, which typically implies that the poll loop is spending too much time processing messages. You can address this either by increasing max.poll.interval.ms or by reducing the maximum size of batches returned in poll() with max.poll.records.
and that was it, never rejoined, only processed 11 messages of the usual 100k, all within 2 seconds. It's unlikely that actual processing took > max.poll.interval - haven't seen that before, there's nothing in the logs.
looks like reactor-kafka
may stuck in some kind of spin loop?
max.poll.interval.ms = 300000
max.poll.records = 500
@peterfigure
Yes, I see similar behavior now in tests, I'll investigate further. What receive*
method did you use? We are using receiveExactlyOnce
@ivyazmitinov 👍 - just .receive()
and doing manual commits.
@peterfigure
I've pushed a fix and intermediate implementation of prefetches, could you, please, test it out?
Essentially I've simply enable unbounded queue on backpressure, so it could cause memory issues, but generally shouldn't because of new default prefetch = 1. You can also play around with the new prefetch setting for receive()
.
thanks @ivyazmitinov - seems to be working (processing messages) but I still see much higher heap usage (even after a full GC) - might be related to #190 - it has 1000s of ReceiverRecords on the heap, it's like it's not respecting backpressure at all. I'll open a separate issue.
@peterfigure
No, I don't think that issue is with back pressure. It is more likely related to the unbounded queue. Could you check heap dump to see, what exactly references the records?
@peterfigure
I couldn't reproduce your issue on synthetic test, could you give more details? What's the average size of the message, how it is processed in the pipeline? Any flatMap
s with a default concurrency or something like that?
@ivyazmitinov started working on a replication project: https://github.com/peterfigure/reactor-kafka-heap - unfortunately don't have more time for the next few hours, if you monitor this in VisualVM you should see 1000s of Pets and 100,000s of Pet in the heap while it's running.
@garyrussell
Actually, I got much worse performance with your PR in my tests and in the project provided by @peterfigure: in the project, with -Xmx750m
the app OOMed almost immediately. With 8gb heap started to grow very quickly. Could you check, please?
My PR does not address this issue, only the poll task memory leak and fixes the pause/resume logic.
I only mentioned it here because @ivyazmitinov mentioned #190.
@garyrussell
Well, there may be bug causing another memory leak, since performance plummeted. I'd still recommend to try your PR on @peterfigure's project.
Oh, sorry; I didn't notice this was your comment 😦 - I thought someone was using my PR thinking that it fixed this back pressure issue with large batches.
All it does is avoid resuming partitions that were paused by the user (via doOnConsumer), but I found when that was fixed, we never resumed because the poll task was only scheduled if requested
is > 0 (in some cases).
I will investigate further.
Are you saying you rebased your fix on top of my PR and that is where you are seeing a degradation?
@garyrussell
No, it is reproducible with your changes alone as well
As I said, I am not surprised by that, my PR does not attempt address the pre-fetch of many large batches of records. I thought that's what your work does.
@garyrussell
@peterfigure's project doesn't use large batches either, there is a simple receive ()
call. I forgot to mention that :)
Also, as I discovered, original memory issue is valid for all receives, because they are polling batches under the hood
Thanks @ivyazmitinov @garyrussell - Ivan, sounds like you were able to replicate what I was seeing? Hopefully I wasn't doing something stupid or missing something 😄
Ok, I'll take a look.
@peterfigure
Yeah, I'm investigating it now. Still, while memory usage is high, it should not lead to OOM with my patch applied. How is your app, didn't you get OOMs yet?
@garyrussell I would expect those GC cycles to bring the heap usage down much lower than 250MB. While it's running I see:
while in the logs at the same time see:
12:52:03.763 [parallel-9] INFO Main - processed key [2808] count [2808]
12:52:03.816 [parallel-10] INFO Main - processed key [2809] count [2809]
12:52:03.871 [parallel-11] INFO Main - processed key [2810] count [2810]
12:52:03.921 [parallel-12] INFO Main - processed key [2811] count [2811]
12:52:03.976 [parallel-13] INFO Main - processed key [2812] count [2812]
Why has the KafkaReceiver already polled 8525 (I suspect it's already polled all 10k) while the Flux has only processed 2800 messages and committing them each individually? Shouldn't back pressure be applied so messages are polled more in step with the processing? Thereby reducing the runtime heap requirements.
I think there's another issue where some garbage is never collected from the old gen but more on than later when I have more data.
@peterfigure
That's not related to the (allegedly) degradation, introduced by @garyrussell, but it is what I'm investigating now. I think, it is somehow related to the fact that from downstream backpressure is applied on the individual consumer records, while kafka polls return batches. So roughly speaking, whenever 1 element is processed, another batch is polled from kafka.
I'll return to this tomorrow
Also, I'm not sure, have @garyrussell deleted his comment or I just can see it for some reason, but that's what I get immediately after start:
23:11:16.247 [reactive-kafka-theTestGroupID-1] ERROR Main - Failed to deserialize proto data [[B@2b94b3f0]
java.lang.OutOfMemoryError: GC overhead limit exceeded
at java.util.Arrays.copyOfRange(Arrays.java:3664)
at java.lang.String.<init>(String.java:207)
at com.google.protobuf.Utf8$UnsafeProcessor.decodeUtf8(Utf8.java:1439)
at com.google.protobuf.Utf8.decodeUtf8(Utf8.java:340)
at com.google.protobuf.CodedInputStream$ArrayDecoder.readStringRequireUtf8(CodedInputStream.java:810)
at io.ys.PetDomain$Pet.<init>(PetDomain.java:86)
at io.ys.PetDomain$Pet.<init>(PetDomain.java:42)
at io.ys.PetDomain$Pet$1.parsePartialFrom(PetDomain.java:635)
at io.ys.PetDomain$Pet$1.parsePartialFrom(PetDomain.java:629)
I think the fundamental truth is we should never be prefetching batches (at least no more than one). My understanding is that is what @ivyazmitinov 's commit is for.
I am actually seeing very different behavior (to my original post that I deleted) now that I have successfully constrained the heap to 750m (gradle is a pain sometimes).
I am still not seeing the OOM that @ivyazmitinov mentions.
@garyrussell
It sort of works on 2gb heap, but still far from optimal:
Even "Perform GC" button doesn't do the trick
gradle is a pain sometimes
I recommend running through Idea, it s much easier to configure there :)
Well, there is clearly something different between our environments.
All I did was add mavenLocal()
to the repos, and changed the reactor-kafka version to 1.3.3-SNAPSHOT (after installing it to my local maven repo) and reactor to 3.4.4-SNAPSHOT, and added
tasks.named<JavaExec>("run") {
doFirst {
jvmArgs = listOf("-Xmx750m")
}
}
to the gradle script.
I was having trouble with IDEA; I used ./gradlew run
on the command line.
I'll reload into IDEA and try again.
@garyrussell
I've also added logging to the ConsumerEventLoop#schedule
method and that's what I got:
23:28:47.017 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Scheduling next batch...
23:28:47.017 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Scheduling next batch...
23:28:47.025 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Scheduling next batch...
23:28:47.025 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Scheduling next batch...
23:28:47.035 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Scheduling next batch...
23:28:47.035 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Scheduling next batch...
23:28:47.039 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Scheduling next batch...
23:28:47.039 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Scheduling next batch...
23:28:47.048 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Scheduling next batch...
23:28:47.048 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Scheduling next batch...
23:28:47.049 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Paused - back pressure
23:28:47.049 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Paused - back pressure
23:28:48.208 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Scheduling next batch...
23:28:48.208 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Scheduling next batch...
23:28:48.308 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Scheduling next batch...
23:28:48.308 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Scheduling next batch...
23:28:48.409 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Scheduling next batch...
23:28:48.409 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Scheduling next batch...
23:28:48.509 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Scheduling next batch...
23:28:48.509 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Scheduling next batch...
23:28:48.610 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Scheduling next batch...
23:28:48.610 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Scheduling next batch...
23:28:48.711 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Scheduling next batch...
23:28:48.711 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Scheduling next batch...
23:28:48.812 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Scheduling next batch...
23:28:48.812 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Scheduling next batch...
23:28:48.912 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Scheduling next batch...
I'm not sure that such frequent scheduling was the intended behaviour
Looks like all the logs are duplicated - I don't see how you can get 2 pauses (and every log above has 2 at the same timestamp).
It's still working fine for me, running out of IDEA.
All I did was add mavenLocal() to the repos, and changed the reactor-kafka version to 1.3.3-SNAPSHOT (after installing it to my local maven repo) and reactor to 3.4.4-SNAPSHOT, and added
Yep, did exactly the same, and tried with gradle even, except reactor version, i've used 3.4.3. Could it be the case?
Also, what version of java are you on? I'm on openjdk version "1.8.0_282"
java -version
openjdk version "14.0.2" 2020-07-14
OpenJDK Runtime Environment AdoptOpenJDK (build 14.0.2+12)
OpenJDK 64-Bit Server VM AdoptOpenJDK (build 14.0.2+12, mixed mode, sharing)
I don't know about the reactor version will affect things - I just used the snapshot because the reactor kafka snap uses it.
I think the fundamental truth is we should never be prefetching batches (at least no more than one).
Agreed
My understanding is that is what @ivyazmitinov 's commit is for.
The patch helps some but I still see a significant spread between messages processed (committed) and what's on the heap, eg 1000 vs 4000.
FWIW, I use 11.0.10 with G1GC
Found the root cause, commented in the PR.
@peterfigure This also seems to fix issue with abundant consumer records in heap, so you could try the PR after my changes are applied. It wasn't related to my issue with large batches after all :)
@ivyazmitinov I have rebased your branch on top of my PR here https://github.com/garyrussell/reactor-kafka/tree/pauseWithBP
And this is the heap profile I see now for a complete rin...
With debug logging for the ConsumerEventLoop
I see things working as I expected
16:47:50.864 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Requesting 1 batches
16:47:50.966 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Requesting 1 batches
16:47:51.030 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - onPartitionsAssigned [test-topic-84, ...] // 100 partitions
16:47:51.081 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Requesting 1 batches
16:47:51.290 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Requesting 1 batches
16:47:51.601 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Paused - back pressure
16:48:16.766 [parallel-8] DEBUG r.k.r.internals.ConsumerEventLoop - Requesting 1 batches
...
16:48:43.394 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Resumed
...
16:49:16.440 [parallel-8] DEBUG r.k.r.internals.ConsumerEventLoop - Requesting 1 batches
16:49:33.695 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Paused - back pressure
However, I am seeing a lot of "Requesting 1 batches" between the pause/resume logs (many in the same millisecond). I am not sure why it's taking so long to resume (but I don't see any slow down of record delivery on the consumer side). This will definitely inflate the requests so that when we resume, we will fetch many batches again. We somehow need to prevent that.
The fact that the requests are coming in on the receiver's event thread indicates it's because we're calling it for each record in the batch, instead of when the batch is complete.
Will continue the investigation tomorrow.
cc/ @peterfigure
@ivyazmitinov Looks like your suggestion on my PR did the trick...
17:22:57.446 [reactive-kafka-theTestGroupID-1] INFO r.k.r.internals.ConsumerEventLoop - SubscribeEvent
17:22:57.455 [main] DEBUG r.k.r.internals.ConsumerEventLoop - Requesting 1 batches
17:22:57.788 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - onPartitionsAssigned [test-topic-84, test-topic-18, test-topic-51, test-topic-76, test-topic-10, test-topic-43, test-topic-68, test-topic-2, test-topic-35, test-topic-60, test-topic-93, test-topic-27, test-topic-50, test-topic-83, test-topic-17, test-topic-42, test-topic-75, test-topic-9, test-topic-34, test-topic-67, test-topic-1, test-topic-92, test-topic-26, test-topic-59, test-topic-20, test-topic-53, test-topic-78, test-topic-12, test-topic-45, test-topic-70, test-topic-4, test-topic-37, test-topic-62, test-topic-95, test-topic-29, test-topic-54, test-topic-87, test-topic-52, test-topic-85, test-topic-19, test-topic-44, test-topic-77, test-topic-11, test-topic-36, test-topic-69, test-topic-3, test-topic-94, test-topic-28, test-topic-61, test-topic-86, test-topic-80, test-topic-14, test-topic-47, test-topic-72, test-topic-6, test-topic-39, test-topic-64, test-topic-97, test-topic-31, test-topic-56, test-topic-89, test-topic-23, test-topic-21, test-topic-46, test-topic-79, test-topic-13, test-topic-38, test-topic-71, test-topic-5, test-topic-96, test-topic-30, test-topic-63, test-topic-88, test-topic-22, test-topic-55, test-topic-82, test-topic-16, test-topic-49, test-topic-74, test-topic-8, test-topic-41, test-topic-66, test-topic-99, test-topic-0, test-topic-33, test-topic-58, test-topic-91, test-topic-25, test-topic-48, test-topic-81, test-topic-15, test-topic-40, test-topic-73, test-topic-7, test-topic-98, test-topic-32, test-topic-65, test-topic-90, test-topic-24, test-topic-57]
17:22:58.118 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Emitting 500 records
17:22:58.120 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Requesting 1 batches
17:22:58.400 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Emitting 500 records
17:22:58.402 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Paused - back pressure
17:23:23.654 [parallel-8] DEBUG r.k.r.internals.ConsumerEventLoop - Requesting 1 batches
17:23:23.697 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Resumed
17:23:23.740 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Emitting 500 records
17:23:23.741 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Paused - back pressure
17:23:50.325 [parallel-8] DEBUG r.k.r.internals.ConsumerEventLoop - Requesting 1 batches
17:23:50.407 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Resumed
17:23:50.430 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Emitting 500 records
17:23:50.430 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Paused - back pressure
17:24:16.902 [parallel-8] DEBUG r.k.r.internals.ConsumerEventLoop - Requesting 1 batches
17:24:16.939 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Resumed
17:24:16.956 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Emitting 500 records
17:24:16.956 [reactive-kafka-theTestGroupID-1] DEBUG r.k.r.internals.ConsumerEventLoop - Paused - back pressure
...
@peterfigure
JFYI, just pushed a fix loosely based on @garyrussell PR and my investigation, was able to run your project on 200m heap without issues (event had to force GC to check is it works at all).
looks much improved!
@garyrussell Does 1.3.3 contain #190 and this fix?
Yes; they are both in 1.3.3; this issue was flagged with the wrong milestone.