[ 
https://issues.apache.org/jira/browse/KAFKA-8154?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17721959#comment-17721959
 ] 

tony mancill commented on KAFKA-8154:
-------------------------------------

Thank you for that observation [~keith.paulson].   We have been seeing this 
issue with BouncyCastle in our environment. We use client versions 2.8.2 and 
3.4.0, but so far I've only tested 2.8.2.  The following patch addresses the 
issue for us with 2.8.2, clearing up both exceptions and the accompanying 
connection resets.
{code:java}
diff --git 
a/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java 
b/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java
index b9879ad6da..effd5fb80d 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java
@@ -344,9 +344,9 @@ public class SslTransportLayer implements TransportLayer {
                     netWriteBuffer.compact();
                     netWriteBuffer = Utils.ensureCapacity(netWriteBuffer, 
currentNetWriteBufferSize);
                     netWriteBuffer.flip();
-                    if (netWriteBuffer.limit() >= currentNetWriteBufferSize) {
+                    if (netWriteBuffer.limit() > currentNetWriteBufferSize) {
                         throw new IllegalStateException("Buffer overflow when 
available data size (" + netWriteBuffer.limit() +
-                                                        ") >= network buffer 
size (" + currentNetWriteBufferSize + ")");
+                                                        ") > network buffer 
size (" + currentNetWriteBufferSize + ")");
                     }
                 } else if (handshakeResult.getStatus() == 
Status.BUFFER_UNDERFLOW) {
                     throw new IllegalStateException("Should not have received 
BUFFER_UNDERFLOW during handshake WRAP.");
@@ -564,6 +564,7 @@ public class SslTransportLayer implements TransportLayer {
                 netReadBuffer.flip();
                 SSLEngineResult unwrapResult;
                 try {
+                    appReadBuffer = Utils.ensureCapacity(appReadBuffer, 
netReadBufferSize());
                     unwrapResult = sslEngine.unwrap(netReadBuffer, 
appReadBuffer);
                     if (state == State.POST_HANDSHAKE && 
appReadBuffer.position() != 0) {
                         // For TLSv1.3, we have finished processing 
post-handshake messages since we are now processing data
@@ -598,7 +599,7 @@ public class SslTransportLayer implements TransportLayer {
                                                         ") >= application 
buffer size (" + currentApplicationBufferSize + ")");
                     }
-                    // appReadBuffer will extended upto 
currentApplicationBufferSize
+                    // appReadBuffer will extend up to 
currentApplicationBufferSize
                     // we need to read the existing content into dst before we 
can do unwrap again. If there are no space in dst
                     // we can break here.
                     if (dst.hasRemaining()) {code}
 

A couple of notes, which is why I haven't submitted a PR yet - that is, I don't 
think the patch is complete yet:
 * First, I'm not sure whether the change in the hunk starting at line 564 is 
needed.  It was in my working tree while I tried to address this previously, so 
I need to test again without it.
 * There are other places in SslTransportLayer.java where 
Utils.ensureCapacity() is called and the following comparison uses >= instead 
of strictly >.  It would be nice to address them all at once.

> 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
(v8.20.10#820010)

Reply via email to