[jira] [Comment Edited] (KAFKA-2045) Memory Management on the consumer
[ https://issues.apache.org/jira/browse/KAFKA-2045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15874896#comment-15874896 ] Armin Braun edited comment on KAFKA-2045 at 2/20/17 6:10 PM: - added a suggestion on how to concretely tackly this and KAFKA-1895 here -> https://issues.apache.org/jira/browse/KAFKA-1895?focusedCommentId=15874894=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15874894 was (Author: original-brownbear): add a suggestion on how to concretely tackly this and KAFKA-1895 here -> https://issues.apache.org/jira/browse/KAFKA-1895?focusedCommentId=15874894=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] [Comment Edited] (KAFKA-2045) Memory Management on the consumer
[ https://issues.apache.org/jira/browse/KAFKA-2045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14385500#comment-14385500 ] Rajiv Kurian edited comment on KAFKA-2045 at 3/28/15 7:40 PM: -- [~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 consumer. 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? was (Author: rzidane): [~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
[jira] [Comment Edited] (KAFKA-2045) Memory Management on the consumer
[ https://issues.apache.org/jira/browse/KAFKA-2045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14385500#comment-14385500 ] Rajiv Kurian edited comment on KAFKA-2045 at 3/28/15 7:40 PM: -- [~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? was (Author: rzidane): [~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
[jira] [Comment Edited] (KAFKA-2045) Memory Management on the consumer
[ https://issues.apache.org/jira/browse/KAFKA-2045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383418#comment-14383418 ] Rajiv Kurian edited comment on KAFKA-2045 at 3/27/15 6:51 AM: -- 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
[jira] [Comment Edited] (KAFKA-2045) Memory Management on the consumer
[ https://issues.apache.org/jira/browse/KAFKA-2045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383418#comment-14383418 ] Rajiv Kurian edited comment on KAFKA-2045 at 3/27/15 6:53 AM: -- 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
[jira] [Comment Edited] (KAFKA-2045) Memory Management on the consumer
[ https://issues.apache.org/jira/browse/KAFKA-2045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383418#comment-14383418 ] Rajiv Kurian edited comment on KAFKA-2045 at 3/27/15 6:53 AM: -- 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
[jira] [Comment Edited] (KAFKA-2045) Memory Management on the consumer
[ https://issues.apache.org/jira/browse/KAFKA-2045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14383418#comment-14383418 ] Rajiv Kurian edited comment on KAFKA-2045 at 3/27/15 6:54 AM: -- 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
[jira] [Comment Edited] (KAFKA-2045) Memory Management on the consumer
[ https://issues.apache.org/jira/browse/KAFKA-2045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14384660#comment-14384660 ] Rajiv Kurian edited comment on KAFKA-2045 at 3/27/15 9:07 PM: -- 1. We can actually make serious performance improvements by improving memory allocation patterns - Yeah this is definitely the crux of it. rantAny 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/rant over. 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 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. I can also use char sequences instead of Java's allocating strings for topics and such just to see how much of a difference they make. 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 and/or decide which parts are worth mangling for the extra performance. Thoughts? was (Author: rzidane): 1. We can actually make serious performance improvements by improving memory allocation patterns - Yeah this is definitely the crux of it. rantAny 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