[ https://issues.apache.org/jira/browse/KAFKA-2672?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14964556#comment-14964556 ]
Ismael Juma commented on KAFKA-2672: ------------------------------------ `poll` in `ConsumerNetworkClient`: {code} private void poll(long timeout, long now) { // send all the requests we can send now pollUnsentRequests(now); // ensure we don't poll any longer than the deadline for // the next scheduled task timeout = Math.min(timeout, delayedTasks.nextTimeout(now)); clientPoll(timeout, now); // execute scheduled tasks now = time.milliseconds(); delayedTasks.poll(now); // try again to send requests since buffer space may have been // cleared or a connect finished in the poll pollUnsentRequests(now); // fail all requests that couldn't be sent clearUnsentRequests(now); } {code} `clearUnsentRequests` raises `SendFailedException` for all unsent requests, which can happen if the handshake is still happening. [~hachikuji], thoughts? > SendFailedException when new consumer is run with SSL > ----------------------------------------------------- > > Key: KAFKA-2672 > URL: https://issues.apache.org/jira/browse/KAFKA-2672 > Project: Kafka > Issue Type: Bug > Components: consumer > Reporter: Rajini Sivaram > Assignee: Neha Narkhede > Fix For: 0.9.0.0 > > > When running new consumer with SSL, debug logs show these exceptions every > time: > {quote} > [2015-10-19 20:57:43,389] DEBUG Fetch failed > (org.apache.kafka.clients.consumer.internals.Fetcher) > org.apache.kafka.clients.consumer.internals.SendFailedException > {quote} > The exception occurs because send is queued before SSL handshake is > complete. I am not sure if the exception is harmless, but it will be good to > avoid the exception either way since it feels like an exception that exists > to handle edge cases like buffer overflow rather than something in a normal > code path. -- This message was sent by Atlassian JIRA (v6.3.4#6332)