[jira] [Commented] (KAFKA-8154) Buffer Overflow exceptions between brokers and with clients

2024-03-19 Thread keith.paulson (Jira)


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

keith.paulson commented on KAFKA-8154:
--

[~tmancill] What is involved with getting this fix implemented?  We've been 
running it since last June on releases up to 3.6 without issue.

> 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)


[jira] [Commented] (KAFKA-8154) Buffer Overflow exceptions between brokers and with clients

2023-06-09 Thread tony mancill (Jira)


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

tony mancill commented on KAFKA-8154:
-

Thank you for the testing and the feedback [~keith.paulson] .  We are now 
starting testing with a more minimal patch:

*2.8.2*

{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..19c8a73eae 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.");
@@ -598,7 +598,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}



*3.4.0*

{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 904c5216a4..8c1abfdf9d 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
@@ -349,9 +349,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.");
@@ -608,7 +608,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}


> 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

[jira] [Commented] (KAFKA-8154) Buffer Overflow exceptions between brokers and with clients

2023-05-24 Thread keith.paulson (Jira)


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

keith.paulson commented on KAFKA-8154:
--

[~tmancill] I used almost the same patch as you did, even the hunk at 564 you 
mention in your first point, though I also adjusted the comparison for the 
application buffer size check right after 564

The change at 564 caused problems; I started getting 
{code:java}
Buffer overflow when available data size (16504) > application buffer size 
(16384)  {code}
Not a +1 issue but much higher.

The conditional involved is 
{code:java}
(appReadBuffer.position() >= currentApplicationBufferSize) {code}
And appReadBuffer is initialized to currentApplicationBufferSize, so this would 
never fire - except for the the 564 ensureCapacity call which will increase it 
to match the netWriteBuffer size.

BC sets app buffer to 1<<14, and net buffer to that plus an offset that varies 
by protocol, but netWriteBuffer will always be bigger than app buffer, and 
trigger that exception in cases of large data blocks.

With that line removed, my half java ssl/half BC-fips cluster has run solid for 
hours.

tl;dr: the 564 hunk needs to be dropped.

> 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)


[jira] [Commented] (KAFKA-8154) Buffer Overflow exceptions between brokers and with clients

2023-05-11 Thread tony mancill (Jira)


[ 
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)


[jira] [Commented] (KAFKA-8154) Buffer Overflow exceptions between brokers and with clients

2023-04-19 Thread keith.paulson (Jira)


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

keith.paulson commented on KAFKA-8154:
--

I'm getting this error now, kafka 3.4 with BouncyCastle FIPS.

The BC libraries uses a fixed 16k for buffer size, as opposed to java ssl, 
which uses a calculation to determine it.

The part that confuses me is that we use Utils.ensureCapacity to adjust 
buffers, and this uses the conditional 
{code:java}
if (newLength > existingBuffer.capacity()) {code}
But the overflow check is
{code:java}
netWriteBuffer.limit() >= currentNetWriteBufferSize {code}
The case where capacity is 16k, and newLength is the same, buffer size won't be 
changed, but then the overflow condition would be immediately hit.

Should the second test be changed to '>' ? 
There should be no terminating character, so x bytes can be written to a buffer 
of size x – or is there a reason we want an extra byte size?

 

> 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)


[jira] [Commented] (KAFKA-8154) Buffer Overflow exceptions between brokers and with clients

2022-11-07 Thread tony mancill (Jira)


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

tony mancill commented on KAFKA-8154:
-

We have observed this this behavior when using either Conscrypt or BouncyCastle 
as SSL providers.  For Conscrypt, we patched around this by changing the 
implementation of getApplicationBufferSize() from:
{code:java}
@Override
public int getApplicationBufferSize() {
return NativeConstants.SSL3_RT_MAX_PLAIN_LENGTH;
}{code}
to
{code:java}
@Override
public int getApplicationBufferSize() {
return NativeConstants.SSL3_RT_MAX_PLAIN_LENGTH - 
NativeConstants.SSL3_RT_HEADER_LENGTH;
} {code}
 

However we don't have a work-around for BouncyCastle, and the simple 1-line 
patch from [~vineethgn] to call ensureCapacity() seems far preferable.  It also 
matches the code used [during the 
handshake|https://github.com/apache/kafka/blob/2.8.1/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java#L371]
 in SslTransportLayer, which was patched as part of KAFKA-5920 in this commit:  
[https://github.com/apache/kafka/commit/d60f011d77ce80a44b02d43bf0889a50a8797dcd]
 

What would help get some eyes on this issue?  Should we submit a PR?

> 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)


[jira] [Commented] (KAFKA-8154) Buffer Overflow exceptions between brokers and with clients

2022-05-31 Thread Vineeth (Jira)


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

Vineeth commented on KAFKA-8154:


I tried patch [9188|https://github.com/apache/kafka/pull/9188]. it didnt solv 
the issue. I tried to reproduce the issue and i can reproduce the issue 
consistently in the following conditions.

1. [Conscrypt|https://github.com/google/conscrypt] as ssl provider 
2. openjdk-11-jdk

I checked by adding additional logs and could find that the packet buffer size 
and app buffer returned by Conscrypt ssl engine mismatches. Below are the logs 
that i got
{code:java}
SSl engine class org.conscrypt.Java8EngineWrapper 
(org.apache.kafka.common.network.SslTransportLayer)
SSl engine session class org.conscrypt.Java8ExtendedSSLSession 
(org.apache.kafka.common.network.SslTransportLayer)
SSl engine session App Buffer Size 16384 
(org.apache.kafka.common.network.SslTransportLayer)
SSl engine session Net Buffer Size 16709 
(org.apache.kafka.common.network.SslTransportLayer) {code}
Since this was a blocker, i modified  the Ssltransportlayer to have 
appreadbuffer atleast the same size as netreadbuffer, before call to unwrap. Is 
it ok to have appReadBuffer atleast the same size as netReadBuffer ?

so far  the clusters are running smoothly without buffer overflow or underflow 
issue. 
{code:java}
while (netReadBuffer.position() > 0) {
             netReadBuffer.flip();
             SSLEngineResult unwrapResult;
             try {
                 appReadBuffer = Utils.ensureCapacity(appReadBuffer, 
netReadBufferSize());
                 unwrapResult = sslEngine.unwrap(netReadBuffer, appReadBuffer); 
{code}

> 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.7#820007)


[jira] [Commented] (KAFKA-8154) Buffer Overflow exceptions between brokers and with clients

2021-08-26 Thread Binh Van Nguyen (Jira)


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

Binh Van Nguyen commented on KAFKA-8154:


[~gordonmessmer] I am using Kafka client version 1.1.1. I modified the code 
based on Alena's patch like this
{code}
@@ -539,9 +539,8 @@ public class SslTransportLayer implements TransportLayer {
 // appReadBuffer will extended upto 
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())
-read += readFromAppBuffer(dst);
-else
+read += readFromAppBuffer(dst);
+if (!dst.hasRemaining())
 break;
 } else if (unwrapResult.getStatus() == 
Status.BUFFER_UNDERFLOW) {
 int currentNetReadBufferSize = netReadBufferSize();

{code}

but it doesn't fix the issue. The exception that I got is this

{code}
21/08/27 06:06:25 WARN network.Selector: [Consumer clientId=consumer-1, 
groupId=kafka1] Unexpected error from ***; closing connection
java.lang.IllegalStateException: Buffer overflow when available data size 
(16384) >= application buffer size (16384)
at 
org.apache.kafka.common.network.SslTransportLayer.read(SslTransportLayer.java:535)
at 
org.apache.kafka.common.network.NetworkReceive.readFromReadableChannel(NetworkReceive.java:122)
at 
org.apache.kafka.common.network.NetworkReceive.readFrom(NetworkReceive.java:93)
at 
org.apache.kafka.common.network.KafkaChannel.receive(KafkaChannel.java:235)
at 
org.apache.kafka.common.network.KafkaChannel.read(KafkaChannel.java:196)
at 
org.apache.kafka.common.network.Selector.attemptRead(Selector.java:559)
at 
org.apache.kafka.common.network.Selector.pollSelectionKeys(Selector.java:495)
at org.apache.kafka.common.network.Selector.poll(Selector.java:424)
at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:460)
at 
org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:261)
at 
org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:233)
at 
org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:209)
at 
org.apache.kafka.clients.consumer.internals.Fetcher.getTopicMetadata(Fetcher.java:271)
at 
org.apache.kafka.clients.consumer.internals.Fetcher.getAllTopicMetadata(Fetcher.java:251)
at 
org.apache.kafka.clients.consumer.KafkaConsumer.listTopics(KafkaConsumer.java:1526)
at 
org.apache.gobblin.kafka.client.Kafka1ConsumerClient.getTopics(Kafka1ConsumerClient.java:119)
at 
org.apache.gobblin.kafka.client.AbstractBaseKafkaConsumerClient.getFilteredTopics(AbstractBaseKafkaConsumerClient.java:97)
at 
org.apache.gobblin.source.extractor.extract.kafka.KafkaSource.getFilteredTopics(KafkaSource.java:798)
at 
org.apache.gobblin.source.extractor.extract.kafka.KafkaSource.getWorkunits(KafkaSource.java:220)
at 
org.apache.gobblin.runtime.SourceDecorator.getWorkunitStream(SourceDecorator.java:81)
at 
org.apache.gobblin.runtime.AbstractJobLauncher.launchJob(AbstractJobLauncher.java:426)
at 
org.apache.gobblin.runtime.mapreduce.CliMRJobLauncher.launchJob(CliMRJobLauncher.java:93)
at 
org.apache.gobblin.runtime.mapreduce.CliMRJobLauncher.run(CliMRJobLauncher.java:70)
at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76)
at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:90)
{code}


> 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.3.4#803005)


[jira] [Commented] (KAFKA-8154) Buffer Overflow exceptions between brokers and with clients

2021-08-26 Thread Gordon (Jira)


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

Gordon commented on KAFKA-8154:
---

What version of the client library are you using in your application?  Would 
you be willing to restore the default dataproc configuration and test a Alena's 
patch to see if that resolves the issue?

> 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.3.4#803005)


[jira] [Commented] (KAFKA-8154) Buffer Overflow exceptions between brokers and with clients

2021-08-26 Thread Binh Van Nguyen (Jira)


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

Binh Van Nguyen commented on KAFKA-8154:


I also faced this issue. In may case, the issue happens when I have 32 or more 
topics in the cluster. I run my app on Google Cloud Dataproc and set the 
property as [~oripwk] mentioned by pass the issue

> 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.3.4#803005)


[jira] [Commented] (KAFKA-8154) Buffer Overflow exceptions between brokers and with clients

2021-03-14 Thread Gordon (Jira)


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

Gordon commented on KAFKA-8154:
---

[~oripwk] Good to know, but it would be better if the bug in Kafka were fixed 
rather than forcing users to work around it forever.  Alena's confirmed that 
her patch fixed the problem in her environment.  I think additional 
confirmations might help push the developers to merge that patch.

> 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.3.4#803005)


[jira] [Commented] (KAFKA-8154) Buffer Overflow exceptions between brokers and with clients

2021-03-08 Thread Ori Popowski (Jira)


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

Ori Popowski commented on KAFKA-8154:
-

For anyone running on Google Cloud Dataproc, setting 
{{dataproc.conscrypt.provider.enable=false}} solved the issue.

> 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.3.4#803005)


[jira] [Commented] (KAFKA-8154) Buffer Overflow exceptions between brokers and with clients

2021-03-02 Thread Ori Popowski (Jira)


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

Ori Popowski commented on KAFKA-8154:
-

Thank you

> 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.3.4#803005)


[jira] [Commented] (KAFKA-8154) Buffer Overflow exceptions between brokers and with clients

2021-03-01 Thread Gordon (Jira)


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

Gordon commented on KAFKA-8154:
---

Rushed, and grabbed the wrong file. Sorry about that.  Added the kafka-clients 
jar, in case that saves you any effort.

> 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.3.4#803005)


[jira] [Commented] (KAFKA-8154) Buffer Overflow exceptions between brokers and with clients

2021-03-01 Thread Ori Popowski (Jira)


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

Ori Popowski commented on KAFKA-8154:
-

[~gordonmessmer] thank you very much for publishing this JAR

> 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.3.4#803005)


[jira] [Commented] (KAFKA-8154) Buffer Overflow exceptions between brokers and with clients

2021-03-01 Thread Gordon (Jira)


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

Gordon commented on KAFKA-8154:
---

Yes, that's the right link.  I've built kafka 2.6.1 with PR 9188 applied:

https://github.com/gordonmessmer/kafkajar

> 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.3.4#803005)


[jira] [Commented] (KAFKA-8154) Buffer Overflow exceptions between brokers and with clients

2021-03-01 Thread Ori Popowski (Jira)


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

Ori Popowski commented on KAFKA-8154:
-

[~gordonmessmer] I don't see any Alena or a patch in your link. Is this the 
correct link?

Additionally, what do you mean by patch? Should I patch my broker/client code 
and build it myself? Because I'm running Confluent

> 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.3.4#803005)


[jira] [Commented] (KAFKA-8154) Buffer Overflow exceptions between brokers and with clients

2021-02-28 Thread Gordon (Jira)


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

Gordon commented on KAFKA-8154:
---

[~oripwk] I recommend applying [Alena's 
patch|https://github.com/apache/kafka/pull/9188].

> 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.3.4#803005)


[jira] [Commented] (KAFKA-8154) Buffer Overflow exceptions between brokers and with clients

2021-02-28 Thread Ori Popowski (Jira)


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

Ori Popowski commented on KAFKA-8154:
-

Hi,

We're hitting this exact issue. Our logs are flooded with these errors and we 
have stability issues with our application.

Is there any workaround we can apply to resolve this?

Thanks

> 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.3.4#803005)


[jira] [Commented] (KAFKA-8154) Buffer Overflow exceptions between brokers and with clients

2020-08-16 Thread Alena Messmer (Jira)


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

Alena Messmer commented on KAFKA-8154:
--

[~pwebb.itrs] [~rnataraja] I've opened a [pull request on 
GitHub|[https://github.com/apache/kafka/pull/9188]]. I no longer see the 
problem after applying this change. Could you apply the change and see if it 
resolves the problem for you as well?

> 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.3.4#803005)


[jira] [Commented] (KAFKA-8154) Buffer Overflow exceptions between brokers and with clients

2020-07-23 Thread Paul Webb (Jira)


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

Paul Webb commented on KAFKA-8154:
--

Hi.  Please note that we were recently affected by this issue.  I tried several 
kafka clients/ brokers and JDKs. The actual cause in the end was that the java 
security providers was being changed.  Specifically:
{code:java}
 Security.insertProviderAt(providerToAdd, 1);
{code}
By adding at position 1, it appeared that the new provider was returning an 
invalid application buffer size. This may explain why this could be difficult 
to reproduce.  When the new provider is added to the back of the list Kafka 
behaved fine. 

Hope this helps. 

 

> 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.3.4#803005)


[jira] [Commented] (KAFKA-8154) Buffer Overflow exceptions between brokers and with clients

2019-04-05 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on KAFKA-8154:
---

Spatterjaaay commented on pull request #6542: attempt to fix KAFKA-8154 buffer 
overflow exceptions
URL: https://github.com/apache/kafka/pull/6542
 
 
   https://issues.apache.org/jira/browse/KAFKA-8154
   
   Replace the inner while loop with an if statement. The purpose of the loop 
if to retry the unwrap after resizing of emptying the AppReadBuffer, however, 
neither of those operations are currently guaranteed. 
   
   The modified jar was used in an environment that reliably reproduced the 
problem when using an unmodified kafka library. The modified library did not 
manifest the problem. 
   
 

This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> 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)


[jira] [Commented] (KAFKA-8154) Buffer Overflow exceptions between brokers and with clients

2019-04-05 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on KAFKA-8154:
---

Spatterjaaay commented on pull request #6541: attempt to fix KAFKA-8154 buffer 
overflow exceptions
URL: https://github.com/apache/kafka/pull/6541
 
 
   https://issues.apache.org/jira/browse/KAFKA-8154
   
   Break the inner while loop after a successful unwrap operation. This allows 
the function to return if dst is full. 
   
   The modified jar was used in an environment that reliably reproduced the 
problem when using an unmodified kafka library. The modified library did not 
manifest the problem. 
   
 

This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> 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)


[jira] [Commented] (KAFKA-8154) Buffer Overflow exceptions between brokers and with clients

2019-04-04 Thread Gordon (JIRA)


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

Gordon commented on KAFKA-8154:
---

Thank you.

That being the case, the BUFFER_UNDERFLOW error that Rajesh posted has a 
different underlying cause.  In that case, one unwrap operation results in 
BUFFER_OVERFLOW, but due to their change, the error isn't handled.  The outer 
loop will terminate because dst is full.  On the next call to read(), the 
unwrap operation will have left data in the network buffer because the 
application buffer was full previously, and there will no longer a full buffer 
worth of space for the readFromSocketChannel operation.  That's when the code 
will raise the BUFFER_UNDERFLOW error.

This condition has nothing to do with buffer size mismatches.

> 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)


[jira] [Commented] (KAFKA-8154) Buffer Overflow exceptions between brokers and with clients

2019-04-04 Thread Rajesh Nataraja (JIRA)


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

Rajesh Nataraja commented on KAFKA-8154:


[~rsivaram] BUFFER_UNDERFLOW was never noticed if we did not make the change 
for BUFFER_OVERFLOW. If we follow the documentation posted by oracle and handle 
buffer_overflow, it creates an underflow. If used the change as suggested by PR 
6495 and only changed the overflow portion instead. Even that created a buffer 
underflow scenario. 

 

> 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)


[jira] [Commented] (KAFKA-8154) Buffer Overflow exceptions between brokers and with clients

2019-04-04 Thread Gordon (JIRA)


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

Gordon commented on KAFKA-8154:
---

[~rnataraja] can you clarify whether the BUFFER_UNDERFLOW you posted occurred 
using an unmodified kafka library, or while you were testing your own changes?

> 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)


[jira] [Commented] (KAFKA-8154) Buffer Overflow exceptions between brokers and with clients

2019-04-04 Thread Rajini Sivaram (JIRA)


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

Rajini Sivaram commented on KAFKA-8154:
---

[~rnataraja] [~gordonmessmer] From the buffer sizes that [~rnataraja] provided, 
read buffer size is {color:#00}18437{color} and application buffer size is 
16384. This looks like the SSLEngine is using buffer sizes that don't match, 
i.e. application buffer size is too small to unwrap a single packet buffer. As 
mentioned earlier, the buffer sizes I looked at (Oracle JDK jdk1.8.0_191 and 
the Kafka system test docker image that uses OpenJDK 1.8.0_181) both have 
smaller packet buffer size and large application buffer size. The SSL 
implementation in Kafka currently does not support mismatched buffer sizes and 
that looks more like an issue with the SSLEngine. It will be good to know if 
the issue [~gordonmessmer] ran into was with the same buffer sizes and the same 
SSL provider.

I looked at several JDKs and couldn't find one that uses buffer sizes that 
don't match. I couldn't find a download for the exact one that [~rnataraja] 
used. If one of you can write a test that recreates the scenario, that would be 
useful before implementing a fix. Otherwise if you do know of an easily 
downloadable JDK with the issue, I can help with writing the test. I would be 
nervous about merging a fix that cannot be tested.

 

 

> 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)


[jira] [Commented] (KAFKA-8154) Buffer Overflow exceptions between brokers and with clients

2019-03-30 Thread Gordon (JIRA)


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

Gordon commented on KAFKA-8154:
---

> We guarantee that we don't allocate a buffer that is larger than the total of 
> SSL session's buffer sizes

That's the root of the problem. getApplicationBufferSize() indicates "buffers 
must be large enough to hold the application data from any inbound network 
application data packet received".  Allocating a buffer of this size is 
acceptable as long as it is empty when unwrap() is called.  The design of the 
read() function, does not necessarily empty the applicationReadBuffer between 
calls to unwrap().  There is a nested loop that calls unwrap() repeatedly while 
netReadBuffer has data.  It also copies data from applicationReadBuffer to dst, 
but if dst is full after one iteration of the inner loop, then 
applicationReadBuffer won't be completely empty and a second call to unwrap() 
may fail because there isn't enough room left in applicationReadBuffer.

There are at least three ways to solve this problem that are immediately 
apparent:

1: Eliminate the nested loop.  I'm pretty sure if you change the inner loop 
from "while (netReadBuffer.position() > 0)" to "if (netReadBuffer.position() > 
0)", then the remaining loop will not continue to iterate when dst is full, 
which means that you won't have a loop iteration where applicationReadBuffer 
still has data in it.  The outer loop already checks the netReadBuffer to see 
if it has room, and does not attempt a network read if it is full, so this 
should be safe.

2: Break the inner loop in the Status.OK case.  This is ugly, but this is 
already being done in the Status.BUFFER_OVERFLOW case.  That section ends with 
a "break" if dst.hasRemaining() is false.  (However, the code in the 
BUFFER_OVERFLOW case may also be somewhat broken in that it first tests to see 
if there is free space and then copies data into the buffer.  It should be 
checking hasRemaining() after copying data, since the readFromAppBuffer(dst) 
call might fill dst.)  In the OK case, if the code checks dst.hasRemaining() 
after the current readFromAppBuffer(dst), it could "break" and avoid the 
problem.

3: Resize the applicationReadBuffer on BUFFER_OVERFLOW.  The fix I proposed in 
PR 5785 is not an "arbitrary" resize of the applicationReadBuffer.  It merely 
ensures that when data is left over in applicationReadBuffer after moving data 
to "dst", and when the inner loop iterates a second time, the buffer will be 
resized so that there is at least getApplicationBufferSize() bytes free in the 
buffer.  This is the recommended behavior if the application read buffer can't 
be emptied and unwrap() retried:

https://docs.oracle.com/javase/8/docs/api/javax/net/ssl/SSLEngine.html#unwrap(java.nio.ByteBuffer,%20java.nio.ByteBuffer)

> 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)


[jira] [Commented] (KAFKA-8154) Buffer Overflow exceptions between brokers and with clients

2019-03-29 Thread Rajesh Nataraja (JIRA)


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

Rajesh Nataraja commented on KAFKA-8154:


[~rsivaram] The infinite loop is exactly what we noticed, when trying to 
implement as per oracle documentation. I tried some ways to avoid infinite 
loop, but all of them end up causing IO exception, because the other side 
closes the connection. Probably because client is unable to complete the read 
operation. But in most of the cases of buffer overflow trying to  increase  the 
application buffer results in buffer underflow. Here are the underflow outputs 
where you can see netread buffersize.

 

{color:#00} WARN org.apache.kafka.common.network.Selector - [Consumer 
clientId=xxx-a60c2c61-282d-4866-a3d1-f88579b44de7-StreamThread-2-consumer, 
groupId=1 ... ] Unexpected error from /[10.10.10.20|http://10.10.10.20/]; 
closing connection{color}
{color:#00}java.lang.IllegalStateException: Buffer underflow when available 
data size (18437) > packet buffer size (18437){color}
{color:#00} at 
org.apache.kafka.common.network.SslTransportLayer.read(SslTransportLayer.java:565){color}
{color:#00} at 
org.apache.kafka.common.network.NetworkReceive.readFrom(NetworkReceive.java:117){color}
{color:#00} at 
org.apache.kafka.common.network.KafkaChannel.receive(KafkaChannel.java:381){color}
{color:#00} at 
org.apache.kafka.common.network.KafkaChannel.read(KafkaChannel.java:342){color}
{color:#00} at 
org.apache.kafka.common.network.Selector.attemptRead(Selector.java:609){color}
{color:#00} at 
org.apache.kafka.common.network.Selector.pollSelectionKeys(Selector.java:541){color}
{color:#00} at 
org.apache.kafka.common.network.Selector.poll(Selector.java:467){color}
{color:#00} at 
org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:535){color}
{color:#00} at 
org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:265){color}
{color:#00} at 
org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:236){color}
{color:#00} at 
org.apache.kafka.clients.consumer.KafkaConsumer.pollForFetches(KafkaConsumer.java:1243){color}
{color:#00} at 
org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1188){color}
{color:#00} at 
org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1164){color}
{color:#00} at 
org.apache.kafka.streams.processor.internals.StreamThread.pollRequests(StreamThread.java:913){color}
{color:#00} at 
org.apache.kafka.streams.processor.internals.StreamThread.runOnce(StreamThread.java:822){color}
{color:#00} at 
org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:777){color}
{color:#00} at 
org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:747){color}

> 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)


[jira] [Commented] (KAFKA-8154) Buffer Overflow exceptions between brokers and with clients

2019-03-29 Thread Rajini Sivaram (JIRA)


[ 
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)


[jira] [Commented] (KAFKA-8154) Buffer Overflow exceptions between brokers and with clients

2019-03-27 Thread Rajesh Nataraja (JIRA)


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

Rajesh Nataraja commented on KAFKA-8154:


[~rsivaram] I will get the outputs soon. But our Application buffer size is 
16384, as you can see here (also in the other pull request)

Unexpected error from /192.168.2.23; closing connection 
(org.apache.kafka.common.network.Selector)
 java.lang.IllegalStateException: Buffer overflow when available data size 
(16384) >= application buffer size (16384)

 

Thanks

Rajesh.

> 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)


[jira] [Commented] (KAFKA-8154) Buffer Overflow exceptions between brokers and with clients

2019-03-27 Thread Rajini Sivaram (JIRA)


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

Rajini Sivaram commented on KAFKA-8154:
---

[~rnataraja] We don't seem to log SSL buffer sizes in SslTransportLayer. Can 
you print out netReadBufferSize(), netWriteBufferSize() and 
applicationBufferSize() in your environment? You could just add a log entry and 
run some tool like console consumer to get the data. Thank you.

> 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)


[jira] [Commented] (KAFKA-8154) Buffer Overflow exceptions between brokers and with clients

2019-03-26 Thread Rajesh Nataraja (JIRA)


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

Rajesh Nataraja commented on KAFKA-8154:


I could show this to you on my setup. I wil ping you on slack. In our case 
Application buffer size is 16384 (16K). 16916 is a bit off. We can then update 
on Jira.

Thanks

> 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)


[jira] [Commented] (KAFKA-8154) Buffer Overflow exceptions between brokers and with clients

2019-03-26 Thread Rajini Sivaram (JIRA)


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

Rajini Sivaram commented on KAFKA-8154:
---

I was trying to recreate this yesterday, but was unable to. Even though it 
looks possible to hit this scenario from the code and we should avoid it, with 
the default buffer sizes, I am not sure how we would get there. Do you by any 
chance have a way to easily recreate this scenario without an extensive set up? 
We need a test to merge a fix, but that could be a just a unit test. I am 
basically trying to understand how your deployment was hitting this error 
frequently, in particular with the Java->Java case with replica fetcher. I was 
trying to recreate on my laptop using Oracle JDK version jdk1.8.0_191. I also 
looked at buffer sizes on the docker container which we use for system tests 
and that uses OpenJDK v8. In both cases, SSL application buffer size is 16916. 
But your exception shows a smaller buffer size of 16384. It will be good to 
understand how it got there and whether we have other issues around SSL buffer 
resize. Thanks.

 

 

 

 

> 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)