[jira] [Commented] (KAFKA-2045) Memory Management on the consumer
[ https://issues.apache.org/jira/browse/KAFKA-2045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15943370#comment-15943370 ] Armin Braun commented on KAFKA-2045: [~guozhang] maybe it would be a good step to add a Consumer JMH benchmark here, now that the basic skeleton for JMH was added? If there is interest in that I'd be happy to do it here or in another issue added to the parent of this one :) > Memory Management on the consumer > - > > Key: KAFKA-2045 > URL: https://issues.apache.org/jira/browse/KAFKA-2045 > Project: Kafka > Issue Type: Sub-task > Components: consumer >Reporter: Guozhang Wang > > We need to add the memory management on the new consumer like we did in the > new producer. This would probably include: > 1. byte buffer re-usage for fetch response partition data. > 2. byte buffer re-usage for on-the-fly de-compression. -- This message was sent by Atlassian JIRA (v6.3.15#6346)
[jira] [Commented] (KAFKA-2045) Memory Management on the consumer
[ https://issues.apache.org/jira/browse/KAFKA-2045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15874896#comment-15874896 ] Armin Braun commented on KAFKA-2045: add a suggestion on how to concretely tackly this and KAFKA-1895 here -> https://issues.apache.org/jira/browse/KAFKA-1895?focusedCommentId=15874894&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15874894 > Memory Management on the consumer > - > > Key: KAFKA-2045 > URL: https://issues.apache.org/jira/browse/KAFKA-2045 > Project: Kafka > Issue Type: Sub-task > Components: consumer >Reporter: Guozhang Wang > > We need to add the memory management on the new consumer like we did in the > new producer. This would probably include: > 1. byte buffer re-usage for fetch response partition data. > 2. byte buffer re-usage for on-the-fly de-compression. -- This message was sent by Atlassian JIRA (v6.3.15#6346)
[jira] [Commented] (KAFKA-2045) Memory Management on the consumer
[ https://issues.apache.org/jira/browse/KAFKA-2045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14483550#comment-14483550 ] Guozhang Wang commented on KAFKA-2045: -- [~rzidane] Although we have some thing like consumer / producer perf classes for testing end2end performance we do not have the ones for testing per-module performances (yet). What people usually do is just to use hprof / etc and check the time split-up on functions. > Memory Management on the consumer > - > > Key: KAFKA-2045 > URL: https://issues.apache.org/jira/browse/KAFKA-2045 > Project: Kafka > Issue Type: Sub-task >Reporter: Guozhang Wang > > We need to add the memory management on the new consumer like we did in the > new producer. This would probably include: > 1. byte buffer re-usage for fetch response partition data. > 2. byte buffer re-usage for on-the-fly de-compression. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-2045) Memory Management on the consumer
[ https://issues.apache.org/jira/browse/KAFKA-2045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14385500#comment-14385500 ] Rajiv Kurian commented on KAFKA-2045: - [~jkreps] Totally agree with you on the concerns with a re-write. I am sure I'll end up re-using most of the code, otherwise it will take too long in any case. But given this is just a prototype, I want the freedom to be able to make changes without being bound by the existing architecture and class hierarchy of the client. Even if I do re-implement some of the parts I'll make sure that the client can (a) Do metadata requests so it can react to leaders moving etc. (b) Actually read from multiple topic/partitions spread across multiple brokers and not just a single broker. Again since this is just a rewrite with the sole purpose of exploring possible performance improvements there can be mainly two consequences: i) It shows no improvements: In that case we end up not spending too much time changing the current code, and the hacky code just gets us to this conclusion faster. ii) It shows interesting improvements: If this were true, we can afford to spend some time seeing which things actually improved performance and make a call on how to integrate best. It might be counterproductive to look at the current client implementation and look at the % of time spent in each of the bottlenecks because those numbers are a consequence of the current memory layout. For example if we do an on the fly CRC check and decompression - CRC check time might go up a bit because now we are not striding over a contiguous ByteBuffer in one sweep. Right now the current client has this pattern --- CRC check on Message1--> CRC check on Message2 --> CRC check on MessageN --> Hand message 1 to consumer --> Hand message N to consumer. Instead with the current proposal we will have a pattern of - Do CRC on a Message1 --> Hand Message1 to consumer --> Do CRC on a Message2 --> Hand Message2 to the consumer . So the CRC checks are separated by potential (certain?) cache floundering during the handling of the message by the client. On the other hand from the perspective of the consumer, the pattern looks like this -- Do CRC and validation on all messages starting with 1 to N --> Hand messages 1 to N to client. Now by the time the Kafka consumer is done with validating and deserializing message N, message 1 is possibly already out of the cache. With the new approach since we hand over a message right after validating it, we give the consumer a hot in cache message, which might improve the consumer processing enough to offset for the loss in CRC striding efficiency. Or it may not. It might just turn out that doing the CRC validation upfront is just a pure win since all the CRC tables will be in cache etc and striding access for the CRC math is worth an extra iteration of the ByteBuffer contents. But it is might still be more profitable to elide copies and prevent creation of objects by doing on the fly decoding and handing out indexes into the actual response ByteBuffer. This result might further be affected by how expensive the deserialization and processing of the message is. If the message is a bloated JSON encoded object that is de-serialized into a POJO and then processed really slowly then none of this will probably matter. On the other hand if the message is a compact and binary encoded and can be processed with minimal cache misses, this stuff might add up. My point is that basing the TODOs on the current profile may not be optimal because the profile is a massive consequence of the current layout and allocation patterns. Also the profile will give %s and we might be able to keep the same %s but just still reduce the overall time taken for the entire consumer processing cycle. Just to belabor the point even further, the current hash map implementations might suffer so many cache misses that they mask an underlying improvement opportunity for the data in the maps. Switching to compact primitive arrays based open hash maps might surface that opportunity again. Is there a performance test that is used to keep track of the new Consumer's performance? If so maybe I can wrap that in a JMH suite and re-use that to test improvements? > Memory Management on the consumer > - > > Key: KAFKA-2045 > URL: https://issues.apache.org/jira/browse/KAFKA-2045 > Project: Kafka > Issue Type: Sub-task >Reporter: Guozhang Wang > > We need to add the memory management on the new consumer like we did in the > new producer. This would probably include: > 1. byte buffer re-usage for fetch response partition data. > 2. byte buffer re-usage for on-the-fly de-compression. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-2045) Memory Management on the consumer
[ https://issues.apache.org/jira/browse/KAFKA-2045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14385387#comment-14385387 ] Jay Kreps commented on KAFKA-2045: -- Okay so since most the discussion here is on optimizing memory allocation let's use this ticket for that. I filed KAFKA-2063 to cover bounding the fetch response size. I think that is a prerequisite for this ticket and also a bigger problem (those big allocations actually often cause OOM). > Memory Management on the consumer > - > > Key: KAFKA-2045 > URL: https://issues.apache.org/jira/browse/KAFKA-2045 > Project: Kafka > Issue Type: Sub-task >Reporter: Guozhang Wang > > We need to add the memory management on the new consumer like we did in the > new producer. This would probably include: > 1. byte buffer re-usage for fetch response partition data. > 2. byte buffer re-usage for on-the-fly de-compression. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-2045) Memory Management on the consumer
[ https://issues.apache.org/jira/browse/KAFKA-2045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14385375#comment-14385375 ] Jay Kreps commented on KAFKA-2045: -- The challenge with a rewrite is that it will be hard to know which optimizations actually mattered. It will likely also be just a partial implementation (i.e. just the fetch request to a single broker) so it will be hard to judge what that would look like or what the performance would be if we integrated it in the main client. My recommendation would be to approach this in a data driven way instead. We have a working client, let's look at where it actually spends time and then improve things that use that time. Example measurement: 1. % of time spent in decompression 2. % of time spent on CRC check 3. % of time spent on GC 4. Etc. It would be easy to implement a slightly broken buffer reuse approach that just preallocated a largish number of buffers bigger than the size needed for experimental purposes and reused those. This would only work for the purposes of the perf test but would let us determine the impact of a more complete pooling implementation quickly. > Memory Management on the consumer > - > > Key: KAFKA-2045 > URL: https://issues.apache.org/jira/browse/KAFKA-2045 > Project: Kafka > Issue Type: Sub-task >Reporter: Guozhang Wang > > We need to add the memory management on the new consumer like we did in the > new producer. This would probably include: > 1. byte buffer re-usage for fetch response partition data. > 2. byte buffer re-usage for on-the-fly de-compression. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-2045) Memory Management on the consumer
[ https://issues.apache.org/jira/browse/KAFKA-2045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14384660#comment-14384660 ] Rajiv Kurian commented on KAFKA-2045: - 1. "We can actually make serious performance improvements by improving memory allocation patterns" - Yeah this is definitely the crux of it. Any performance improvements should also look at long term effects like GC activity, longest GC pause etc in addition to just throughput. Even the throughput and latency numbers will have to be looked at for a long time especially in an application where things don't fit in the L1 or L2 caches. I have usually found that with Java most benchmarks (even ones conducted with JMH) lie because of how short in duration they are. Since Java has a Thread Local Allocation Buffer, objects allocated in quick succession get allocated next to each other in memory too. So even though an ArrayList of objects is an array of pointers to objects, the fact that these objects were allocated next to each other means they get 95% (hand wave hand wave) of the benefits of an equivalent std::vector of structs in C++. The nice memory-striding effects of sequential buffers holds even if it is a linked list of Objects again given that the Objects themselves were next to each other. But over time even if a single Object is actually not deleted/shuffled in the ArrayList, a garbage collection is very likely to move them around in memory and when this happens they don't move as an entire unit but separately. Now what began as sequential access degenerates into an array of pointers to randomly laid out objects. And performance of these is an order of magnitude lower than arrays of sequentially laid out structs in C. A ByteBuffer/sun.misc.Unsafe based approach on the other hand never changes memory layout so the benefits continue to hold. This is why in my experience the 99.99th and above percentiles of typical POJO based solutions tanks and is orders of magnitude worse than the 99th etc, whereas solutions based on ByteBuffers and sun.misc.Unsafe have 99.99s that are maybe 4-5 times worse than the 99th. But again there might (will?) be other bottlenecks like the network or CRC that might show up before one can get the max out of such a design. 2. "We don't mangle the code to badly in doing so" - I am planning to write a prototype using my own code from scratch that would include things like on the fly protocol parsing, buffer management and socket management. I'll keep looking at /copy the existing code to ensure that I handle errors correctly. It is just easier to start from fresh - that way I can work solely on getting this to work rather than worrying about how to fit this design in the current class hierarchy. A separate no strings prototype will also probably provide the best platform for a performance demo since I can use things like primitive array based open hash-maps and other non-allocating primitives based data structures for metadata management. It just gives me a lot of options without messing with trunk. If this works out and we see an improvement in performance that seems interesting, we can work on how best to not mangle the code etc. Thoughts? > Memory Management on the consumer > - > > Key: KAFKA-2045 > URL: https://issues.apache.org/jira/browse/KAFKA-2045 > Project: Kafka > Issue Type: Sub-task >Reporter: Guozhang Wang > > We need to add the memory management on the new consumer like we did in the > new producer. This would probably include: > 1. byte buffer re-usage for fetch response partition data. > 2. byte buffer re-usage for on-the-fly de-compression. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-2045) Memory Management on the consumer
[ https://issues.apache.org/jira/browse/KAFKA-2045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14384558#comment-14384558 ] Jay Kreps commented on KAFKA-2045: -- Yeah, as you say, I think bounding memory would still be possible. Once a certain amount of memory was in use you would not begin new socket reads until more memory was available. The issue with this is just that it cuts across many layers so it may be tricky to implement. Anyhow, consider all these approaches. I think the real things we have to establish are: 1. We can actually make serious performance improvements by improving memory allocation patterns 2. We don't mangle the code to badly in doing so > Memory Management on the consumer > - > > Key: KAFKA-2045 > URL: https://issues.apache.org/jira/browse/KAFKA-2045 > Project: Kafka > Issue Type: Sub-task >Reporter: Guozhang Wang > > We need to add the memory management on the new consumer like we did in the > new producer. This would probably include: > 1. byte buffer re-usage for fetch response partition data. > 2. byte buffer re-usage for on-the-fly de-compression. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-2045) Memory Management on the consumer
[ https://issues.apache.org/jira/browse/KAFKA-2045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14384544#comment-14384544 ] Rajiv Kurian commented on KAFKA-2045: - [~jkreps] the simple pool of ByteBuffers definitely sounds like an easier thing to start out with. Like you said a nice thing that a single buffer offers is absolute memory bounds, but I am sure there are other ways to tackle that. I could just have a setting for highest number of concurrent requests which is equal to the highest number of concurrent buffers per broker. We can then create buffers lazily (up to the max) and rotate between them in order. So for 3 buffers we could go 0->1->2->0 etc. The consumer would still have an index into this pool as would the network producer. The network producer will not be able to re-use a response buffer that is still being iterated upon so the consumption of a response cannot be delayed forever without causing poll calls to run out of buffers and just return empty iterators. Your proposed API for ConsumerRecords reuse sounds fine. This gives me enough to work on a prototype, which I hope I can do soon with permission from the bosses. > Memory Management on the consumer > - > > Key: KAFKA-2045 > URL: https://issues.apache.org/jira/browse/KAFKA-2045 > Project: Kafka > Issue Type: Sub-task >Reporter: Guozhang Wang > > We need to add the memory management on the new consumer like we did in the > new producer. This would probably include: > 1. byte buffer re-usage for fetch response partition data. > 2. byte buffer re-usage for on-the-fly de-compression. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-2045) Memory Management on the consumer
[ https://issues.apache.org/jira/browse/KAFKA-2045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14384100#comment-14384100 ] Jay Kreps commented on KAFKA-2045: -- [~rzidane] I do think a prototype would help show whether there was a real perf gain here or not. Given the other cpu expenses we have like CRC checks it's not obvious that memory will be an issue, but you just never know till someone tries. I think one approach that might be simpler would just be to pool the ByteBuffers rather than trying to force it to be the case that there is exactly one. The pool will be super trivial since the requests will all be approximately the same size (so just an ArrayList). We don't need to try to make the pool block or support threads or anything like that. We currently have this api: {code} ConsumerRecords recs = consumer.poll(100); {code} We would add another version of that api to facilitate reuse: {code} ConsumerRecords recs = consumer.poll(100, recs); {code} The second parameter is a ConsumerRecords instance that the client is "recycling". If recs=null this api is the same as the current poll api. If non-null we would grab the underlying ByteBuffers from the ConsumerRecords instance and add it to our pool for reuse. I think this would allow both lazy deserialization (which I suspect on its own is enough to avoid issues with the ConsumerRecords) and reuse at the network level as we read requests. It is true that the memory bound is a little loose since you can have two requests at any time (one being read and one being given out to the consumer), but that is fine. Thoughts? > Memory Management on the consumer > - > > Key: KAFKA-2045 > URL: https://issues.apache.org/jira/browse/KAFKA-2045 > Project: Kafka > Issue Type: Sub-task >Reporter: Guozhang Wang > > We need to add the memory management on the new consumer like we did in the > new producer. This would probably include: > 1. byte buffer re-usage for fetch response partition data. > 2. byte buffer re-usage for on-the-fly de-compression. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-2045) Memory Management on the consumer
[ https://issues.apache.org/jira/browse/KAFKA-2045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14383418#comment-14383418 ] Rajiv Kurian commented on KAFKA-2045: - Copying from the email list and expanding here. My proposal is a single RequestBuffer and a single ResponseBuffer per broker per Consumer. We also need another ByteBuffer to write decompressed message sets (only one message set at a time) to. Another part of the proposal is that when we get a complete response we iterate through the ResponseBuffer and hand out pointers into the buffer to the main low level iterator. The work flow will look a bit like this: i) Re-use the same request buffer to create a request and write to the socket. ii) On poll re-use the same response buffer to read in the request till it is complete. iii) When the response is complete respond with an iterator to the response ByteBuffer. The consumer must now consume the entire ByteBuffer on this thread since we use the a single mutable iterator to go through the ByteBuffer. It is tricker when we consider that during iteration the consumer might send more kafka requests and call poll further. I have a proposal to handle this and still allow requests/responses to be pipelined. I have written something like this for another application and since this is all happening in a single thread it is a bit easier. Here is my proposed design: The response buffer looks a bit like this: |___:___|_} : is the consumer iterator i.e. the position of the next message to be consumed. This is always at the start of a new response, new message set, new message in a message set, end of a response etc. Because iterating on the fly means we will go from one token to the next one. | is the network producer iterator i.e. the position of the next byte from the broker. This can be any arbitrary byte boundary really. } is the end of the buffer. Some details: i) Most of the times the consumer iterator (:) remains behind the network iterator(|). It will catch up when we have consumed all messages. ii) Sometimes we will have fewer bytes than required for a complete response at the end of the buffer. In such a case we will have to wait till we have enough space in the front of the buffer i.e. consumer iterator has moved on enough to create enough space. In such a case we will write some special value at the index where we skipped to the end. This will let the consumer know that it needs to skip ahead to the front of the buffer. This means that every response HAS to be prepended by a special header (can be a single byte) which says if the following bytes are a valid message or not. Say 1 means valid, 0 means invalid. The consumer will only know that there is more to read when the network-producer sequence has gone ahead of the consumer sequence. And it will either read the message right there (if the header says 1) or skip to the beginning of the buffer (if the header says 0). iii) Every time the network producer prepares to write a new response to an index in the buffer it needs to ensure that there is at least 4 bytes (size of message field) + 1 byte for the header + some other minimum amount we can use as a heuristic before it considers the buffer slice usable. If the buffer slice is not usable it has to write the skip ahead header (0) and increment its sequence to point exactly to the end of the buffer. Once the network producer finds enough space in the thread it should wait till at least 4 bytes are read so that it can definitively know the request size. When it reads the size it is certain how many contiguous bytes are required (size of message + 1 byte for header) . Now it can decide with certainty whether it can continue with the slice of the buffer it has (i.e from current pos till end of buffer) or if it has to write the skip ahead header (0) and wait till it gets more contiguous space. If it can continue then it will wait till the entire response is read into the buffer (i,e bytes read == size of response). When this happens, it needs to increment its sequence by size of response + 1 (1 for the header ) and also set the header to 1 to indicate that there is a readable response. iv) A ConsumerRecordIterator is only reset/created once we have an entire contiguous response. Each ConsumerRecordIterator will have a pointer to the beginning of the response and its size. The iterator will hand out ConsumerRecord messages (or reuse them). Each ConsumerRecord also has a pointer to the beginning of the message it is pointing to and a size/pointer to the end. It can also have a mutable reference field for the Topic and an int for the partition. All fields are mutable so that these flyweights can be re-used. v) Once an entire response has been iterated through ( i.e bytes iterated == si
[jira] [Commented] (KAFKA-2045) Memory Management on the consumer
[ https://issues.apache.org/jira/browse/KAFKA-2045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14383344#comment-14383344 ] Jay Kreps commented on KAFKA-2045: -- Hey [~rzidane], one statically allocated ByteBuffer per node is theoretically sufficient sufficient once we have some limit on the response size. It won't be a trivial change though as that reuse will have to go through the NetworkClient and Selector layers so it will require careful design if we attempt it. Currently there are two main memory uses: we allocate the big ByteBuffer allocations we do for the responses, then we parse these into many many ConsumerRecord instances which are stored internally until we can give them out to the user. Not sure which is worse the big chunks or the umpteen little records. I suspect the ConsumerRecord allocation would be addressed by KAFKA-1895 but that would actually complicate ByteBuffer reuse since now we would hand these buffers out to the user. You could potentially implement both but you would need to change consumer.poll to allow passing back in the ConsumerRecords instance for reuse when you are done with it. > Memory Management on the consumer > - > > Key: KAFKA-2045 > URL: https://issues.apache.org/jira/browse/KAFKA-2045 > Project: Kafka > Issue Type: Sub-task >Reporter: Guozhang Wang > > We need to add the memory management on the new consumer like we did in the > new producer. This would probably include: > 1. byte buffer re-usage for fetch response partition data. > 2. byte buffer re-usage for on-the-fly de-compression. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-2045) Memory Management on the consumer
[ https://issues.apache.org/jira/browse/KAFKA-2045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14383147#comment-14383147 ] Rajiv Kurian commented on KAFKA-2045: - Bounding the ByteBuffers and statically allocating them would be great. On consumers do we need any more than a ByteBuffer per broker that the client is talking to? Why do we need a buffer per topic/partition? Even if the leader for a topic/partition changes, we will ultimately know about it and ask the new leader for data. This data will still be after the previous data for the topic/partition that moved so to the consumer it will just look like another message set and order per topic/partition is still maintained. > Memory Management on the consumer > - > > Key: KAFKA-2045 > URL: https://issues.apache.org/jira/browse/KAFKA-2045 > Project: Kafka > Issue Type: Sub-task >Reporter: Guozhang Wang > > We need to add the memory management on the new consumer like we did in the > new producer. This would probably include: > 1. byte buffer re-usage for fetch response partition data. > 2. byte buffer re-usage for on-the-fly de-compression. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-2045) Memory Management on the consumer
[ https://issues.apache.org/jira/browse/KAFKA-2045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14380718#comment-14380718 ] Guozhang Wang commented on KAFKA-2045: -- Agreed, and I remember we did have some discussions regarding (1) and proposed (b). > Memory Management on the consumer > - > > Key: KAFKA-2045 > URL: https://issues.apache.org/jira/browse/KAFKA-2045 > Project: Kafka > Issue Type: Sub-task >Reporter: Guozhang Wang > > We need to add the memory management on the new consumer like we did in the > new producer. This would probably include: > 1. byte buffer re-usage for fetch response partition data. > 2. byte buffer re-usage for on-the-fly de-compression. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-2045) Memory Management on the consumer
[ https://issues.apache.org/jira/browse/KAFKA-2045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14378476#comment-14378476 ] Jay Kreps commented on KAFKA-2045: -- There are really two issues: 1. Bounding fetch size while still guaranteeing that you eventually get data from each partition 2. Pooling and reusing byte buffers I actually think (1) is really pressing, but (2) is just an optimization that may or may not have high payoff. (1) is what leads to the huge memory allocations and sudden OOM when a consumer falls behind and then suddenly has lots of data or when partition assignment changes. For (1) I think we need to figure out whether this is (a) some heuristic in the consumer which decides to only do fetches for a subset of topic/partitions or (b) a new parameter in the fetch request that gives a total bound on the request size. I think we discussed this a while back and agreed on (b), but I can't remember now. The argument if I recall was that that was the only way for the server to monitor all the subscribed topics and avoid blocking on an empty topic while non-empty partitions have data. Bounding the allocations should help performance a lot too. If we do this bounding then I think reuse will be a lot easier to since each response will use at most that many bytes and you could potentially even just statically allocate the byte buffer for each partition and reuse it. > Memory Management on the consumer > - > > Key: KAFKA-2045 > URL: https://issues.apache.org/jira/browse/KAFKA-2045 > Project: Kafka > Issue Type: Sub-task >Reporter: Guozhang Wang > > We need to add the memory management on the new consumer like we did in the > new producer. This would probably include: > 1. byte buffer re-usage for fetch response partition data. > 2. byte buffer re-usage for on-the-fly de-compression. -- This message was sent by Atlassian JIRA (v6.3.4#6332)