[Kerberos] JAAS module content not generated? javax.security.auth.callback.UnsupportedCallbackException: Could not login: the client is being asked for a password, but the Kafka client code does not currently support obtaining a password from the user.

classic Classic list List threaded Threaded
4 messages Options
Reply | Threaded
Open this post in threaded view
|

[Kerberos] JAAS module content not generated? javax.security.auth.callback.UnsupportedCallbackException: Could not login: the client is being asked for a password, but the Kafka client code does not currently support obtaining a password from the user.

Sebastien Pereira
Hi,

We are using Flink 1.5.3 where the Kafka producer talks with a kerberized kafka (kerberos only, no SSL).

It fails to connect to kafka with a root Exception: javax.security.auth.callback.UnsupportedCallbackException: Could not login: the client is being asked for a password, but the Kafka client code does not currently support obtaining a password from the user.

We have the following configuration for kerberos in flink-conf.yaml:
# ----------------------------------------------
security.kerberos.login.use-ticket-cache: false
security.kerberos.login.keytab:  /etc/krb5/flink.keytab
security.kerberos.login.principal: kafka/[hidden email]
security.kerberos.login.contexts: KafkaClient
# ----------------------------------------------

We use org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer011 with the following properties for kerberos:
# ----------------------------------------------
security.protocol=SASL_PLAINTEXT
sasl.kerberos.service.name=kafka
# ----------------------------------------------

From job/task managers hosts we can login with the same user which runs flink processes, and successfully get a kerberos ticket:

# ----------------------------------------------
kubectl exec -it <manager> -- /bin/bash
$ kinit kafka/[hidden email] -k -t /etc/krb5/flink.keytab
                                                                                   
Done!
New ticket is stored in cache file /opt/flink/krb5cc_bai
$ klist

Credentials cache: /opt/flink/krb5cc_bai
Default principal: kafka/[hidden email]
Number of entries: 1

[1] Service principal: krbtgt/[hidden email]
        Valid starting: Monday, September 10, 2018 at 4:58:29 PM
        Expires: Tuesday, September 11, 2018 at 4:58:29 PM
# ----------------------------------------------

However,
When we check the content of the JAAS file generated in /temp, we see no content apart the comments:

/tmp$ cat jaas-4651713797960840940.conf
/**
################################################################################
#  Licensed to the Apache Software Foundation (ASF) under one
#  or more contributor license agreements.  See the NOTICE file
#  distributed with this work for additional information
#  regarding copyright ownership.  The ASF licenses this file
#  to you under the Apache License, Version 2.0 (the
#  "License"); you may not use this file except in compliance
#  with the License.  You may obtain a copy of the License at
#
#      http://www.apache.org/licenses/LICENSE-2.0
#
#  Unless required by applicable law or agreed to in writing, software
#  distributed under the License is distributed on an "AS IS" BASIS,
#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
#  See the License for the specific language governing permissions and
# limitations under the License.
################################################################################
# We are using this file as an workaround for the Kafka and ZK SASL implementation
# since they explicitly look for java.security.auth.login.config property
# Please do not edit/delete this file - See FLINK-3929
**/

/tmp$

- Could you confirm that we should have more in the generated JAAS file?
- We strongly suspect the UnsupportedCallbackException is caused by missing content in the generated JAAS file.

Thanks,

Sebastien Pereira
Reply | Threaded
Open this post in threaded view
|

Re: [Kerberos] JAAS module content not generated? javax.security.auth.callback.UnsupportedCallbackException: Could not login: the client is being asked for a password, but the Kafka client code does not currently support obtaining a password from the user.

Aljoscha Krettek
Hi,

this definitely needs some investigation! Did you try setting the JAAS settings in the sasl.jaas.config property as described in https://docs.confluent.io/current/kafka/authentication_sasl/index.html#jaas-configurations?

Best,
Aljoscha

On 11. Sep 2018, at 10:08, Sebastien Pereira <[hidden email]> wrote:

Hi,

We are using Flink 1.5.3 where the Kafka producer talks with a kerberized kafka (kerberos only, no SSL).

It fails to connect to kafka with a root Exception: javax.security.auth.callback.UnsupportedCallbackException: Could not login: the client is being asked for a password, but the Kafka client code does not currently support obtaining a password from the user.

We have the following configuration for kerberos in flink-conf.yaml:
# ----------------------------------------------
security.kerberos.login.use-ticket-cache: false
security.kerberos.login.keytab:  /etc/krb5/flink.keytab
security.kerberos.login.principal: [hidden email]
security.kerberos.login.contexts: KafkaClient
# ----------------------------------------------

We use org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer011 with the following properties for kerberos:
# ----------------------------------------------
security.protocol=SASL_PLAINTEXT
sasl.kerberos.service.name=kafka
# ----------------------------------------------

From job/task managers hosts we can login with the same user which runs flink processes, and successfully get a kerberos ticket:

# ----------------------------------------------
kubectl exec -it <manager> -- /bin/bash
$ kinit [hidden email] -k -t /etc/krb5/flink.keytab

Done!
New ticket is stored in cache file /opt/flink/krb5cc_bai
$ klist

Credentials cache: /opt/flink/krb5cc_bai
Default principal: [hidden email]
Number of entries: 1

[1] Service principal: [hidden email]
Valid starting: Monday, September 10, 2018 at 4:58:29 PM
Expires: Tuesday, September 11, 2018 at 4:58:29 PM
# ----------------------------------------------

However,
When we check the content of the JAAS file generated in /temp, we see no content apart the comments:

/tmp$ cat jaas-4651713797960840940.conf
/**
################################################################################
#  Licensed to the Apache Software Foundation (ASF) under one
#  or more contributor license agreements.  See the NOTICE file
#  distributed with this work for additional information
#  regarding copyright ownership.  The ASF licenses this file
#  to you under the Apache License, Version 2.0 (the
#  "License"); you may not use this file except in compliance
#  with the License.  You may obtain a copy of the License at
#
#      http://www.apache.org/licenses/LICENSE-2.0
#
#  Unless required by applicable law or agreed to in writing, software
#  distributed under the License is distributed on an "AS IS" BASIS,
#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
#  See the License for the specific language governing permissions and
# limitations under the License.
################################################################################
# We are using this file as an workaround for the Kafka and ZK SASL implementation
# since they explicitly look for java.security.auth.login.config property
# Please do not edit/delete this file - See FLINK-3929
**/

/tmp$

- Could you confirm that we should have more in the generated JAAS file?
- We strongly suspect the UnsupportedCallbackException is caused by missing content in the generated JAAS file.

Thanks,

Sebastien Pereira

Reply | Threaded
Open this post in threaded view
|

Re: [Kerberos] JAAS module content not generated? javax.security.auth.callback.UnsupportedCallbackException: Could not login: the client is being asked for a password, but the Kafka client code does not currently support obtaining a password from the user.

Dawid Wysakowicz-2

Hi Sebastien,

I don't think empty jaas.conf file is a problem here. This file is created just to satisfy some preconditions which check for existence of that file. The actual values are passed in memory.

Usually when I had problems with kerberos it was due to hostname checks. Make sure you use FQDN for kafka brokers.

Best,

Dawid


On 13/09/18 16:17, Aljoscha Krettek wrote:
Hi,

this definitely needs some investigation! Did you try setting the JAAS settings in the sasl.jaas.config property as described in https://docs.confluent.io/current/kafka/authentication_sasl/index.html#jaas-configurations?

Best,
Aljoscha

On 11. Sep 2018, at 10:08, Sebastien Pereira <[hidden email]> wrote:

Hi,

We are using Flink 1.5.3 where the Kafka producer talks with a kerberized kafka (kerberos only, no SSL).

It fails to connect to kafka with a root Exception: javax.security.auth.callback.UnsupportedCallbackException: Could not login: the client is being asked for a password, but the Kafka client code does not currently support obtaining a password from the user.

We have the following configuration for kerberos in flink-conf.yaml:
# ----------------------------------------------
security.kerberos.login.use-ticket-cache: false
security.kerberos.login.keytab:  /etc/krb5/flink.keytab
security.kerberos.login.principal: [hidden email]
security.kerberos.login.contexts: KafkaClient
# ----------------------------------------------

We use org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer011 with the following properties for kerberos:
# ----------------------------------------------
security.protocol=SASL_PLAINTEXT
sasl.kerberos.service.name=kafka
# ----------------------------------------------

From job/task managers hosts we can login with the same user which runs flink processes, and successfully get a kerberos ticket:

# ----------------------------------------------
kubectl exec -it <manager> -- /bin/bash
$ kinit [hidden email] -k -t /etc/krb5/flink.keytab

Done!
New ticket is stored in cache file /opt/flink/krb5cc_bai
$ klist

Credentials cache: /opt/flink/krb5cc_bai
Default principal: [hidden email]
Number of entries: 1

[1] Service principal: [hidden email]
Valid starting: Monday, September 10, 2018 at 4:58:29 PM
Expires: Tuesday, September 11, 2018 at 4:58:29 PM
# ----------------------------------------------

However,
When we check the content of the JAAS file generated in /temp, we see no content apart the comments:

/tmp$ cat jaas-4651713797960840940.conf
/**
################################################################################
#  Licensed to the Apache Software Foundation (ASF) under one
#  or more contributor license agreements.  See the NOTICE file
#  distributed with this work for additional information
#  regarding copyright ownership.  The ASF licenses this file
#  to you under the Apache License, Version 2.0 (the
#  "License"); you may not use this file except in compliance
#  with the License.  You may obtain a copy of the License at
#
#      http://www.apache.org/licenses/LICENSE-2.0
#
#  Unless required by applicable law or agreed to in writing, software
#  distributed under the License is distributed on an "AS IS" BASIS,
#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
#  See the License for the specific language governing permissions and
# limitations under the License.
################################################################################
# We are using this file as an workaround for the Kafka and ZK SASL implementation
# since they explicitly look for java.security.auth.login.config property
# Please do not edit/delete this file - See FLINK-3929
**/

/tmp$

- Could you confirm that we should have more in the generated JAAS file?
- We strongly suspect the UnsupportedCallbackException is caused by missing content in the generated JAAS file.

Thanks,

Sebastien Pereira



signature.asc (849 bytes) Download Attachment
Reply | Threaded
Open this post in threaded view
|

Re: [Kerberos] JAAS module content not generated? javax.security.auth.callback.UnsupportedCallbackException: Could not login: the client is being asked for a password, but the Kafka client code does not currently support obtaining a password from the user.

Sebastien Pereira
In reply to this post by Sebastien Pereira
The root cause was the property `sasl.mechanism` that we forced to `PLAIN`. For Kerberos authentication the value must be `GSSAPI` (default value).

FYI from the source code it seems normal that the JAAS file have no content: the configuration is dynamically set, ie: https://github.com/apache/flink/blob/release-1.5.3/flink-runtime/src/main/java/org/apache/flink/runtime/security/modules/JaasModule.java#L74

With `sasl.mechanism=GSSAPI` the connection to Kafka with kerberos authentication succeeds.

Regards,

Sebastien

> On September 11, 2018 at 8:08 AM Sebastien Pereira <[hidden email]> wrote:
>
>
> Hi,
>
> We are using Flink 1.5.3 where the Kafka producer talks with a kerberized kafka (kerberos only, no SSL).
>
> It fails to connect to kafka with a root Exception: javax.security.auth.callback.UnsupportedCallbackException: Could not login: the client is being asked for a password, but the Kafka client code does not currently support obtaining a password from the user.
>
> We have the following configuration for kerberos in flink-conf.yaml:
> # ----------------------------------------------
> security.kerberos.login.use-ticket-cache: false
> security.kerberos.login.keytab:  /etc/krb5/flink.keytab
> security.kerberos.login.principal: kafka/[hidden email]
> security.kerberos.login.contexts: KafkaClient
> # ----------------------------------------------
>
> We use org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer011 with the following properties for kerberos:
> # ----------------------------------------------
> security.protocol=SASL_PLAINTEXT
> sasl.kerberos.service.name=kafka
> # ----------------------------------------------
>
> From job/task managers hosts we can login with the same user which runs flink processes, and successfully get a kerberos ticket:
>
> # ----------------------------------------------
> kubectl exec -it <manager> -- /bin/bash
> $ kinit kafka/[hidden email] -k -t /etc/krb5/flink.keytab
>                                                                                    
> Done!
> New ticket is stored in cache file /opt/flink/krb5cc_bai
> $ klist
>
> Credentials cache: /opt/flink/krb5cc_bai
> Default principal: kafka/[hidden email]
> Number of entries: 1
>
> [1] Service principal: krbtgt/[hidden email]
> Valid starting: Monday, September 10, 2018 at 4:58:29 PM
> Expires: Tuesday, September 11, 2018 at 4:58:29 PM
> # ----------------------------------------------
>
> However,
> When we check the content of the JAAS file generated in /temp, we see no content apart the comments:
>
> /tmp$ cat jaas-4651713797960840940.conf
> /**
> ################################################################################
> #  Licensed to the Apache Software Foundation (ASF) under one
> #  or more contributor license agreements.  See the NOTICE file
> #  distributed with this work for additional information
> #  regarding copyright ownership.  The ASF licenses this file
> #  to you under the Apache License, Version 2.0 (the
> #  "License"); you may not use this file except in compliance
> #  with the License.  You may obtain a copy of the License at
> #
> #      http://www.apache.org/licenses/LICENSE-2.0
> #
> #  Unless required by applicable law or agreed to in writing, software
> #  distributed under the License is distributed on an "AS IS" BASIS,
> #  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> #  See the License for the specific language governing permissions and
> # limitations under the License.
> ################################################################################
> # We are using this file as an workaround for the Kafka and ZK SASL implementation
> # since they explicitly look for java.security.auth.login.config property
> # Please do not edit/delete this file - See FLINK-3929
> **/
>
> /tmp$
>
> - Could you confirm that we should have more in the generated JAAS file?
> - We strongly suspect the UnsupportedCallbackException is caused by missing content in the generated JAAS file.
>
> Thanks,
>
> Sebastien Pereira