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

Reply via email to