You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by "huxihx (JIRA)" <ji...@apache.org> on 2017/06/13 08:29:01 UTC

[jira] [Commented] (KAFKA-5007) Kafka Replica Fetcher Thread- Resource Leak

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

huxihx commented on KAFKA-5007:
-------------------------------

[~junrao] is it possible that it 's caused by the code snippet below:


{code:java}
...
SocketChannel socketChannel = SocketChannel.open();
        socketChannel.configureBlocking(false);
        Socket socket = socketChannel.socket();
        socket.setKeepAlive(true);
        if (sendBufferSize != Selectable.USE_DEFAULT_BUFFER_SIZE)
            socket.setSendBufferSize(sendBufferSize);
        if (receiveBufferSize != Selectable.USE_DEFAULT_BUFFER_SIZE)
            socket.setReceiveBufferSize(receiveBufferSize);
        socket.setTcpNoDelay(true);
        boolean connected;
        try {
            connected = socketChannel.connect(address);
        } catch (UnresolvedAddressException e) {
            socketChannel.close();
            throw new IOException("Can't resolve address: " + address, e);
        } catch (IOException e) {
            socketChannel.close();
            throw e;
        }
{code}

The code did not capture all possible exceptions so `socketChannel` got failed to be closed.


> Kafka Replica Fetcher Thread- Resource Leak
> -------------------------------------------
>
>                 Key: KAFKA-5007
>                 URL: https://issues.apache.org/jira/browse/KAFKA-5007
>             Project: Kafka
>          Issue Type: Bug
>          Components: core, network
>    Affects Versions: 0.10.0.0, 0.10.1.1, 0.10.2.0
>         Environment: Centos 7
> Jave 8
>            Reporter: Joseph Aliase
>            Priority: Critical
>              Labels: reliability
>         Attachments: jstack-kafka.out, jstack-zoo.out, lsofkafka.txt, lsofzookeeper.txt
>
>
> Kafka is running out of open file descriptor when system network interface is done.
> Issue description:
> We have a Kafka Cluster of 5 node running on version 0.10.1.1. The open file descriptor for the account running Kafka is set to 100000.
> During an upgrade, network interface went down. Outage continued for 12 hours eventually all the broker crashed with java.io.IOException: Too many open files error.
> We repeated the test in a lower environment and observed that Open Socket count keeps on increasing while the NIC is down.
> We have around 13 topics with max partition size of 120 and number of replica fetcher thread is set to 8.
> Using an internal monitoring tool we observed that Open Socket descriptor   for the broker pid continued to increase although NIC was down leading to  Open File descriptor error. 



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)