[ 
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:#000000} 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:#000000}java.lang.IllegalStateException: Buffer underflow when available 
data size (18437) > packet buffer size (18437){color}
{color:#000000} at 
org.apache.kafka.common.network.SslTransportLayer.read(SslTransportLayer.java:565){color}
{color:#000000} at 
org.apache.kafka.common.network.NetworkReceive.readFrom(NetworkReceive.java:117){color}
{color:#000000} at 
org.apache.kafka.common.network.KafkaChannel.receive(KafkaChannel.java:381){color}
{color:#000000} at 
org.apache.kafka.common.network.KafkaChannel.read(KafkaChannel.java:342){color}
{color:#000000} at 
org.apache.kafka.common.network.Selector.attemptRead(Selector.java:609){color}
{color:#000000} at 
org.apache.kafka.common.network.Selector.pollSelectionKeys(Selector.java:541){color}
{color:#000000} at 
org.apache.kafka.common.network.Selector.poll(Selector.java:467){color}
{color:#000000} at 
org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:535){color}
{color:#000000} at 
org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:265){color}
{color:#000000} at 
org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:236){color}
{color:#000000} at 
org.apache.kafka.clients.consumer.KafkaConsumer.pollForFetches(KafkaConsumer.java:1243){color}
{color:#000000} at 
org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1188){color}
{color:#000000} at 
org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1164){color}
{color:#000000} at 
org.apache.kafka.streams.processor.internals.StreamThread.pollRequests(StreamThread.java:913){color}
{color:#000000} at 
org.apache.kafka.streams.processor.internals.StreamThread.runOnce(StreamThread.java:822){color}
{color:#000000} at 
org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:777){color}
{color:#000000} 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)

Reply via email to