Péter Gergő Barna created KAFKA-8568: ----------------------------------------
Summary: MirrorMaker 2.0 resource leak Key: KAFKA-8568 URL: https://issues.apache.org/jira/browse/KAFKA-8568 Project: Kafka Issue Type: Bug Components: KafkaConnect, mirrormaker Affects Versions: 2.2.2 Reporter: Péter Gergő Barna This issue produced by the branch KIP-382 (I am not sure which version is affected by that) While MirrorMaker 2.0 is running, the following command returns a number that is getting larger and larger. {noformat} lsof -p <Mirrormaker 2.0 pid> | grep ESTABLISHED | wc -l{noformat} In the error log, NullPointers pops up from the MirrorSourceTask.cleanup, because either the consumer or the producer is null when the cleanup method tries to close it. {noformat} Exception in thread "Thread-790" java.lang.NullPointerException at org.apache.kafka.connect.mirror.MirrorSourceTask.cleanup(MirrorSourceTask.java:110) at java.lang.Thread.run(Thread.java:748) Exception in thread "Thread-792" java.lang.NullPointerException at org.apache.kafka.connect.mirror.MirrorSourceTask.cleanup(MirrorSourceTask.java:110) at java.lang.Thread.run(Thread.java:748) Exception in thread "Thread-791" java.lang.NullPointerException at org.apache.kafka.connect.mirror.MirrorSourceTask.cleanup(MirrorSourceTask.java:116) at java.lang.Thread.run(Thread.java:748) Exception in thread "Thread-793" java.lang.NullPointerException at org.apache.kafka.connect.mirror.MirrorSourceTask.cleanup(MirrorSourceTask.java:110) at java.lang.Thread.run(Thread.java:748){noformat} When the number of the established connections (returned by lsof) reaches a certain limit, new exceptions start to pop up in the logs: Too many open files {noformat} [2019-06-19 12:56:43,949] ERROR WorkerSourceTask{id=MirrorHeartbeatConnector-0} failed to send record to heartbeats: {} (org.apache.kafka.connect.runtime.WorkerSourceTask) org.apache.kafka.common.errors.SaslAuthenticationException: An error: (java.security.PrivilegedActionException: javax.security.sasl.SaslException: GSS initiate failed [Caused by GSSException: No valid credentials provided (Mechanism level: Too many open files)]) occurred when evaluating SASL token received from the Kafka Broker. Kafka Client will go to A UTHENTICATION_FAILED state. Caused by: javax.security.sasl.SaslException: GSS initiate failed [Caused by GSSException: No valid credentials provided (Mechanism level: Too many open files)] at com.sun.security.sasl.gsskerb.GssKrb5Client.evaluateChallenge(GssKrb5Client.java:211) at org.apache.kafka.common.security.authenticator.SaslClientAuthenticator.lambda$createSaslToken$1(SaslClientAuthenticator.java:461) at java.security.AccessController.doPrivileged(Native Method) at javax.security.auth.Subject.doAs(Subject.java:422) at org.apache.kafka.common.security.authenticator.SaslClientAuthenticator.createSaslToken(SaslClientAuthenticator.java:461) at org.apache.kafka.common.security.authenticator.SaslClientAuthenticator.sendSaslClientToken(SaslClientAuthenticator.java:370) at org.apache.kafka.common.security.authenticator.SaslClientAuthenticator.sendInitialToken(SaslClientAuthenticator.java:290) at org.apache.kafka.common.security.authenticator.SaslClientAuthenticator.authenticate(SaslClientAuthenticator.java:230) at org.apache.kafka.common.network.KafkaChannel.prepare(KafkaChannel.java:173) at org.apache.kafka.common.network.Selector.pollSelectionKeys(Selector.java:536) at org.apache.kafka.common.network.Selector.poll(Selector.java:472) at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:535) at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:311) at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:235) at java.lang.Thread.run(Thread.java:748) Caused by: GSSException: No valid credentials provided (Mechanism level: Too many open files) at sun.security.jgss.krb5.Krb5Context.initSecContext(Krb5Context.java:775) at sun.security.jgss.GSSContextImpl.initSecContext(GSSContextImpl.java:248) at sun.security.jgss.GSSContextImpl.initSecContext(GSSContextImpl.java:179) at com.sun.security.sasl.gsskerb.GssKrb5Client.evaluateChallenge(GssKrb5Client.java:192) ... 14 more Caused by: java.net.SocketException: Too many open files at java.net.Socket.createImpl(Socket.java:460) at java.net.Socket.connect(Socket.java:587) at sun.security.krb5.internal.TCPClient.<init>(NetClient.java:63) at sun.security.krb5.internal.NetClient.getInstance(NetClient.java:43) at sun.security.krb5.KdcComm$KdcCommunication.run(KdcComm.java:393) at sun.security.krb5.KdcComm$KdcCommunication.run(KdcComm.java:364) at java.security.AccessController.doPrivileged(Native Method) at sun.security.krb5.KdcComm.send(KdcComm.java:348) at sun.security.krb5.KdcComm.sendIfPossible(KdcComm.java:253) at sun.security.krb5.KdcComm.send(KdcComm.java:229) at sun.security.krb5.KdcComm.send(KdcComm.java:200) at sun.security.krb5.KrbTgsReq.send(KrbTgsReq.java:246) at sun.security.krb5.KrbTgsReq.sendAndGetCreds(KrbTgsReq.java:261) at sun.security.krb5.internal.CredentialsUtil.serviceCreds(CredentialsUtil.java:308) at sun.security.krb5.internal.CredentialsUtil.acquireServiceCreds(CredentialsUtil.java:126) at sun.security.krb5.Credentials.acquireServiceCreds(Credentials.java:458 at sun.security.jgss.krb5.Krb5Context.initSecContext(Krb5Context.java:693) ... 17 more {noformat} From this point, MirrorMaker 2.0 starts failing to mirror new topics. -- This message was sent by Atlassian JIRA (v7.6.3#76005)