You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2017/06/02 14:08:04 UTC
[jira] [Commented] (KAFKA-5325) Connection Lose during Kafka
Kerberos Renewal process
[ https://issues.apache.org/jira/browse/KAFKA-5325?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16034742#comment-16034742 ]
ASF GitHub Bot commented on KAFKA-5325:
---------------------------------------
GitHub user rajinisivaram opened a pull request:
https://github.com/apache/kafka/pull/3208
KAFKA-5325: Avoid and handle exceptions for Kerberos re-login
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/rajinisivaram/kafka KAFKA-5325
Alternatively you can review and apply these changes as the patch at:
https://github.com/apache/kafka/pull/3208.patch
To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:
This closes #3208
----
commit ad8afca300bda010a4fdbd9bbdcdf80bfda17250
Author: Rajini Sivaram <ra...@googlemail.com>
Date: 2017-06-02T11:31:48Z
KAFKA-5325: Avoid and handle exceptions for Kerberos re-login
----
> Connection Lose during Kafka Kerberos Renewal process
> -----------------------------------------------------
>
> Key: KAFKA-5325
> URL: https://issues.apache.org/jira/browse/KAFKA-5325
> Project: Kafka
> Issue Type: Bug
> Components: producer
> Affects Versions: 0.9.0.0
> Reporter: MuthuKumar
> Assignee: Rajini Sivaram
>
> During Kerberos Ticket renewal, all requests reaching the server interim Kerberos renewal ticket logout & re-login is getting failed with below mentioned error.
> kafka-clients-0.9.0.0.jar is being used for producer end. Reason for using Kafka version 0.9.0.0 at producer end as the server is running in 0.10.0.x
> OS: Oracle Linux Server release 6.7
> Kerberos Configuration - Producer end
> -------------------------------------------------
> KafkaClient {
> com.sun.security.auth.module.Krb5LoginModule required
> refreshKrb5Config=true
> principal="user@xxxx.COM"
> useKeyTab=true
> serviceName="kafka"
> keyTab="xxxxx.keytab"
> client=true;
> };
> Application Log
> -------------------
> 2017-05-25 02:20:37,515 INF [Login.java:354] Initiating logout for user@xxxx.COM
> 2017-05-25 02:20:37,515 INF [Login.java:365] Initiating re-login for user@xxxx.COM
> 2017-05-25 02:20:37,525 INF [SaslChannelBuilder.java:91] Failed to create channel due to
> org.apache.kafka.common.KafkaException: Failed to configure SaslClientAuthenticator
> at org.apache.kafka.common.security.authenticator.SaslClientAuthenticator.configure(SaslClientAuthenticator.java:94)
> at org.apache.kafka.common.network.SaslChannelBuilder.buildChannel(SaslChannelBuilder.java:88)
> at org.apache.kafka.common.network.Selector.connect(Selector.java:162)
> at org.apache.kafka.clients.NetworkClient.initiateConnect(NetworkClient.java:514)
> at org.apache.kafka.clients.NetworkClient.ready(NetworkClient.java:169)
> at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:180)
> at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:128)
> at java.lang.Thread.run(Thread.java:745)
> Caused by: java.util.NoSuchElementException: null
> at java.util.LinkedList$ListItr.next(LinkedList.java:890)
> at javax.security.auth.Subject$SecureSet$1.next(Subject.java:1056)
> at org.apache.kafka.common.security.authenticator.SaslClientAuthenticator.configure(SaslClientAuthenticator.java:90)
> ... 7 common frames omitted
> 2017-05-25 02:20:37,526 ERR [Sender.java:130] Uncaught error in kafka producer I/O thread:
> org.apache.kafka.common.KafkaException: org.apache.kafka.common.KafkaException: Failed to configure SaslClientAuthenticator
> at org.apache.kafka.common.network.SaslChannelBuilder.buildChannel(SaslChannelBuilder.java:92)
> at org.apache.kafka.common.network.Selector.connect(Selector.java:162)
> at org.apache.kafka.clients.NetworkClient.initiateConnect(NetworkClient.java:514)
> at org.apache.kafka.clients.NetworkClient.ready(NetworkClient.java:169)
> at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:180)
> at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:128)
> at java.lang.Thread.run(Thread.java:745)
> Caused by: org.apache.kafka.common.KafkaException: Failed to configure SaslClientAuthenticator
> at org.apache.kafka.common.security.authenticator.SaslClientAuthenticator.configure(SaslClientAuthenticator.java:94)
> at org.apache.kafka.common.network.SaslChannelBuilder.buildChannel(SaslChannelBuilder.java:88)
> ... 6 common frames omitted
> Caused by: java.util.NoSuchElementException: null
> at java.util.LinkedList$ListItr.next(LinkedList.java:890)
> at javax.security.auth.Subject$SecureSet$1.next(Subject.java:1056)
> at org.apache.kafka.common.security.authenticator.SaslClientAuthenticator.configure(SaslClientAuthenticator.java:90)
> ... 7 common frames omitted
> 2017-05-25 02:20:37,536 ERR [Sender.java:130] Uncaught error in kafka producer I/O thread:
> java.lang.NullPointerException: null
> 2017-05-25 02:20:37,536 ERR [Sender.java:130] Uncaught error in kafka producer I/O thread:
> java.lang.NullPointerException: null
> 2017-05-25 02:20:37,536 ERR [Sender.java:130] Uncaught error in kafka producer I/O thread:
> java.lang.NullPointerException: null
--
This message was sent by Atlassian JIRA
(v6.3.15#6346)