[ https://issues.apache.org/jira/browse/KAFKA-8154?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16804777#comment-16804777 ]
Rajini Sivaram commented on KAFKA-8154: --------------------------------------- The assumption we make is that an incoming packet with the SSL session's packet buffer size can be unwrapped into an application buffer with the SSL session's application buffer size and the application buffer size is bigger than the unwrapped data (hence the >= check). We guarantee that we don't allocate a buffer that is larger than the total of SSL session's buffer sizes per-connection since this is typically used to calculate the total memory required for brokers based on maximum connection count and can lead to OOM if we arbitrarily increased the value. So I dont think we can apply the fix in the PR [https://github.com/apache/kafka/pull/5785]. With the standard SSL session packet size in normal JDKs, in order to hit the exception in PR [https://github.com/apache/kafka/pull/6495,] application buffer size needs to be 16384 (lower than the value typically used). But if we set application buffer size to 16384 and use the normal network read buffer size, for example with an Oracle 1.8 JDK, instead of throwing IllegalStateException, we would end up with a infinite loop with large packets that cannot be unwrapped with an application buffer size of 16384. I think we need to better understand the issue here. It will be good to know if the issues encountered in both the failing scenarios (in the two PRs) were the same. A stack trace from the first one will be helpful for this. It will also be useful to get buffer sizes (netReadBufferSize(), netWriteBufferSize() and applicationBufferSize()) for the two environments. At the moment, we just have the one number 16384 as application buffer size and we dont expect that to work. Obviously if we can recreate with an unit test, that would be good, but it may be easier to write an unit test once we can identify the exact issue that we need to fix. > Buffer Overflow exceptions between brokers and with clients > ----------------------------------------------------------- > > Key: KAFKA-8154 > URL: https://issues.apache.org/jira/browse/KAFKA-8154 > Project: Kafka > Issue Type: Bug > Components: clients > Affects Versions: 2.1.0 > Reporter: Rajesh Nataraja > Priority: Major > Attachments: server.properties.txt > > > https://github.com/apache/kafka/pull/6495 > https://github.com/apache/kafka/pull/5785 -- This message was sent by Atlassian JIRA (v7.6.3#76005)