You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by Dave Ariens <da...@blackberry.com> on 2015/12/10 16:15:41 UTC

Interacting with a secured Kafka cluster via GSS-API

Hi devs!

I'm working on enhancing a custom 0.8.2.1 producer/consumer to support establishing connections a secured 0.9.0 cluster with strict ACLs on each topic.  I'm pretty new to (read: first day working with) GSS-API/JAAS and not really sure how to approach this problem.  Our existing implementation is pretty straight forward, establish a socket to the broker and send/receive byte arrays conforming to the Kafka protocol accordingly.

Now--with 0.9.0 brokers configured to listen on 9093 with SASL_PLAINTEXT.  I'm performing a auth with Kerberos on our existing Kerberos infrastructure that's in a known functional state and I'm running into a world of confusion.

My initial experiments consisted of sending a metadata fetch request for a single topic only that has ACLs restricting metadata, consuming and producing to only a single principal.  Something that should be pretty straight forward.

I'm launching my client with the following system properties configured:

JAVA_OPTS="$JAVA_OPTS -Djavax.security.auth.useSubjectCredsOnly=false"
JAVA_OPTS="$JAVA_OPTS -Djava.security.auth.login.config=$CONFIGDIR/jaas.conf"
JAVA_OPTS="$JAVA_OPTS -Djava.security.debug=gssloginconfig,configfile,configparser,logincontext"
JAVA_OPTS="$JAVA_OPTS -Djava.security.krb5.realm=MY.REALM"
JAVA_OPTS="$JAVA_OPTS -Djava.security.krb5.kdc=my.kdc"

...and my jaas.conf contains:

com.sun.security.jgss.krb5.initiate {
    com.sun.security.auth.module.Krb5LoginModule required
    principal="dariens@MY.REALM"
    useKeyTab=true
    keyTab="/opt/klogger/config/dariens.keytab"
    useTicketCache=false
    doNotPrompt=true
    storeKey=true;
};

I then open the socket as I did with 0.8.2.1 and then use GSS-API to authenticate myself with a little util that accepts the socket then returns the GSSContext (code below).  I then tried a number of things (being unfamiliar with everything in this space I relied on trial and error/brute force to attempt to get it working).

I tried sending/receiving byte arrays as I would have with 0.8.2.1 with combinations of encrypt/verifyIntegrity, I tried wrapping the original byte array with the context and sending/verifying the token, all to no avail and gave up after a number of hours.

FWIW, I do see that I am authenticating via GSS-API properly regardless of what I then do over the socket.

Could anyone advise how far off my approach is?  I'll be reading up on this more today and have some packet captures that I'll examine to glean more information on but in the meantime some help would be greatly appreciated!

Logs:

[2015-12-10 14:59:03,716] INFO Listening on port 2002 (com.blackberry.bdp.klogger.TcpListener)
[2015-12-10 14:59:05,408] INFO Creating new producer for topic dariens-only-rw, key 1 (com.blackberry.bdp.krackle.producer.Producer)
[2015-12-10 14:59:05,624] INFO Updating metadata (com.blackberry.bdp.krackle.producer.Producer)
[2015-12-10 14:59:05,626] INFO Getting metadata for dariens-only-rw (com.blackberry.bdp.krackle.meta.MetaData)
[2015-12-10 14:59:05,629] INFO established socket to k1-kafka090.kafka.lab:9093 (com.blackberry.bdp.krackle.meta.MetaData)
[2015-12-10 14:59:05,776] INFO sending token of size 572 from initSecContext (com.blackberry.bdp.krackle.security.GssKrb5AuthHandler)
[2015-12-10 14:59:05,776] INFO context established (com.blackberry.bdp.krackle.security.GssKrb5AuthHandler)
[2015-12-10 14:59:05,776] INFO client: dariens@MY.REALM (com.blackberry.bdp.krackle.security.GssKrb5AuthHandler)
[2015-12-10 14:59:05,776] INFO server: kafka  (com.blackberry.bdp.krackle.security.GssKrb5AuthHandler)
[2015-12-10 14:59:05,776] INFO mutual auth took place?: false (com.blackberry.bdp.krackle.security.GssKrb5AuthHandler)
[2015-12-10 14:59:05,776] INFO context supports data integrity?: true (com.blackberry.bdp.krackle.security.GssKrb5AuthHandler)
[2015-12-10 14:59:05,776] INFO context supports data confidentiality?: false (com.blackberry.bdp.krackle.security.GssKrb5AuthHandler)

configfile: reading file:/opt/klogger/config/jaas.conf
configparser:     Reading next config entry: com.sun.security.jgss.krb5.initiate
configparser:         com.sun.security.auth.module.Krb5LoginModule, required
configparser:             storeKey=true
configparser:             useKeyTab=true
configparser:             principal=dariens@MY.REALM
configparser:             keyTab=/opt/klogger/config/dariens.keytab
configparser:             useTicketCache=false
configparser:             doNotPrompt=true
    [GSS LoginConfigImpl]: Trying com.sun.security.jgss.krb5.initiate: Found!
    [LoginContext]: login success
    [LoginContext]: commit success

Some GSS-API related source:

public GssKrb5AuthHandler() throws GSSException {
  krb5Oid = new Oid("1.2.840.113554.1.2.2");
  manager = GSSManager.getInstance();
  requestMutualAuth = false;
  encrypt = false;
  verifyIntegrity = true;
}

public GSSContext getContext(Socket sock,
  String peerPrincipal)
    throws GSSException, IOException {
  DataInputStream inStream = new DataInputStream(sock.getInputStream());
  DataOutputStream outStream = new DataOutputStream(sock.getOutputStream());
  GSSName peerName = manager.createName(peerPrincipal, null);
  GSSContext context = manager.createContext(peerName,
    krb5Oid,
    null,
    GSSContext.DEFAULT_LIFETIME);
  context.requestMutualAuth(requestMutualAuth);  // Mutual authentication
  context.requestConf(encrypt);  // Confidentility/encryption
  context.requestInteg(verifyIntegrity); // Integrity(i.e. require wrap() and getMICmethods()

  byte[] token = new byte[0];
  while (!context.isEstablished()) {
    token = context.initSecContext(token, 0, token.length);
    if (token != null) {
      LOG.info("sending token of size {} from initSecContext", token.length);
      outStream.writeInt(token.length);
      outStream.write(token);
      outStream.flush();
    }
    if (!context.isEstablished()) {
      token = new byte[inStream.readInt()];
      LOG.info("read input token (size {}) for processing by initSecContext", token.length);
      inStream.readFully(token);
    }
  }
  LOG.info("context established");
  LOG.info("client: {}", context.getSrcName());
  LOG.info("server: {} ", context.getTargName());
  LOG.info("mutual auth took place?: {}", context.getMutualAuthState());
  LOG.info("context supports data integrity?: {}", context.getIntegState());
  LOG.info("context supports data confidentiality?: {}", context.getConfState());
  return context;
}





RE: Interacting with a secured Kafka cluster via GSS-API

Posted by Dave Ariens <da...@blackberry.com>.
Hey folks,

If anyone can quickly comment on the below producer calls?  If this is worth more discussion I'll raise it on the users list under a more appropriate subject.

Thanks

________________________________________
From: Dave Ariens
Sent: Thursday, December 10, 2015 1:40 PM
To: dev@kafka.apache.org
Subject: RE: Interacting with a secured Kafka cluster via GSS-API

> Interested to find out if the new producer and consumer fare
> better

Regarding that, what's the best way to use the new producer with as little overhead as possible?

import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.ProducerRecord;

producer = new KafkaProducer<byte[], byte[]>(conf.getProperties());

The code containing the producer is listening on a TCP socket and reading into a byte buffer until there's a new line character.  It then treats the contents of the byte array up to the position of the buffer as a message.

byte[] payload = Arrays.copyOf(bytes, sendBuffer.position());
ProducerRecord<byte[], byte[]> record = new ProducerRecord<>(topic, payload);
producer.send(record);

...is there a more elegant way to produce that message?  Having to create a new object by itself would be too costly in our use case, let alone the array copy...

(another disclaimer: I really haven't played around with the new producer much at all--and even my 0.8.2.1 knowledge is mainly focused on Krackle)
________________________________________
From: ismaelj@gmail.com [ismaelj@gmail.com] on behalf of Ismael Juma [ismael@juma.me.uk]
Sent: Thursday, December 10, 2015 1:10 PM
To: dev@kafka.apache.org
Subject: Re: Interacting with a secured Kafka cluster via GSS-API

On Thu, Dec 10, 2015 at 6:06 PM, Dave Ariens <da...@blackberry.com> wrote:

> Sure, the custom producer and consumer library I'm updating is Krackle:
> https://github.com/blackberry/Krackle
> ...
> We found that neither the Kafka proper producers nor Kafka could keep up
> with the stock packages when we initially looked (pre 0.8)
>

Thanks, Dave. Interested to find out if the new producer and consumer fare
better. :)

Ismael

RE: Interacting with a secured Kafka cluster via GSS-API

Posted by Dave Ariens <da...@blackberry.com>.
> Interested to find out if the new producer and consumer fare
> better

Regarding that, what's the best way to use the new producer with as little overhead as possible?

import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.ProducerRecord;

producer = new KafkaProducer<byte[], byte[]>(conf.getProperties());

The code containing the producer is listening on a TCP socket and reading into a byte buffer until there's a new line character.  It then treats the contents of the byte array up to the position of the buffer as a message.

byte[] payload = Arrays.copyOf(bytes, sendBuffer.position());
ProducerRecord<byte[], byte[]> record = new ProducerRecord<>(topic, payload);
producer.send(record); 

...is there a more elegant way to produce that message?  Having to create a new object by itself would be too costly in our use case, let alone the array copy...

(another disclaimer: I really haven't played around with the new producer much at all--and even my 0.8.2.1 knowledge is mainly focused on Krackle)
________________________________________
From: ismaelj@gmail.com [ismaelj@gmail.com] on behalf of Ismael Juma [ismael@juma.me.uk]
Sent: Thursday, December 10, 2015 1:10 PM
To: dev@kafka.apache.org
Subject: Re: Interacting with a secured Kafka cluster via GSS-API

On Thu, Dec 10, 2015 at 6:06 PM, Dave Ariens <da...@blackberry.com> wrote:

> Sure, the custom producer and consumer library I'm updating is Krackle:
> https://github.com/blackberry/Krackle
> ...
> We found that neither the Kafka proper producers nor Kafka could keep up
> with the stock packages when we initially looked (pre 0.8)
>

Thanks, Dave. Interested to find out if the new producer and consumer fare
better. :)

Ismael

Re: Interacting with a secured Kafka cluster via GSS-API

Posted by Ismael Juma <is...@juma.me.uk>.
On Thu, Dec 10, 2015 at 6:06 PM, Dave Ariens <da...@blackberry.com> wrote:

> Sure, the custom producer and consumer library I'm updating is Krackle:
> https://github.com/blackberry/Krackle
> ...
> We found that neither the Kafka proper producers nor Kafka could keep up
> with the stock packages when we initially looked (pre 0.8)
>

Thanks, Dave. Interested to find out if the new producer and consumer fare
better. :)

Ismael

RE: Interacting with a secured Kafka cluster via GSS-API

Posted by Dave Ariens <da...@blackberry.com>.
Sure, the custom producer and consumer library I'm updating is Krackle:  https://github.com/blackberry/Krackle

It's a very limited low overhead library that we're using to get syslog style messages from the Blackberry service provider infrastructure into Kafka.

We have dozens of pools of servers containing dozens of hosts each which each host receiving and forwarding ~50K msg/sec into Kafka.

We found that neither the Kafka proper producers nor Kafka could keep up with the stock packages when we initially looked (pre 0.8)


________________________________________
From: ismaelj@gmail.com [ismaelj@gmail.com] on behalf of Ismael Juma [ismael@juma.me.uk]
Sent: Thursday, December 10, 2015 12:44 PM
To: dev@kafka.apache.org
Subject: Re: Interacting with a secured Kafka cluster via GSS-API

On Thu, Dec 10, 2015 at 5:24 PM, Dave Ariens <da...@blackberry.com> wrote:

> Absolutely, currently I'm hoping to get authentication working and then
> ultimately work towards encryption.   We're also testing performance of
> more out-of-the-box Kafka components but I fear our message volume will
> require us to maintain our custom producers and consumers.
>

If it turns out that the out of the box components are not good enough, it
would be interesting to know how your custom clients differ. Is that
documented somewhere?

Thanks,
Ismael

Re: Interacting with a secured Kafka cluster via GSS-API

Posted by Ismael Juma <is...@juma.me.uk>.
On Thu, Dec 10, 2015 at 5:24 PM, Dave Ariens <da...@blackberry.com> wrote:

> Absolutely, currently I'm hoping to get authentication working and then
> ultimately work towards encryption.   We're also testing performance of
> more out-of-the-box Kafka components but I fear our message volume will
> require us to maintain our custom producers and consumers.
>

If it turns out that the out of the box components are not good enough, it
would be interesting to know how your custom clients differ. Is that
documented somewhere?

Thanks,
Ismael

RE: Interacting with a secured Kafka cluster via GSS-API

Posted by Dave Ariens <da...@blackberry.com>.
Yes, I have read through the documentation and I am attempting SASL authentication with Kerberos.    I don't blame anyone for assuming I hadn't :)

I have my keytab generated and I am launching with my client with a JAAS configuration accordingly.  The missing piece for me was I was attempting to use GSS-API to perform that authentication ans not the javax.security.sasl packages as per the way it's performed in org.apache.kafka.common.security.authenticator.SaslClientAuthenticator.

I'm ripping out my GSS-API code and replacing it with an implementation similar to Kafka's.

Like I said, I only started playing around with the security stuff yesterday so I have a lot to learn.

Thanks for your patience.

________________________________________
From: Flavio Junqueira [fpj@apache.org]
Sent: Thursday, December 10, 2015 12:38 PM
To: dev@kafka.apache.org
Subject: Re: Interacting with a secured Kafka cluster via GSS-API

Hi Dave,

I apologize for the obvious question, but have you had a look at the documentation:

http://kafka.apache.org/documentation.html#security <http://kafka.apache.org/documentation.html#security>

It is possible that you're not aware that it is there, so I'm just confirming. But, if you did have a look and the content didn't work for you, let us know why so that we can fix it.

Thanks,
-Flavio

> On 10 Dec 2015, at 17:24, Dave Ariens <da...@blackberry.com> wrote:
>
> Absolutely, currently I'm hoping to get authentication working and then ultimately work towards encryption.   We're also testing performance of more out-of-the-box Kafka components but I fear our message volume will require us to maintain our custom producers and consumers.
>
>
> ________________________________________
> From: Andrew Schofield [andrew_schofield@uk.ibm.com]
> Sent: Thursday, December 10, 2015 10:52 AM
> To: dev@kafka.apache.org
> Subject: RE: Interacting with a secured Kafka cluster via GSS-API
>
> Wouldn't you use TLS to secure the connections? Encrypting just the
> credentials but not the connection seems brave.
>
> Andrew
>
>
>
> From:   Dave Ariens <da...@blackberry.com>
> To:     "dev@kafka.apache.org" <de...@kafka.apache.org>
> Date:   10/12/2015 15:43
> Subject:        RE: Interacting with a secured Kafka cluster via GSS-API
>
>
>
>> Is there a reason why you are using GSS-API directly instead of via
> SASL?
>
> There sure is--because I have no clue what I'm doing :)
>
> Our Kafka 0.9.0 cluster is currently only configured for SASL_PLAINTEXT so
> we're not encrypting anything at the moment.  I'll take a look through
> SaslClientAuthenticator and try and come back with either confirmation
> that everything is working as expected (hopefully) or at least more
> intelligent questions...
>
> Thanks!
>
> ________________________________________
> From: ismaelj@gmail.com [ismaelj@gmail.com] on behalf of Ismael Juma
> [ismael@juma.me.uk]
> Sent: Thursday, December 10, 2015 10:36 AM
> To: dev@kafka.apache.org
> Subject: Re: Interacting with a secured Kafka cluster via GSS-API
>
> Hi Dave,
>
> Is there a reason why you are using GSS-API directly instead of via SASL?
> It should still work, but if you do the latter, you can potentially reuse
> the existing code (or at least use it as inspiration), see
> `org.apache.kafka.common.security.authenticator.SaslClientAuthenticator`.
>
> Also, please keep in mind that we are only using SASL for authentication
> and that to encrypt the communication, you have to use SASL_SSL (ie we
> don't support the SASL confidentiality QOP, for example).
>
> I hope this helps.
>
> Ismael
>
>
>
> Unless stated otherwise above:
> IBM United Kingdom Limited - Registered in England and Wales with number
> 741598.
> Registered office: PO Box 41, North Harbour, Portsmouth, Hampshire PO6 3AU


Re: Interacting with a secured Kafka cluster via GSS-API

Posted by Flavio Junqueira <fp...@apache.org>.
Hi Dave,

I apologize for the obvious question, but have you had a look at the documentation:

http://kafka.apache.org/documentation.html#security <http://kafka.apache.org/documentation.html#security>

It is possible that you're not aware that it is there, so I'm just confirming. But, if you did have a look and the content didn't work for you, let us know why so that we can fix it.

Thanks,
-Flavio

> On 10 Dec 2015, at 17:24, Dave Ariens <da...@blackberry.com> wrote:
> 
> Absolutely, currently I'm hoping to get authentication working and then ultimately work towards encryption.   We're also testing performance of more out-of-the-box Kafka components but I fear our message volume will require us to maintain our custom producers and consumers.
> 
> 
> ________________________________________
> From: Andrew Schofield [andrew_schofield@uk.ibm.com]
> Sent: Thursday, December 10, 2015 10:52 AM
> To: dev@kafka.apache.org
> Subject: RE: Interacting with a secured Kafka cluster via GSS-API
> 
> Wouldn't you use TLS to secure the connections? Encrypting just the
> credentials but not the connection seems brave.
> 
> Andrew
> 
> 
> 
> From:   Dave Ariens <da...@blackberry.com>
> To:     "dev@kafka.apache.org" <de...@kafka.apache.org>
> Date:   10/12/2015 15:43
> Subject:        RE: Interacting with a secured Kafka cluster via GSS-API
> 
> 
> 
>> Is there a reason why you are using GSS-API directly instead of via
> SASL?
> 
> There sure is--because I have no clue what I'm doing :)
> 
> Our Kafka 0.9.0 cluster is currently only configured for SASL_PLAINTEXT so
> we're not encrypting anything at the moment.  I'll take a look through
> SaslClientAuthenticator and try and come back with either confirmation
> that everything is working as expected (hopefully) or at least more
> intelligent questions...
> 
> Thanks!
> 
> ________________________________________
> From: ismaelj@gmail.com [ismaelj@gmail.com] on behalf of Ismael Juma
> [ismael@juma.me.uk]
> Sent: Thursday, December 10, 2015 10:36 AM
> To: dev@kafka.apache.org
> Subject: Re: Interacting with a secured Kafka cluster via GSS-API
> 
> Hi Dave,
> 
> Is there a reason why you are using GSS-API directly instead of via SASL?
> It should still work, but if you do the latter, you can potentially reuse
> the existing code (or at least use it as inspiration), see
> `org.apache.kafka.common.security.authenticator.SaslClientAuthenticator`.
> 
> Also, please keep in mind that we are only using SASL for authentication
> and that to encrypt the communication, you have to use SASL_SSL (ie we
> don't support the SASL confidentiality QOP, for example).
> 
> I hope this helps.
> 
> Ismael
> 
> 
> 
> Unless stated otherwise above:
> IBM United Kingdom Limited - Registered in England and Wales with number
> 741598.
> Registered office: PO Box 41, North Harbour, Portsmouth, Hampshire PO6 3AU


RE: Interacting with a secured Kafka cluster via GSS-API

Posted by Dave Ariens <da...@blackberry.com>.
Absolutely, currently I'm hoping to get authentication working and then ultimately work towards encryption.   We're also testing performance of more out-of-the-box Kafka components but I fear our message volume will require us to maintain our custom producers and consumers.


________________________________________
From: Andrew Schofield [andrew_schofield@uk.ibm.com]
Sent: Thursday, December 10, 2015 10:52 AM
To: dev@kafka.apache.org
Subject: RE: Interacting with a secured Kafka cluster via GSS-API

Wouldn't you use TLS to secure the connections? Encrypting just the
credentials but not the connection seems brave.

Andrew



From:   Dave Ariens <da...@blackberry.com>
To:     "dev@kafka.apache.org" <de...@kafka.apache.org>
Date:   10/12/2015 15:43
Subject:        RE: Interacting with a secured Kafka cluster via GSS-API



> Is there a reason why you are using GSS-API directly instead of via
SASL?

There sure is--because I have no clue what I'm doing :)

Our Kafka 0.9.0 cluster is currently only configured for SASL_PLAINTEXT so
we're not encrypting anything at the moment.  I'll take a look through
SaslClientAuthenticator and try and come back with either confirmation
that everything is working as expected (hopefully) or at least more
intelligent questions...

Thanks!

________________________________________
From: ismaelj@gmail.com [ismaelj@gmail.com] on behalf of Ismael Juma
[ismael@juma.me.uk]
Sent: Thursday, December 10, 2015 10:36 AM
To: dev@kafka.apache.org
Subject: Re: Interacting with a secured Kafka cluster via GSS-API

Hi Dave,

Is there a reason why you are using GSS-API directly instead of via SASL?
It should still work, but if you do the latter, you can potentially reuse
the existing code (or at least use it as inspiration), see
`org.apache.kafka.common.security.authenticator.SaslClientAuthenticator`.

Also, please keep in mind that we are only using SASL for authentication
and that to encrypt the communication, you have to use SASL_SSL (ie we
don't support the SASL confidentiality QOP, for example).

I hope this helps.

Ismael



Unless stated otherwise above:
IBM United Kingdom Limited - Registered in England and Wales with number
741598.
Registered office: PO Box 41, North Harbour, Portsmouth, Hampshire PO6 3AU

RE: Interacting with a secured Kafka cluster via GSS-API

Posted by Andrew Schofield <an...@uk.ibm.com>.
Wouldn't you use TLS to secure the connections? Encrypting just the 
credentials but not the connection seems brave.

Andrew



From:   Dave Ariens <da...@blackberry.com>
To:     "dev@kafka.apache.org" <de...@kafka.apache.org>
Date:   10/12/2015 15:43
Subject:        RE: Interacting with a secured Kafka cluster via GSS-API



> Is there a reason why you are using GSS-API directly instead of via 
SASL?

There sure is--because I have no clue what I'm doing :)

Our Kafka 0.9.0 cluster is currently only configured for SASL_PLAINTEXT so 
we're not encrypting anything at the moment.  I'll take a look through 
SaslClientAuthenticator and try and come back with either confirmation 
that everything is working as expected (hopefully) or at least more 
intelligent questions...

Thanks!

________________________________________
From: ismaelj@gmail.com [ismaelj@gmail.com] on behalf of Ismael Juma 
[ismael@juma.me.uk]
Sent: Thursday, December 10, 2015 10:36 AM
To: dev@kafka.apache.org
Subject: Re: Interacting with a secured Kafka cluster via GSS-API

Hi Dave,

Is there a reason why you are using GSS-API directly instead of via SASL?
It should still work, but if you do the latter, you can potentially reuse
the existing code (or at least use it as inspiration), see
`org.apache.kafka.common.security.authenticator.SaslClientAuthenticator`.

Also, please keep in mind that we are only using SASL for authentication
and that to encrypt the communication, you have to use SASL_SSL (ie we
don't support the SASL confidentiality QOP, for example).

I hope this helps.

Ismael



Unless stated otherwise above:
IBM United Kingdom Limited - Registered in England and Wales with number 
741598. 
Registered office: PO Box 41, North Harbour, Portsmouth, Hampshire PO6 3AU

RE: Interacting with a secured Kafka cluster via GSS-API

Posted by Dave Ariens <da...@blackberry.com>.
> Is there a reason why you are using GSS-API directly instead of via SASL?

There sure is--because I have no clue what I'm doing :)

Our Kafka 0.9.0 cluster is currently only configured for SASL_PLAINTEXT so we're not encrypting anything at the moment.  I'll take a look through SaslClientAuthenticator and try and come back with either confirmation that everything is working as expected (hopefully) or at least more intelligent questions...

Thanks!

________________________________________
From: ismaelj@gmail.com [ismaelj@gmail.com] on behalf of Ismael Juma [ismael@juma.me.uk]
Sent: Thursday, December 10, 2015 10:36 AM
To: dev@kafka.apache.org
Subject: Re: Interacting with a secured Kafka cluster via GSS-API

Hi Dave,

Is there a reason why you are using GSS-API directly instead of via SASL?
It should still work, but if you do the latter, you can potentially reuse
the existing code (or at least use it as inspiration), see
`org.apache.kafka.common.security.authenticator.SaslClientAuthenticator`.

Also, please keep in mind that we are only using SASL for authentication
and that to encrypt the communication, you have to use SASL_SSL (ie we
don't support the SASL confidentiality QOP, for example).

I hope this helps.

Ismael

Re: Interacting with a secured Kafka cluster via GSS-API

Posted by Ismael Juma <is...@juma.me.uk>.
Hi Dave,

Is there a reason why you are using GSS-API directly instead of via SASL?
It should still work, but if you do the latter, you can potentially reuse
the existing code (or at least use it as inspiration), see
`org.apache.kafka.common.security.authenticator.SaslClientAuthenticator`.

Also, please keep in mind that we are only using SASL for authentication
and that to encrypt the communication, you have to use SASL_SSL (ie we
don't support the SASL confidentiality QOP, for example).

I hope this helps.

Ismael