Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-05-03 Thread Ismael Juma
Hi Gwen,

Very good question! There's a PR here:

https://github.com/apache/kafka/pull/1232

:)

Ismael

On Tue, May 3, 2016 at 5:20 PM, Gwen Shapira  wrote:

> Are we planning on updating the security section in Kafka documentation?
>
> On Tue, May 3, 2016 at 12:18 AM, Rajini Sivaram
>  wrote:
> > Magnus,
> >
> > Yes, you are absolutely right. I have fixed the wiki page. Thank you for
> > pointing it out.
> >
> > Regards,
> >
> > Rajini
> >
> > On Mon, May 2, 2016 at 11:41 PM, Magnus Edenhill 
> wrote:
> >
> >> Rajini,
> >>
> >> I think I found a small documentation error on the KIP-43 wiki page, it
> >> says the SASL framing size is int16, but I believe it should be int32.
> >>
> >> Can you verify?
> >>
> >> Regards,
> >> Magnus
> >>
> >>
> >> 2016-04-25 15:38 GMT+02:00 Rajini Sivaram  >:
> >>
> >> > Magnus,
> >> >
> >> > I have updated KIP-43 to include a section with the handshake
> >> > request/response format. Have also added some more text to distinguish
> >> the
> >> > actual authentication flow from the Kafka handshake/request flow.
> >> >
> >> > Thank you,
> >> >
> >> > Rajini
> >> >
> >> >
> >> > On Mon, Apr 25, 2016 at 3:41 AM, Magnus Edenhill 
> >> > wrote:
> >> >
> >> > > Rajini,
> >> > >
> >> > > the KIP wiki is a bit unclear on the protocol changes.
> >> > > Could you document the proposed Kafka protocol requests&responses in
> >> the
> >> > > standard format (as on "A guide to the Kafka protocol").
> >> > > This information should also be added to that page when the KIP is
> >> > > accepted.
> >> > > I think it would also be good to clarify what SASL handshake means,
> if
> >> > that
> >> > > is the Kafka-leved SASL mechanism handshake or the opaque SASL data
> >> > > handshake performed by the SASL libraries.
> >> > >
> >> > > Thanks,
> >> > > Magnus
> >> > >
> >> > > 2016-04-19 17:20 GMT-07:00 Jun Rao :
> >> > >
> >> > > > Just to close the loop on this. Discussed with Magnus offline on
> how
> >> > > KIP-43
> >> > > > and KIP-35 can play together. We agreed upon the following
> proposal.
> >> > > >
> >> > > > On a SASL port,
> >> > > >
> >> > > > client sends:
> >> > > >
> >> > > > ApiVersionRequest (optional), SaslHandshakeRequest, SASL
> tokens
> >> > (size
> >> > > > delimited as being done now), regular api requests
> >> > > >
> >> > > > client receives:
> >> > > >
> >> > > > ApiVersionResponse (optional), SaslHandshakeResponse, SASL
> tokens
> >> > > (size
> >> > > > delimited as being done now), regular api responses
> >> > > >
> >> > > > The format of SaslHandshakeRequest is what's currently described
> in
> >> > > > KIP-43. There
> >> > > > will be some minor tweaks on ApiVersionResponse, which Magnus will
> >> > follow
> >> > > > up in the KIP-35 thread itself.
> >> > > >
> >> > > > Thanks,
> >> > > >
> >> > > > Jun
> >> > > >
> >> > > >
> >> > > > On Wed, Apr 13, 2016 at 5:59 AM, Rajini Sivaram <
> >> > > > rajinisiva...@googlemail.com> wrote:
> >> > > >
> >> > > > > I have updated the PR (https://github.com/apache/kafka/pull/812
> )
> >> and
> >> > > > > KIP-43
> >> > > > > to use standard Kafka format for the new request/response added
> by
> >> > > > KIP-43.
> >> > > > > I haven't changed the overall structure of the Java code.
> Feedback
> >> is
> >> > > > > appreciated.
> >> > > > >
> >> > > > > Thanks,
> >> > > > >
> >> > > > > Rajini
> >> > > > >
> >> > > > > On Tue, Apr 12, 2016 at 3:52 PM, Ismael Juma  >
> >> > > wrote:
> >> > > > >
> >> > > > > > Hi Jun,
> >> > > > > >
> >> > > > > > Comments inline.
> >> > > > > >
> >> > > > > > On Mon, Apr 11, 2016 at 1:57 AM, Jun Rao 
> >> wrote:
> >> > > > > >
> >> > > > > > > Yes, that should be fine right? Since the new api key will
> >> start
> >> > > with
> >> > > > > a 0
> >> > > > > > > byte, it actually guarantees that it's different from 0x60
> (1st
> >> > > byte
> >> > > > in
> >> > > > > > the
> >> > > > > > > old protocol) even if we change the request version id in
> the
> >> > > future.
> >> > > > > >
> >> > > > > >
> >> > > > > > Yes, this is true. Also, the GSS API library will throw an
> >> > exception
> >> > > if
> >> > > > > the
> >> > > > > > first byte is not 0x60 (for the case where newer clients
> connect
> >> to
> >> > > > older
> >> > > > > > brokers):
> >> > > > > >
> >> > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> https://github.com/frohoff/jdk8u-dev-jdk/blob/master/src/share/classes/sun/security/jgss/GSSHeader.java#L97
> >> > > > > >
> >> > > > > >
> >> > > > > > And the DEFECTIVE_TOKEN status code is specified in both RFC
> >> > 2743[1]
> >> > > > and
> >> > > > > > RFC 5653[2]. Section 3.1 of RFC 2743 specifies that the token
> tag
> >> > > > > consists
> >> > > > > > of the following elements, in order:
> >> > > > > >
> >> > > > > > 1. 0x60 -- Tag for [APPLICATION 0] SEQUENCE; indicates that
> >> > > > > >   -- constructed form, definite length encoding follows.
> >> > > > > >
> >> > > > > > 2. Token length octets ...
> >> > > > > >
> >> > > > > >
> >> > > > > >

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-05-03 Thread Gwen Shapira
Are we planning on updating the security section in Kafka documentation?

On Tue, May 3, 2016 at 12:18 AM, Rajini Sivaram
 wrote:
> Magnus,
>
> Yes, you are absolutely right. I have fixed the wiki page. Thank you for
> pointing it out.
>
> Regards,
>
> Rajini
>
> On Mon, May 2, 2016 at 11:41 PM, Magnus Edenhill  wrote:
>
>> Rajini,
>>
>> I think I found a small documentation error on the KIP-43 wiki page, it
>> says the SASL framing size is int16, but I believe it should be int32.
>>
>> Can you verify?
>>
>> Regards,
>> Magnus
>>
>>
>> 2016-04-25 15:38 GMT+02:00 Rajini Sivaram :
>>
>> > Magnus,
>> >
>> > I have updated KIP-43 to include a section with the handshake
>> > request/response format. Have also added some more text to distinguish
>> the
>> > actual authentication flow from the Kafka handshake/request flow.
>> >
>> > Thank you,
>> >
>> > Rajini
>> >
>> >
>> > On Mon, Apr 25, 2016 at 3:41 AM, Magnus Edenhill 
>> > wrote:
>> >
>> > > Rajini,
>> > >
>> > > the KIP wiki is a bit unclear on the protocol changes.
>> > > Could you document the proposed Kafka protocol requests&responses in
>> the
>> > > standard format (as on "A guide to the Kafka protocol").
>> > > This information should also be added to that page when the KIP is
>> > > accepted.
>> > > I think it would also be good to clarify what SASL handshake means, if
>> > that
>> > > is the Kafka-leved SASL mechanism handshake or the opaque SASL data
>> > > handshake performed by the SASL libraries.
>> > >
>> > > Thanks,
>> > > Magnus
>> > >
>> > > 2016-04-19 17:20 GMT-07:00 Jun Rao :
>> > >
>> > > > Just to close the loop on this. Discussed with Magnus offline on how
>> > > KIP-43
>> > > > and KIP-35 can play together. We agreed upon the following proposal.
>> > > >
>> > > > On a SASL port,
>> > > >
>> > > > client sends:
>> > > >
>> > > > ApiVersionRequest (optional), SaslHandshakeRequest, SASL tokens
>> > (size
>> > > > delimited as being done now), regular api requests
>> > > >
>> > > > client receives:
>> > > >
>> > > > ApiVersionResponse (optional), SaslHandshakeResponse, SASL tokens
>> > > (size
>> > > > delimited as being done now), regular api responses
>> > > >
>> > > > The format of SaslHandshakeRequest is what's currently described in
>> > > > KIP-43. There
>> > > > will be some minor tweaks on ApiVersionResponse, which Magnus will
>> > follow
>> > > > up in the KIP-35 thread itself.
>> > > >
>> > > > Thanks,
>> > > >
>> > > > Jun
>> > > >
>> > > >
>> > > > On Wed, Apr 13, 2016 at 5:59 AM, Rajini Sivaram <
>> > > > rajinisiva...@googlemail.com> wrote:
>> > > >
>> > > > > I have updated the PR (https://github.com/apache/kafka/pull/812)
>> and
>> > > > > KIP-43
>> > > > > to use standard Kafka format for the new request/response added by
>> > > > KIP-43.
>> > > > > I haven't changed the overall structure of the Java code. Feedback
>> is
>> > > > > appreciated.
>> > > > >
>> > > > > Thanks,
>> > > > >
>> > > > > Rajini
>> > > > >
>> > > > > On Tue, Apr 12, 2016 at 3:52 PM, Ismael Juma 
>> > > wrote:
>> > > > >
>> > > > > > Hi Jun,
>> > > > > >
>> > > > > > Comments inline.
>> > > > > >
>> > > > > > On Mon, Apr 11, 2016 at 1:57 AM, Jun Rao 
>> wrote:
>> > > > > >
>> > > > > > > Yes, that should be fine right? Since the new api key will
>> start
>> > > with
>> > > > > a 0
>> > > > > > > byte, it actually guarantees that it's different from 0x60 (1st
>> > > byte
>> > > > in
>> > > > > > the
>> > > > > > > old protocol) even if we change the request version id in the
>> > > future.
>> > > > > >
>> > > > > >
>> > > > > > Yes, this is true. Also, the GSS API library will throw an
>> > exception
>> > > if
>> > > > > the
>> > > > > > first byte is not 0x60 (for the case where newer clients connect
>> to
>> > > > older
>> > > > > > brokers):
>> > > > > >
>> > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> https://github.com/frohoff/jdk8u-dev-jdk/blob/master/src/share/classes/sun/security/jgss/GSSHeader.java#L97
>> > > > > >
>> > > > > >
>> > > > > > And the DEFECTIVE_TOKEN status code is specified in both RFC
>> > 2743[1]
>> > > > and
>> > > > > > RFC 5653[2]. Section 3.1 of RFC 2743 specifies that the token tag
>> > > > > consists
>> > > > > > of the following elements, in order:
>> > > > > >
>> > > > > > 1. 0x60 -- Tag for [APPLICATION 0] SEQUENCE; indicates that
>> > > > > >   -- constructed form, definite length encoding follows.
>> > > > > >
>> > > > > > 2. Token length octets ...
>> > > > > >
>> > > > > >
>> > > > > > Ismael
>> > > > > >
>> > > > > > [1] Generic Security Service Application Program Interface
>> Version
>> > 2,
>> > > > > > Update 1: https://tools.ietf.org/html/rfc2743
>> > > > > > [2] Generic Security Service API Version 2: Java Bindings Update:
>> > > > > > https://tools.ietf.org/html/rfc5653
>> > > > > >
>> > > > > > Ismael
>> > > > > >
>> > > > >
>> > > > >
>> > > > >
>> > > > > --
>> > > > > Regards,
>> > > > >
>> > > > > Rajini
>> > > > >
>> > > >
>> > >
>> >
>> >
>> >
>> > --
>> > Regards,
>> >
>> 

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-05-03 Thread Rajini Sivaram
Magnus,

Yes, you are absolutely right. I have fixed the wiki page. Thank you for
pointing it out.

Regards,

Rajini

On Mon, May 2, 2016 at 11:41 PM, Magnus Edenhill  wrote:

> Rajini,
>
> I think I found a small documentation error on the KIP-43 wiki page, it
> says the SASL framing size is int16, but I believe it should be int32.
>
> Can you verify?
>
> Regards,
> Magnus
>
>
> 2016-04-25 15:38 GMT+02:00 Rajini Sivaram :
>
> > Magnus,
> >
> > I have updated KIP-43 to include a section with the handshake
> > request/response format. Have also added some more text to distinguish
> the
> > actual authentication flow from the Kafka handshake/request flow.
> >
> > Thank you,
> >
> > Rajini
> >
> >
> > On Mon, Apr 25, 2016 at 3:41 AM, Magnus Edenhill 
> > wrote:
> >
> > > Rajini,
> > >
> > > the KIP wiki is a bit unclear on the protocol changes.
> > > Could you document the proposed Kafka protocol requests&responses in
> the
> > > standard format (as on "A guide to the Kafka protocol").
> > > This information should also be added to that page when the KIP is
> > > accepted.
> > > I think it would also be good to clarify what SASL handshake means, if
> > that
> > > is the Kafka-leved SASL mechanism handshake or the opaque SASL data
> > > handshake performed by the SASL libraries.
> > >
> > > Thanks,
> > > Magnus
> > >
> > > 2016-04-19 17:20 GMT-07:00 Jun Rao :
> > >
> > > > Just to close the loop on this. Discussed with Magnus offline on how
> > > KIP-43
> > > > and KIP-35 can play together. We agreed upon the following proposal.
> > > >
> > > > On a SASL port,
> > > >
> > > > client sends:
> > > >
> > > > ApiVersionRequest (optional), SaslHandshakeRequest, SASL tokens
> > (size
> > > > delimited as being done now), regular api requests
> > > >
> > > > client receives:
> > > >
> > > > ApiVersionResponse (optional), SaslHandshakeResponse, SASL tokens
> > > (size
> > > > delimited as being done now), regular api responses
> > > >
> > > > The format of SaslHandshakeRequest is what's currently described in
> > > > KIP-43. There
> > > > will be some minor tweaks on ApiVersionResponse, which Magnus will
> > follow
> > > > up in the KIP-35 thread itself.
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > >
> > > > On Wed, Apr 13, 2016 at 5:59 AM, Rajini Sivaram <
> > > > rajinisiva...@googlemail.com> wrote:
> > > >
> > > > > I have updated the PR (https://github.com/apache/kafka/pull/812)
> and
> > > > > KIP-43
> > > > > to use standard Kafka format for the new request/response added by
> > > > KIP-43.
> > > > > I haven't changed the overall structure of the Java code. Feedback
> is
> > > > > appreciated.
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Rajini
> > > > >
> > > > > On Tue, Apr 12, 2016 at 3:52 PM, Ismael Juma 
> > > wrote:
> > > > >
> > > > > > Hi Jun,
> > > > > >
> > > > > > Comments inline.
> > > > > >
> > > > > > On Mon, Apr 11, 2016 at 1:57 AM, Jun Rao 
> wrote:
> > > > > >
> > > > > > > Yes, that should be fine right? Since the new api key will
> start
> > > with
> > > > > a 0
> > > > > > > byte, it actually guarantees that it's different from 0x60 (1st
> > > byte
> > > > in
> > > > > > the
> > > > > > > old protocol) even if we change the request version id in the
> > > future.
> > > > > >
> > > > > >
> > > > > > Yes, this is true. Also, the GSS API library will throw an
> > exception
> > > if
> > > > > the
> > > > > > first byte is not 0x60 (for the case where newer clients connect
> to
> > > > older
> > > > > > brokers):
> > > > > >
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://github.com/frohoff/jdk8u-dev-jdk/blob/master/src/share/classes/sun/security/jgss/GSSHeader.java#L97
> > > > > >
> > > > > >
> > > > > > And the DEFECTIVE_TOKEN status code is specified in both RFC
> > 2743[1]
> > > > and
> > > > > > RFC 5653[2]. Section 3.1 of RFC 2743 specifies that the token tag
> > > > > consists
> > > > > > of the following elements, in order:
> > > > > >
> > > > > > 1. 0x60 -- Tag for [APPLICATION 0] SEQUENCE; indicates that
> > > > > >   -- constructed form, definite length encoding follows.
> > > > > >
> > > > > > 2. Token length octets ...
> > > > > >
> > > > > >
> > > > > > Ismael
> > > > > >
> > > > > > [1] Generic Security Service Application Program Interface
> Version
> > 2,
> > > > > > Update 1: https://tools.ietf.org/html/rfc2743
> > > > > > [2] Generic Security Service API Version 2: Java Bindings Update:
> > > > > > https://tools.ietf.org/html/rfc5653
> > > > > >
> > > > > > Ismael
> > > > > >
> > > > >
> > > > >
> > > > >
> > > > > --
> > > > > Regards,
> > > > >
> > > > > Rajini
> > > > >
> > > >
> > >
> >
> >
> >
> > --
> > Regards,
> >
> > Rajini
> >
>



-- 
Regards,

Rajini


Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-05-02 Thread Magnus Edenhill
Rajini,

I think I found a small documentation error on the KIP-43 wiki page, it
says the SASL framing size is int16, but I believe it should be int32.

Can you verify?

Regards,
Magnus


2016-04-25 15:38 GMT+02:00 Rajini Sivaram :

> Magnus,
>
> I have updated KIP-43 to include a section with the handshake
> request/response format. Have also added some more text to distinguish the
> actual authentication flow from the Kafka handshake/request flow.
>
> Thank you,
>
> Rajini
>
>
> On Mon, Apr 25, 2016 at 3:41 AM, Magnus Edenhill 
> wrote:
>
> > Rajini,
> >
> > the KIP wiki is a bit unclear on the protocol changes.
> > Could you document the proposed Kafka protocol requests&responses in the
> > standard format (as on "A guide to the Kafka protocol").
> > This information should also be added to that page when the KIP is
> > accepted.
> > I think it would also be good to clarify what SASL handshake means, if
> that
> > is the Kafka-leved SASL mechanism handshake or the opaque SASL data
> > handshake performed by the SASL libraries.
> >
> > Thanks,
> > Magnus
> >
> > 2016-04-19 17:20 GMT-07:00 Jun Rao :
> >
> > > Just to close the loop on this. Discussed with Magnus offline on how
> > KIP-43
> > > and KIP-35 can play together. We agreed upon the following proposal.
> > >
> > > On a SASL port,
> > >
> > > client sends:
> > >
> > > ApiVersionRequest (optional), SaslHandshakeRequest, SASL tokens
> (size
> > > delimited as being done now), regular api requests
> > >
> > > client receives:
> > >
> > > ApiVersionResponse (optional), SaslHandshakeResponse, SASL tokens
> > (size
> > > delimited as being done now), regular api responses
> > >
> > > The format of SaslHandshakeRequest is what's currently described in
> > > KIP-43. There
> > > will be some minor tweaks on ApiVersionResponse, which Magnus will
> follow
> > > up in the KIP-35 thread itself.
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > >
> > > On Wed, Apr 13, 2016 at 5:59 AM, Rajini Sivaram <
> > > rajinisiva...@googlemail.com> wrote:
> > >
> > > > I have updated the PR (https://github.com/apache/kafka/pull/812) and
> > > > KIP-43
> > > > to use standard Kafka format for the new request/response added by
> > > KIP-43.
> > > > I haven't changed the overall structure of the Java code. Feedback is
> > > > appreciated.
> > > >
> > > > Thanks,
> > > >
> > > > Rajini
> > > >
> > > > On Tue, Apr 12, 2016 at 3:52 PM, Ismael Juma 
> > wrote:
> > > >
> > > > > Hi Jun,
> > > > >
> > > > > Comments inline.
> > > > >
> > > > > On Mon, Apr 11, 2016 at 1:57 AM, Jun Rao  wrote:
> > > > >
> > > > > > Yes, that should be fine right? Since the new api key will start
> > with
> > > > a 0
> > > > > > byte, it actually guarantees that it's different from 0x60 (1st
> > byte
> > > in
> > > > > the
> > > > > > old protocol) even if we change the request version id in the
> > future.
> > > > >
> > > > >
> > > > > Yes, this is true. Also, the GSS API library will throw an
> exception
> > if
> > > > the
> > > > > first byte is not 0x60 (for the case where newer clients connect to
> > > older
> > > > > brokers):
> > > > >
> > > > >
> > > > >
> > > >
> > >
> >
> https://github.com/frohoff/jdk8u-dev-jdk/blob/master/src/share/classes/sun/security/jgss/GSSHeader.java#L97
> > > > >
> > > > >
> > > > > And the DEFECTIVE_TOKEN status code is specified in both RFC
> 2743[1]
> > > and
> > > > > RFC 5653[2]. Section 3.1 of RFC 2743 specifies that the token tag
> > > > consists
> > > > > of the following elements, in order:
> > > > >
> > > > > 1. 0x60 -- Tag for [APPLICATION 0] SEQUENCE; indicates that
> > > > >   -- constructed form, definite length encoding follows.
> > > > >
> > > > > 2. Token length octets ...
> > > > >
> > > > >
> > > > > Ismael
> > > > >
> > > > > [1] Generic Security Service Application Program Interface Version
> 2,
> > > > > Update 1: https://tools.ietf.org/html/rfc2743
> > > > > [2] Generic Security Service API Version 2: Java Bindings Update:
> > > > > https://tools.ietf.org/html/rfc5653
> > > > >
> > > > > Ismael
> > > > >
> > > >
> > > >
> > > >
> > > > --
> > > > Regards,
> > > >
> > > > Rajini
> > > >
> > >
> >
>
>
>
> --
> Regards,
>
> Rajini
>


Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-04-25 Thread Rajini Sivaram
Magnus,

I have updated KIP-43 to include a section with the handshake
request/response format. Have also added some more text to distinguish the
actual authentication flow from the Kafka handshake/request flow.

Thank you,

Rajini


On Mon, Apr 25, 2016 at 3:41 AM, Magnus Edenhill  wrote:

> Rajini,
>
> the KIP wiki is a bit unclear on the protocol changes.
> Could you document the proposed Kafka protocol requests&responses in the
> standard format (as on "A guide to the Kafka protocol").
> This information should also be added to that page when the KIP is
> accepted.
> I think it would also be good to clarify what SASL handshake means, if that
> is the Kafka-leved SASL mechanism handshake or the opaque SASL data
> handshake performed by the SASL libraries.
>
> Thanks,
> Magnus
>
> 2016-04-19 17:20 GMT-07:00 Jun Rao :
>
> > Just to close the loop on this. Discussed with Magnus offline on how
> KIP-43
> > and KIP-35 can play together. We agreed upon the following proposal.
> >
> > On a SASL port,
> >
> > client sends:
> >
> > ApiVersionRequest (optional), SaslHandshakeRequest, SASL tokens (size
> > delimited as being done now), regular api requests
> >
> > client receives:
> >
> > ApiVersionResponse (optional), SaslHandshakeResponse, SASL tokens
> (size
> > delimited as being done now), regular api responses
> >
> > The format of SaslHandshakeRequest is what's currently described in
> > KIP-43. There
> > will be some minor tweaks on ApiVersionResponse, which Magnus will follow
> > up in the KIP-35 thread itself.
> >
> > Thanks,
> >
> > Jun
> >
> >
> > On Wed, Apr 13, 2016 at 5:59 AM, Rajini Sivaram <
> > rajinisiva...@googlemail.com> wrote:
> >
> > > I have updated the PR (https://github.com/apache/kafka/pull/812) and
> > > KIP-43
> > > to use standard Kafka format for the new request/response added by
> > KIP-43.
> > > I haven't changed the overall structure of the Java code. Feedback is
> > > appreciated.
> > >
> > > Thanks,
> > >
> > > Rajini
> > >
> > > On Tue, Apr 12, 2016 at 3:52 PM, Ismael Juma 
> wrote:
> > >
> > > > Hi Jun,
> > > >
> > > > Comments inline.
> > > >
> > > > On Mon, Apr 11, 2016 at 1:57 AM, Jun Rao  wrote:
> > > >
> > > > > Yes, that should be fine right? Since the new api key will start
> with
> > > a 0
> > > > > byte, it actually guarantees that it's different from 0x60 (1st
> byte
> > in
> > > > the
> > > > > old protocol) even if we change the request version id in the
> future.
> > > >
> > > >
> > > > Yes, this is true. Also, the GSS API library will throw an exception
> if
> > > the
> > > > first byte is not 0x60 (for the case where newer clients connect to
> > older
> > > > brokers):
> > > >
> > > >
> > > >
> > >
> >
> https://github.com/frohoff/jdk8u-dev-jdk/blob/master/src/share/classes/sun/security/jgss/GSSHeader.java#L97
> > > >
> > > >
> > > > And the DEFECTIVE_TOKEN status code is specified in both RFC 2743[1]
> > and
> > > > RFC 5653[2]. Section 3.1 of RFC 2743 specifies that the token tag
> > > consists
> > > > of the following elements, in order:
> > > >
> > > > 1. 0x60 -- Tag for [APPLICATION 0] SEQUENCE; indicates that
> > > >   -- constructed form, definite length encoding follows.
> > > >
> > > > 2. Token length octets ...
> > > >
> > > >
> > > > Ismael
> > > >
> > > > [1] Generic Security Service Application Program Interface Version 2,
> > > > Update 1: https://tools.ietf.org/html/rfc2743
> > > > [2] Generic Security Service API Version 2: Java Bindings Update:
> > > > https://tools.ietf.org/html/rfc5653
> > > >
> > > > Ismael
> > > >
> > >
> > >
> > >
> > > --
> > > Regards,
> > >
> > > Rajini
> > >
> >
>



-- 
Regards,

Rajini


Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-04-24 Thread Magnus Edenhill
Rajini,

the KIP wiki is a bit unclear on the protocol changes.
Could you document the proposed Kafka protocol requests&responses in the
standard format (as on "A guide to the Kafka protocol").
This information should also be added to that page when the KIP is accepted.
I think it would also be good to clarify what SASL handshake means, if that
is the Kafka-leved SASL mechanism handshake or the opaque SASL data
handshake performed by the SASL libraries.

Thanks,
Magnus

2016-04-19 17:20 GMT-07:00 Jun Rao :

> Just to close the loop on this. Discussed with Magnus offline on how KIP-43
> and KIP-35 can play together. We agreed upon the following proposal.
>
> On a SASL port,
>
> client sends:
>
> ApiVersionRequest (optional), SaslHandshakeRequest, SASL tokens (size
> delimited as being done now), regular api requests
>
> client receives:
>
> ApiVersionResponse (optional), SaslHandshakeResponse, SASL tokens (size
> delimited as being done now), regular api responses
>
> The format of SaslHandshakeRequest is what's currently described in
> KIP-43. There
> will be some minor tweaks on ApiVersionResponse, which Magnus will follow
> up in the KIP-35 thread itself.
>
> Thanks,
>
> Jun
>
>
> On Wed, Apr 13, 2016 at 5:59 AM, Rajini Sivaram <
> rajinisiva...@googlemail.com> wrote:
>
> > I have updated the PR (https://github.com/apache/kafka/pull/812) and
> > KIP-43
> > to use standard Kafka format for the new request/response added by
> KIP-43.
> > I haven't changed the overall structure of the Java code. Feedback is
> > appreciated.
> >
> > Thanks,
> >
> > Rajini
> >
> > On Tue, Apr 12, 2016 at 3:52 PM, Ismael Juma  wrote:
> >
> > > Hi Jun,
> > >
> > > Comments inline.
> > >
> > > On Mon, Apr 11, 2016 at 1:57 AM, Jun Rao  wrote:
> > >
> > > > Yes, that should be fine right? Since the new api key will start with
> > a 0
> > > > byte, it actually guarantees that it's different from 0x60 (1st byte
> in
> > > the
> > > > old protocol) even if we change the request version id in the future.
> > >
> > >
> > > Yes, this is true. Also, the GSS API library will throw an exception if
> > the
> > > first byte is not 0x60 (for the case where newer clients connect to
> older
> > > brokers):
> > >
> > >
> > >
> >
> https://github.com/frohoff/jdk8u-dev-jdk/blob/master/src/share/classes/sun/security/jgss/GSSHeader.java#L97
> > >
> > >
> > > And the DEFECTIVE_TOKEN status code is specified in both RFC 2743[1]
> and
> > > RFC 5653[2]. Section 3.1 of RFC 2743 specifies that the token tag
> > consists
> > > of the following elements, in order:
> > >
> > > 1. 0x60 -- Tag for [APPLICATION 0] SEQUENCE; indicates that
> > >   -- constructed form, definite length encoding follows.
> > >
> > > 2. Token length octets ...
> > >
> > >
> > > Ismael
> > >
> > > [1] Generic Security Service Application Program Interface Version 2,
> > > Update 1: https://tools.ietf.org/html/rfc2743
> > > [2] Generic Security Service API Version 2: Java Bindings Update:
> > > https://tools.ietf.org/html/rfc5653
> > >
> > > Ismael
> > >
> >
> >
> >
> > --
> > Regards,
> >
> > Rajini
> >
>


Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-04-19 Thread Jun Rao
Just to close the loop on this. Discussed with Magnus offline on how KIP-43
and KIP-35 can play together. We agreed upon the following proposal.

On a SASL port,

client sends:

ApiVersionRequest (optional), SaslHandshakeRequest, SASL tokens (size
delimited as being done now), regular api requests

client receives:

ApiVersionResponse (optional), SaslHandshakeResponse, SASL tokens (size
delimited as being done now), regular api responses

The format of SaslHandshakeRequest is what's currently described in
KIP-43. There
will be some minor tweaks on ApiVersionResponse, which Magnus will follow
up in the KIP-35 thread itself.

Thanks,

Jun


On Wed, Apr 13, 2016 at 5:59 AM, Rajini Sivaram <
rajinisiva...@googlemail.com> wrote:

> I have updated the PR (https://github.com/apache/kafka/pull/812) and
> KIP-43
> to use standard Kafka format for the new request/response added by KIP-43.
> I haven't changed the overall structure of the Java code. Feedback is
> appreciated.
>
> Thanks,
>
> Rajini
>
> On Tue, Apr 12, 2016 at 3:52 PM, Ismael Juma  wrote:
>
> > Hi Jun,
> >
> > Comments inline.
> >
> > On Mon, Apr 11, 2016 at 1:57 AM, Jun Rao  wrote:
> >
> > > Yes, that should be fine right? Since the new api key will start with
> a 0
> > > byte, it actually guarantees that it's different from 0x60 (1st byte in
> > the
> > > old protocol) even if we change the request version id in the future.
> >
> >
> > Yes, this is true. Also, the GSS API library will throw an exception if
> the
> > first byte is not 0x60 (for the case where newer clients connect to older
> > brokers):
> >
> >
> >
> https://github.com/frohoff/jdk8u-dev-jdk/blob/master/src/share/classes/sun/security/jgss/GSSHeader.java#L97
> >
> >
> > And the DEFECTIVE_TOKEN status code is specified in both RFC 2743[1] and
> > RFC 5653[2]. Section 3.1 of RFC 2743 specifies that the token tag
> consists
> > of the following elements, in order:
> >
> > 1. 0x60 -- Tag for [APPLICATION 0] SEQUENCE; indicates that
> >   -- constructed form, definite length encoding follows.
> >
> > 2. Token length octets ...
> >
> >
> > Ismael
> >
> > [1] Generic Security Service Application Program Interface Version 2,
> > Update 1: https://tools.ietf.org/html/rfc2743
> > [2] Generic Security Service API Version 2: Java Bindings Update:
> > https://tools.ietf.org/html/rfc5653
> >
> > Ismael
> >
>
>
>
> --
> Regards,
>
> Rajini
>


Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-04-13 Thread Rajini Sivaram
I have updated the PR (https://github.com/apache/kafka/pull/812) and KIP-43
to use standard Kafka format for the new request/response added by KIP-43.
I haven't changed the overall structure of the Java code. Feedback is
appreciated.

Thanks,

Rajini

On Tue, Apr 12, 2016 at 3:52 PM, Ismael Juma  wrote:

> Hi Jun,
>
> Comments inline.
>
> On Mon, Apr 11, 2016 at 1:57 AM, Jun Rao  wrote:
>
> > Yes, that should be fine right? Since the new api key will start with a 0
> > byte, it actually guarantees that it's different from 0x60 (1st byte in
> the
> > old protocol) even if we change the request version id in the future.
>
>
> Yes, this is true. Also, the GSS API library will throw an exception if the
> first byte is not 0x60 (for the case where newer clients connect to older
> brokers):
>
>
> https://github.com/frohoff/jdk8u-dev-jdk/blob/master/src/share/classes/sun/security/jgss/GSSHeader.java#L97
>
>
> And the DEFECTIVE_TOKEN status code is specified in both RFC 2743[1] and
> RFC 5653[2]. Section 3.1 of RFC 2743 specifies that the token tag consists
> of the following elements, in order:
>
> 1. 0x60 -- Tag for [APPLICATION 0] SEQUENCE; indicates that
>   -- constructed form, definite length encoding follows.
>
> 2. Token length octets ...
>
>
> Ismael
>
> [1] Generic Security Service Application Program Interface Version 2,
> Update 1: https://tools.ietf.org/html/rfc2743
> [2] Generic Security Service API Version 2: Java Bindings Update:
> https://tools.ietf.org/html/rfc5653
>
> Ismael
>



-- 
Regards,

Rajini


Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-04-12 Thread Ismael Juma
Hi Jun,

Comments inline.

On Mon, Apr 11, 2016 at 1:57 AM, Jun Rao  wrote:

> Yes, that should be fine right? Since the new api key will start with a 0
> byte, it actually guarantees that it's different from 0x60 (1st byte in the
> old protocol) even if we change the request version id in the future.


Yes, this is true. Also, the GSS API library will throw an exception if the
first byte is not 0x60 (for the case where newer clients connect to older
brokers):

https://github.com/frohoff/jdk8u-dev-jdk/blob/master/src/share/classes/sun/security/jgss/GSSHeader.java#L97


And the DEFECTIVE_TOKEN status code is specified in both RFC 2743[1] and
RFC 5653[2]. Section 3.1 of RFC 2743 specifies that the token tag consists
of the following elements, in order:

1. 0x60 -- Tag for [APPLICATION 0] SEQUENCE; indicates that
  -- constructed form, definite length encoding follows.

2. Token length octets ...


Ismael

[1] Generic Security Service Application Program Interface Version 2,
Update 1: https://tools.ietf.org/html/rfc2743
[2] Generic Security Service API Version 2: Java Bindings Update:
https://tools.ietf.org/html/rfc5653

Ismael


Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-04-10 Thread Jun Rao
Ismael,

My responses are inlined below.

On Sun, Apr 10, 2016 at 12:25 PM, Ismael Juma  wrote:

> Hi Jun,
>
> A couple of points below.
>
> On Sat, Apr 9, 2016 at 12:19 AM, Jun Rao  wrote:
>
> > 5. Your main request is how can a client know that the broker is now
> > supporting new SASL mechanisms. One way to support that is to adjust
> KIP-43
> > slightly. We can model the SaslMechanismRequest as a regular request
> (with
> > standard request header) and add that to our protocol definition.
> Version 0
> >
>
> The current compatibility story for older clients in KIP-43 is that we send
> the mechanism first as that can be distinguished from the bytes sent by the
> GSSAPI in 0.9.0.0. If we use the standard request header for
> SaslMechanismRequest (which I agree would be a nice thing to do) then we
> would be sending the api key (INT16) first.
>

Yes, that should be fine right? Since the new api key will start with a 0
byte, it actually guarantees that it's different from 0x60 (1st byte in the
old protocol) even if we change the request version id in the future.


>
> of this request indicates that it supports GSSAPI and SASL Plain. If we
> > support any additional mechanism in the future, we will bump up the
> version
> > of SaslMechanismRequest. We also add in the protocol documentation that
> the
> > SASL authentication protocol is SaslMechanismRequest followed by token
> > exchange from SASL library. If we pick the current proposal in KIP-35,
> when
> > the client issues ApiRequest, we will return the supported versions
> > for SaslMechanismRequest as well. Does this work for you?
> >
>
> Currently, authentication would have to succeed before any application
> layer request can be sent. To make sure I understand correctly, are you
> suggesting that we would change it so that an ApiVersionRequest would be
> possible before authentication happens (so that the client would then know
> the supported versions of SaslMechanismRequest)?
>
>
No, I was thinking that you still need to be able to authenticate before
you can issue ApiVersionRequest. But you made me think a bit more on
ApiVersionRequest. Will reply directly to the KIP-35 thread.


> Thanks,
> Ismael
>

Thanks,

Jun


Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-04-10 Thread Ismael Juma
Hi Jun,

A couple of points below.

On Sat, Apr 9, 2016 at 12:19 AM, Jun Rao  wrote:

> 5. Your main request is how can a client know that the broker is now
> supporting new SASL mechanisms. One way to support that is to adjust KIP-43
> slightly. We can model the SaslMechanismRequest as a regular request (with
> standard request header) and add that to our protocol definition. Version 0
>

The current compatibility story for older clients in KIP-43 is that we send
the mechanism first as that can be distinguished from the bytes sent by the
GSSAPI in 0.9.0.0. If we use the standard request header for
SaslMechanismRequest (which I agree would be a nice thing to do) then we
would be sending the api key (INT16) first.


> of this request indicates that it supports GSSAPI and SASL Plain. If we
> support any additional mechanism in the future, we will bump up the version
> of SaslMechanismRequest. We also add in the protocol documentation that the
> SASL authentication protocol is SaslMechanismRequest followed by token
> exchange from SASL library. If we pick the current proposal in KIP-35, when
> the client issues ApiRequest, we will return the supported versions
> for SaslMechanismRequest as well. Does this work for you?
>

Currently, authentication would have to succeed before any application
layer request can be sent. To make sure I understand correctly, are you
suggesting that we would change it so that an ApiVersionRequest would be
possible before authentication happens (so that the client would then know
the supported versions of SaslMechanismRequest)?

Thanks,
Ismael


Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-04-10 Thread Jun Rao
Magnus proposal versions each SASL token. Perhaps you had something
different in mind? I agree that we want to make the part that Kafka
controls in SASL extensible. For now, the mechanism part is the only thing
that we had in mind. Is there anything else you think would be useful? We
could rename SaslMechanismRequest to sth more general
like SaslNegotiationRequest if we anticipate additional info in the
future. One difference between Kafka and Zookeeper is that Zookeeper
chooses to use a single port for both plaintext and SASL. So Zookeeper
client will need to further communicate whether plaintext or SASL is used
for a connection. In Kafka, we have a separate port for SASL.

Thanks,

Jun

On Sun, Apr 10, 2016 at 10:43 AM, Jay Kreps  wrote:

> I don't think we are versioning their bytes but providing a framework for
> any additional data needed (in this case the mechanism at the least is
> needed but it could be more), right? I think this is what the authenticate
> api in zookeeper does, right?
>
> -Jay
>
> On Sun, Apr 10, 2016 at 10:08 AM, Jun Rao  wrote:
>
> > The way that we want to extend SASL mechanism is to first add the
> mechanism
> > exchange part. Once the mechanism is selected, we will exchange the SASL
> > tokens specific to the selected mechanism. Magnus proposes to model both
> > the mechanism exchange and token exchange as independent Kafka
> > request/response. My concern is that modeling the token exchange part as
> a
> > separate request feels weird. Those tokens are provided by the SASL
> > library. Once a mechanism is selected, Kafka doesn't control their
> content.
> > So why would Kafka want to version them? If you look at Magnus's
> proposal,
> > the server essentially ignores the header in the token exchange request
> > since it always expects the same header once a given mechanism is
> > determined.
> >
> > Based on that, I was suggesting an alternative approach to just model the
> > mechanism exchange part as a Kafka request, since this is the only part
> > that Kafka controls. We then just document the authentication protocol to
> > be SaslMechanismRequest followed by standard token exchanges from the
> SASL
> > library based on the agreed upon SASL mechanism. This allows us to extend
> > mechanisms in the future. It is a bit ad hoc. However, the authentication
> > logic doesn't completely fit into the independent Kafka request protocol.
> >
> > Thanks,
> >
> > Jun
> >
> > On Sun, Apr 10, 2016 at 9:02 AM, Jay Kreps  wrote:
> >
> > > I understood Magnus's complaint to be introducing a non-extensible ad
> hoc
> > > protocol. I would second that. The responses seem to be all about how
> the
> > > java code is organized (do we process things in the KafkaApis layer,
> > etc).
> > > These are separate questions, right?
> > >
> > > -Jay
> > >
> > > On Fri, Apr 8, 2016 at 4:19 PM, Jun Rao  wrote:
> > >
> > > > Hi, Magnus,
> > > >
> > > > You brought up a few things in your proposal. I am trying to itemize
> > them
> > > > below so that we can discuss them individually.
> > > >
> > > > 1. You are proposing moving the SASL authentication logic into the
> > > > application and modeling it just as regular requests such as
> > > produce/fetch.
> > > > The issue is that there is dependency between the authentication part
> > and
> > > > regular requests. The client is not expected to send any regular
> > request
> > > > before the authentication completes. If we mix them together, the
> > > > application layer will need to maintain additional states to manage
> > such
> > > > dependency. In the current implementation, authentication is done
> below
> > > the
> > > > application layer. So, when a connection is ready, the application is
> > > free
> > > > to send any regular requests. This simplifies the logic in the
> > > application
> > > > layer.
> > > >
> > > > 2. You are proposing having a single port that supports both
> plaintext
> > > and
> > > > SASL, instead of our current approach of having separate ports.
> Indeed,
> > > > there are other projects like Zookeeper do it that way. I think both
> > > > approaches are possible, but it's not clear to me that one is clearly
> > > > better than the other. Even if you have just a single port, people
> > would
> > > > likely want to have the ability to disable plaintext connections if
> > SASL
> > > is
> > > > enabled. So, you would still need some configuration on the server
> > side.
> > > > This is then not too much different from the multiple port approach.
> > > >
> > > > 3. You are proposing to give a client the ability to support multiple
> > > SASL
> > > > mechanisms and choose one to use dynamically. I am not sure if there
> > is a
> > > > real use case for that. Setting the credential for one authentication
> > > > mechanism can already be tricky. Configuring multiple of those in a
> > > single
> > > > client can cause more confusing. Also, different SASL mechanisms are
> > > > typically associated with different users. It seems unlik

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-04-10 Thread Jay Kreps
I don't think we are versioning their bytes but providing a framework for
any additional data needed (in this case the mechanism at the least is
needed but it could be more), right? I think this is what the authenticate
api in zookeeper does, right?

-Jay

On Sun, Apr 10, 2016 at 10:08 AM, Jun Rao  wrote:

> The way that we want to extend SASL mechanism is to first add the mechanism
> exchange part. Once the mechanism is selected, we will exchange the SASL
> tokens specific to the selected mechanism. Magnus proposes to model both
> the mechanism exchange and token exchange as independent Kafka
> request/response. My concern is that modeling the token exchange part as a
> separate request feels weird. Those tokens are provided by the SASL
> library. Once a mechanism is selected, Kafka doesn't control their content.
> So why would Kafka want to version them? If you look at Magnus's proposal,
> the server essentially ignores the header in the token exchange request
> since it always expects the same header once a given mechanism is
> determined.
>
> Based on that, I was suggesting an alternative approach to just model the
> mechanism exchange part as a Kafka request, since this is the only part
> that Kafka controls. We then just document the authentication protocol to
> be SaslMechanismRequest followed by standard token exchanges from the SASL
> library based on the agreed upon SASL mechanism. This allows us to extend
> mechanisms in the future. It is a bit ad hoc. However, the authentication
> logic doesn't completely fit into the independent Kafka request protocol.
>
> Thanks,
>
> Jun
>
> On Sun, Apr 10, 2016 at 9:02 AM, Jay Kreps  wrote:
>
> > I understood Magnus's complaint to be introducing a non-extensible ad hoc
> > protocol. I would second that. The responses seem to be all about how the
> > java code is organized (do we process things in the KafkaApis layer,
> etc).
> > These are separate questions, right?
> >
> > -Jay
> >
> > On Fri, Apr 8, 2016 at 4:19 PM, Jun Rao  wrote:
> >
> > > Hi, Magnus,
> > >
> > > You brought up a few things in your proposal. I am trying to itemize
> them
> > > below so that we can discuss them individually.
> > >
> > > 1. You are proposing moving the SASL authentication logic into the
> > > application and modeling it just as regular requests such as
> > produce/fetch.
> > > The issue is that there is dependency between the authentication part
> and
> > > regular requests. The client is not expected to send any regular
> request
> > > before the authentication completes. If we mix them together, the
> > > application layer will need to maintain additional states to manage
> such
> > > dependency. In the current implementation, authentication is done below
> > the
> > > application layer. So, when a connection is ready, the application is
> > free
> > > to send any regular requests. This simplifies the logic in the
> > application
> > > layer.
> > >
> > > 2. You are proposing having a single port that supports both plaintext
> > and
> > > SASL, instead of our current approach of having separate ports. Indeed,
> > > there are other projects like Zookeeper do it that way. I think both
> > > approaches are possible, but it's not clear to me that one is clearly
> > > better than the other. Even if you have just a single port, people
> would
> > > likely want to have the ability to disable plaintext connections if
> SASL
> > is
> > > enabled. So, you would still need some configuration on the server
> side.
> > > This is then not too much different from the multiple port approach.
> > >
> > > 3. You are proposing to give a client the ability to support multiple
> > SASL
> > > mechanisms and choose one to use dynamically. I am not sure if there
> is a
> > > real use case for that. Setting the credential for one authentication
> > > mechanism can already be tricky. Configuring multiple of those in a
> > single
> > > client can cause more confusing. Also, different SASL mechanisms are
> > > typically associated with different users. It seems unlikely for an
> > > administrator to give the same application two different user ids for
> > > authorization. The reason that the current proposal returns a list of
> > > enabled mechanisms is not for client to make a choice, but rather for
> the
> > > client to know the available options if the client gets an
> > > UnsupportedMechanism exception. Gwen gave an example that a client may
> > want
> > > to switch from one authentication mechanism to another. I am not sure
> if
> > we
> > > need to optimize for the transition phase. A simpler approach is to
> have
> > > the new authentication mechanism enabled and new ACL added on the
> broker
> > > side first. Then enable the new authentication mechanism on the client.
> > >
> > > 4. You are proposing to wrap the tokens from the SASL library in Kafka
> > > request protocol. It feels a bit weird to put a Kafka version on each
> > SASL
> > > token. Those tokens are generated by the SASL library,

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-04-10 Thread Jun Rao
The way that we want to extend SASL mechanism is to first add the mechanism
exchange part. Once the mechanism is selected, we will exchange the SASL
tokens specific to the selected mechanism. Magnus proposes to model both
the mechanism exchange and token exchange as independent Kafka
request/response. My concern is that modeling the token exchange part as a
separate request feels weird. Those tokens are provided by the SASL
library. Once a mechanism is selected, Kafka doesn't control their content.
So why would Kafka want to version them? If you look at Magnus's proposal,
the server essentially ignores the header in the token exchange request
since it always expects the same header once a given mechanism is
determined.

Based on that, I was suggesting an alternative approach to just model the
mechanism exchange part as a Kafka request, since this is the only part
that Kafka controls. We then just document the authentication protocol to
be SaslMechanismRequest followed by standard token exchanges from the SASL
library based on the agreed upon SASL mechanism. This allows us to extend
mechanisms in the future. It is a bit ad hoc. However, the authentication
logic doesn't completely fit into the independent Kafka request protocol.

Thanks,

Jun

On Sun, Apr 10, 2016 at 9:02 AM, Jay Kreps  wrote:

> I understood Magnus's complaint to be introducing a non-extensible ad hoc
> protocol. I would second that. The responses seem to be all about how the
> java code is organized (do we process things in the KafkaApis layer, etc).
> These are separate questions, right?
>
> -Jay
>
> On Fri, Apr 8, 2016 at 4:19 PM, Jun Rao  wrote:
>
> > Hi, Magnus,
> >
> > You brought up a few things in your proposal. I am trying to itemize them
> > below so that we can discuss them individually.
> >
> > 1. You are proposing moving the SASL authentication logic into the
> > application and modeling it just as regular requests such as
> produce/fetch.
> > The issue is that there is dependency between the authentication part and
> > regular requests. The client is not expected to send any regular request
> > before the authentication completes. If we mix them together, the
> > application layer will need to maintain additional states to manage such
> > dependency. In the current implementation, authentication is done below
> the
> > application layer. So, when a connection is ready, the application is
> free
> > to send any regular requests. This simplifies the logic in the
> application
> > layer.
> >
> > 2. You are proposing having a single port that supports both plaintext
> and
> > SASL, instead of our current approach of having separate ports. Indeed,
> > there are other projects like Zookeeper do it that way. I think both
> > approaches are possible, but it's not clear to me that one is clearly
> > better than the other. Even if you have just a single port, people would
> > likely want to have the ability to disable plaintext connections if SASL
> is
> > enabled. So, you would still need some configuration on the server side.
> > This is then not too much different from the multiple port approach.
> >
> > 3. You are proposing to give a client the ability to support multiple
> SASL
> > mechanisms and choose one to use dynamically. I am not sure if there is a
> > real use case for that. Setting the credential for one authentication
> > mechanism can already be tricky. Configuring multiple of those in a
> single
> > client can cause more confusing. Also, different SASL mechanisms are
> > typically associated with different users. It seems unlikely for an
> > administrator to give the same application two different user ids for
> > authorization. The reason that the current proposal returns a list of
> > enabled mechanisms is not for client to make a choice, but rather for the
> > client to know the available options if the client gets an
> > UnsupportedMechanism exception. Gwen gave an example that a client may
> want
> > to switch from one authentication mechanism to another. I am not sure if
> we
> > need to optimize for the transition phase. A simpler approach is to have
> > the new authentication mechanism enabled and new ACL added on the broker
> > side first. Then enable the new authentication mechanism on the client.
> >
> > 4. You are proposing to wrap the tokens from the SASL library in Kafka
> > request protocol. It feels a bit weird to put a Kafka version on each
> SASL
> > token. Those tokens are generated by the SASL library, which is
> responsible
> > for maintaining the compatibility.
> >
> > 5. Your main request is how can a client know that the broker is now
> > supporting new SASL mechanisms. One way to support that is to adjust
> KIP-43
> > slightly. We can model the SaslMechanismRequest as a regular request
> (with
> > standard request header) and add that to our protocol definition.
> Version 0
> > of this request indicates that it supports GSSAPI and SASL Plain. If we
> > support any additional mechanism in the futu

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-04-10 Thread Jay Kreps
I understood Magnus's complaint to be introducing a non-extensible ad hoc
protocol. I would second that. The responses seem to be all about how the
java code is organized (do we process things in the KafkaApis layer, etc).
These are separate questions, right?

-Jay

On Fri, Apr 8, 2016 at 4:19 PM, Jun Rao  wrote:

> Hi, Magnus,
>
> You brought up a few things in your proposal. I am trying to itemize them
> below so that we can discuss them individually.
>
> 1. You are proposing moving the SASL authentication logic into the
> application and modeling it just as regular requests such as produce/fetch.
> The issue is that there is dependency between the authentication part and
> regular requests. The client is not expected to send any regular request
> before the authentication completes. If we mix them together, the
> application layer will need to maintain additional states to manage such
> dependency. In the current implementation, authentication is done below the
> application layer. So, when a connection is ready, the application is free
> to send any regular requests. This simplifies the logic in the application
> layer.
>
> 2. You are proposing having a single port that supports both plaintext and
> SASL, instead of our current approach of having separate ports. Indeed,
> there are other projects like Zookeeper do it that way. I think both
> approaches are possible, but it's not clear to me that one is clearly
> better than the other. Even if you have just a single port, people would
> likely want to have the ability to disable plaintext connections if SASL is
> enabled. So, you would still need some configuration on the server side.
> This is then not too much different from the multiple port approach.
>
> 3. You are proposing to give a client the ability to support multiple SASL
> mechanisms and choose one to use dynamically. I am not sure if there is a
> real use case for that. Setting the credential for one authentication
> mechanism can already be tricky. Configuring multiple of those in a single
> client can cause more confusing. Also, different SASL mechanisms are
> typically associated with different users. It seems unlikely for an
> administrator to give the same application two different user ids for
> authorization. The reason that the current proposal returns a list of
> enabled mechanisms is not for client to make a choice, but rather for the
> client to know the available options if the client gets an
> UnsupportedMechanism exception. Gwen gave an example that a client may want
> to switch from one authentication mechanism to another. I am not sure if we
> need to optimize for the transition phase. A simpler approach is to have
> the new authentication mechanism enabled and new ACL added on the broker
> side first. Then enable the new authentication mechanism on the client.
>
> 4. You are proposing to wrap the tokens from the SASL library in Kafka
> request protocol. It feels a bit weird to put a Kafka version on each SASL
> token. Those tokens are generated by the SASL library, which is responsible
> for maintaining the compatibility.
>
> 5. Your main request is how can a client know that the broker is now
> supporting new SASL mechanisms. One way to support that is to adjust KIP-43
> slightly. We can model the SaslMechanismRequest as a regular request (with
> standard request header) and add that to our protocol definition. Version 0
> of this request indicates that it supports GSSAPI and SASL Plain. If we
> support any additional mechanism in the future, we will bump up the version
> of SaslMechanismRequest. We also add in the protocol documentation that the
> SASL authentication protocol is SaslMechanismRequest followed by token
> exchange from SASL library. If we pick the current proposal in KIP-35, when
> the client issues ApiRequest, we will return the supported versions
> for SaslMechanismRequest as well. Does this work for you?
>
> Thanks,
>
> Jun
>
> On Wed, Apr 6, 2016 at 3:33 PM, Magnus Edenhill 
> wrote:
>
> > 2016-04-06 19:16 GMT+02:00 Rajini Sivaram  >:
> >
> > > Magnus,
> > >
> > > I have not looked at your proposal in detail yet,
> > >
> >
> > Please do :)
> >
> >
> > > but I have a few comments:
> > >
> > >1. We need to support SSL and SASL_SSL within the same broker (for
> > >instance, to support certificate-based authentication for
> replication
> > > and
> > >SASL for external connections). At the moment, there is a one-to-one
> > >mapping between port and security protocol. If SASL is a Kafka-level
> > > option
> > >on the same port rather than a security protocol, this would be
> harder
> > > to
> > >achieve. With separate protocols, we are also able to turn off SSL
> > > client
> > >auth for SASL.
> > >
> >
> > Why would it be harder to achieve? And is it harder for the Kafka
> developer
> > or the user?
> > Wouldnt it be possible to accept both non-SSL-auth and SSL-auth clients
> on
> > the same port?
> >
> >
> >
> >
> > >2. SASL is

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-04-08 Thread Jun Rao
Hi, Magnus,

You brought up a few things in your proposal. I am trying to itemize them
below so that we can discuss them individually.

1. You are proposing moving the SASL authentication logic into the
application and modeling it just as regular requests such as produce/fetch.
The issue is that there is dependency between the authentication part and
regular requests. The client is not expected to send any regular request
before the authentication completes. If we mix them together, the
application layer will need to maintain additional states to manage such
dependency. In the current implementation, authentication is done below the
application layer. So, when a connection is ready, the application is free
to send any regular requests. This simplifies the logic in the application
layer.

2. You are proposing having a single port that supports both plaintext and
SASL, instead of our current approach of having separate ports. Indeed,
there are other projects like Zookeeper do it that way. I think both
approaches are possible, but it's not clear to me that one is clearly
better than the other. Even if you have just a single port, people would
likely want to have the ability to disable plaintext connections if SASL is
enabled. So, you would still need some configuration on the server side.
This is then not too much different from the multiple port approach.

3. You are proposing to give a client the ability to support multiple SASL
mechanisms and choose one to use dynamically. I am not sure if there is a
real use case for that. Setting the credential for one authentication
mechanism can already be tricky. Configuring multiple of those in a single
client can cause more confusing. Also, different SASL mechanisms are
typically associated with different users. It seems unlikely for an
administrator to give the same application two different user ids for
authorization. The reason that the current proposal returns a list of
enabled mechanisms is not for client to make a choice, but rather for the
client to know the available options if the client gets an
UnsupportedMechanism exception. Gwen gave an example that a client may want
to switch from one authentication mechanism to another. I am not sure if we
need to optimize for the transition phase. A simpler approach is to have
the new authentication mechanism enabled and new ACL added on the broker
side first. Then enable the new authentication mechanism on the client.

4. You are proposing to wrap the tokens from the SASL library in Kafka
request protocol. It feels a bit weird to put a Kafka version on each SASL
token. Those tokens are generated by the SASL library, which is responsible
for maintaining the compatibility.

5. Your main request is how can a client know that the broker is now
supporting new SASL mechanisms. One way to support that is to adjust KIP-43
slightly. We can model the SaslMechanismRequest as a regular request (with
standard request header) and add that to our protocol definition. Version 0
of this request indicates that it supports GSSAPI and SASL Plain. If we
support any additional mechanism in the future, we will bump up the version
of SaslMechanismRequest. We also add in the protocol documentation that the
SASL authentication protocol is SaslMechanismRequest followed by token
exchange from SASL library. If we pick the current proposal in KIP-35, when
the client issues ApiRequest, we will return the supported versions
for SaslMechanismRequest as well. Does this work for you?

Thanks,

Jun

On Wed, Apr 6, 2016 at 3:33 PM, Magnus Edenhill  wrote:

> 2016-04-06 19:16 GMT+02:00 Rajini Sivaram :
>
> > Magnus,
> >
> > I have not looked at your proposal in detail yet,
> >
>
> Please do :)
>
>
> > but I have a few comments:
> >
> >1. We need to support SSL and SASL_SSL within the same broker (for
> >instance, to support certificate-based authentication for replication
> > and
> >SASL for external connections). At the moment, there is a one-to-one
> >mapping between port and security protocol. If SASL is a Kafka-level
> > option
> >on the same port rather than a security protocol, this would be harder
> > to
> >achieve. With separate protocols, we are also able to turn off SSL
> > client
> >auth for SASL.
> >
>
> Why would it be harder to achieve? And is it harder for the Kafka developer
> or the user?
> Wouldnt it be possible to accept both non-SSL-auth and SSL-auth clients on
> the same port?
>
>
>
>
> >2. SASL is a challenge-response protocol. Server generates a
> challenge,
> >client responds. Fitting this into Kafka API is possible, but feels
> >unnatural.
> >
>
> I dont see any problems with, the handshake is half-duplex and driven by
> the SASL library, all the client needs
> to do is "long poll" the server for data.
>
> Something like this in pseudo code (where sasl. is the SASL library
> context):
>until sasl.auth_done() {
>  sendbuf = sasl.wants_to_send()
>  if (!sendbuf) // Nothing

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-04-08 Thread Gwen Shapira
I think we do need at least some capability to dynamically select
mechanisms:

Storm currently supports Kerberos. For various reasons they will need to
support delegation tokens. In order to allow any Storm version to work with
any Kafka versions, a new Storm will need to know whether it should check
for new authentication mechanisms (by doing KIP-43 type login) or to resort
to Kerberos (because, alas, it is an old broker).

Otherwise they will face a very painful compatibility issue.

Gwen


On Fri, Apr 8, 2016 at 8:20 AM, Rajini Sivaram  wrote:

> Gwen,
>
> Kafka 0.9.0.x expect the first token from the client (immediately following
> connection for SASL_PLAINTEXT, immediately following TLS handsshake for
> SASL_SSL) to be a GSSAPI token. This is passed directly to the SASL GSSAPI
> engine. Hence a client which may connect to SASL-enabled Kafka 0.9.0.x has
> to either use GSSAPI, or handle authentication failure when attempting to
> connect using another mechanism. Any additional behaviour we add in 0.10
> cannot change that. Given that SASL configuration in Kafka is based on
> static JVM-wide JAAS configuration, I am not sure we need to address
> dynamic selection of mechanisms in the client.
>
>
>
> On Thu, Apr 7, 2016 at 7:24 PM, Gwen Shapira  wrote:
>
> > Harsha and Rajini,
> >
> > We need a good way to advertise to clients whether the broker they are
> > talking to supports the new KIP-43 methods or not. So a Storm client will
> > know whether it can use a delegation token or needs to revert to older
> > implementation.
> >
> > Alternative proposals are more than welcome at the moment.
> >
> > Gwen
> >
> > On Thu, Apr 7, 2016 at 11:11 AM, Harsha  wrote:
> >
> > > Magnus,
> > >   I am not in favor moving this SASL/SSL hand shake to
> > >   KafkaApllication level protocol. I am not sure what evolution
> > >   of the current handshake  needs versioning and also merging
> > >   Application level protocol with SSL/SASL handshake doesn't
> > >   seem right.
> > > -Harsha
> > >
> > > On Thu, Apr 7, 2016, at 05:46 AM, Rajini Sivaram wrote:
> > > > Magnus,
> > > >
> > > >
> > > >
> > > >
> > > > *"Why would it be harder to achieve? And is it harder for the Kafka
> > > > developeror the user?Wouldnt it be possible to accept both
> non-SSL-auth
> > > > and
> > > > SSL-auth clients onthe same port?"*
> > > >
> > > > Maybe I am missing something, so perhaps it would be easier if I
> > > > explained
> > > > our scenario and you could tell me how it would work with the new
> > design?
> > > >
> > > >
> > > > a) Inter-broker communication
> > > >
> > > >
> > > >- uses SSL (without SASL)
> > > >- is on an internal network
> > > >- advertised host is internal IP address
> > > >- broker uses TLS client auth
> > > >
> > > > b) External clients
> > > >
> > > >
> > > >-   use SASL_SSL
> > > >-   are on the public network (and traffic goes through an
> > additional
> > > >TLS proxy for security)
> > > >-   advertised host is a registered host name also included in the
> > > >certificate and validated by the client during TLS handshake
> > > >-   cannot use TLS client auth since that would require a complex
> > PKI
> > > >infrastructure for certificate management
> > > >-   should not be allowed to connect to Kafka without SASL
> > > >authentication (broker should prevent connection without SASL)
> > > >
> > > >
> > > > Questions:
> > > >
> > > >
> > > >1.   How do I configure a) and b) to be on different network
> > > >interfaces?
> > > >2.   How do I advertise different hosts for a) and b)?
> > > >3.   How do I configure broker to disable TLS client auth for b)
> > while
> > > >enabling it for a)?
> > > >4.   How does the broker prevent external clients from connecting
> > > >without SASL while allowing internal clients to connect without
> > SASL?
> > > >
> > > > Thanks,
> > > >
> > > > Rajini
> > > >
> > > > On Wed, Apr 6, 2016 at 8:33 PM, Magnus Edenhill 
> > > > wrote:
> > > >
> > > > > 2016-04-06 19:16 GMT+02:00 Rajini Sivaram <
> > > rajinisiva...@googlemail.com>:
> > > > >
> > > > > > Magnus,
> > > > > >
> > > > > > I have not looked at your proposal in detail yet,
> > > > > >
> > > > >
> > > > > Please do :)
> > > > >
> > > > >
> > > > > > but I have a few comments:
> > > > > >
> > > > > >1. We need to support SSL and SASL_SSL within the same broker
> > (for
> > > > > >instance, to support certificate-based authentication for
> > > replication
> > > > > > and
> > > > > >SASL for external connections). At the moment, there is a
> > > one-to-one
> > > > > >mapping between port and security protocol. If SASL is a
> > > Kafka-level
> > > > > > option
> > > > > >on the same port rather than a security protocol, this would
> be
> > > harder
> > > > > > to
> > > > > >achieve. With separate protocols, we are also able to turn off
> > SSL
> > > > > > client
> > > > > >auth for 

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-04-08 Thread Rajini Sivaram
Gwen,

Kafka 0.9.0.x expect the first token from the client (immediately following
connection for SASL_PLAINTEXT, immediately following TLS handsshake for
SASL_SSL) to be a GSSAPI token. This is passed directly to the SASL GSSAPI
engine. Hence a client which may connect to SASL-enabled Kafka 0.9.0.x has
to either use GSSAPI, or handle authentication failure when attempting to
connect using another mechanism. Any additional behaviour we add in 0.10
cannot change that. Given that SASL configuration in Kafka is based on
static JVM-wide JAAS configuration, I am not sure we need to address
dynamic selection of mechanisms in the client.



On Thu, Apr 7, 2016 at 7:24 PM, Gwen Shapira  wrote:

> Harsha and Rajini,
>
> We need a good way to advertise to clients whether the broker they are
> talking to supports the new KIP-43 methods or not. So a Storm client will
> know whether it can use a delegation token or needs to revert to older
> implementation.
>
> Alternative proposals are more than welcome at the moment.
>
> Gwen
>
> On Thu, Apr 7, 2016 at 11:11 AM, Harsha  wrote:
>
> > Magnus,
> >   I am not in favor moving this SASL/SSL hand shake to
> >   KafkaApllication level protocol. I am not sure what evolution
> >   of the current handshake  needs versioning and also merging
> >   Application level protocol with SSL/SASL handshake doesn't
> >   seem right.
> > -Harsha
> >
> > On Thu, Apr 7, 2016, at 05:46 AM, Rajini Sivaram wrote:
> > > Magnus,
> > >
> > >
> > >
> > >
> > > *"Why would it be harder to achieve? And is it harder for the Kafka
> > > developeror the user?Wouldnt it be possible to accept both non-SSL-auth
> > > and
> > > SSL-auth clients onthe same port?"*
> > >
> > > Maybe I am missing something, so perhaps it would be easier if I
> > > explained
> > > our scenario and you could tell me how it would work with the new
> design?
> > >
> > >
> > > a) Inter-broker communication
> > >
> > >
> > >- uses SSL (without SASL)
> > >- is on an internal network
> > >- advertised host is internal IP address
> > >- broker uses TLS client auth
> > >
> > > b) External clients
> > >
> > >
> > >-   use SASL_SSL
> > >-   are on the public network (and traffic goes through an
> additional
> > >TLS proxy for security)
> > >-   advertised host is a registered host name also included in the
> > >certificate and validated by the client during TLS handshake
> > >-   cannot use TLS client auth since that would require a complex
> PKI
> > >infrastructure for certificate management
> > >-   should not be allowed to connect to Kafka without SASL
> > >authentication (broker should prevent connection without SASL)
> > >
> > >
> > > Questions:
> > >
> > >
> > >1.   How do I configure a) and b) to be on different network
> > >interfaces?
> > >2.   How do I advertise different hosts for a) and b)?
> > >3.   How do I configure broker to disable TLS client auth for b)
> while
> > >enabling it for a)?
> > >4.   How does the broker prevent external clients from connecting
> > >without SASL while allowing internal clients to connect without
> SASL?
> > >
> > > Thanks,
> > >
> > > Rajini
> > >
> > > On Wed, Apr 6, 2016 at 8:33 PM, Magnus Edenhill 
> > > wrote:
> > >
> > > > 2016-04-06 19:16 GMT+02:00 Rajini Sivaram <
> > rajinisiva...@googlemail.com>:
> > > >
> > > > > Magnus,
> > > > >
> > > > > I have not looked at your proposal in detail yet,
> > > > >
> > > >
> > > > Please do :)
> > > >
> > > >
> > > > > but I have a few comments:
> > > > >
> > > > >1. We need to support SSL and SASL_SSL within the same broker
> (for
> > > > >instance, to support certificate-based authentication for
> > replication
> > > > > and
> > > > >SASL for external connections). At the moment, there is a
> > one-to-one
> > > > >mapping between port and security protocol. If SASL is a
> > Kafka-level
> > > > > option
> > > > >on the same port rather than a security protocol, this would be
> > harder
> > > > > to
> > > > >achieve. With separate protocols, we are also able to turn off
> SSL
> > > > > client
> > > > >auth for SASL.
> > > > >
> > > >
> > > > Why would it be harder to achieve? And is it harder for the Kafka
> > developer
> > > > or the user?
> > > > Wouldnt it be possible to accept both non-SSL-auth and SSL-auth
> > clients on
> > > > the same port?
> > > >
> > > >
> > > >
> > > >
> > > > >2. SASL is a challenge-response protocol. Server generates a
> > > > challenge,
> > > > >client responds. Fitting this into Kafka API is possible, but
> > feels
> > > > >unnatural.
> > > > >
> > > >
> > > > I dont see any problems with, the handshake is half-duplex and driven
> > by
> > > > the SASL library, all the client needs
> > > > to do is "long poll" the server for data.
> > > >
> > > > Something like this in pseudo code (where sasl. is the SASL library
> > > > context):
> > > >until sasl.auth_don

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-04-07 Thread Gwen Shapira
Harsha and Rajini,

We need a good way to advertise to clients whether the broker they are
talking to supports the new KIP-43 methods or not. So a Storm client will
know whether it can use a delegation token or needs to revert to older
implementation.

Alternative proposals are more than welcome at the moment.

Gwen

On Thu, Apr 7, 2016 at 11:11 AM, Harsha  wrote:

> Magnus,
>   I am not in favor moving this SASL/SSL hand shake to
>   KafkaApllication level protocol. I am not sure what evolution
>   of the current handshake  needs versioning and also merging
>   Application level protocol with SSL/SASL handshake doesn't
>   seem right.
> -Harsha
>
> On Thu, Apr 7, 2016, at 05:46 AM, Rajini Sivaram wrote:
> > Magnus,
> >
> >
> >
> >
> > *"Why would it be harder to achieve? And is it harder for the Kafka
> > developeror the user?Wouldnt it be possible to accept both non-SSL-auth
> > and
> > SSL-auth clients onthe same port?"*
> >
> > Maybe I am missing something, so perhaps it would be easier if I
> > explained
> > our scenario and you could tell me how it would work with the new design?
> >
> >
> > a) Inter-broker communication
> >
> >
> >- uses SSL (without SASL)
> >- is on an internal network
> >- advertised host is internal IP address
> >- broker uses TLS client auth
> >
> > b) External clients
> >
> >
> >-   use SASL_SSL
> >-   are on the public network (and traffic goes through an additional
> >TLS proxy for security)
> >-   advertised host is a registered host name also included in the
> >certificate and validated by the client during TLS handshake
> >-   cannot use TLS client auth since that would require a complex PKI
> >infrastructure for certificate management
> >-   should not be allowed to connect to Kafka without SASL
> >authentication (broker should prevent connection without SASL)
> >
> >
> > Questions:
> >
> >
> >1.   How do I configure a) and b) to be on different network
> >interfaces?
> >2.   How do I advertise different hosts for a) and b)?
> >3.   How do I configure broker to disable TLS client auth for b) while
> >enabling it for a)?
> >4.   How does the broker prevent external clients from connecting
> >without SASL while allowing internal clients to connect without SASL?
> >
> > Thanks,
> >
> > Rajini
> >
> > On Wed, Apr 6, 2016 at 8:33 PM, Magnus Edenhill 
> > wrote:
> >
> > > 2016-04-06 19:16 GMT+02:00 Rajini Sivaram <
> rajinisiva...@googlemail.com>:
> > >
> > > > Magnus,
> > > >
> > > > I have not looked at your proposal in detail yet,
> > > >
> > >
> > > Please do :)
> > >
> > >
> > > > but I have a few comments:
> > > >
> > > >1. We need to support SSL and SASL_SSL within the same broker (for
> > > >instance, to support certificate-based authentication for
> replication
> > > > and
> > > >SASL for external connections). At the moment, there is a
> one-to-one
> > > >mapping between port and security protocol. If SASL is a
> Kafka-level
> > > > option
> > > >on the same port rather than a security protocol, this would be
> harder
> > > > to
> > > >achieve. With separate protocols, we are also able to turn off SSL
> > > > client
> > > >auth for SASL.
> > > >
> > >
> > > Why would it be harder to achieve? And is it harder for the Kafka
> developer
> > > or the user?
> > > Wouldnt it be possible to accept both non-SSL-auth and SSL-auth
> clients on
> > > the same port?
> > >
> > >
> > >
> > >
> > > >2. SASL is a challenge-response protocol. Server generates a
> > > challenge,
> > > >client responds. Fitting this into Kafka API is possible, but
> feels
> > > >unnatural.
> > > >
> > >
> > > I dont see any problems with, the handshake is half-duplex and driven
> by
> > > the SASL library, all the client needs
> > > to do is "long poll" the server for data.
> > >
> > > Something like this in pseudo code (where sasl. is the SASL library
> > > context):
> > >until sasl.auth_done() {
> > >  sendbuf = sasl.wants_to_send()
> > >  if (!sendbuf) // Nothing to send, send empty payload to poll
> > > server for data
> > >  response = SaslHandshakeRequest(payload=none)
> > >  else
> > > response = SaslHandshakeRequest(payload=sendbuf)
> > >
> > > // handle response (possibly empty) from server and continue
> > > handshake.
> > >  sasl.parse(response)
> > > }
> > >
> > >
> > >3. Personally, I like the current Network|Security|Application
> layering
> > > >of the code in Kafka. Handshake code is quite complicated and
> hard to
> > > > debug
> > > >as it is, moving that to the application layer could make it
> worse.
> > > With
> > > >handshake in the application layer, you would need to set
> Principal
> > > > based
> > > >on SSL channel for non-SASL and sometime later on for SASL. It
> may be
> > > > just
> > > >me, but it doesn't feel quite right

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-04-07 Thread Harsha
Magnus,
  I am not in favor moving this SASL/SSL hand shake to
  KafkaApllication level protocol. I am not sure what evolution
  of the current handshake  needs versioning and also merging
  Application level protocol with SSL/SASL handshake doesn't
  seem right.
-Harsha

On Thu, Apr 7, 2016, at 05:46 AM, Rajini Sivaram wrote:
> Magnus,
> 
> 
> 
> 
> *"Why would it be harder to achieve? And is it harder for the Kafka
> developeror the user?Wouldnt it be possible to accept both non-SSL-auth
> and
> SSL-auth clients onthe same port?"*
> 
> Maybe I am missing something, so perhaps it would be easier if I
> explained
> our scenario and you could tell me how it would work with the new design?
> 
> 
> a) Inter-broker communication
> 
> 
>- uses SSL (without SASL)
>- is on an internal network
>- advertised host is internal IP address
>- broker uses TLS client auth
> 
> b) External clients
> 
> 
>-   use SASL_SSL
>-   are on the public network (and traffic goes through an additional
>TLS proxy for security)
>-   advertised host is a registered host name also included in the
>certificate and validated by the client during TLS handshake
>-   cannot use TLS client auth since that would require a complex PKI
>infrastructure for certificate management
>-   should not be allowed to connect to Kafka without SASL
>authentication (broker should prevent connection without SASL)
> 
> 
> Questions:
> 
> 
>1.   How do I configure a) and b) to be on different network
>interfaces?
>2.   How do I advertise different hosts for a) and b)?
>3.   How do I configure broker to disable TLS client auth for b) while
>enabling it for a)?
>4.   How does the broker prevent external clients from connecting
>without SASL while allowing internal clients to connect without SASL?
> 
> Thanks,
> 
> Rajini
> 
> On Wed, Apr 6, 2016 at 8:33 PM, Magnus Edenhill 
> wrote:
> 
> > 2016-04-06 19:16 GMT+02:00 Rajini Sivaram :
> >
> > > Magnus,
> > >
> > > I have not looked at your proposal in detail yet,
> > >
> >
> > Please do :)
> >
> >
> > > but I have a few comments:
> > >
> > >1. We need to support SSL and SASL_SSL within the same broker (for
> > >instance, to support certificate-based authentication for replication
> > > and
> > >SASL for external connections). At the moment, there is a one-to-one
> > >mapping between port and security protocol. If SASL is a Kafka-level
> > > option
> > >on the same port rather than a security protocol, this would be harder
> > > to
> > >achieve. With separate protocols, we are also able to turn off SSL
> > > client
> > >auth for SASL.
> > >
> >
> > Why would it be harder to achieve? And is it harder for the Kafka developer
> > or the user?
> > Wouldnt it be possible to accept both non-SSL-auth and SSL-auth clients on
> > the same port?
> >
> >
> >
> >
> > >2. SASL is a challenge-response protocol. Server generates a
> > challenge,
> > >client responds. Fitting this into Kafka API is possible, but feels
> > >unnatural.
> > >
> >
> > I dont see any problems with, the handshake is half-duplex and driven by
> > the SASL library, all the client needs
> > to do is "long poll" the server for data.
> >
> > Something like this in pseudo code (where sasl. is the SASL library
> > context):
> >until sasl.auth_done() {
> >  sendbuf = sasl.wants_to_send()
> >  if (!sendbuf) // Nothing to send, send empty payload to poll
> > server for data
> >  response = SaslHandshakeRequest(payload=none)
> >  else
> > response = SaslHandshakeRequest(payload=sendbuf)
> >
> > // handle response (possibly empty) from server and continue
> > handshake.
> >  sasl.parse(response)
> > }
> >
> >
> >3. Personally, I like the current Network|Security|Application layering
> > >of the code in Kafka. Handshake code is quite complicated and hard to
> > > debug
> > >as it is, moving that to the application layer could make it worse.
> > With
> > >handshake in the application layer, you would need to set Principal
> > > based
> > >on SSL channel for non-SASL and sometime later on for SASL. It may be
> > > just
> > >me, but it doesn't feel quite right.
> > >
> >
> > I actually think reusing the existing Kafka protocol implementation for
> > this makes it
> > more robust and easier to debug and understand.
> >
> > How do you plan on handling feature discovery for new requests or request
> > versions in this special protocol?
> > I.e., what KIP-35 provides for the Kafka protocol.
> >
> >
> > Regards,
> > Magnus
> >
> >
> > >
> > > On Wed, Apr 6, 2016 at 12:47 PM, Magnus Edenhill 
> > > wrote:
> > >
> > > > Engineers,
> > > >
> > > > sorry to stir up some dust on this KIP but following yesterday's
> > > discussion
> > > > on the KIP call I'm a bit concerned about the way the SASL handshake
> > > > protocol is div

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-04-07 Thread Rajini Sivaram
Magnus,




*"Why would it be harder to achieve? And is it harder for the Kafka
developeror the user?Wouldnt it be possible to accept both non-SSL-auth and
SSL-auth clients onthe same port?"*

Maybe I am missing something, so perhaps it would be easier if I explained
our scenario and you could tell me how it would work with the new design?


a) Inter-broker communication


   - uses SSL (without SASL)
   - is on an internal network
   - advertised host is internal IP address
   - broker uses TLS client auth

b) External clients


   -   use SASL_SSL
   -   are on the public network (and traffic goes through an additional
   TLS proxy for security)
   -   advertised host is a registered host name also included in the
   certificate and validated by the client during TLS handshake
   -   cannot use TLS client auth since that would require a complex PKI
   infrastructure for certificate management
   -   should not be allowed to connect to Kafka without SASL
   authentication (broker should prevent connection without SASL)


Questions:


   1.   How do I configure a) and b) to be on different network interfaces?
   2.   How do I advertise different hosts for a) and b)?
   3.   How do I configure broker to disable TLS client auth for b) while
   enabling it for a)?
   4.   How does the broker prevent external clients from connecting
   without SASL while allowing internal clients to connect without SASL?

Thanks,

Rajini

On Wed, Apr 6, 2016 at 8:33 PM, Magnus Edenhill  wrote:

> 2016-04-06 19:16 GMT+02:00 Rajini Sivaram :
>
> > Magnus,
> >
> > I have not looked at your proposal in detail yet,
> >
>
> Please do :)
>
>
> > but I have a few comments:
> >
> >1. We need to support SSL and SASL_SSL within the same broker (for
> >instance, to support certificate-based authentication for replication
> > and
> >SASL for external connections). At the moment, there is a one-to-one
> >mapping between port and security protocol. If SASL is a Kafka-level
> > option
> >on the same port rather than a security protocol, this would be harder
> > to
> >achieve. With separate protocols, we are also able to turn off SSL
> > client
> >auth for SASL.
> >
>
> Why would it be harder to achieve? And is it harder for the Kafka developer
> or the user?
> Wouldnt it be possible to accept both non-SSL-auth and SSL-auth clients on
> the same port?
>
>
>
>
> >2. SASL is a challenge-response protocol. Server generates a
> challenge,
> >client responds. Fitting this into Kafka API is possible, but feels
> >unnatural.
> >
>
> I dont see any problems with, the handshake is half-duplex and driven by
> the SASL library, all the client needs
> to do is "long poll" the server for data.
>
> Something like this in pseudo code (where sasl. is the SASL library
> context):
>until sasl.auth_done() {
>  sendbuf = sasl.wants_to_send()
>  if (!sendbuf) // Nothing to send, send empty payload to poll
> server for data
>  response = SaslHandshakeRequest(payload=none)
>  else
> response = SaslHandshakeRequest(payload=sendbuf)
>
> // handle response (possibly empty) from server and continue
> handshake.
>  sasl.parse(response)
> }
>
>
>3. Personally, I like the current Network|Security|Application layering
> >of the code in Kafka. Handshake code is quite complicated and hard to
> > debug
> >as it is, moving that to the application layer could make it worse.
> With
> >handshake in the application layer, you would need to set Principal
> > based
> >on SSL channel for non-SASL and sometime later on for SASL. It may be
> > just
> >me, but it doesn't feel quite right.
> >
>
> I actually think reusing the existing Kafka protocol implementation for
> this makes it
> more robust and easier to debug and understand.
>
> How do you plan on handling feature discovery for new requests or request
> versions in this special protocol?
> I.e., what KIP-35 provides for the Kafka protocol.
>
>
> Regards,
> Magnus
>
>
> >
> > On Wed, Apr 6, 2016 at 12:47 PM, Magnus Edenhill 
> > wrote:
> >
> > > Engineers,
> > >
> > > sorry to stir up some dust on this KIP but following yesterday's
> > discussion
> > > on the KIP call I'm a bit concerned about the way the SASL handshake
> > > protocol is diverging even more from the main Kafka protocol.
> > > The SASL handshake is already its own special protocol and the latest
> > > addition of SaslMechanisms request in KIP-43
> > > adds yet another protocol on top of that, making assumptions on what
> SASL
> > > libraries are communicating in the opaque data channel.
> > > Unlike TLS, SASL has no standard handshake protocol so conceiving a
> > special
> > > non-Kafka protocol just for SASL in Kafka is unwarranted given that the
> > > Kafka protocol natively supports the same semantics.
> > > There's also a non-neglectable maintenance aspect of maintaining a
> > separate
> > > protocol, in regards to documentation

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-04-06 Thread Magnus Edenhill
2016-04-06 19:16 GMT+02:00 Rajini Sivaram :

> Magnus,
>
> I have not looked at your proposal in detail yet,
>

Please do :)


> but I have a few comments:
>
>1. We need to support SSL and SASL_SSL within the same broker (for
>instance, to support certificate-based authentication for replication
> and
>SASL for external connections). At the moment, there is a one-to-one
>mapping between port and security protocol. If SASL is a Kafka-level
> option
>on the same port rather than a security protocol, this would be harder
> to
>achieve. With separate protocols, we are also able to turn off SSL
> client
>auth for SASL.
>

Why would it be harder to achieve? And is it harder for the Kafka developer
or the user?
Wouldnt it be possible to accept both non-SSL-auth and SSL-auth clients on
the same port?




>2. SASL is a challenge-response protocol. Server generates a challenge,
>client responds. Fitting this into Kafka API is possible, but feels
>unnatural.
>

I dont see any problems with, the handshake is half-duplex and driven by
the SASL library, all the client needs
to do is "long poll" the server for data.

Something like this in pseudo code (where sasl. is the SASL library
context):
   until sasl.auth_done() {
 sendbuf = sasl.wants_to_send()
 if (!sendbuf) // Nothing to send, send empty payload to poll
server for data
 response = SaslHandshakeRequest(payload=none)
 else
response = SaslHandshakeRequest(payload=sendbuf)

// handle response (possibly empty) from server and continue
handshake.
 sasl.parse(response)
}


   3. Personally, I like the current Network|Security|Application layering
>of the code in Kafka. Handshake code is quite complicated and hard to
> debug
>as it is, moving that to the application layer could make it worse. With
>handshake in the application layer, you would need to set Principal
> based
>on SSL channel for non-SASL and sometime later on for SASL. It may be
> just
>me, but it doesn't feel quite right.
>

I actually think reusing the existing Kafka protocol implementation for
this makes it
more robust and easier to debug and understand.

How do you plan on handling feature discovery for new requests or request
versions in this special protocol?
I.e., what KIP-35 provides for the Kafka protocol.


Regards,
Magnus


>
> On Wed, Apr 6, 2016 at 12:47 PM, Magnus Edenhill 
> wrote:
>
> > Engineers,
> >
> > sorry to stir up some dust on this KIP but following yesterday's
> discussion
> > on the KIP call I'm a bit concerned about the way the SASL handshake
> > protocol is diverging even more from the main Kafka protocol.
> > The SASL handshake is already its own special protocol and the latest
> > addition of SaslMechanisms request in KIP-43
> > adds yet another protocol on top of that, making assumptions on what SASL
> > libraries are communicating in the opaque data channel.
> > Unlike TLS, SASL has no standard handshake protocol so conceiving a
> special
> > non-Kafka protocol just for SASL in Kafka is unwarranted given that the
> > Kafka protocol natively supports the same semantics.
> > There's also a non-neglectable maintenance aspect of maintaining a
> separate
> > protocol, in regards to documentation, versioning, testing, protocol
> stack
> > implementations (broker and clients * N), etc.
> >
> >
> >
> > So let me suggest a different approach that pulls the SASL handshake into
> > the Kafka protocol properly and makes the
> > authentication an in-band connection setup step.
> >
> >
> > Create two new proper Kafka requests:
> >
> > SaslMechanismsRequest V0 {
> >   // empty (see discussion below)
> > }
> >
> > SaslMechanismResponse V0 {
> >  int16 errroCode; // e.g. AuthenticationNotRequired (when SASL is
> > not configured) - or maybe just return an empty list in that case
> > Array[String] mechanisms;   // server's supported mechanisms. Client
> > may choose which one to use.
> > }
> >
> >
> >  SaslHandshakeRequest V0 {
> >  string selected_mechanism;  // client's selected mechanism, MUST not
> > change during a handshake (ignore in sub-sequent requests?)
> >  bytes data;// opaque SASL library handshake data.
> >  }
> >
> > SaslHandshakeResponse V0 {
> > int16 errorCode;   // e.g. UnsupportedMechanism, and arbitrary SASL
> > library exceptions, etc.
> > bytes data;   // opaque SASL library handshake data.
> > }
> >
> > (Note: I'll leave the exact format of the above Request/Responses to the
> > people in the know, this is a proof of concept layout.)
> >
> > The protocol semantics of the current SASL handshake, which is driven by
> > the underlying SASL libraries,  remains in place,
> > such as sending zero length requests to "poll" the remote side, etc.
> >
> >
> >
> > Regarding empty SaslMechanismRequest:
> > A previous proposal suggests sending the client's configured mechanism in
> > the SaslMechanismRequest, which is fine 

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-04-06 Thread Rajini Sivaram
Magnus,

I have not looked at your proposal in detail yet, but I have a few comments:


   1. We need to support SSL and SASL_SSL within the same broker (for
   instance, to support certificate-based authentication for replication and
   SASL for external connections). At the moment, there is a one-to-one
   mapping between port and security protocol. If SASL is a Kafka-level option
   on the same port rather than a security protocol, this would be harder to
   achieve. With separate protocols, we are also able to turn off SSL client
   auth for SASL.
   2. SASL is a challenge-response protocol. Server generates a challenge,
   client responds. Fitting this into Kafka API is possible, but feels
   unnatural.
   3. Personally, I like the current Network|Security|Application layering
   of the code in Kafka. Handshake code is quite complicated and hard to debug
   as it is, moving that to the application layer could make it worse. With
   handshake in the application layer, you would need to set Principal based
   on SSL channel for non-SASL and sometime later on for SASL. It may be just
   me, but it doesn't feel quite right.


On Wed, Apr 6, 2016 at 12:47 PM, Magnus Edenhill  wrote:

> Engineers,
>
> sorry to stir up some dust on this KIP but following yesterday's discussion
> on the KIP call I'm a bit concerned about the way the SASL handshake
> protocol is diverging even more from the main Kafka protocol.
> The SASL handshake is already its own special protocol and the latest
> addition of SaslMechanisms request in KIP-43
> adds yet another protocol on top of that, making assumptions on what SASL
> libraries are communicating in the opaque data channel.
> Unlike TLS, SASL has no standard handshake protocol so conceiving a special
> non-Kafka protocol just for SASL in Kafka is unwarranted given that the
> Kafka protocol natively supports the same semantics.
> There's also a non-neglectable maintenance aspect of maintaining a separate
> protocol, in regards to documentation, versioning, testing, protocol stack
> implementations (broker and clients * N), etc.
>
>
>
> So let me suggest a different approach that pulls the SASL handshake into
> the Kafka protocol properly and makes the
> authentication an in-band connection setup step.
>
>
> Create two new proper Kafka requests:
>
> SaslMechanismsRequest V0 {
>   // empty (see discussion below)
> }
>
> SaslMechanismResponse V0 {
>  int16 errroCode; // e.g. AuthenticationNotRequired (when SASL is
> not configured) - or maybe just return an empty list in that case
> Array[String] mechanisms;   // server's supported mechanisms. Client
> may choose which one to use.
> }
>
>
>  SaslHandshakeRequest V0 {
>  string selected_mechanism;  // client's selected mechanism, MUST not
> change during a handshake (ignore in sub-sequent requests?)
>  bytes data;// opaque SASL library handshake data.
>  }
>
> SaslHandshakeResponse V0 {
> int16 errorCode;   // e.g. UnsupportedMechanism, and arbitrary SASL
> library exceptions, etc.
> bytes data;   // opaque SASL library handshake data.
> }
>
> (Note: I'll leave the exact format of the above Request/Responses to the
> people in the know, this is a proof of concept layout.)
>
> The protocol semantics of the current SASL handshake, which is driven by
> the underlying SASL libraries,  remains in place,
> such as sending zero length requests to "poll" the remote side, etc.
>
>
>
> Regarding empty SaslMechanismRequest:
> A previous proposal suggests sending the client's configured mechanism in
> the SaslMechanismRequest, which is fine but if we do that there  is no
> point in returning the broker's list of mechanisms since the client has
> already chosen - only the error code is needed.
>  For clients wanting to support multiple mechanisms it can iterate over its
> configured mechanisms and send a  SaslMechanismRequest for each one, but
> that just adds complexity at no gain, and at the end of the day the
> desired_mechanism check in the broker is purely cosmetic.
>
>  If we instead skip the desired mechanism and let the broker always return
> its list of supported mechanisms the client can choose the one it needs,
> this allows a client to optionally support multiple mechanisms and also
> puts the decision logic in the client rather than the server.
>
>  If the argument is that a client will only support one mechanism (which I
> think is a bold claim to make) then the SaslMechanismRequest isn't needed
> at all, the client can jump directly SaslHandshakeRequest {
> selected_mechanism="mymechanism" } and look for UnsupportedMechanism in the
> response's errorCode.
>
>
>
>
>
> Typical client MO using the above requests:
>
> # Use ApiVersion info to select the proper SaslMechanismRequest version to
> use (for future use..)
>  Client > Broker:   ApiVersionRequest  // (optional) query for
> supported API versions, possibly limited to
> {SaslHandshakeReq,SaslMechanismsReq..}
>  Broker < Client:   ApiVersio

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-04-06 Thread Magnus Edenhill
Engineers,

sorry to stir up some dust on this KIP but following yesterday's discussion
on the KIP call I'm a bit concerned about the way the SASL handshake
protocol is diverging even more from the main Kafka protocol.
The SASL handshake is already its own special protocol and the latest
addition of SaslMechanisms request in KIP-43
adds yet another protocol on top of that, making assumptions on what SASL
libraries are communicating in the opaque data channel.
Unlike TLS, SASL has no standard handshake protocol so conceiving a special
non-Kafka protocol just for SASL in Kafka is unwarranted given that the
Kafka protocol natively supports the same semantics.
There's also a non-neglectable maintenance aspect of maintaining a separate
protocol, in regards to documentation, versioning, testing, protocol stack
implementations (broker and clients * N), etc.



So let me suggest a different approach that pulls the SASL handshake into
the Kafka protocol properly and makes the
authentication an in-band connection setup step.


Create two new proper Kafka requests:

SaslMechanismsRequest V0 {
  // empty (see discussion below)
}

SaslMechanismResponse V0 {
 int16 errroCode; // e.g. AuthenticationNotRequired (when SASL is
not configured) - or maybe just return an empty list in that case
Array[String] mechanisms;   // server's supported mechanisms. Client
may choose which one to use.
}


 SaslHandshakeRequest V0 {
 string selected_mechanism;  // client's selected mechanism, MUST not
change during a handshake (ignore in sub-sequent requests?)
 bytes data;// opaque SASL library handshake data.
 }

SaslHandshakeResponse V0 {
int16 errorCode;   // e.g. UnsupportedMechanism, and arbitrary SASL
library exceptions, etc.
bytes data;   // opaque SASL library handshake data.
}

(Note: I'll leave the exact format of the above Request/Responses to the
people in the know, this is a proof of concept layout.)

The protocol semantics of the current SASL handshake, which is driven by
the underlying SASL libraries,  remains in place,
such as sending zero length requests to "poll" the remote side, etc.



Regarding empty SaslMechanismRequest:
A previous proposal suggests sending the client's configured mechanism in
the SaslMechanismRequest, which is fine but if we do that there  is no
point in returning the broker's list of mechanisms since the client has
already chosen - only the error code is needed.
 For clients wanting to support multiple mechanisms it can iterate over its
configured mechanisms and send a  SaslMechanismRequest for each one, but
that just adds complexity at no gain, and at the end of the day the
desired_mechanism check in the broker is purely cosmetic.

 If we instead skip the desired mechanism and let the broker always return
its list of supported mechanisms the client can choose the one it needs,
this allows a client to optionally support multiple mechanisms and also
puts the decision logic in the client rather than the server.

 If the argument is that a client will only support one mechanism (which I
think is a bold claim to make) then the SaslMechanismRequest isn't needed
at all, the client can jump directly SaslHandshakeRequest {
selected_mechanism="mymechanism" } and look for UnsupportedMechanism in the
response's errorCode.





Typical client MO using the above requests:

# Use ApiVersion info to select the proper SaslMechanismRequest version to
use (for future use..)
 Client > Broker:   ApiVersionRequest  // (optional) query for
supported API versions, possibly limited to
{SaslHandshakeReq,SaslMechanismsReq..}
 Broker < Client:   ApiVersionResponse { .. }


# Request broker's supported mechanisms list
 Client > Broker:  SaslMechanismRequest { }
 Broker < Client:  SaslMechanismResponse { mechanisms = { "GSSAPI", "KRB",
.. }

# If broker responded with an empty mechanisms list, alternative an
AuthenticationNotRequired error code, the client skips to its operational
phase.
# If the client can't find a supported mechanism in the returned mechanisms
list it raises an error to the application.

# Otherwise the client selects a proper mechanism and the auth handshake
begins:
 Client > Broker: SaslHandshakeRequest { mechanism="KRB", data =
KRB-specific data.. }
 Broker > Client: SaslHandshakeResponse { data = KRB-specific data }
repeat as necessary

# Client is now authenticated and may start its usual operation of
MetadataRequest, Produce, Fetch, etc...



Pros:
 + Using the standard Kafka protocol provides proper versioning,
documentation, feature detection, etc, out of the box.
 + Less complex protcol implementations in both client and brokers.
Standard protocol stack can be used.
 + No need to define a specific SASL port, SASL is now an in-band operation
on any type of connection (PLAINTEXT, SSL)
 + Requires proper per-API auth enforcement in the broker, which is a good
thing. (i.e., if SASL is configured an unauthenticated client should not be
permitted to call any API but

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-03-15 Thread Rajini Sivaram
Both the KIP and the PR have been updated to a cut-down version as
discussed in the KIP meeting today.

Any feedback is appreciated.

On Tue, Mar 15, 2016 at 7:39 PM, Rajini Sivaram <
rajinisiva...@googlemail.com> wrote:

> Following on from the discussions in the KIP meeting today, the suggestion
> is to implement a cut-down version of KIP-43 for 0.10.0.0 with a follow-on
> KIP after the release to address support for custom mechanisms.
>
> Changes to be removed from KIP-43:
>
>1. Remove the configuration for CallbackHandler. The callback handler
>implementation in Kafka will support Kerberos, PLAIN and Digest-MD5. It
>will not support custom or more complex mechanisms which require additional
>callbacks.
>2. Remove the configuration for Login. The Login implementation in
>Kafka will support Kerberos and any other mechanism (PLAIN, Digest-MD5 etc)
>that doesn't require functionality like token refresh.
>
> Changes included in KIP-43:
>
>1. Configurable mechanism
>2. Support for multiple mechanisms in the broker
>3. Implementation of SASL/PLAIN
>
> If there are no objections to this, I can update the KIP and the PR by
> tomorrow. And move the support for custom mechanisms into another KIP and
> PR for review after the release of 0.10.0.0.
>
>
> On Mon, Mar 14, 2016 at 7:48 AM, Rajini Sivaram <
> rajinisiva...@googlemail.com> wrote:
>
>> Harsha,
>>
>> You are right, we don't expect to override callback handler or login for
>> Digest-MD5.
>>
>> Pluggable CallbackHandler and Login modules enable custom SASL mechanisms
>> to be implemented without modifying Kafka. For instance, it would enable
>> KIP-44 (
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-44+-+Allow+Kafka+to+have+a+customized+security+protocol)
>> to be implemented without making the whole security protocol pluggable. Tao
>> Xiao has already confirmed earlier in this discussion thread that the
>> proposed callback handler and login interfaces are suitable for their
>> custom authentication.
>>
>>
>>
>> On Sun, Mar 13, 2016 at 6:59 PM, Harsha  wrote:
>>
>>> Agree with Gwen here. I feel like these additional pluggable Login
>>> Modules are making this KIP complex. Since the main goal of the KIP is
>>> to enable additional mechanism , can we limit the scope to that and If
>>> we feel necessary for pluggable Login and callback handler classes we
>>> can address in another JIRA.
>>>
>>> Adding digest-md5 ,password callbacks can be done to existing
>>> callbackhandler without  expose it as pluggable class. It would be
>>> useful to have broker support multiple mechanisms.  I haven't seen
>>> anyone using more than this in hadoop . It might be different for Kafka
>>> but I personally haven't seen anyone asking for this yet.
>>>
>>> Thanks,
>>> Harsha
>>>
>>>
>>> On Thu, Mar 10, 2016, at 01:44 AM, Rajini Sivaram wrote:
>>> > Gwen,
>>> >
>>> > Just to be clear, the alternative would be:
>>> >
>>> > *jaas.conf:*
>>> >
>>> > GssapiKafkaServer {
>>> >
>>> > com.ibm.security.auth.module.Krb5LoginModule required
>>> > credsType=both
>>> > useKeytab="file:/kafka/key.tab"
>>> > principal="kafka/localh...@example.com ";
>>> >
>>> > };
>>> >
>>> > SmartcardKafkaServer {
>>> >
>>> >   example.SmartcardLoginModule required
>>> >
>>> >   cardNumber=123;
>>> >
>>> > };
>>> >
>>> >
>>> > *KafkaConfig*
>>> >
>>> >
>>> >
>>> >- login.context.map={"GSSAPI="GssapiKafkaServer",
>>> >   "SMARTCARD"=SmartcardKafkaServer}
>>> >   - login.class.map={"GSSAPI=GssapiLogin.class,
>>> >   "SMARTCARD"=SmartcardLogin.class}
>>> >   -
>>> callback.handler.class.map={"GSSAPI"=GssapiCallbackHandler.class,
>>> >   "SMARTCARD"=SmartcardCallbackHandler.class}
>>> >
>>> > *Client Config *
>>> > Same as the server, but with only one entry allowed in each map and
>>> > jaas.conf
>>> >
>>> >
>>> >
>>> > This is a different model from the Java standard for supporting
>>> multiple
>>> > logins. As a developer, I am inclined to stick with approaches that are
>>> > widely in use like JSSE. But this alternative can be made to work if
>>> the
>>> > Kafka community feels it is more appropriate for Kafka. If you know of
>>> > other systems which use this approach, that would be helpful.
>>> >
>>> >
>>> >
>>> > On Thu, Mar 10, 2016 at 2:07 AM, Gwen Shapira 
>>> wrote:
>>> >
>>> > > What I'm hearing is that:
>>> > >
>>> > > 1. In order to support authentication mechanisms that were not
>>> written
>>> > > specifically with Kafka in mind, someone will need to write the
>>> > > integration between the mechanism and Kafka. This may include Login
>>> > > and CallbackHandler classes. This can be the mechanism vendor, the
>>> > > user or a 3rd party vendor.
>>> > > 2. If someone wrote the code to support a mechanism in Kafka, and a
>>> > > user will want to use more than one mechanism, they will still need
>>> to
>>> > > write a wrapper.
>>> > > 3. In reality, #2 will not be necessary ("edge-case") because Kafka
>>> >

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-03-15 Thread Rajini Sivaram
Following on from the discussions in the KIP meeting today, the suggestion
is to implement a cut-down version of KIP-43 for 0.10.0.0 with a follow-on
KIP after the release to address support for custom mechanisms.

Changes to be removed from KIP-43:

   1. Remove the configuration for CallbackHandler. The callback handler
   implementation in Kafka will support Kerberos, PLAIN and Digest-MD5. It
   will not support custom or more complex mechanisms which require additional
   callbacks.
   2. Remove the configuration for Login. The Login implementation in Kafka
   will support Kerberos and any other mechanism (PLAIN, Digest-MD5 etc) that
   doesn't require functionality like token refresh.

Changes included in KIP-43:

   1. Configurable mechanism
   2. Support for multiple mechanisms in the broker
   3. Implementation of SASL/PLAIN

If there are no objections to this, I can update the KIP and the PR by
tomorrow. And move the support for custom mechanisms into another KIP and
PR for review after the release of 0.10.0.0.


On Mon, Mar 14, 2016 at 7:48 AM, Rajini Sivaram <
rajinisiva...@googlemail.com> wrote:

> Harsha,
>
> You are right, we don't expect to override callback handler or login for
> Digest-MD5.
>
> Pluggable CallbackHandler and Login modules enable custom SASL mechanisms
> to be implemented without modifying Kafka. For instance, it would enable
> KIP-44 (
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-44+-+Allow+Kafka+to+have+a+customized+security+protocol)
> to be implemented without making the whole security protocol pluggable. Tao
> Xiao has already confirmed earlier in this discussion thread that the
> proposed callback handler and login interfaces are suitable for their
> custom authentication.
>
>
>
> On Sun, Mar 13, 2016 at 6:59 PM, Harsha  wrote:
>
>> Agree with Gwen here. I feel like these additional pluggable Login
>> Modules are making this KIP complex. Since the main goal of the KIP is
>> to enable additional mechanism , can we limit the scope to that and If
>> we feel necessary for pluggable Login and callback handler classes we
>> can address in another JIRA.
>>
>> Adding digest-md5 ,password callbacks can be done to existing
>> callbackhandler without  expose it as pluggable class. It would be
>> useful to have broker support multiple mechanisms.  I haven't seen
>> anyone using more than this in hadoop . It might be different for Kafka
>> but I personally haven't seen anyone asking for this yet.
>>
>> Thanks,
>> Harsha
>>
>>
>> On Thu, Mar 10, 2016, at 01:44 AM, Rajini Sivaram wrote:
>> > Gwen,
>> >
>> > Just to be clear, the alternative would be:
>> >
>> > *jaas.conf:*
>> >
>> > GssapiKafkaServer {
>> >
>> > com.ibm.security.auth.module.Krb5LoginModule required
>> > credsType=both
>> > useKeytab="file:/kafka/key.tab"
>> > principal="kafka/localh...@example.com ";
>> >
>> > };
>> >
>> > SmartcardKafkaServer {
>> >
>> >   example.SmartcardLoginModule required
>> >
>> >   cardNumber=123;
>> >
>> > };
>> >
>> >
>> > *KafkaConfig*
>> >
>> >
>> >
>> >- login.context.map={"GSSAPI="GssapiKafkaServer",
>> >   "SMARTCARD"=SmartcardKafkaServer}
>> >   - login.class.map={"GSSAPI=GssapiLogin.class,
>> >   "SMARTCARD"=SmartcardLogin.class}
>> >   -
>> callback.handler.class.map={"GSSAPI"=GssapiCallbackHandler.class,
>> >   "SMARTCARD"=SmartcardCallbackHandler.class}
>> >
>> > *Client Config *
>> > Same as the server, but with only one entry allowed in each map and
>> > jaas.conf
>> >
>> >
>> >
>> > This is a different model from the Java standard for supporting multiple
>> > logins. As a developer, I am inclined to stick with approaches that are
>> > widely in use like JSSE. But this alternative can be made to work if the
>> > Kafka community feels it is more appropriate for Kafka. If you know of
>> > other systems which use this approach, that would be helpful.
>> >
>> >
>> >
>> > On Thu, Mar 10, 2016 at 2:07 AM, Gwen Shapira 
>> wrote:
>> >
>> > > What I'm hearing is that:
>> > >
>> > > 1. In order to support authentication mechanisms that were not written
>> > > specifically with Kafka in mind, someone will need to write the
>> > > integration between the mechanism and Kafka. This may include Login
>> > > and CallbackHandler classes. This can be the mechanism vendor, the
>> > > user or a 3rd party vendor.
>> > > 2. If someone wrote the code to support a mechanism in Kafka, and a
>> > > user will want to use more than one mechanism, they will still need to
>> > > write a wrapper.
>> > > 3. In reality, #2 will not be necessary ("edge-case") because Kafka
>> > > will actually already provide the callback needed (and presumably also
>> > > the code to load the LoginModule provided by Example.com)?
>> > >
>> > > Tradeoff #1 sounds reasonable.
>> > > #2 and #3 do not sound reasonable considering one of the goals of the
>> > > patch is to support multiple mechanisms. I don't think we should force
>> > > our users to write code just to avoid w

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-03-14 Thread Rajini Sivaram
Harsha,

You are right, we don't expect to override callback handler or login for
Digest-MD5.

Pluggable CallbackHandler and Login modules enable custom SASL mechanisms
to be implemented without modifying Kafka. For instance, it would enable
KIP-44 (
https://cwiki.apache.org/confluence/display/KAFKA/KIP-44+-+Allow+Kafka+to+have+a+customized+security+protocol)
to be implemented without making the whole security protocol pluggable. Tao
Xiao has already confirmed earlier in this discussion thread that the
proposed callback handler and login interfaces are suitable for their
custom authentication.



On Sun, Mar 13, 2016 at 6:59 PM, Harsha  wrote:

> Agree with Gwen here. I feel like these additional pluggable Login
> Modules are making this KIP complex. Since the main goal of the KIP is
> to enable additional mechanism , can we limit the scope to that and If
> we feel necessary for pluggable Login and callback handler classes we
> can address in another JIRA.
>
> Adding digest-md5 ,password callbacks can be done to existing
> callbackhandler without  expose it as pluggable class. It would be
> useful to have broker support multiple mechanisms.  I haven't seen
> anyone using more than this in hadoop . It might be different for Kafka
> but I personally haven't seen anyone asking for this yet.
>
> Thanks,
> Harsha
>
>
> On Thu, Mar 10, 2016, at 01:44 AM, Rajini Sivaram wrote:
> > Gwen,
> >
> > Just to be clear, the alternative would be:
> >
> > *jaas.conf:*
> >
> > GssapiKafkaServer {
> >
> > com.ibm.security.auth.module.Krb5LoginModule required
> > credsType=both
> > useKeytab="file:/kafka/key.tab"
> > principal="kafka/localh...@example.com ";
> >
> > };
> >
> > SmartcardKafkaServer {
> >
> >   example.SmartcardLoginModule required
> >
> >   cardNumber=123;
> >
> > };
> >
> >
> > *KafkaConfig*
> >
> >
> >
> >- login.context.map={"GSSAPI="GssapiKafkaServer",
> >   "SMARTCARD"=SmartcardKafkaServer}
> >   - login.class.map={"GSSAPI=GssapiLogin.class,
> >   "SMARTCARD"=SmartcardLogin.class}
> >   - callback.handler.class.map={"GSSAPI"=GssapiCallbackHandler.class,
> >   "SMARTCARD"=SmartcardCallbackHandler.class}
> >
> > *Client Config *
> > Same as the server, but with only one entry allowed in each map and
> > jaas.conf
> >
> >
> >
> > This is a different model from the Java standard for supporting multiple
> > logins. As a developer, I am inclined to stick with approaches that are
> > widely in use like JSSE. But this alternative can be made to work if the
> > Kafka community feels it is more appropriate for Kafka. If you know of
> > other systems which use this approach, that would be helpful.
> >
> >
> >
> > On Thu, Mar 10, 2016 at 2:07 AM, Gwen Shapira  wrote:
> >
> > > What I'm hearing is that:
> > >
> > > 1. In order to support authentication mechanisms that were not written
> > > specifically with Kafka in mind, someone will need to write the
> > > integration between the mechanism and Kafka. This may include Login
> > > and CallbackHandler classes. This can be the mechanism vendor, the
> > > user or a 3rd party vendor.
> > > 2. If someone wrote the code to support a mechanism in Kafka, and a
> > > user will want to use more than one mechanism, they will still need to
> > > write a wrapper.
> > > 3. In reality, #2 will not be necessary ("edge-case") because Kafka
> > > will actually already provide the callback needed (and presumably also
> > > the code to load the LoginModule provided by Example.com)?
> > >
> > > Tradeoff #1 sounds reasonable.
> > > #2 and #3 do not sound reasonable considering one of the goals of the
> > > patch is to support multiple mechanisms. I don't think we should force
> > > our users to write code just to avoid writing it ourselves.
> > > Configuring security is complex enough as is.
> > > Furthermore, if we believe that "Smartcard is likely to use standard
> > > NameCallback and PasswordCallback already implemented in Kafka" - why
> > > do we even provide configuration for Login and CallbackHandler
> > > classes? Either we support multiple mechanisms written by different
> > > vendors, or we don't.
> > >
> > > Gwen
> > >
> > >
> > > On Wed, Mar 9, 2016 at 12:32 AM, Rajini Sivaram
> > >  wrote:
> > > > I am not saying that the developer at Example Inc. would develop a
> Login
> > > > implementation that combines Smartcard and Kerberos because Retailer
> uses
> > > > both. I am saying that Example Inc develops the LoginModule (similar
> to
> > > JVM
> > > > security providers developing Kerberos modules). But there is no
> standard
> > > > interface for Login to allow ticket refresh. So, it is very unlikely
> that
> > > > Example Inc would develop a Login implementation that works with an
> > > Apache
> > > > Kafka defined interface ( Kafka developers wrote this code for
> Kerberos).
> > > > For a custom integration, the user (i.e. Retailer) would be expected
> to
> > > > develop this code if required.
> > > >
> > > > You could imagine t

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-03-13 Thread Harsha
Agree with Gwen here. I feel like these additional pluggable Login
Modules are making this KIP complex. Since the main goal of the KIP is
to enable additional mechanism , can we limit the scope to that and If
we feel necessary for pluggable Login and callback handler classes we
can address in another JIRA.

Adding digest-md5 ,password callbacks can be done to existing
callbackhandler without  expose it as pluggable class. It would be
useful to have broker support multiple mechanisms.  I haven't seen
anyone using more than this in hadoop . It might be different for Kafka
but I personally haven't seen anyone asking for this yet. 

Thanks,
Harsha


On Thu, Mar 10, 2016, at 01:44 AM, Rajini Sivaram wrote:
> Gwen,
> 
> Just to be clear, the alternative would be:
> 
> *jaas.conf:*
> 
> GssapiKafkaServer {
> 
> com.ibm.security.auth.module.Krb5LoginModule required
> credsType=both
> useKeytab="file:/kafka/key.tab"
> principal="kafka/localh...@example.com ";
> 
> };
> 
> SmartcardKafkaServer {
> 
>   example.SmartcardLoginModule required
> 
>   cardNumber=123;
> 
> };
> 
> 
> *KafkaConfig*
> 
> 
> 
>- login.context.map={"GSSAPI="GssapiKafkaServer",
>   "SMARTCARD"=SmartcardKafkaServer}
>   - login.class.map={"GSSAPI=GssapiLogin.class,
>   "SMARTCARD"=SmartcardLogin.class}
>   - callback.handler.class.map={"GSSAPI"=GssapiCallbackHandler.class,
>   "SMARTCARD"=SmartcardCallbackHandler.class}
> 
> *Client Config *
> Same as the server, but with only one entry allowed in each map and
> jaas.conf
> 
> 
> 
> This is a different model from the Java standard for supporting multiple
> logins. As a developer, I am inclined to stick with approaches that are
> widely in use like JSSE. But this alternative can be made to work if the
> Kafka community feels it is more appropriate for Kafka. If you know of
> other systems which use this approach, that would be helpful.
> 
> 
> 
> On Thu, Mar 10, 2016 at 2:07 AM, Gwen Shapira  wrote:
> 
> > What I'm hearing is that:
> >
> > 1. In order to support authentication mechanisms that were not written
> > specifically with Kafka in mind, someone will need to write the
> > integration between the mechanism and Kafka. This may include Login
> > and CallbackHandler classes. This can be the mechanism vendor, the
> > user or a 3rd party vendor.
> > 2. If someone wrote the code to support a mechanism in Kafka, and a
> > user will want to use more than one mechanism, they will still need to
> > write a wrapper.
> > 3. In reality, #2 will not be necessary ("edge-case") because Kafka
> > will actually already provide the callback needed (and presumably also
> > the code to load the LoginModule provided by Example.com)?
> >
> > Tradeoff #1 sounds reasonable.
> > #2 and #3 do not sound reasonable considering one of the goals of the
> > patch is to support multiple mechanisms. I don't think we should force
> > our users to write code just to avoid writing it ourselves.
> > Configuring security is complex enough as is.
> > Furthermore, if we believe that "Smartcard is likely to use standard
> > NameCallback and PasswordCallback already implemented in Kafka" - why
> > do we even provide configuration for Login and CallbackHandler
> > classes? Either we support multiple mechanisms written by different
> > vendors, or we don't.
> >
> > Gwen
> >
> >
> > On Wed, Mar 9, 2016 at 12:32 AM, Rajini Sivaram
> >  wrote:
> > > I am not saying that the developer at Example Inc. would develop a Login
> > > implementation that combines Smartcard and Kerberos because Retailer uses
> > > both. I am saying that Example Inc develops the LoginModule (similar to
> > JVM
> > > security providers developing Kerberos modules). But there is no standard
> > > interface for Login to allow ticket refresh. So, it is very unlikely that
> > > Example Inc would develop a Login implementation that works with an
> > Apache
> > > Kafka defined interface ( Kafka developers wrote this code for Kerberos).
> > > For a custom integration, the user (i.e. Retailer) would be expected to
> > > develop this code if required.
> > >
> > > You could imagine that Smartcard is a commonly used mechanism and a 3rd
> > > party develops code for integrating Smartcard with Kafka and makes the
> > > integration code (Login and CallbackHandler implementation) widely
> > > available, If Retailer wants to use clients or a broker with just
> > Smartcard
> > > enabled in their broker, they configure Kafka to use the 3rd party code,
> > > with no additional code development. But to combine Smartcard and
> > Kerberos,
> > > Retailer needs to write a few lines of code to incorporate both Smartcard
> > > and Kerberos. I believe this is an edge case.
> > >
> > > Smartcard is likely to use standard NameCallback and PasswordCallback
> > > already implemented in Kafka and Kerberos support exists in Kafka. So it
> > is
> > > very likely that Retailer doesn't need to override Login or
> > CallbackHandler
> > > in this case. An

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-03-10 Thread Rajini Sivaram
Gwen,

Just to be clear, the alternative would be:

*jaas.conf:*

GssapiKafkaServer {

com.ibm.security.auth.module.Krb5LoginModule required
credsType=both
useKeytab="file:/kafka/key.tab"
principal="kafka/localh...@example.com ";

};

SmartcardKafkaServer {

  example.SmartcardLoginModule required

  cardNumber=123;

};


*KafkaConfig*



   - login.context.map={"GSSAPI="GssapiKafkaServer",
  "SMARTCARD"=SmartcardKafkaServer}
  - login.class.map={"GSSAPI=GssapiLogin.class,
  "SMARTCARD"=SmartcardLogin.class}
  - callback.handler.class.map={"GSSAPI"=GssapiCallbackHandler.class,
  "SMARTCARD"=SmartcardCallbackHandler.class}

*Client Config *
Same as the server, but with only one entry allowed in each map and
jaas.conf



This is a different model from the Java standard for supporting multiple
logins. As a developer, I am inclined to stick with approaches that are
widely in use like JSSE. But this alternative can be made to work if the
Kafka community feels it is more appropriate for Kafka. If you know of
other systems which use this approach, that would be helpful.



On Thu, Mar 10, 2016 at 2:07 AM, Gwen Shapira  wrote:

> What I'm hearing is that:
>
> 1. In order to support authentication mechanisms that were not written
> specifically with Kafka in mind, someone will need to write the
> integration between the mechanism and Kafka. This may include Login
> and CallbackHandler classes. This can be the mechanism vendor, the
> user or a 3rd party vendor.
> 2. If someone wrote the code to support a mechanism in Kafka, and a
> user will want to use more than one mechanism, they will still need to
> write a wrapper.
> 3. In reality, #2 will not be necessary ("edge-case") because Kafka
> will actually already provide the callback needed (and presumably also
> the code to load the LoginModule provided by Example.com)?
>
> Tradeoff #1 sounds reasonable.
> #2 and #3 do not sound reasonable considering one of the goals of the
> patch is to support multiple mechanisms. I don't think we should force
> our users to write code just to avoid writing it ourselves.
> Configuring security is complex enough as is.
> Furthermore, if we believe that "Smartcard is likely to use standard
> NameCallback and PasswordCallback already implemented in Kafka" - why
> do we even provide configuration for Login and CallbackHandler
> classes? Either we support multiple mechanisms written by different
> vendors, or we don't.
>
> Gwen
>
>
> On Wed, Mar 9, 2016 at 12:32 AM, Rajini Sivaram
>  wrote:
> > I am not saying that the developer at Example Inc. would develop a Login
> > implementation that combines Smartcard and Kerberos because Retailer uses
> > both. I am saying that Example Inc develops the LoginModule (similar to
> JVM
> > security providers developing Kerberos modules). But there is no standard
> > interface for Login to allow ticket refresh. So, it is very unlikely that
> > Example Inc would develop a Login implementation that works with an
> Apache
> > Kafka defined interface ( Kafka developers wrote this code for Kerberos).
> > For a custom integration, the user (i.e. Retailer) would be expected to
> > develop this code if required.
> >
> > You could imagine that Smartcard is a commonly used mechanism and a 3rd
> > party develops code for integrating Smartcard with Kafka and makes the
> > integration code (Login and CallbackHandler implementation) widely
> > available, If Retailer wants to use clients or a broker with just
> Smartcard
> > enabled in their broker, they configure Kafka to use the 3rd party code,
> > with no additional code development. But to combine Smartcard and
> Kerberos,
> > Retailer needs to write a few lines of code to incorporate both Smartcard
> > and Kerberos. I believe this is an edge case.
> >
> > Smartcard is likely to use standard NameCallback and PasswordCallback
> > already implemented in Kafka and Kerberos support exists in Kafka. So it
> is
> > very likely that Retailer doesn't need to override Login or
> CallbackHandler
> > in this case. And it would just be a matter of configuring the
> mechanisms.
> >
> > On Wed, Mar 9, 2016 at 12:48 AM, Gwen Shapira  wrote:
> >
> >> "Since smart card logins are not built into Kafka (or the JDK), you
> need a
> >> developer to build the login module. So the developer implements
> >> example.SmartcardLoginModule. In addition, the developer may also
> implement
> >> callback handlers for the SASL client or server  and a login class to
> keep
> >> this login refreshed. The callback handlers and login implementation
> >> support all the mechanisms that the organisation supports - in this case
> >> Kerberos and smart card."
> >>
> >> In this case, the developer works for Example Inc (which develops
> >> SmartCard authentication modules), while I work for Retailer and need
> >> to use his module.
> >> You assume that developer from Example Inc knows about all the
> >> mechanisms that I have enabled in my Kafka cluster and h

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-03-09 Thread Gwen Shapira
What I'm hearing is that:

1. In order to support authentication mechanisms that were not written
specifically with Kafka in mind, someone will need to write the
integration between the mechanism and Kafka. This may include Login
and CallbackHandler classes. This can be the mechanism vendor, the
user or a 3rd party vendor.
2. If someone wrote the code to support a mechanism in Kafka, and a
user will want to use more than one mechanism, they will still need to
write a wrapper.
3. In reality, #2 will not be necessary ("edge-case") because Kafka
will actually already provide the callback needed (and presumably also
the code to load the LoginModule provided by Example.com)?

Tradeoff #1 sounds reasonable.
#2 and #3 do not sound reasonable considering one of the goals of the
patch is to support multiple mechanisms. I don't think we should force
our users to write code just to avoid writing it ourselves.
Configuring security is complex enough as is.
Furthermore, if we believe that "Smartcard is likely to use standard
NameCallback and PasswordCallback already implemented in Kafka" - why
do we even provide configuration for Login and CallbackHandler
classes? Either we support multiple mechanisms written by different
vendors, or we don't.

Gwen


On Wed, Mar 9, 2016 at 12:32 AM, Rajini Sivaram
 wrote:
> I am not saying that the developer at Example Inc. would develop a Login
> implementation that combines Smartcard and Kerberos because Retailer uses
> both. I am saying that Example Inc develops the LoginModule (similar to JVM
> security providers developing Kerberos modules). But there is no standard
> interface for Login to allow ticket refresh. So, it is very unlikely that
> Example Inc would develop a Login implementation that works with an Apache
> Kafka defined interface ( Kafka developers wrote this code for Kerberos).
> For a custom integration, the user (i.e. Retailer) would be expected to
> develop this code if required.
>
> You could imagine that Smartcard is a commonly used mechanism and a 3rd
> party develops code for integrating Smartcard with Kafka and makes the
> integration code (Login and CallbackHandler implementation) widely
> available, If Retailer wants to use clients or a broker with just Smartcard
> enabled in their broker, they configure Kafka to use the 3rd party code,
> with no additional code development. But to combine Smartcard and Kerberos,
> Retailer needs to write a few lines of code to incorporate both Smartcard
> and Kerberos. I believe this is an edge case.
>
> Smartcard is likely to use standard NameCallback and PasswordCallback
> already implemented in Kafka and Kerberos support exists in Kafka. So it is
> very likely that Retailer doesn't need to override Login or CallbackHandler
> in this case. And it would just be a matter of configuring the mechanisms.
>
> On Wed, Mar 9, 2016 at 12:48 AM, Gwen Shapira  wrote:
>
>> "Since smart card logins are not built into Kafka (or the JDK), you need a
>> developer to build the login module. So the developer implements
>> example.SmartcardLoginModule. In addition, the developer may also implement
>> callback handlers for the SASL client or server  and a login class to keep
>> this login refreshed. The callback handlers and login implementation
>> support all the mechanisms that the organisation supports - in this case
>> Kerberos and smart card."
>>
>> In this case, the developer works for Example Inc (which develops
>> SmartCard authentication modules), while I work for Retailer and need
>> to use his module.
>> You assume that developer from Example Inc knows about all the
>> mechanisms that I have enabled in my Kafka cluster and he is capable
>> of developing a Login class and maybe a callback handler that supports
>> all of them. I think this is an unreasonable expectation.
>>
>> I think that allowing plug in of Login method and single Callback
>> handler is not a good extensible solution for multiple mechanisms. We
>> should probably supply a Kafka-based LoginContext and CallbackHandler
>> that will be able to delegate to implementations of LoginModules and
>> Callback Handlers based on mechanism out of a list supplied by a user.
>> I agree it is more complex, but otherwise we are asking of too much
>> coordination from whoever implements CallbackHandlers and Login
>> methods.
>>
>> Gwen
>>
>> On Tue, Mar 8, 2016 at 3:35 PM, Rajini Sivaram
>>  wrote:
>> > As an admin, you would configure login modules using standard JAAS
>> > configuration. So, if you are supporting Kerberos and smart card login
>> > modules within a single broker, in your jaas.conf you would specify:
>> >
>> > KafkaServer {
>> >
>> > com.ibm.security.auth.module.Krb5LoginModule required
>> > credsType=both
>> > useKeytab="file:/kafka/key.tab"
>> > principal="kafka/localh...@example.com";
>> >
>> >   example.SmartcardLoginModule required
>> >
>> >   cardNumber=123
>> >
>> > };
>> >
>> >
>> > Since smart card logins are not built into Kafka (or the JDK), you need a
>> 

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-03-09 Thread Rajini Sivaram
I am not saying that the developer at Example Inc. would develop a Login
implementation that combines Smartcard and Kerberos because Retailer uses
both. I am saying that Example Inc develops the LoginModule (similar to JVM
security providers developing Kerberos modules). But there is no standard
interface for Login to allow ticket refresh. So, it is very unlikely that
Example Inc would develop a Login implementation that works with an Apache
Kafka defined interface ( Kafka developers wrote this code for Kerberos).
For a custom integration, the user (i.e. Retailer) would be expected to
develop this code if required.

You could imagine that Smartcard is a commonly used mechanism and a 3rd
party develops code for integrating Smartcard with Kafka and makes the
integration code (Login and CallbackHandler implementation) widely
available, If Retailer wants to use clients or a broker with just Smartcard
enabled in their broker, they configure Kafka to use the 3rd party code,
with no additional code development. But to combine Smartcard and Kerberos,
Retailer needs to write a few lines of code to incorporate both Smartcard
and Kerberos. I believe this is an edge case.

Smartcard is likely to use standard NameCallback and PasswordCallback
already implemented in Kafka and Kerberos support exists in Kafka. So it is
very likely that Retailer doesn't need to override Login or CallbackHandler
in this case. And it would just be a matter of configuring the mechanisms.

On Wed, Mar 9, 2016 at 12:48 AM, Gwen Shapira  wrote:

> "Since smart card logins are not built into Kafka (or the JDK), you need a
> developer to build the login module. So the developer implements
> example.SmartcardLoginModule. In addition, the developer may also implement
> callback handlers for the SASL client or server  and a login class to keep
> this login refreshed. The callback handlers and login implementation
> support all the mechanisms that the organisation supports - in this case
> Kerberos and smart card."
>
> In this case, the developer works for Example Inc (which develops
> SmartCard authentication modules), while I work for Retailer and need
> to use his module.
> You assume that developer from Example Inc knows about all the
> mechanisms that I have enabled in my Kafka cluster and he is capable
> of developing a Login class and maybe a callback handler that supports
> all of them. I think this is an unreasonable expectation.
>
> I think that allowing plug in of Login method and single Callback
> handler is not a good extensible solution for multiple mechanisms. We
> should probably supply a Kafka-based LoginContext and CallbackHandler
> that will be able to delegate to implementations of LoginModules and
> Callback Handlers based on mechanism out of a list supplied by a user.
> I agree it is more complex, but otherwise we are asking of too much
> coordination from whoever implements CallbackHandlers and Login
> methods.
>
> Gwen
>
> On Tue, Mar 8, 2016 at 3:35 PM, Rajini Sivaram
>  wrote:
> > As an admin, you would configure login modules using standard JAAS
> > configuration. So, if you are supporting Kerberos and smart card login
> > modules within a single broker, in your jaas.conf you would specify:
> >
> > KafkaServer {
> >
> > com.ibm.security.auth.module.Krb5LoginModule required
> > credsType=both
> > useKeytab="file:/kafka/key.tab"
> > principal="kafka/localh...@example.com";
> >
> >   example.SmartcardLoginModule required
> >
> >   cardNumber=123
> >
> > };
> >
> >
> > Since smart card logins are not built into Kafka (or the JDK), you need a
> > developer to build the login module. So the developer implements
> > example.SmartcardLoginModule. In addition, the developer may also
> implement
> > callback handlers for the SASL client or server  and a login class to
> keep
> > this login refreshed. The callback handlers and login implementation
> > support all the mechanisms that the organisation supports - in this case
> > Kerberos and smart card.
> >
> > As an admin, if you are using mechanisms built into Kafka, you simply
> > specify a JAAS config with the combination of mechanisms you want and you
> > are done. If you are adding new mechanisms, it clearly can't be done
> > without some additional code to implement that mechanism. The callback
> and
> > login interfaces are configurable to ensure that (as a developer), you
> can
> > build custom mechanisms without modifying Kafka. SASL was designed
> > precisely to do this. But at the moment, Kafka restricts what type of
> > authentication you can perform with SASL.
> >
> > Back to the questions:
> > 1. *How does the proposed Login class interact with LoginContext and *
> > *LoginModule.*
> > Login class performs *LoginContext.login(). *This performs login for all
> > the login modules of that context (this is standard Java function). Login
> > is cached with a reference counter to keep it alive.
> >
> > 2. *Do we really need a pluggable Login class, when users
> **(administrato

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-03-08 Thread Gwen Shapira
"Since smart card logins are not built into Kafka (or the JDK), you need a
developer to build the login module. So the developer implements
example.SmartcardLoginModule. In addition, the developer may also implement
callback handlers for the SASL client or server  and a login class to keep
this login refreshed. The callback handlers and login implementation
support all the mechanisms that the organisation supports - in this case
Kerberos and smart card."

In this case, the developer works for Example Inc (which develops
SmartCard authentication modules), while I work for Retailer and need
to use his module.
You assume that developer from Example Inc knows about all the
mechanisms that I have enabled in my Kafka cluster and he is capable
of developing a Login class and maybe a callback handler that supports
all of them. I think this is an unreasonable expectation.

I think that allowing plug in of Login method and single Callback
handler is not a good extensible solution for multiple mechanisms. We
should probably supply a Kafka-based LoginContext and CallbackHandler
that will be able to delegate to implementations of LoginModules and
Callback Handlers based on mechanism out of a list supplied by a user.
I agree it is more complex, but otherwise we are asking of too much
coordination from whoever implements CallbackHandlers and Login
methods.

Gwen

On Tue, Mar 8, 2016 at 3:35 PM, Rajini Sivaram
 wrote:
> As an admin, you would configure login modules using standard JAAS
> configuration. So, if you are supporting Kerberos and smart card login
> modules within a single broker, in your jaas.conf you would specify:
>
> KafkaServer {
>
> com.ibm.security.auth.module.Krb5LoginModule required
> credsType=both
> useKeytab="file:/kafka/key.tab"
> principal="kafka/localh...@example.com";
>
>   example.SmartcardLoginModule required
>
>   cardNumber=123
>
> };
>
>
> Since smart card logins are not built into Kafka (or the JDK), you need a
> developer to build the login module. So the developer implements
> example.SmartcardLoginModule. In addition, the developer may also implement
> callback handlers for the SASL client or server  and a login class to keep
> this login refreshed. The callback handlers and login implementation
> support all the mechanisms that the organisation supports - in this case
> Kerberos and smart card.
>
> As an admin, if you are using mechanisms built into Kafka, you simply
> specify a JAAS config with the combination of mechanisms you want and you
> are done. If you are adding new mechanisms, it clearly can't be done
> without some additional code to implement that mechanism. The callback and
> login interfaces are configurable to ensure that (as a developer), you can
> build custom mechanisms without modifying Kafka. SASL was designed
> precisely to do this. But at the moment, Kafka restricts what type of
> authentication you can perform with SASL.
>
> Back to the questions:
> 1. *How does the proposed Login class interact with LoginContext and *
> *LoginModule.*
> Login class performs *LoginContext.login(). *This performs login for all
> the login modules of that context (this is standard Java function). Login
> is cached with a reference counter to keep it alive.
>
> 2. *Do we really need a pluggable Login class, when users **(administrators,
> not developers) are more likely to want to specify LoginModules?*
> Administrators specify *LoginModules. *Developers implement new mechanisms.
> The pluggable *Login* class enables the developer to implement new
> mechanisms without changing Kafka. If Kafka didn't have a default Kerberos
> implementation and a user wanted to use Kerberos, pluggability of *Login* 
> class
> enables the user to include Kerberos without changing Kafka. Not that we
> want another Kerberos, but Tao had a requirement to enable a custom
> mechanism with functionality similar to Kerberos (see KIP-44). Pluggable
> *Login* supports this use case without replacing the entire authentication
> mechanism in Kafka.
>
> *3. How will users get to specify multiple LoginModules?*
> Standard Java JAAS configuration - see example above.
>
>
> *4. It sounds like different LoginModules may need
> different CallbackHandlers. Since we only allow a single
> pluggableCallbackHandler class in the KIP-43 configuration, how will this
> be handled?*
> One LoginContext, one subject, one callback handler, multiple login
> modules. This is the Java model for login. And it works well because
> callback handlers tend to be fairly standard. Even if the smart card SASL
> callbacks needed to do something wildly different and you wanted a broker
> that combined this with Kerberos, you would implement a callback that
> looked like the code below.
>
> private CallbackHandler handler;
>
> public void configure(Map configs, Mode mode, Subject
> subject, String mechanism) {public void configure(Map configs,
> Mode mode, Subject subject, String mechanism) {
>
> switch (mechanism) {

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-03-08 Thread Rajini Sivaram
As an admin, you would configure login modules using standard JAAS
configuration. So, if you are supporting Kerberos and smart card login
modules within a single broker, in your jaas.conf you would specify:

KafkaServer {

com.ibm.security.auth.module.Krb5LoginModule required
credsType=both
useKeytab="file:/kafka/key.tab"
principal="kafka/localh...@example.com";

  example.SmartcardLoginModule required

  cardNumber=123

};


Since smart card logins are not built into Kafka (or the JDK), you need a
developer to build the login module. So the developer implements
example.SmartcardLoginModule. In addition, the developer may also implement
callback handlers for the SASL client or server  and a login class to keep
this login refreshed. The callback handlers and login implementation
support all the mechanisms that the organisation supports - in this case
Kerberos and smart card.

As an admin, if you are using mechanisms built into Kafka, you simply
specify a JAAS config with the combination of mechanisms you want and you
are done. If you are adding new mechanisms, it clearly can't be done
without some additional code to implement that mechanism. The callback and
login interfaces are configurable to ensure that (as a developer), you can
build custom mechanisms without modifying Kafka. SASL was designed
precisely to do this. But at the moment, Kafka restricts what type of
authentication you can perform with SASL.

Back to the questions:
1. *How does the proposed Login class interact with LoginContext and *
*LoginModule.*
Login class performs *LoginContext.login(). *This performs login for all
the login modules of that context (this is standard Java function). Login
is cached with a reference counter to keep it alive.

2. *Do we really need a pluggable Login class, when users **(administrators,
not developers) are more likely to want to specify LoginModules?*
Administrators specify *LoginModules. *Developers implement new mechanisms.
The pluggable *Login* class enables the developer to implement new
mechanisms without changing Kafka. If Kafka didn't have a default Kerberos
implementation and a user wanted to use Kerberos, pluggability of *Login* class
enables the user to include Kerberos without changing Kafka. Not that we
want another Kerberos, but Tao had a requirement to enable a custom
mechanism with functionality similar to Kerberos (see KIP-44). Pluggable
*Login* supports this use case without replacing the entire authentication
mechanism in Kafka.

*3. How will users get to specify multiple LoginModules?*
Standard Java JAAS configuration - see example above.


*4. It sounds like different LoginModules may need
different CallbackHandlers. Since we only allow a single
pluggableCallbackHandler class in the KIP-43 configuration, how will this
be handled?*
One LoginContext, one subject, one callback handler, multiple login
modules. This is the Java model for login. And it works well because
callback handlers tend to be fairly standard. Even if the smart card SASL
callbacks needed to do something wildly different and you wanted a broker
that combined this with Kerberos, you would implement a callback that
looked like the code below.

private CallbackHandler handler;

public void configure(Map configs, Mode mode, Subject
subject, String mechanism) {public void configure(Map configs,
Mode mode, Subject subject, String mechanism) {

switch (mechanism) {

case "GSSAPI" : handler = new GssapiCallbackHandler(); break
;

case "SMARTCARD" : handler = new SmartcardCallbackHandler();
break;

}

}


public void handle(Callback[] callbacks) throws IOException,
UnsupportedCallbackException {

handler.handle(callbacks);

}

But I don't see this as just an admin configuration. This is some code
developed alongside a new mechanism. Yes, you can add a new configuration
type in Kafka that maps mechanism to callback handler class to avoid the
switch statement. But that doesn't remove the need for new code altogether
since you still need to implement SmartcardCallbackHandler. Multiple
conflicting callback handlers within one broker to support complete
different behaviour is an edge case. Since the design supports
full flexibility in terms of functionality and you have to write new code
for new mechanisms anyway, this feels like the simplest solution.



On Tue, Mar 8, 2016 at 8:23 PM, Gwen Shapira  wrote:

> I guess what is unclear to me is:
>
> 1. How does the proposed Login class interact with LoginContext and
> LoginModule.
> 2. Do we really need a pluggable Login class, when users
> (administrators, not developers) are more likely to want to specify
> LoginModules?
> 3. How will users get to specify multiple LoginModules?
> 4. It sounds like different LoginModules may need different
> CallbackHandlers. Since we only allow a single pluggable
> CallbackHandler class in the KIP-43 configuration, how will this be
> handled?
>

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-03-08 Thread Gwen Shapira
I guess what is unclear to me is:

1. How does the proposed Login class interact with LoginContext and
LoginModule.
2. Do we really need a pluggable Login class, when users
(administrators, not developers) are more likely to want to specify
LoginModules?
3. How will users get to specify multiple LoginModules?
4. It sounds like different LoginModules may need different
CallbackHandlers. Since we only allow a single pluggable
CallbackHandler class in the KIP-43 configuration, how will this be
handled?

In other words, suppose I want my Kafka installation to support both
Kerberos and SmartCard mechanisms, can you tell me how to configure
Kafka? (as admin, not developer)?

Gwen



On Tue, Mar 8, 2016 at 11:46 AM, Rajini Sivaram
 wrote:
> Gwen,
>
> I am not sure I understand your concern. Java login contexts support
> multiple login modules precisely to support this type of real-world
> scenario.
>
> From Javadoc of *javax.security.auth.login.LoginContext*:
>
> *In addition to supporting pluggable authentication, this class also
> supports the notion of stacked authentication. Applications may be
> configured to use more than one LoginModule. For example, one could
> configure both a Kerberos LoginModule and a smart card LoginModule under an
> application. *
>
> * A typical caller instantiates a LoginContext with a name and a
> CallbackHandler. LoginContext uses the name as the index into a
> Configuration to determine which LoginModules should be used, and which
> ones must succeed in order for the overall authentication to succeed. The
> CallbackHandler is passed to the underlying LoginModules so they may
> communicate and interact with users (prompting for a username and password
> via a graphical user interface, for example).*
>
> If it is good enough for Java, surely it must be good enough for Kafka. We
> don't expect users to directly use existing classes when they want to
> support complex new mechanisms. Login is not a standard interface, and we
> want to make callback handlers configurable. So we do require users to wrap
> their existing classes in our interface. And the interfaces make it
> possible to plugin any number of mechanisms. Am I missing something?
>
>
>
> On Tue, Mar 8, 2016 at 4:49 PM, Gwen Shapira  wrote:
>
>> Yes, I understand that.
>>
>> However, because the current configuration only allows a single
>> callback and a single login module, I need to supply a class that
>> supports callbacks or logins for every single mechanism that I need to
>> support. I question whether this is applicable in real-world scenario
>> where a user may need to support mechanisms that were written by
>> different vendors and where a single class that implements all the
>> necessary modules or callsbacks is very unlikely to exist.
>>
>> I agree that supporting a list of classes and dynamically using the
>> correct callbacks and modules based on configuration is complex, but I
>> don't think this feature is useful enough without it.
>>
>> Gwen
>>
>> On Tue, Mar 8, 2016 at 2:40 AM, Rajini Sivaram
>>  wrote:
>> > Gwen,
>> >
>> > Sorry if this is not clear in the KIP. The goal of the KIP is to enable
>> new
>> > mechanisms to be added without any changes to Apache Kafka. The
>> > configuration proposed by the KIP is based on the configurability of the
>> > Java SASL API and what it would take to add a new mechanism similar to
>> > Kerberos/PLAIN/Digest-MD5 and the requirements that Tao raised for custom
>> > authentication.
>> >
>> > The PR includes a test for Digest-MD5 which is not included in Kafka, but
>> > is tested using the configuration options that allow new mechanisms to be
>> > added.
>> >
>> > If you want to add a more complex mechanism which requires integration
>> with
>> > some external authentication libraries for login or callback handlers,
>> you
>> > can do that by configuring your own callback handler and login
>> > implementation. The login class allows you to add additional threads (for
>> > instance for token refresh), while the callback handler extends the
>> > standard Java CallbackHandler interface, enabling you to override any of
>> > the default callbacks in Kafka to provide your own behaviour. If you want
>> > to add your own login modules, SASL server or SASL client implementation
>> > for a mechanism that is not supported in your JDK, you can do that by
>> > adding new Java security modules, using the built-in pluggability of Java
>> > security.
>> >
>> > And finally, if you want to enable multiple mechanisms in your Kafka
>> > brokers, you can specify a callback handler class that either provides
>> the
>> > handlers for all the mechanisms (as the PR does for Kerberos and PLAIN)
>> or
>> > a delegator that calls different callback handlers based on the
>> mechanism.
>> > And for login, you provide a single or a compound class that provides any
>> > additional login logic for all the login modules that you have specified
>> > for KafkaServer.
>> >
>> > Hope that helps.
>> >
>> >
>> 

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-03-08 Thread Rajini Sivaram
Gwen,

I am not sure I understand your concern. Java login contexts support
multiple login modules precisely to support this type of real-world
scenario.

>From Javadoc of *javax.security.auth.login.LoginContext*:

*In addition to supporting pluggable authentication, this class also
supports the notion of stacked authentication. Applications may be
configured to use more than one LoginModule. For example, one could
configure both a Kerberos LoginModule and a smart card LoginModule under an
application. *

* A typical caller instantiates a LoginContext with a name and a
CallbackHandler. LoginContext uses the name as the index into a
Configuration to determine which LoginModules should be used, and which
ones must succeed in order for the overall authentication to succeed. The
CallbackHandler is passed to the underlying LoginModules so they may
communicate and interact with users (prompting for a username and password
via a graphical user interface, for example).*

If it is good enough for Java, surely it must be good enough for Kafka. We
don't expect users to directly use existing classes when they want to
support complex new mechanisms. Login is not a standard interface, and we
want to make callback handlers configurable. So we do require users to wrap
their existing classes in our interface. And the interfaces make it
possible to plugin any number of mechanisms. Am I missing something?



On Tue, Mar 8, 2016 at 4:49 PM, Gwen Shapira  wrote:

> Yes, I understand that.
>
> However, because the current configuration only allows a single
> callback and a single login module, I need to supply a class that
> supports callbacks or logins for every single mechanism that I need to
> support. I question whether this is applicable in real-world scenario
> where a user may need to support mechanisms that were written by
> different vendors and where a single class that implements all the
> necessary modules or callsbacks is very unlikely to exist.
>
> I agree that supporting a list of classes and dynamically using the
> correct callbacks and modules based on configuration is complex, but I
> don't think this feature is useful enough without it.
>
> Gwen
>
> On Tue, Mar 8, 2016 at 2:40 AM, Rajini Sivaram
>  wrote:
> > Gwen,
> >
> > Sorry if this is not clear in the KIP. The goal of the KIP is to enable
> new
> > mechanisms to be added without any changes to Apache Kafka. The
> > configuration proposed by the KIP is based on the configurability of the
> > Java SASL API and what it would take to add a new mechanism similar to
> > Kerberos/PLAIN/Digest-MD5 and the requirements that Tao raised for custom
> > authentication.
> >
> > The PR includes a test for Digest-MD5 which is not included in Kafka, but
> > is tested using the configuration options that allow new mechanisms to be
> > added.
> >
> > If you want to add a more complex mechanism which requires integration
> with
> > some external authentication libraries for login or callback handlers,
> you
> > can do that by configuring your own callback handler and login
> > implementation. The login class allows you to add additional threads (for
> > instance for token refresh), while the callback handler extends the
> > standard Java CallbackHandler interface, enabling you to override any of
> > the default callbacks in Kafka to provide your own behaviour. If you want
> > to add your own login modules, SASL server or SASL client implementation
> > for a mechanism that is not supported in your JDK, you can do that by
> > adding new Java security modules, using the built-in pluggability of Java
> > security.
> >
> > And finally, if you want to enable multiple mechanisms in your Kafka
> > brokers, you can specify a callback handler class that either provides
> the
> > handlers for all the mechanisms (as the PR does for Kerberos and PLAIN)
> or
> > a delegator that calls different callback handlers based on the
> mechanism.
> > And for login, you provide a single or a compound class that provides any
> > additional login logic for all the login modules that you have specified
> > for KafkaServer.
> >
> > Hope that helps.
> >
> >
> > On Tue, Mar 8, 2016 at 1:17 AM, Gwen Shapira  wrote:
> >
> >> Can you explain the process for a adding a new mechanism based on
> current
> >> KIP?
> >>
> >> My thought is that if it requires modifying Apache Kafka code, it is
> >> not pluggable enough.
> >>
> >> On Mon, Mar 7, 2016 at 4:04 PM, Rajini Sivaram
> >>  wrote:
> >> > Gwen,
> >> >
> >> > In cases where you want completely different callbacks for different
> >> > mechanisms, I was thinking that the choice would be between a map of
> >> > classes (mechanism -> callbackHandler class) or a delegator class that
> >> > chooses the appropriate callback handler class based on mechanism. I
> >> chose
> >> > the latter since it makes it easier to configure in Kafka. Since we
> >> create
> >> > a callback handler for each channel and configure it with the
> >> > client-selected mechanism, it is 

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-03-08 Thread Gwen Shapira
Yes, I understand that.

However, because the current configuration only allows a single
callback and a single login module, I need to supply a class that
supports callbacks or logins for every single mechanism that I need to
support. I question whether this is applicable in real-world scenario
where a user may need to support mechanisms that were written by
different vendors and where a single class that implements all the
necessary modules or callsbacks is very unlikely to exist.

I agree that supporting a list of classes and dynamically using the
correct callbacks and modules based on configuration is complex, but I
don't think this feature is useful enough without it.

Gwen

On Tue, Mar 8, 2016 at 2:40 AM, Rajini Sivaram
 wrote:
> Gwen,
>
> Sorry if this is not clear in the KIP. The goal of the KIP is to enable new
> mechanisms to be added without any changes to Apache Kafka. The
> configuration proposed by the KIP is based on the configurability of the
> Java SASL API and what it would take to add a new mechanism similar to
> Kerberos/PLAIN/Digest-MD5 and the requirements that Tao raised for custom
> authentication.
>
> The PR includes a test for Digest-MD5 which is not included in Kafka, but
> is tested using the configuration options that allow new mechanisms to be
> added.
>
> If you want to add a more complex mechanism which requires integration with
> some external authentication libraries for login or callback handlers, you
> can do that by configuring your own callback handler and login
> implementation. The login class allows you to add additional threads (for
> instance for token refresh), while the callback handler extends the
> standard Java CallbackHandler interface, enabling you to override any of
> the default callbacks in Kafka to provide your own behaviour. If you want
> to add your own login modules, SASL server or SASL client implementation
> for a mechanism that is not supported in your JDK, you can do that by
> adding new Java security modules, using the built-in pluggability of Java
> security.
>
> And finally, if you want to enable multiple mechanisms in your Kafka
> brokers, you can specify a callback handler class that either provides the
> handlers for all the mechanisms (as the PR does for Kerberos and PLAIN) or
> a delegator that calls different callback handlers based on the mechanism.
> And for login, you provide a single or a compound class that provides any
> additional login logic for all the login modules that you have specified
> for KafkaServer.
>
> Hope that helps.
>
>
> On Tue, Mar 8, 2016 at 1:17 AM, Gwen Shapira  wrote:
>
>> Can you explain the process for a adding a new mechanism based on current
>> KIP?
>>
>> My thought is that if it requires modifying Apache Kafka code, it is
>> not pluggable enough.
>>
>> On Mon, Mar 7, 2016 at 4:04 PM, Rajini Sivaram
>>  wrote:
>> > Gwen,
>> >
>> > In cases where you want completely different callbacks for different
>> > mechanisms, I was thinking that the choice would be between a map of
>> > classes (mechanism -> callbackHandler class) or a delegator class that
>> > chooses the appropriate callback handler class based on mechanism. I
>> chose
>> > the latter since it makes it easier to configure in Kafka. Since we
>> create
>> > a callback handler for each channel and configure it with the
>> > client-selected mechanism, it is straightforward to have one wrapper
>> class
>> > that delegates to the right mechanism-specific class to handle callbacks.
>> > In many cases, a single class may be sufficient (the PR uses a single
>> > callback class for Kerberos and PLAIN). I do see your point about the
>> > flexibility that multiple classes would provide, but since you need to be
>> > able to associate the callback with a mechanism for this to be useful, I
>> am
>> > not sure if just a list would add value.
>> >
>> > Login class is slightly different since the proposal is to use a single
>> > login context with multiple login modules to handle multiple mechanisms.
>> In
>> > this case, you want to perform login for all the mechanisms that are
>> > enabled. And you want to call loginContext.login() only once. Again, you
>> > can delegate to multiple classes if you wish to add some complex
>> > mechanism-specific logic, but a single login class makes the mapping to a
>> > single login context and the login cache more obvious (the PR has a test
>> > that includes Kerberos and PLAIN).
>> >
>> > Thoughts?
>> >
>> > On Mon, Mar 7, 2016 at 9:57 PM, Gwen Shapira  wrote:
>> >
>> >> Hi,
>> >>
>> >> Before I can vote on this KIP, I have two additional questions /
>> >> comments on the new configuration:
>> >>
>> >> 1. sasl.callback.handler.class - it looks like we want a single class
>> >> that implements all mechanisms. I think this will make it difficult to
>> >> extend since the only way I can add a mechanism will be by
>> >> implementing every single existing mechanism (otherwise customers will
>> >> need to choose between new and exist

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-03-08 Thread Rajini Sivaram
Gwen,

Sorry if this is not clear in the KIP. The goal of the KIP is to enable new
mechanisms to be added without any changes to Apache Kafka. The
configuration proposed by the KIP is based on the configurability of the
Java SASL API and what it would take to add a new mechanism similar to
Kerberos/PLAIN/Digest-MD5 and the requirements that Tao raised for custom
authentication.

The PR includes a test for Digest-MD5 which is not included in Kafka, but
is tested using the configuration options that allow new mechanisms to be
added.

If you want to add a more complex mechanism which requires integration with
some external authentication libraries for login or callback handlers, you
can do that by configuring your own callback handler and login
implementation. The login class allows you to add additional threads (for
instance for token refresh), while the callback handler extends the
standard Java CallbackHandler interface, enabling you to override any of
the default callbacks in Kafka to provide your own behaviour. If you want
to add your own login modules, SASL server or SASL client implementation
for a mechanism that is not supported in your JDK, you can do that by
adding new Java security modules, using the built-in pluggability of Java
security.

And finally, if you want to enable multiple mechanisms in your Kafka
brokers, you can specify a callback handler class that either provides the
handlers for all the mechanisms (as the PR does for Kerberos and PLAIN) or
a delegator that calls different callback handlers based on the mechanism.
And for login, you provide a single or a compound class that provides any
additional login logic for all the login modules that you have specified
for KafkaServer.

Hope that helps.


On Tue, Mar 8, 2016 at 1:17 AM, Gwen Shapira  wrote:

> Can you explain the process for a adding a new mechanism based on current
> KIP?
>
> My thought is that if it requires modifying Apache Kafka code, it is
> not pluggable enough.
>
> On Mon, Mar 7, 2016 at 4:04 PM, Rajini Sivaram
>  wrote:
> > Gwen,
> >
> > In cases where you want completely different callbacks for different
> > mechanisms, I was thinking that the choice would be between a map of
> > classes (mechanism -> callbackHandler class) or a delegator class that
> > chooses the appropriate callback handler class based on mechanism. I
> chose
> > the latter since it makes it easier to configure in Kafka. Since we
> create
> > a callback handler for each channel and configure it with the
> > client-selected mechanism, it is straightforward to have one wrapper
> class
> > that delegates to the right mechanism-specific class to handle callbacks.
> > In many cases, a single class may be sufficient (the PR uses a single
> > callback class for Kerberos and PLAIN). I do see your point about the
> > flexibility that multiple classes would provide, but since you need to be
> > able to associate the callback with a mechanism for this to be useful, I
> am
> > not sure if just a list would add value.
> >
> > Login class is slightly different since the proposal is to use a single
> > login context with multiple login modules to handle multiple mechanisms.
> In
> > this case, you want to perform login for all the mechanisms that are
> > enabled. And you want to call loginContext.login() only once. Again, you
> > can delegate to multiple classes if you wish to add some complex
> > mechanism-specific logic, but a single login class makes the mapping to a
> > single login context and the login cache more obvious (the PR has a test
> > that includes Kerberos and PLAIN).
> >
> > Thoughts?
> >
> > On Mon, Mar 7, 2016 at 9:57 PM, Gwen Shapira  wrote:
> >
> >> Hi,
> >>
> >> Before I can vote on this KIP, I have two additional questions /
> >> comments on the new configuration:
> >>
> >> 1. sasl.callback.handler.class - it looks like we want a single class
> >> that implements all mechanisms. I think this will make it difficult to
> >> extend since the only way I can add a mechanism will be by
> >> implementing every single existing mechanism (otherwise customers will
> >> need to choose between new and existing when selecting which class to
> >> use). If Microsoft releases a proprietary "AD Mechanism" and Oracle
> >> releases "OID mechanism", there will be no class that implements both.
> >> Can we make it a list of classes instead? I realize this complicates
> >> the code a bit (some kind of factory will be required to choose the
> >> right class to use), but important IMO.
> >> 2. similar for sasl.login.class - if I have a class for Kerberos (with
> >> refresh thread) and a class for "plain", we need to be able to load
> >> both.
> >>
> >> Gwen
> >>
> >> On Wed, Mar 2, 2016 at 12:30 AM, Rajini Sivaram
> >>  wrote:
> >> > Jun,
> >> >
> >> > Thanks, I have added a note to the KIP. I will add a comment in the
> >> > implementation and also add a unit test to ensure that conflicts are
> >> > avoided when version number is modified.
> >> >
> >> > On Tue, Ma

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-03-07 Thread Gwen Shapira
Can you explain the process for a adding a new mechanism based on current KIP?

My thought is that if it requires modifying Apache Kafka code, it is
not pluggable enough.

On Mon, Mar 7, 2016 at 4:04 PM, Rajini Sivaram
 wrote:
> Gwen,
>
> In cases where you want completely different callbacks for different
> mechanisms, I was thinking that the choice would be between a map of
> classes (mechanism -> callbackHandler class) or a delegator class that
> chooses the appropriate callback handler class based on mechanism. I chose
> the latter since it makes it easier to configure in Kafka. Since we create
> a callback handler for each channel and configure it with the
> client-selected mechanism, it is straightforward to have one wrapper class
> that delegates to the right mechanism-specific class to handle callbacks.
> In many cases, a single class may be sufficient (the PR uses a single
> callback class for Kerberos and PLAIN). I do see your point about the
> flexibility that multiple classes would provide, but since you need to be
> able to associate the callback with a mechanism for this to be useful, I am
> not sure if just a list would add value.
>
> Login class is slightly different since the proposal is to use a single
> login context with multiple login modules to handle multiple mechanisms. In
> this case, you want to perform login for all the mechanisms that are
> enabled. And you want to call loginContext.login() only once. Again, you
> can delegate to multiple classes if you wish to add some complex
> mechanism-specific logic, but a single login class makes the mapping to a
> single login context and the login cache more obvious (the PR has a test
> that includes Kerberos and PLAIN).
>
> Thoughts?
>
> On Mon, Mar 7, 2016 at 9:57 PM, Gwen Shapira  wrote:
>
>> Hi,
>>
>> Before I can vote on this KIP, I have two additional questions /
>> comments on the new configuration:
>>
>> 1. sasl.callback.handler.class - it looks like we want a single class
>> that implements all mechanisms. I think this will make it difficult to
>> extend since the only way I can add a mechanism will be by
>> implementing every single existing mechanism (otherwise customers will
>> need to choose between new and existing when selecting which class to
>> use). If Microsoft releases a proprietary "AD Mechanism" and Oracle
>> releases "OID mechanism", there will be no class that implements both.
>> Can we make it a list of classes instead? I realize this complicates
>> the code a bit (some kind of factory will be required to choose the
>> right class to use), but important IMO.
>> 2. similar for sasl.login.class - if I have a class for Kerberos (with
>> refresh thread) and a class for "plain", we need to be able to load
>> both.
>>
>> Gwen
>>
>> On Wed, Mar 2, 2016 at 12:30 AM, Rajini Sivaram
>>  wrote:
>> > Jun,
>> >
>> > Thanks, I have added a note to the KIP. I will add a comment in the
>> > implementation and also add a unit test to ensure that conflicts are
>> > avoided when version number is modified.
>> >
>> > On Tue, Mar 1, 2016 at 5:43 PM, Jun Rao  wrote:
>> >
>> >> Rajini,
>> >>
>> >> Thanks for the explanation. For 1, this implies that we have to be
>> careful
>> >> with changing the 2-byte version in the future to avoid conflict. Could
>> you
>> >> document this in the KIP and also in the implementation?
>> >>
>> >> Jun
>> >>
>> >> On Tue, Mar 1, 2016 at 2:47 AM, Rajini Sivaram <
>> >> rajinisiva...@googlemail.com
>> >> > wrote:
>> >>
>> >> > Jun,
>> >> >
>> >> > Thank you for the review.
>> >> >
>> >> >
>> >> >1. With GSSAPI, the first context establishment packet starts with
>> the
>> >> >byte 0x60 (tag for APPLICATION-0) followed by a variable-length
>> >> encoded
>> >> >size, followed by various tags and contents. And the packet also
>> >> > contains a
>> >> >checksum. This is completely different from the mechanism packet
>> from
>> >> > Kafka
>> >> >clients which start with a two-byte version set to zero currently,
>> >> > followed
>> >> >by just a String mechanism.
>> >> >2. Agreed, I have removed the version from the server response in
>> the
>> >> >KIP. Thanks.
>> >> >
>> >> >
>> >> > On Tue, Mar 1, 2016 at 2:33 AM, Jun Rao  wrote:
>> >> >
>> >> > > Rajini,
>> >> > >
>> >> > > Thanks for the updates. Just a couple of minor comments.
>> >> > >
>> >> > > 1. With the default GSSAPI, what's the first packet that the client
>> >> sends
>> >> > > to the server? Is that completely different from the packet format
>> that
>> >> > we
>> >> > > will use for non-GSSAPI mechanisms?
>> >> > >
>> >> > > 2. In the server response, it doesn't seem that we need to include
>> the
>> >> > > version since the client knows the version of the request that it
>> >> sends.
>> >> > >
>> >> > > Jun
>> >> > >
>> >> > > On Mon, Feb 29, 2016 at 10:14 AM, Rajini Sivaram <
>> >> > > rajinisiva...@googlemail.com> wrote:
>> >> > >
>> >> > > > Harsha,
>> >> > > >
>> >> > > > Thank you for the review. I will wait a

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-03-07 Thread Rajini Sivaram
Gwen,

In cases where you want completely different callbacks for different
mechanisms, I was thinking that the choice would be between a map of
classes (mechanism -> callbackHandler class) or a delegator class that
chooses the appropriate callback handler class based on mechanism. I chose
the latter since it makes it easier to configure in Kafka. Since we create
a callback handler for each channel and configure it with the
client-selected mechanism, it is straightforward to have one wrapper class
that delegates to the right mechanism-specific class to handle callbacks.
In many cases, a single class may be sufficient (the PR uses a single
callback class for Kerberos and PLAIN). I do see your point about the
flexibility that multiple classes would provide, but since you need to be
able to associate the callback with a mechanism for this to be useful, I am
not sure if just a list would add value.

Login class is slightly different since the proposal is to use a single
login context with multiple login modules to handle multiple mechanisms. In
this case, you want to perform login for all the mechanisms that are
enabled. And you want to call loginContext.login() only once. Again, you
can delegate to multiple classes if you wish to add some complex
mechanism-specific logic, but a single login class makes the mapping to a
single login context and the login cache more obvious (the PR has a test
that includes Kerberos and PLAIN).

Thoughts?

On Mon, Mar 7, 2016 at 9:57 PM, Gwen Shapira  wrote:

> Hi,
>
> Before I can vote on this KIP, I have two additional questions /
> comments on the new configuration:
>
> 1. sasl.callback.handler.class - it looks like we want a single class
> that implements all mechanisms. I think this will make it difficult to
> extend since the only way I can add a mechanism will be by
> implementing every single existing mechanism (otherwise customers will
> need to choose between new and existing when selecting which class to
> use). If Microsoft releases a proprietary "AD Mechanism" and Oracle
> releases "OID mechanism", there will be no class that implements both.
> Can we make it a list of classes instead? I realize this complicates
> the code a bit (some kind of factory will be required to choose the
> right class to use), but important IMO.
> 2. similar for sasl.login.class - if I have a class for Kerberos (with
> refresh thread) and a class for "plain", we need to be able to load
> both.
>
> Gwen
>
> On Wed, Mar 2, 2016 at 12:30 AM, Rajini Sivaram
>  wrote:
> > Jun,
> >
> > Thanks, I have added a note to the KIP. I will add a comment in the
> > implementation and also add a unit test to ensure that conflicts are
> > avoided when version number is modified.
> >
> > On Tue, Mar 1, 2016 at 5:43 PM, Jun Rao  wrote:
> >
> >> Rajini,
> >>
> >> Thanks for the explanation. For 1, this implies that we have to be
> careful
> >> with changing the 2-byte version in the future to avoid conflict. Could
> you
> >> document this in the KIP and also in the implementation?
> >>
> >> Jun
> >>
> >> On Tue, Mar 1, 2016 at 2:47 AM, Rajini Sivaram <
> >> rajinisiva...@googlemail.com
> >> > wrote:
> >>
> >> > Jun,
> >> >
> >> > Thank you for the review.
> >> >
> >> >
> >> >1. With GSSAPI, the first context establishment packet starts with
> the
> >> >byte 0x60 (tag for APPLICATION-0) followed by a variable-length
> >> encoded
> >> >size, followed by various tags and contents. And the packet also
> >> > contains a
> >> >checksum. This is completely different from the mechanism packet
> from
> >> > Kafka
> >> >clients which start with a two-byte version set to zero currently,
> >> > followed
> >> >by just a String mechanism.
> >> >2. Agreed, I have removed the version from the server response in
> the
> >> >KIP. Thanks.
> >> >
> >> >
> >> > On Tue, Mar 1, 2016 at 2:33 AM, Jun Rao  wrote:
> >> >
> >> > > Rajini,
> >> > >
> >> > > Thanks for the updates. Just a couple of minor comments.
> >> > >
> >> > > 1. With the default GSSAPI, what's the first packet that the client
> >> sends
> >> > > to the server? Is that completely different from the packet format
> that
> >> > we
> >> > > will use for non-GSSAPI mechanisms?
> >> > >
> >> > > 2. In the server response, it doesn't seem that we need to include
> the
> >> > > version since the client knows the version of the request that it
> >> sends.
> >> > >
> >> > > Jun
> >> > >
> >> > > On Mon, Feb 29, 2016 at 10:14 AM, Rajini Sivaram <
> >> > > rajinisiva...@googlemail.com> wrote:
> >> > >
> >> > > > Harsha,
> >> > > >
> >> > > > Thank you for the review. I will wait another day to see if there
> is
> >> > more
> >> > > > feedback and then start a voting thread.
> >> > > >
> >> > > > Rajini
> >> > > >
> >> > > > On Mon, Feb 29, 2016 at 2:51 PM, Harsha  wrote:
> >> > > >
> >> > > > > Rajini,
> >> > > > >   Thanks for the changes to the KIP. It looks good
> to
> >> > me. I
> >> > > > >   think we can move to voti

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-03-07 Thread Gwen Shapira
Hi,

Before I can vote on this KIP, I have two additional questions /
comments on the new configuration:

1. sasl.callback.handler.class - it looks like we want a single class
that implements all mechanisms. I think this will make it difficult to
extend since the only way I can add a mechanism will be by
implementing every single existing mechanism (otherwise customers will
need to choose between new and existing when selecting which class to
use). If Microsoft releases a proprietary "AD Mechanism" and Oracle
releases "OID mechanism", there will be no class that implements both.
Can we make it a list of classes instead? I realize this complicates
the code a bit (some kind of factory will be required to choose the
right class to use), but important IMO.
2. similar for sasl.login.class - if I have a class for Kerberos (with
refresh thread) and a class for "plain", we need to be able to load
both.

Gwen

On Wed, Mar 2, 2016 at 12:30 AM, Rajini Sivaram
 wrote:
> Jun,
>
> Thanks, I have added a note to the KIP. I will add a comment in the
> implementation and also add a unit test to ensure that conflicts are
> avoided when version number is modified.
>
> On Tue, Mar 1, 2016 at 5:43 PM, Jun Rao  wrote:
>
>> Rajini,
>>
>> Thanks for the explanation. For 1, this implies that we have to be careful
>> with changing the 2-byte version in the future to avoid conflict. Could you
>> document this in the KIP and also in the implementation?
>>
>> Jun
>>
>> On Tue, Mar 1, 2016 at 2:47 AM, Rajini Sivaram <
>> rajinisiva...@googlemail.com
>> > wrote:
>>
>> > Jun,
>> >
>> > Thank you for the review.
>> >
>> >
>> >1. With GSSAPI, the first context establishment packet starts with the
>> >byte 0x60 (tag for APPLICATION-0) followed by a variable-length
>> encoded
>> >size, followed by various tags and contents. And the packet also
>> > contains a
>> >checksum. This is completely different from the mechanism packet from
>> > Kafka
>> >clients which start with a two-byte version set to zero currently,
>> > followed
>> >by just a String mechanism.
>> >2. Agreed, I have removed the version from the server response in the
>> >KIP. Thanks.
>> >
>> >
>> > On Tue, Mar 1, 2016 at 2:33 AM, Jun Rao  wrote:
>> >
>> > > Rajini,
>> > >
>> > > Thanks for the updates. Just a couple of minor comments.
>> > >
>> > > 1. With the default GSSAPI, what's the first packet that the client
>> sends
>> > > to the server? Is that completely different from the packet format that
>> > we
>> > > will use for non-GSSAPI mechanisms?
>> > >
>> > > 2. In the server response, it doesn't seem that we need to include the
>> > > version since the client knows the version of the request that it
>> sends.
>> > >
>> > > Jun
>> > >
>> > > On Mon, Feb 29, 2016 at 10:14 AM, Rajini Sivaram <
>> > > rajinisiva...@googlemail.com> wrote:
>> > >
>> > > > Harsha,
>> > > >
>> > > > Thank you for the review. I will wait another day to see if there is
>> > more
>> > > > feedback and then start a voting thread.
>> > > >
>> > > > Rajini
>> > > >
>> > > > On Mon, Feb 29, 2016 at 2:51 PM, Harsha  wrote:
>> > > >
>> > > > > Rajini,
>> > > > >   Thanks for the changes to the KIP. It looks good to
>> > me. I
>> > > > >   think we can move to voting.
>> > > > > Thanks,
>> > > > > Harsha
>> > > > >
>> > > > > On Mon, Feb 29, 2016, at 12:43 AM, Rajini Sivaram wrote:
>> > > > > > I have added some more detail to the KIP based on the discussion
>> in
>> > > the
>> > > > > > last KIP meeting to simplify support for multiple mechanisms.
>> Have
>> > > also
>> > > > > > changed the property names to reflect this.
>> > > > > >
>> > > > > > Also updated the PR in
>> > > > https://issues.apache.org/jira/browse/KAFKA-3149
>> > > > > > to
>> > > > > > reflect the KIP.
>> > > > > >
>> > > > > > Any feedback is appreciated.
>> > > > > >
>> > > > > >
>> > > > > > On Tue, Feb 23, 2016 at 9:36 PM, Rajini Sivaram <
>> > > > > > rajinisiva...@googlemail.com> wrote:
>> > > > > >
>> > > > > > > I have updated the KIP based on the discussion in the KIP
>> meeting
>> > > > > today.
>> > > > > > >
>> > > > > > > Comments and feedback are welcome.
>> > > > > > >
>> > > > > > > On Wed, Feb 3, 2016 at 7:20 PM, Rajini Sivaram <
>> > > > > > > rajinisiva...@googlemail.com> wrote:
>> > > > > > >
>> > > > > > >> Hi Harsha,
>> > > > > > >>
>> > > > > > >> Thank you for the review. Can you clarify - I think you are
>> > saying
>> > > > > that
>> > > > > > >> the client should send its mechanism over the wire to the
>> > server.
>> > > Is
>> > > > > that
>> > > > > > >> correct? The exchange is slightly different in the KIP (the PR
>> > > > > matches the
>> > > > > > >> KIP) from the one you described to enable interoperability
>> with
>> > > > > 0.9.0.0.
>> > > > > > >>
>> > > > > > >>
>> > > > > > >> On Wed, Feb 3, 2016 at 1:56 PM, Harsha 
>> wrote:
>> > > > > > >>
>> > > > > > >>> Rajini,
>> > > > > > >>>I looked at the PR you have. I think its better

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-03-02 Thread Rajini Sivaram
Jun,

Thanks, I have added a note to the KIP. I will add a comment in the
implementation and also add a unit test to ensure that conflicts are
avoided when version number is modified.

On Tue, Mar 1, 2016 at 5:43 PM, Jun Rao  wrote:

> Rajini,
>
> Thanks for the explanation. For 1, this implies that we have to be careful
> with changing the 2-byte version in the future to avoid conflict. Could you
> document this in the KIP and also in the implementation?
>
> Jun
>
> On Tue, Mar 1, 2016 at 2:47 AM, Rajini Sivaram <
> rajinisiva...@googlemail.com
> > wrote:
>
> > Jun,
> >
> > Thank you for the review.
> >
> >
> >1. With GSSAPI, the first context establishment packet starts with the
> >byte 0x60 (tag for APPLICATION-0) followed by a variable-length
> encoded
> >size, followed by various tags and contents. And the packet also
> > contains a
> >checksum. This is completely different from the mechanism packet from
> > Kafka
> >clients which start with a two-byte version set to zero currently,
> > followed
> >by just a String mechanism.
> >2. Agreed, I have removed the version from the server response in the
> >KIP. Thanks.
> >
> >
> > On Tue, Mar 1, 2016 at 2:33 AM, Jun Rao  wrote:
> >
> > > Rajini,
> > >
> > > Thanks for the updates. Just a couple of minor comments.
> > >
> > > 1. With the default GSSAPI, what's the first packet that the client
> sends
> > > to the server? Is that completely different from the packet format that
> > we
> > > will use for non-GSSAPI mechanisms?
> > >
> > > 2. In the server response, it doesn't seem that we need to include the
> > > version since the client knows the version of the request that it
> sends.
> > >
> > > Jun
> > >
> > > On Mon, Feb 29, 2016 at 10:14 AM, Rajini Sivaram <
> > > rajinisiva...@googlemail.com> wrote:
> > >
> > > > Harsha,
> > > >
> > > > Thank you for the review. I will wait another day to see if there is
> > more
> > > > feedback and then start a voting thread.
> > > >
> > > > Rajini
> > > >
> > > > On Mon, Feb 29, 2016 at 2:51 PM, Harsha  wrote:
> > > >
> > > > > Rajini,
> > > > >   Thanks for the changes to the KIP. It looks good to
> > me. I
> > > > >   think we can move to voting.
> > > > > Thanks,
> > > > > Harsha
> > > > >
> > > > > On Mon, Feb 29, 2016, at 12:43 AM, Rajini Sivaram wrote:
> > > > > > I have added some more detail to the KIP based on the discussion
> in
> > > the
> > > > > > last KIP meeting to simplify support for multiple mechanisms.
> Have
> > > also
> > > > > > changed the property names to reflect this.
> > > > > >
> > > > > > Also updated the PR in
> > > > https://issues.apache.org/jira/browse/KAFKA-3149
> > > > > > to
> > > > > > reflect the KIP.
> > > > > >
> > > > > > Any feedback is appreciated.
> > > > > >
> > > > > >
> > > > > > On Tue, Feb 23, 2016 at 9:36 PM, Rajini Sivaram <
> > > > > > rajinisiva...@googlemail.com> wrote:
> > > > > >
> > > > > > > I have updated the KIP based on the discussion in the KIP
> meeting
> > > > > today.
> > > > > > >
> > > > > > > Comments and feedback are welcome.
> > > > > > >
> > > > > > > On Wed, Feb 3, 2016 at 7:20 PM, Rajini Sivaram <
> > > > > > > rajinisiva...@googlemail.com> wrote:
> > > > > > >
> > > > > > >> Hi Harsha,
> > > > > > >>
> > > > > > >> Thank you for the review. Can you clarify - I think you are
> > saying
> > > > > that
> > > > > > >> the client should send its mechanism over the wire to the
> > server.
> > > Is
> > > > > that
> > > > > > >> correct? The exchange is slightly different in the KIP (the PR
> > > > > matches the
> > > > > > >> KIP) from the one you described to enable interoperability
> with
> > > > > 0.9.0.0.
> > > > > > >>
> > > > > > >>
> > > > > > >> On Wed, Feb 3, 2016 at 1:56 PM, Harsha 
> wrote:
> > > > > > >>
> > > > > > >>> Rajini,
> > > > > > >>>I looked at the PR you have. I think its better
> with
> > > > your
> > > > > > >>>earlier approach rather than extending the
> protocol.
> > > > > > >>> What I was thinking initially is, Broker has a config option
> of
> > > say
> > > > > > >>> sasl.mechanism = GSSAPI, PLAIN
> > > > > > >>> and the client can have similar config of
> sasl.mechanism=PLAIN.
> > > > > Client
> > > > > > >>> can send its sasl mechanism before the handshake starts and
> if
> > > the
> > > > > > >>> broker accepts that particular mechanism than it can go ahead
> > > with
> > > > > > >>> handshake otherwise return a error saying that the mechanism
> > not
> > > > > > >>> allowed.
> > > > > > >>>
> > > > > > >>> Thanks,
> > > > > > >>> Harsha
> > > > > > >>>
> > > > > > >>> On Wed, Feb 3, 2016, at 04:58 AM, Rajini Sivaram wrote:
> > > > > > >>> > A slightly different approach for supporting different SASL
> > > > > mechanisms
> > > > > > >>> > within a broker is to allow the same "*security protocol*"
> to
> > > be
> > > > > used
> > > > > > >>> on
> > > > > > >>> > different ports with different configuration options. An
> > > > advantage
> > >

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-03-01 Thread Jun Rao
Rajini,

Thanks for the explanation. For 1, this implies that we have to be careful
with changing the 2-byte version in the future to avoid conflict. Could you
document this in the KIP and also in the implementation?

Jun

On Tue, Mar 1, 2016 at 2:47 AM, Rajini Sivaram  wrote:

> Jun,
>
> Thank you for the review.
>
>
>1. With GSSAPI, the first context establishment packet starts with the
>byte 0x60 (tag for APPLICATION-0) followed by a variable-length encoded
>size, followed by various tags and contents. And the packet also
> contains a
>checksum. This is completely different from the mechanism packet from
> Kafka
>clients which start with a two-byte version set to zero currently,
> followed
>by just a String mechanism.
>2. Agreed, I have removed the version from the server response in the
>KIP. Thanks.
>
>
> On Tue, Mar 1, 2016 at 2:33 AM, Jun Rao  wrote:
>
> > Rajini,
> >
> > Thanks for the updates. Just a couple of minor comments.
> >
> > 1. With the default GSSAPI, what's the first packet that the client sends
> > to the server? Is that completely different from the packet format that
> we
> > will use for non-GSSAPI mechanisms?
> >
> > 2. In the server response, it doesn't seem that we need to include the
> > version since the client knows the version of the request that it sends.
> >
> > Jun
> >
> > On Mon, Feb 29, 2016 at 10:14 AM, Rajini Sivaram <
> > rajinisiva...@googlemail.com> wrote:
> >
> > > Harsha,
> > >
> > > Thank you for the review. I will wait another day to see if there is
> more
> > > feedback and then start a voting thread.
> > >
> > > Rajini
> > >
> > > On Mon, Feb 29, 2016 at 2:51 PM, Harsha  wrote:
> > >
> > > > Rajini,
> > > >   Thanks for the changes to the KIP. It looks good to
> me. I
> > > >   think we can move to voting.
> > > > Thanks,
> > > > Harsha
> > > >
> > > > On Mon, Feb 29, 2016, at 12:43 AM, Rajini Sivaram wrote:
> > > > > I have added some more detail to the KIP based on the discussion in
> > the
> > > > > last KIP meeting to simplify support for multiple mechanisms. Have
> > also
> > > > > changed the property names to reflect this.
> > > > >
> > > > > Also updated the PR in
> > > https://issues.apache.org/jira/browse/KAFKA-3149
> > > > > to
> > > > > reflect the KIP.
> > > > >
> > > > > Any feedback is appreciated.
> > > > >
> > > > >
> > > > > On Tue, Feb 23, 2016 at 9:36 PM, Rajini Sivaram <
> > > > > rajinisiva...@googlemail.com> wrote:
> > > > >
> > > > > > I have updated the KIP based on the discussion in the KIP meeting
> > > > today.
> > > > > >
> > > > > > Comments and feedback are welcome.
> > > > > >
> > > > > > On Wed, Feb 3, 2016 at 7:20 PM, Rajini Sivaram <
> > > > > > rajinisiva...@googlemail.com> wrote:
> > > > > >
> > > > > >> Hi Harsha,
> > > > > >>
> > > > > >> Thank you for the review. Can you clarify - I think you are
> saying
> > > > that
> > > > > >> the client should send its mechanism over the wire to the
> server.
> > Is
> > > > that
> > > > > >> correct? The exchange is slightly different in the KIP (the PR
> > > > matches the
> > > > > >> KIP) from the one you described to enable interoperability with
> > > > 0.9.0.0.
> > > > > >>
> > > > > >>
> > > > > >> On Wed, Feb 3, 2016 at 1:56 PM, Harsha  wrote:
> > > > > >>
> > > > > >>> Rajini,
> > > > > >>>I looked at the PR you have. I think its better with
> > > your
> > > > > >>>earlier approach rather than extending the protocol.
> > > > > >>> What I was thinking initially is, Broker has a config option of
> > say
> > > > > >>> sasl.mechanism = GSSAPI, PLAIN
> > > > > >>> and the client can have similar config of sasl.mechanism=PLAIN.
> > > > Client
> > > > > >>> can send its sasl mechanism before the handshake starts and if
> > the
> > > > > >>> broker accepts that particular mechanism than it can go ahead
> > with
> > > > > >>> handshake otherwise return a error saying that the mechanism
> not
> > > > > >>> allowed.
> > > > > >>>
> > > > > >>> Thanks,
> > > > > >>> Harsha
> > > > > >>>
> > > > > >>> On Wed, Feb 3, 2016, at 04:58 AM, Rajini Sivaram wrote:
> > > > > >>> > A slightly different approach for supporting different SASL
> > > > mechanisms
> > > > > >>> > within a broker is to allow the same "*security protocol*" to
> > be
> > > > used
> > > > > >>> on
> > > > > >>> > different ports with different configuration options. An
> > > advantage
> > > > of
> > > > > >>> > this
> > > > > >>> > approach is that it extends the configurability of not just
> > SASL,
> > > > but
> > > > > >>> any
> > > > > >>> > protocol. For instance, it would enable the use of SSL with
> > > mutual
> > > > > >>> client
> > > > > >>> > authentication on one port or different certificate chains on
> > > > another.
> > > > > >>> > And
> > > > > >>> > it avoids the need for SASL mechanism negotiation.
> > > > > >>> >
> > > > > >>> > Kafka would have the same "*security protocols" *defined as
> > > today,
> > > > but
> > > > > >>> 

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-03-01 Thread Rajini Sivaram
Ismael,

Thank you, will make it clear in the docs that mechanism can be passed even
for GSSAPI from 0.10.0.0.

On Tue, Mar 1, 2016 at 2:59 PM, Ismael Juma  wrote:

> Hi Rajini,
>
> Thanks for clarifying. Comments inline.
>
> On Tue, Mar 1, 2016 at 2:21 PM, Rajini Sivaram <
> rajinisiva...@googlemail.com
> > wrote:
> >
> > Since we want to support arbitrary custom mechanisms, it feels better to
> > use mechanism names rather than Strings. IDs would require ensuring that
> > client and server have the same mapping. Since the mechanism name Strings
> > are not particularly long and they are useful for debugging, I feel it is
> > worthwhile to retain Strings rather than IDs.
> >
>
> Fair enough.
>
> I wasn't sure whether Kafka attempted to retain compatibility in both
> > directions. I would have preferred to send the mechanism even for GSSAPI
> to
> > keep the code consistent, but went for version compatibility instead. I
> was
> > thinking mainly of replication. If you have a cluster that uses 0.9.0.x
> > with SASL replication, it would be easier to upgrade if new clients
> worked
> > with old brokers.
> >
>
> That makes sense. We could use inter.broker.protocol.version for the
> replication case, but `SaslClientAuthenticator` is shared between clients
> and the broker and the way you did is probably simpler. However, the other
> side of the coin is how we document the protocol for clients that are
> distributed separately from Kafka itself. It would be good to make it clear
> in such documentation that one can pass the mechanism even for the GSSAPI
> case from 0.10.0.0 onwards.
>
> Ismael
>



-- 
Regards,

Rajini


Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-03-01 Thread Ismael Juma
Hi Rajini,

Thanks for clarifying. Comments inline.

On Tue, Mar 1, 2016 at 2:21 PM, Rajini Sivaram  wrote:
>
> Since we want to support arbitrary custom mechanisms, it feels better to
> use mechanism names rather than Strings. IDs would require ensuring that
> client and server have the same mapping. Since the mechanism name Strings
> are not particularly long and they are useful for debugging, I feel it is
> worthwhile to retain Strings rather than IDs.
>

Fair enough.

I wasn't sure whether Kafka attempted to retain compatibility in both
> directions. I would have preferred to send the mechanism even for GSSAPI to
> keep the code consistent, but went for version compatibility instead. I was
> thinking mainly of replication. If you have a cluster that uses 0.9.0.x
> with SASL replication, it would be easier to upgrade if new clients worked
> with old brokers.
>

That makes sense. We could use inter.broker.protocol.version for the
replication case, but `SaslClientAuthenticator` is shared between clients
and the broker and the way you did is probably simpler. However, the other
side of the coin is how we document the protocol for clients that are
distributed separately from Kafka itself. It would be good to make it clear
in such documentation that one can pass the mechanism even for the GSSAPI
case from 0.10.0.0 onwards.

Ismael


Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-03-01 Thread Rajini Sivaram
Ismael,

Thank you for the review.

"*Would it be better to assign an id to each mechanism and pass that
instead **of the String? That would be more space-efficient.*"

Since we want to support arbitrary custom mechanisms, it feels better to
use mechanism names rather than Strings. IDs would require ensuring that
client and server have the same mapping. Since the mechanism name Strings
are not particularly long and they are useful for debugging, I feel it is
worthwhile to retain Strings rather than IDs.

"
*The currently proposed way has the benefit that newer clients would
support older brokers _if_ thesasl.mechanism was GSSAPI. Is this important?
For Kafka, brokers have to be upgraded before clients, generally. Brokers
would still support older **clients that didn't send the mechanism either
way.*"

I wasn't sure whether Kafka attempted to retain compatibility in both
directions. I would have preferred to send the mechanism even for GSSAPI to
keep the code consistent, but went for version compatibility instead. I was
thinking mainly of replication. If you have a cluster that uses 0.9.0.x
with SASL replication, it would be easier to upgrade if new clients worked
with old brokers.

Thoughts?


On Tue, Mar 1, 2016 at 11:44 AM, Ismael Juma  wrote:

> Also, with regards to the client flow, it says:
>
> "If sasl.mechanism is not GSSAPI, send a packet with the mechanism name to
> the server. Otherwise go to Step 3."
>
> It sounds like it would be more regular and simpler for clients if they
> always sent the mechanism, even if GSSAPI, right? The currently proposed
> way has the benefit that newer clients would support older brokers _if_ the
> sasl.mechanism was GSSAPI. Is this important? For Kafka, brokers have to be
> upgraded before clients, generally. Brokers would still support older
> clients that didn't send the mechanism either way.
>
> Ismael
>
>
> On Tue, Mar 1, 2016 at 3:11 AM, Ismael Juma  wrote:
>
> > Hi Rajini,
> >
> > One question below.
> >
> > On Tue, Mar 1, 2016 at 2:47 AM, Rajini Sivaram <
> > rajinisiva...@googlemail.com> wrote:
> >
> >>1. With GSSAPI, the first context establishment packet starts with
> the
> >>byte 0x60 (tag for APPLICATION-0) followed by a variable-length
> encoded
> >>size, followed by various tags and contents. And the packet also
> >> contains a
> >>checksum. This is completely different from the mechanism packet from
> >> Kafka
> >>clients which start with a two-byte version set to zero currently,
> >> followed
> >>by just a String mechanism.
> >>
> >
> > Would it be better to assign an id to each mechanism and pass that
> instead
> > of the String? That would be more space-efficient.
> >
> > Ismael
> >
>



-- 
Regards,

Rajini


Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-03-01 Thread Ismael Juma
Also, with regards to the client flow, it says:

"If sasl.mechanism is not GSSAPI, send a packet with the mechanism name to
the server. Otherwise go to Step 3."

It sounds like it would be more regular and simpler for clients if they
always sent the mechanism, even if GSSAPI, right? The currently proposed
way has the benefit that newer clients would support older brokers _if_ the
sasl.mechanism was GSSAPI. Is this important? For Kafka, brokers have to be
upgraded before clients, generally. Brokers would still support older
clients that didn't send the mechanism either way.

Ismael


On Tue, Mar 1, 2016 at 3:11 AM, Ismael Juma  wrote:

> Hi Rajini,
>
> One question below.
>
> On Tue, Mar 1, 2016 at 2:47 AM, Rajini Sivaram <
> rajinisiva...@googlemail.com> wrote:
>
>>1. With GSSAPI, the first context establishment packet starts with the
>>byte 0x60 (tag for APPLICATION-0) followed by a variable-length encoded
>>size, followed by various tags and contents. And the packet also
>> contains a
>>checksum. This is completely different from the mechanism packet from
>> Kafka
>>clients which start with a two-byte version set to zero currently,
>> followed
>>by just a String mechanism.
>>
>
> Would it be better to assign an id to each mechanism and pass that instead
> of the String? That would be more space-efficient.
>
> Ismael
>


Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-03-01 Thread Ismael Juma
Hi Rajini,

One question below.

On Tue, Mar 1, 2016 at 2:47 AM, Rajini Sivaram  wrote:

>1. With GSSAPI, the first context establishment packet starts with the
>byte 0x60 (tag for APPLICATION-0) followed by a variable-length encoded
>size, followed by various tags and contents. And the packet also
> contains a
>checksum. This is completely different from the mechanism packet from
> Kafka
>clients which start with a two-byte version set to zero currently,
> followed
>by just a String mechanism.
>

Would it be better to assign an id to each mechanism and pass that instead
of the String? That would be more space-efficient.

Ismael


Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-03-01 Thread Rajini Sivaram
Jun,

Thank you for the review.


   1. With GSSAPI, the first context establishment packet starts with the
   byte 0x60 (tag for APPLICATION-0) followed by a variable-length encoded
   size, followed by various tags and contents. And the packet also contains a
   checksum. This is completely different from the mechanism packet from Kafka
   clients which start with a two-byte version set to zero currently, followed
   by just a String mechanism.
   2. Agreed, I have removed the version from the server response in the
   KIP. Thanks.


On Tue, Mar 1, 2016 at 2:33 AM, Jun Rao  wrote:

> Rajini,
>
> Thanks for the updates. Just a couple of minor comments.
>
> 1. With the default GSSAPI, what's the first packet that the client sends
> to the server? Is that completely different from the packet format that we
> will use for non-GSSAPI mechanisms?
>
> 2. In the server response, it doesn't seem that we need to include the
> version since the client knows the version of the request that it sends.
>
> Jun
>
> On Mon, Feb 29, 2016 at 10:14 AM, Rajini Sivaram <
> rajinisiva...@googlemail.com> wrote:
>
> > Harsha,
> >
> > Thank you for the review. I will wait another day to see if there is more
> > feedback and then start a voting thread.
> >
> > Rajini
> >
> > On Mon, Feb 29, 2016 at 2:51 PM, Harsha  wrote:
> >
> > > Rajini,
> > >   Thanks for the changes to the KIP. It looks good to me. I
> > >   think we can move to voting.
> > > Thanks,
> > > Harsha
> > >
> > > On Mon, Feb 29, 2016, at 12:43 AM, Rajini Sivaram wrote:
> > > > I have added some more detail to the KIP based on the discussion in
> the
> > > > last KIP meeting to simplify support for multiple mechanisms. Have
> also
> > > > changed the property names to reflect this.
> > > >
> > > > Also updated the PR in
> > https://issues.apache.org/jira/browse/KAFKA-3149
> > > > to
> > > > reflect the KIP.
> > > >
> > > > Any feedback is appreciated.
> > > >
> > > >
> > > > On Tue, Feb 23, 2016 at 9:36 PM, Rajini Sivaram <
> > > > rajinisiva...@googlemail.com> wrote:
> > > >
> > > > > I have updated the KIP based on the discussion in the KIP meeting
> > > today.
> > > > >
> > > > > Comments and feedback are welcome.
> > > > >
> > > > > On Wed, Feb 3, 2016 at 7:20 PM, Rajini Sivaram <
> > > > > rajinisiva...@googlemail.com> wrote:
> > > > >
> > > > >> Hi Harsha,
> > > > >>
> > > > >> Thank you for the review. Can you clarify - I think you are saying
> > > that
> > > > >> the client should send its mechanism over the wire to the server.
> Is
> > > that
> > > > >> correct? The exchange is slightly different in the KIP (the PR
> > > matches the
> > > > >> KIP) from the one you described to enable interoperability with
> > > 0.9.0.0.
> > > > >>
> > > > >>
> > > > >> On Wed, Feb 3, 2016 at 1:56 PM, Harsha  wrote:
> > > > >>
> > > > >>> Rajini,
> > > > >>>I looked at the PR you have. I think its better with
> > your
> > > > >>>earlier approach rather than extending the protocol.
> > > > >>> What I was thinking initially is, Broker has a config option of
> say
> > > > >>> sasl.mechanism = GSSAPI, PLAIN
> > > > >>> and the client can have similar config of sasl.mechanism=PLAIN.
> > > Client
> > > > >>> can send its sasl mechanism before the handshake starts and if
> the
> > > > >>> broker accepts that particular mechanism than it can go ahead
> with
> > > > >>> handshake otherwise return a error saying that the mechanism not
> > > > >>> allowed.
> > > > >>>
> > > > >>> Thanks,
> > > > >>> Harsha
> > > > >>>
> > > > >>> On Wed, Feb 3, 2016, at 04:58 AM, Rajini Sivaram wrote:
> > > > >>> > A slightly different approach for supporting different SASL
> > > mechanisms
> > > > >>> > within a broker is to allow the same "*security protocol*" to
> be
> > > used
> > > > >>> on
> > > > >>> > different ports with different configuration options. An
> > advantage
> > > of
> > > > >>> > this
> > > > >>> > approach is that it extends the configurability of not just
> SASL,
> > > but
> > > > >>> any
> > > > >>> > protocol. For instance, it would enable the use of SSL with
> > mutual
> > > > >>> client
> > > > >>> > authentication on one port or different certificate chains on
> > > another.
> > > > >>> > And
> > > > >>> > it avoids the need for SASL mechanism negotiation.
> > > > >>> >
> > > > >>> > Kafka would have the same "*security protocols" *defined as
> > today,
> > > but
> > > > >>> > with
> > > > >>> > (a single) configurable SASL mechanism. To have different
> > > > >>> configurations
> > > > >>> > of
> > > > >>> > a protocol within a broker, users can define new protocol names
> > > which
> > > > >>> are
> > > > >>> > configured versions of existing protocols, perhaps using just
> > > > >>> > configuration
> > > > >>> > entries and no additional code.
> > > > >>> >
> > > > >>> > For example:
> > > > >>> >
> > > > >>> > A single mechanism broker would be configured as:
> > > > >>> >
> > > > >>> > listeners=SASL_SSL://:9092
> > > > 

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-02-29 Thread Jun Rao
Rajini,

Thanks for the updates. Just a couple of minor comments.

1. With the default GSSAPI, what's the first packet that the client sends
to the server? Is that completely different from the packet format that we
will use for non-GSSAPI mechanisms?

2. In the server response, it doesn't seem that we need to include the
version since the client knows the version of the request that it sends.

Jun

On Mon, Feb 29, 2016 at 10:14 AM, Rajini Sivaram <
rajinisiva...@googlemail.com> wrote:

> Harsha,
>
> Thank you for the review. I will wait another day to see if there is more
> feedback and then start a voting thread.
>
> Rajini
>
> On Mon, Feb 29, 2016 at 2:51 PM, Harsha  wrote:
>
> > Rajini,
> >   Thanks for the changes to the KIP. It looks good to me. I
> >   think we can move to voting.
> > Thanks,
> > Harsha
> >
> > On Mon, Feb 29, 2016, at 12:43 AM, Rajini Sivaram wrote:
> > > I have added some more detail to the KIP based on the discussion in the
> > > last KIP meeting to simplify support for multiple mechanisms. Have also
> > > changed the property names to reflect this.
> > >
> > > Also updated the PR in
> https://issues.apache.org/jira/browse/KAFKA-3149
> > > to
> > > reflect the KIP.
> > >
> > > Any feedback is appreciated.
> > >
> > >
> > > On Tue, Feb 23, 2016 at 9:36 PM, Rajini Sivaram <
> > > rajinisiva...@googlemail.com> wrote:
> > >
> > > > I have updated the KIP based on the discussion in the KIP meeting
> > today.
> > > >
> > > > Comments and feedback are welcome.
> > > >
> > > > On Wed, Feb 3, 2016 at 7:20 PM, Rajini Sivaram <
> > > > rajinisiva...@googlemail.com> wrote:
> > > >
> > > >> Hi Harsha,
> > > >>
> > > >> Thank you for the review. Can you clarify - I think you are saying
> > that
> > > >> the client should send its mechanism over the wire to the server. Is
> > that
> > > >> correct? The exchange is slightly different in the KIP (the PR
> > matches the
> > > >> KIP) from the one you described to enable interoperability with
> > 0.9.0.0.
> > > >>
> > > >>
> > > >> On Wed, Feb 3, 2016 at 1:56 PM, Harsha  wrote:
> > > >>
> > > >>> Rajini,
> > > >>>I looked at the PR you have. I think its better with
> your
> > > >>>earlier approach rather than extending the protocol.
> > > >>> What I was thinking initially is, Broker has a config option of say
> > > >>> sasl.mechanism = GSSAPI, PLAIN
> > > >>> and the client can have similar config of sasl.mechanism=PLAIN.
> > Client
> > > >>> can send its sasl mechanism before the handshake starts and if the
> > > >>> broker accepts that particular mechanism than it can go ahead with
> > > >>> handshake otherwise return a error saying that the mechanism not
> > > >>> allowed.
> > > >>>
> > > >>> Thanks,
> > > >>> Harsha
> > > >>>
> > > >>> On Wed, Feb 3, 2016, at 04:58 AM, Rajini Sivaram wrote:
> > > >>> > A slightly different approach for supporting different SASL
> > mechanisms
> > > >>> > within a broker is to allow the same "*security protocol*" to be
> > used
> > > >>> on
> > > >>> > different ports with different configuration options. An
> advantage
> > of
> > > >>> > this
> > > >>> > approach is that it extends the configurability of not just SASL,
> > but
> > > >>> any
> > > >>> > protocol. For instance, it would enable the use of SSL with
> mutual
> > > >>> client
> > > >>> > authentication on one port or different certificate chains on
> > another.
> > > >>> > And
> > > >>> > it avoids the need for SASL mechanism negotiation.
> > > >>> >
> > > >>> > Kafka would have the same "*security protocols" *defined as
> today,
> > but
> > > >>> > with
> > > >>> > (a single) configurable SASL mechanism. To have different
> > > >>> configurations
> > > >>> > of
> > > >>> > a protocol within a broker, users can define new protocol names
> > which
> > > >>> are
> > > >>> > configured versions of existing protocols, perhaps using just
> > > >>> > configuration
> > > >>> > entries and no additional code.
> > > >>> >
> > > >>> > For example:
> > > >>> >
> > > >>> > A single mechanism broker would be configured as:
> > > >>> >
> > > >>> > listeners=SASL_SSL://:9092
> > > >>> > sasl.mechanism=GSSAPI
> > > >>> > sasl.kerberos.class.name=kafka
> > > >>> > ...
> > > >>> >
> > > >>> >
> > > >>> > And a multi-mechanism broker would be configured as:
> > > >>> >
> > > >>> > listeners=gssapi://:9092,plain://:9093,custom://:9094
> > > >>> > gssapi.security.protocol=SASL_SSL
> > > >>> > gssapi.sasl.mechanism=GSSAPI
> > > >>> > gssapi.sasl.kerberos.class.name=kafka
> > > >>> > ...
> > > >>> > plain.security.protocol=SASL_SSL
> > > >>> > plain.sasl.mechanism=PLAIN
> > > >>> > ..
> > > >>> > custom.security.protocol=SASL_PLAINTEXT
> > > >>> > custom.sasl.mechanism=CUSTOM
> > > >>> > custom.sasl.callback.handler.class=example.CustomCallbackHandler
> > > >>> >
> > > >>> >
> > > >>> >
> > > >>> > This is still a big change because it affects the currently fixed
> > > >>> > enumeration of security protocol definitions, but o

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-02-29 Thread Rajini Sivaram
Harsha,

Thank you for the review. I will wait another day to see if there is more
feedback and then start a voting thread.

Rajini

On Mon, Feb 29, 2016 at 2:51 PM, Harsha  wrote:

> Rajini,
>   Thanks for the changes to the KIP. It looks good to me. I
>   think we can move to voting.
> Thanks,
> Harsha
>
> On Mon, Feb 29, 2016, at 12:43 AM, Rajini Sivaram wrote:
> > I have added some more detail to the KIP based on the discussion in the
> > last KIP meeting to simplify support for multiple mechanisms. Have also
> > changed the property names to reflect this.
> >
> > Also updated the PR in https://issues.apache.org/jira/browse/KAFKA-3149
> > to
> > reflect the KIP.
> >
> > Any feedback is appreciated.
> >
> >
> > On Tue, Feb 23, 2016 at 9:36 PM, Rajini Sivaram <
> > rajinisiva...@googlemail.com> wrote:
> >
> > > I have updated the KIP based on the discussion in the KIP meeting
> today.
> > >
> > > Comments and feedback are welcome.
> > >
> > > On Wed, Feb 3, 2016 at 7:20 PM, Rajini Sivaram <
> > > rajinisiva...@googlemail.com> wrote:
> > >
> > >> Hi Harsha,
> > >>
> > >> Thank you for the review. Can you clarify - I think you are saying
> that
> > >> the client should send its mechanism over the wire to the server. Is
> that
> > >> correct? The exchange is slightly different in the KIP (the PR
> matches the
> > >> KIP) from the one you described to enable interoperability with
> 0.9.0.0.
> > >>
> > >>
> > >> On Wed, Feb 3, 2016 at 1:56 PM, Harsha  wrote:
> > >>
> > >>> Rajini,
> > >>>I looked at the PR you have. I think its better with your
> > >>>earlier approach rather than extending the protocol.
> > >>> What I was thinking initially is, Broker has a config option of say
> > >>> sasl.mechanism = GSSAPI, PLAIN
> > >>> and the client can have similar config of sasl.mechanism=PLAIN.
> Client
> > >>> can send its sasl mechanism before the handshake starts and if the
> > >>> broker accepts that particular mechanism than it can go ahead with
> > >>> handshake otherwise return a error saying that the mechanism not
> > >>> allowed.
> > >>>
> > >>> Thanks,
> > >>> Harsha
> > >>>
> > >>> On Wed, Feb 3, 2016, at 04:58 AM, Rajini Sivaram wrote:
> > >>> > A slightly different approach for supporting different SASL
> mechanisms
> > >>> > within a broker is to allow the same "*security protocol*" to be
> used
> > >>> on
> > >>> > different ports with different configuration options. An advantage
> of
> > >>> > this
> > >>> > approach is that it extends the configurability of not just SASL,
> but
> > >>> any
> > >>> > protocol. For instance, it would enable the use of SSL with mutual
> > >>> client
> > >>> > authentication on one port or different certificate chains on
> another.
> > >>> > And
> > >>> > it avoids the need for SASL mechanism negotiation.
> > >>> >
> > >>> > Kafka would have the same "*security protocols" *defined as today,
> but
> > >>> > with
> > >>> > (a single) configurable SASL mechanism. To have different
> > >>> configurations
> > >>> > of
> > >>> > a protocol within a broker, users can define new protocol names
> which
> > >>> are
> > >>> > configured versions of existing protocols, perhaps using just
> > >>> > configuration
> > >>> > entries and no additional code.
> > >>> >
> > >>> > For example:
> > >>> >
> > >>> > A single mechanism broker would be configured as:
> > >>> >
> > >>> > listeners=SASL_SSL://:9092
> > >>> > sasl.mechanism=GSSAPI
> > >>> > sasl.kerberos.class.name=kafka
> > >>> > ...
> > >>> >
> > >>> >
> > >>> > And a multi-mechanism broker would be configured as:
> > >>> >
> > >>> > listeners=gssapi://:9092,plain://:9093,custom://:9094
> > >>> > gssapi.security.protocol=SASL_SSL
> > >>> > gssapi.sasl.mechanism=GSSAPI
> > >>> > gssapi.sasl.kerberos.class.name=kafka
> > >>> > ...
> > >>> > plain.security.protocol=SASL_SSL
> > >>> > plain.sasl.mechanism=PLAIN
> > >>> > ..
> > >>> > custom.security.protocol=SASL_PLAINTEXT
> > >>> > custom.sasl.mechanism=CUSTOM
> > >>> > custom.sasl.callback.handler.class=example.CustomCallbackHandler
> > >>> >
> > >>> >
> > >>> >
> > >>> > This is still a big change because it affects the currently fixed
> > >>> > enumeration of security protocol definitions, but one that is
> perhaps
> > >>> > more
> > >>> > flexible than defining every new SASL mechanism as a new security
> > >>> > protocol.
> > >>> >
> > >>> > Thoughts?
> > >>> >
> > >>> >
> > >>> > On Tue, Feb 2, 2016 at 12:20 PM, Rajini Sivaram <
> > >>> > rajinisiva...@googlemail.com> wrote:
> > >>> >
> > >>> > > As Ismael has said, we do not have a requirement to support
> multiple
> > >>> > > protocols in a broker. But I agree with Jun's observation that
> some
> > >>> > > companies might want to support a different authentication
> mechanism
> > >>> for
> > >>> > > internal users or partners. For instance, we do use two different
> > >>> > > authentication mechanisms, it just so happens that we are able
> to use
> > >>> > > certificate-based a

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-02-29 Thread Harsha
Rajini,
  Thanks for the changes to the KIP. It looks good to me. I
  think we can move to voting.
Thanks,
Harsha

On Mon, Feb 29, 2016, at 12:43 AM, Rajini Sivaram wrote:
> I have added some more detail to the KIP based on the discussion in the
> last KIP meeting to simplify support for multiple mechanisms. Have also
> changed the property names to reflect this.
> 
> Also updated the PR in https://issues.apache.org/jira/browse/KAFKA-3149
> to
> reflect the KIP.
> 
> Any feedback is appreciated.
> 
> 
> On Tue, Feb 23, 2016 at 9:36 PM, Rajini Sivaram <
> rajinisiva...@googlemail.com> wrote:
> 
> > I have updated the KIP based on the discussion in the KIP meeting today.
> >
> > Comments and feedback are welcome.
> >
> > On Wed, Feb 3, 2016 at 7:20 PM, Rajini Sivaram <
> > rajinisiva...@googlemail.com> wrote:
> >
> >> Hi Harsha,
> >>
> >> Thank you for the review. Can you clarify - I think you are saying that
> >> the client should send its mechanism over the wire to the server. Is that
> >> correct? The exchange is slightly different in the KIP (the PR matches the
> >> KIP) from the one you described to enable interoperability with 0.9.0.0.
> >>
> >>
> >> On Wed, Feb 3, 2016 at 1:56 PM, Harsha  wrote:
> >>
> >>> Rajini,
> >>>I looked at the PR you have. I think its better with your
> >>>earlier approach rather than extending the protocol.
> >>> What I was thinking initially is, Broker has a config option of say
> >>> sasl.mechanism = GSSAPI, PLAIN
> >>> and the client can have similar config of sasl.mechanism=PLAIN. Client
> >>> can send its sasl mechanism before the handshake starts and if the
> >>> broker accepts that particular mechanism than it can go ahead with
> >>> handshake otherwise return a error saying that the mechanism not
> >>> allowed.
> >>>
> >>> Thanks,
> >>> Harsha
> >>>
> >>> On Wed, Feb 3, 2016, at 04:58 AM, Rajini Sivaram wrote:
> >>> > A slightly different approach for supporting different SASL mechanisms
> >>> > within a broker is to allow the same "*security protocol*" to be used
> >>> on
> >>> > different ports with different configuration options. An advantage of
> >>> > this
> >>> > approach is that it extends the configurability of not just SASL, but
> >>> any
> >>> > protocol. For instance, it would enable the use of SSL with mutual
> >>> client
> >>> > authentication on one port or different certificate chains on another.
> >>> > And
> >>> > it avoids the need for SASL mechanism negotiation.
> >>> >
> >>> > Kafka would have the same "*security protocols" *defined as today, but
> >>> > with
> >>> > (a single) configurable SASL mechanism. To have different
> >>> configurations
> >>> > of
> >>> > a protocol within a broker, users can define new protocol names which
> >>> are
> >>> > configured versions of existing protocols, perhaps using just
> >>> > configuration
> >>> > entries and no additional code.
> >>> >
> >>> > For example:
> >>> >
> >>> > A single mechanism broker would be configured as:
> >>> >
> >>> > listeners=SASL_SSL://:9092
> >>> > sasl.mechanism=GSSAPI
> >>> > sasl.kerberos.class.name=kafka
> >>> > ...
> >>> >
> >>> >
> >>> > And a multi-mechanism broker would be configured as:
> >>> >
> >>> > listeners=gssapi://:9092,plain://:9093,custom://:9094
> >>> > gssapi.security.protocol=SASL_SSL
> >>> > gssapi.sasl.mechanism=GSSAPI
> >>> > gssapi.sasl.kerberos.class.name=kafka
> >>> > ...
> >>> > plain.security.protocol=SASL_SSL
> >>> > plain.sasl.mechanism=PLAIN
> >>> > ..
> >>> > custom.security.protocol=SASL_PLAINTEXT
> >>> > custom.sasl.mechanism=CUSTOM
> >>> > custom.sasl.callback.handler.class=example.CustomCallbackHandler
> >>> >
> >>> >
> >>> >
> >>> > This is still a big change because it affects the currently fixed
> >>> > enumeration of security protocol definitions, but one that is perhaps
> >>> > more
> >>> > flexible than defining every new SASL mechanism as a new security
> >>> > protocol.
> >>> >
> >>> > Thoughts?
> >>> >
> >>> >
> >>> > On Tue, Feb 2, 2016 at 12:20 PM, Rajini Sivaram <
> >>> > rajinisiva...@googlemail.com> wrote:
> >>> >
> >>> > > As Ismael has said, we do not have a requirement to support multiple
> >>> > > protocols in a broker. But I agree with Jun's observation that some
> >>> > > companies might want to support a different authentication mechanism
> >>> for
> >>> > > internal users or partners. For instance, we do use two different
> >>> > > authentication mechanisms, it just so happens that we are able to use
> >>> > > certificate-based authentication for internal users, and hence don't
> >>> > > require multiple SASL mechanisms in a broker.
> >>> > >
> >>> > > As Tao has pointed out, mechanism negotiation is a common usage
> >>> pattern.
> >>> > > Many existing protocols that support SASL do already use this
> >>> pattern. AMQP
> >>> > > (
> >>> > >
> >>> http://docs.oasis-open.org/amqp/core/v1.0/os/amqp-core-security-v1.0-os.html#type-sasl-mechanisms
> >>> ),
> >>> > > which, as a me

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-02-29 Thread Rajini Sivaram
I have added some more detail to the KIP based on the discussion in the
last KIP meeting to simplify support for multiple mechanisms. Have also
changed the property names to reflect this.

Also updated the PR in https://issues.apache.org/jira/browse/KAFKA-3149 to
reflect the KIP.

Any feedback is appreciated.


On Tue, Feb 23, 2016 at 9:36 PM, Rajini Sivaram <
rajinisiva...@googlemail.com> wrote:

> I have updated the KIP based on the discussion in the KIP meeting today.
>
> Comments and feedback are welcome.
>
> On Wed, Feb 3, 2016 at 7:20 PM, Rajini Sivaram <
> rajinisiva...@googlemail.com> wrote:
>
>> Hi Harsha,
>>
>> Thank you for the review. Can you clarify - I think you are saying that
>> the client should send its mechanism over the wire to the server. Is that
>> correct? The exchange is slightly different in the KIP (the PR matches the
>> KIP) from the one you described to enable interoperability with 0.9.0.0.
>>
>>
>> On Wed, Feb 3, 2016 at 1:56 PM, Harsha  wrote:
>>
>>> Rajini,
>>>I looked at the PR you have. I think its better with your
>>>earlier approach rather than extending the protocol.
>>> What I was thinking initially is, Broker has a config option of say
>>> sasl.mechanism = GSSAPI, PLAIN
>>> and the client can have similar config of sasl.mechanism=PLAIN. Client
>>> can send its sasl mechanism before the handshake starts and if the
>>> broker accepts that particular mechanism than it can go ahead with
>>> handshake otherwise return a error saying that the mechanism not
>>> allowed.
>>>
>>> Thanks,
>>> Harsha
>>>
>>> On Wed, Feb 3, 2016, at 04:58 AM, Rajini Sivaram wrote:
>>> > A slightly different approach for supporting different SASL mechanisms
>>> > within a broker is to allow the same "*security protocol*" to be used
>>> on
>>> > different ports with different configuration options. An advantage of
>>> > this
>>> > approach is that it extends the configurability of not just SASL, but
>>> any
>>> > protocol. For instance, it would enable the use of SSL with mutual
>>> client
>>> > authentication on one port or different certificate chains on another.
>>> > And
>>> > it avoids the need for SASL mechanism negotiation.
>>> >
>>> > Kafka would have the same "*security protocols" *defined as today, but
>>> > with
>>> > (a single) configurable SASL mechanism. To have different
>>> configurations
>>> > of
>>> > a protocol within a broker, users can define new protocol names which
>>> are
>>> > configured versions of existing protocols, perhaps using just
>>> > configuration
>>> > entries and no additional code.
>>> >
>>> > For example:
>>> >
>>> > A single mechanism broker would be configured as:
>>> >
>>> > listeners=SASL_SSL://:9092
>>> > sasl.mechanism=GSSAPI
>>> > sasl.kerberos.class.name=kafka
>>> > ...
>>> >
>>> >
>>> > And a multi-mechanism broker would be configured as:
>>> >
>>> > listeners=gssapi://:9092,plain://:9093,custom://:9094
>>> > gssapi.security.protocol=SASL_SSL
>>> > gssapi.sasl.mechanism=GSSAPI
>>> > gssapi.sasl.kerberos.class.name=kafka
>>> > ...
>>> > plain.security.protocol=SASL_SSL
>>> > plain.sasl.mechanism=PLAIN
>>> > ..
>>> > custom.security.protocol=SASL_PLAINTEXT
>>> > custom.sasl.mechanism=CUSTOM
>>> > custom.sasl.callback.handler.class=example.CustomCallbackHandler
>>> >
>>> >
>>> >
>>> > This is still a big change because it affects the currently fixed
>>> > enumeration of security protocol definitions, but one that is perhaps
>>> > more
>>> > flexible than defining every new SASL mechanism as a new security
>>> > protocol.
>>> >
>>> > Thoughts?
>>> >
>>> >
>>> > On Tue, Feb 2, 2016 at 12:20 PM, Rajini Sivaram <
>>> > rajinisiva...@googlemail.com> wrote:
>>> >
>>> > > As Ismael has said, we do not have a requirement to support multiple
>>> > > protocols in a broker. But I agree with Jun's observation that some
>>> > > companies might want to support a different authentication mechanism
>>> for
>>> > > internal users or partners. For instance, we do use two different
>>> > > authentication mechanisms, it just so happens that we are able to use
>>> > > certificate-based authentication for internal users, and hence don't
>>> > > require multiple SASL mechanisms in a broker.
>>> > >
>>> > > As Tao has pointed out, mechanism negotiation is a common usage
>>> pattern.
>>> > > Many existing protocols that support SASL do already use this
>>> pattern. AMQP
>>> > > (
>>> > >
>>> http://docs.oasis-open.org/amqp/core/v1.0/os/amqp-core-security-v1.0-os.html#type-sasl-mechanisms
>>> ),
>>> > > which, as a messaging protocol maybe closer to Kafka in use cases
>>> than
>>> > > Zookeeper, is an example. Other examples where the client negotiates
>>> or
>>> > > sends SASL mechanism to server include ACAP that is used as an
>>> example in
>>> > > the SASL RFCs, POP3, LDAP, SMTP etc. This is not to say that Kafka
>>> > > shouldn't use a different type of mechanism selection that fits
>>> better with
>>> > > the existing Kafka design. Just that negotiati

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-02-23 Thread Rajini Sivaram
I have updated the KIP based on the discussion in the KIP meeting today.

Comments and feedback are welcome.

On Wed, Feb 3, 2016 at 7:20 PM, Rajini Sivaram  wrote:

> Hi Harsha,
>
> Thank you for the review. Can you clarify - I think you are saying that
> the client should send its mechanism over the wire to the server. Is that
> correct? The exchange is slightly different in the KIP (the PR matches the
> KIP) from the one you described to enable interoperability with 0.9.0.0.
>
>
> On Wed, Feb 3, 2016 at 1:56 PM, Harsha  wrote:
>
>> Rajini,
>>I looked at the PR you have. I think its better with your
>>earlier approach rather than extending the protocol.
>> What I was thinking initially is, Broker has a config option of say
>> sasl.mechanism = GSSAPI, PLAIN
>> and the client can have similar config of sasl.mechanism=PLAIN. Client
>> can send its sasl mechanism before the handshake starts and if the
>> broker accepts that particular mechanism than it can go ahead with
>> handshake otherwise return a error saying that the mechanism not
>> allowed.
>>
>> Thanks,
>> Harsha
>>
>> On Wed, Feb 3, 2016, at 04:58 AM, Rajini Sivaram wrote:
>> > A slightly different approach for supporting different SASL mechanisms
>> > within a broker is to allow the same "*security protocol*" to be used on
>> > different ports with different configuration options. An advantage of
>> > this
>> > approach is that it extends the configurability of not just SASL, but
>> any
>> > protocol. For instance, it would enable the use of SSL with mutual
>> client
>> > authentication on one port or different certificate chains on another.
>> > And
>> > it avoids the need for SASL mechanism negotiation.
>> >
>> > Kafka would have the same "*security protocols" *defined as today, but
>> > with
>> > (a single) configurable SASL mechanism. To have different configurations
>> > of
>> > a protocol within a broker, users can define new protocol names which
>> are
>> > configured versions of existing protocols, perhaps using just
>> > configuration
>> > entries and no additional code.
>> >
>> > For example:
>> >
>> > A single mechanism broker would be configured as:
>> >
>> > listeners=SASL_SSL://:9092
>> > sasl.mechanism=GSSAPI
>> > sasl.kerberos.class.name=kafka
>> > ...
>> >
>> >
>> > And a multi-mechanism broker would be configured as:
>> >
>> > listeners=gssapi://:9092,plain://:9093,custom://:9094
>> > gssapi.security.protocol=SASL_SSL
>> > gssapi.sasl.mechanism=GSSAPI
>> > gssapi.sasl.kerberos.class.name=kafka
>> > ...
>> > plain.security.protocol=SASL_SSL
>> > plain.sasl.mechanism=PLAIN
>> > ..
>> > custom.security.protocol=SASL_PLAINTEXT
>> > custom.sasl.mechanism=CUSTOM
>> > custom.sasl.callback.handler.class=example.CustomCallbackHandler
>> >
>> >
>> >
>> > This is still a big change because it affects the currently fixed
>> > enumeration of security protocol definitions, but one that is perhaps
>> > more
>> > flexible than defining every new SASL mechanism as a new security
>> > protocol.
>> >
>> > Thoughts?
>> >
>> >
>> > On Tue, Feb 2, 2016 at 12:20 PM, Rajini Sivaram <
>> > rajinisiva...@googlemail.com> wrote:
>> >
>> > > As Ismael has said, we do not have a requirement to support multiple
>> > > protocols in a broker. But I agree with Jun's observation that some
>> > > companies might want to support a different authentication mechanism
>> for
>> > > internal users or partners. For instance, we do use two different
>> > > authentication mechanisms, it just so happens that we are able to use
>> > > certificate-based authentication for internal users, and hence don't
>> > > require multiple SASL mechanisms in a broker.
>> > >
>> > > As Tao has pointed out, mechanism negotiation is a common usage
>> pattern.
>> > > Many existing protocols that support SASL do already use this
>> pattern. AMQP
>> > > (
>> > >
>> http://docs.oasis-open.org/amqp/core/v1.0/os/amqp-core-security-v1.0-os.html#type-sasl-mechanisms
>> ),
>> > > which, as a messaging protocol maybe closer to Kafka in use cases than
>> > > Zookeeper, is an example. Other examples where the client negotiates
>> or
>> > > sends SASL mechanism to server include ACAP that is used as an
>> example in
>> > > the SASL RFCs, POP3, LDAP, SMTP etc. This is not to say that Kafka
>> > > shouldn't use a different type of mechanism selection that fits
>> better with
>> > > the existing Kafka design. Just that negotiation is a common pattern
>> and
>> > > since we typically turn on javax.net.debug to debug TLS negotiation
>> issues,
>> > > having to use Kafka logging to debug SASL negotiation issues is not
>> that
>> > > dissimilar.
>> > >
>> > >
>> > > On Tue, Feb 2, 2016 at 6:12 AM, tao xiao 
>> wrote:
>> > >
>> > >> I am the author of KIP-44. I hope my use case will add some values
>> to this
>> > >> discussion. The reason I raised KIP44 is that I want to be able to
>> > >> implement a custom security protocol that can fulfill the need of my
>> > >> company. As pointed ou

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-02-03 Thread Rajini Sivaram
Hi Harsha,

Thank you for the review. Can you clarify - I think you are saying that the
client should send its mechanism over the wire to the server. Is that
correct? The exchange is slightly different in the KIP (the PR matches the
KIP) from the one you described to enable interoperability with 0.9.0.0.


On Wed, Feb 3, 2016 at 1:56 PM, Harsha  wrote:

> Rajini,
>I looked at the PR you have. I think its better with your
>earlier approach rather than extending the protocol.
> What I was thinking initially is, Broker has a config option of say
> sasl.mechanism = GSSAPI, PLAIN
> and the client can have similar config of sasl.mechanism=PLAIN. Client
> can send its sasl mechanism before the handshake starts and if the
> broker accepts that particular mechanism than it can go ahead with
> handshake otherwise return a error saying that the mechanism not
> allowed.
>
> Thanks,
> Harsha
>
> On Wed, Feb 3, 2016, at 04:58 AM, Rajini Sivaram wrote:
> > A slightly different approach for supporting different SASL mechanisms
> > within a broker is to allow the same "*security protocol*" to be used on
> > different ports with different configuration options. An advantage of
> > this
> > approach is that it extends the configurability of not just SASL, but any
> > protocol. For instance, it would enable the use of SSL with mutual client
> > authentication on one port or different certificate chains on another.
> > And
> > it avoids the need for SASL mechanism negotiation.
> >
> > Kafka would have the same "*security protocols" *defined as today, but
> > with
> > (a single) configurable SASL mechanism. To have different configurations
> > of
> > a protocol within a broker, users can define new protocol names which are
> > configured versions of existing protocols, perhaps using just
> > configuration
> > entries and no additional code.
> >
> > For example:
> >
> > A single mechanism broker would be configured as:
> >
> > listeners=SASL_SSL://:9092
> > sasl.mechanism=GSSAPI
> > sasl.kerberos.class.name=kafka
> > ...
> >
> >
> > And a multi-mechanism broker would be configured as:
> >
> > listeners=gssapi://:9092,plain://:9093,custom://:9094
> > gssapi.security.protocol=SASL_SSL
> > gssapi.sasl.mechanism=GSSAPI
> > gssapi.sasl.kerberos.class.name=kafka
> > ...
> > plain.security.protocol=SASL_SSL
> > plain.sasl.mechanism=PLAIN
> > ..
> > custom.security.protocol=SASL_PLAINTEXT
> > custom.sasl.mechanism=CUSTOM
> > custom.sasl.callback.handler.class=example.CustomCallbackHandler
> >
> >
> >
> > This is still a big change because it affects the currently fixed
> > enumeration of security protocol definitions, but one that is perhaps
> > more
> > flexible than defining every new SASL mechanism as a new security
> > protocol.
> >
> > Thoughts?
> >
> >
> > On Tue, Feb 2, 2016 at 12:20 PM, Rajini Sivaram <
> > rajinisiva...@googlemail.com> wrote:
> >
> > > As Ismael has said, we do not have a requirement to support multiple
> > > protocols in a broker. But I agree with Jun's observation that some
> > > companies might want to support a different authentication mechanism
> for
> > > internal users or partners. For instance, we do use two different
> > > authentication mechanisms, it just so happens that we are able to use
> > > certificate-based authentication for internal users, and hence don't
> > > require multiple SASL mechanisms in a broker.
> > >
> > > As Tao has pointed out, mechanism negotiation is a common usage
> pattern.
> > > Many existing protocols that support SASL do already use this pattern.
> AMQP
> > > (
> > >
> http://docs.oasis-open.org/amqp/core/v1.0/os/amqp-core-security-v1.0-os.html#type-sasl-mechanisms
> ),
> > > which, as a messaging protocol maybe closer to Kafka in use cases than
> > > Zookeeper, is an example. Other examples where the client negotiates or
> > > sends SASL mechanism to server include ACAP that is used as an example
> in
> > > the SASL RFCs, POP3, LDAP, SMTP etc. This is not to say that Kafka
> > > shouldn't use a different type of mechanism selection that fits better
> with
> > > the existing Kafka design. Just that negotiation is a common pattern
> and
> > > since we typically turn on javax.net.debug to debug TLS negotiation
> issues,
> > > having to use Kafka logging to debug SASL negotiation issues is not
> that
> > > dissimilar.
> > >
> > >
> > > On Tue, Feb 2, 2016 at 6:12 AM, tao xiao  wrote:
> > >
> > >> I am the author of KIP-44. I hope my use case will add some values to
> this
> > >> discussion. The reason I raised KIP44 is that I want to be able to
> > >> implement a custom security protocol that can fulfill the need of my
> > >> company. As pointed out by Ismael KIP-43 now supports a pluggable way
> to
> > >> inject custom security provider to SASL I think it is enough to cover
> the
> > >> use case I have and address the concerns raised in KIP-44.
> > >>
> > >> For multiple security protocols support simultaneously it is not
> needed in
> > >> my use ca

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-02-03 Thread Harsha
Rajini,
   I looked at the PR you have. I think its better with your
   earlier approach rather than extending the protocol.
What I was thinking initially is, Broker has a config option of say
sasl.mechanism = GSSAPI, PLAIN
and the client can have similar config of sasl.mechanism=PLAIN. Client
can send its sasl mechanism before the handshake starts and if the
broker accepts that particular mechanism than it can go ahead with
handshake otherwise return a error saying that the mechanism not
allowed. 

Thanks,
Harsha

On Wed, Feb 3, 2016, at 04:58 AM, Rajini Sivaram wrote:
> A slightly different approach for supporting different SASL mechanisms
> within a broker is to allow the same "*security protocol*" to be used on
> different ports with different configuration options. An advantage of
> this
> approach is that it extends the configurability of not just SASL, but any
> protocol. For instance, it would enable the use of SSL with mutual client
> authentication on one port or different certificate chains on another.
> And
> it avoids the need for SASL mechanism negotiation.
> 
> Kafka would have the same "*security protocols" *defined as today, but
> with
> (a single) configurable SASL mechanism. To have different configurations
> of
> a protocol within a broker, users can define new protocol names which are
> configured versions of existing protocols, perhaps using just
> configuration
> entries and no additional code.
> 
> For example:
> 
> A single mechanism broker would be configured as:
> 
> listeners=SASL_SSL://:9092
> sasl.mechanism=GSSAPI
> sasl.kerberos.class.name=kafka
> ...
> 
> 
> And a multi-mechanism broker would be configured as:
> 
> listeners=gssapi://:9092,plain://:9093,custom://:9094
> gssapi.security.protocol=SASL_SSL
> gssapi.sasl.mechanism=GSSAPI
> gssapi.sasl.kerberos.class.name=kafka
> ...
> plain.security.protocol=SASL_SSL
> plain.sasl.mechanism=PLAIN
> ..
> custom.security.protocol=SASL_PLAINTEXT
> custom.sasl.mechanism=CUSTOM
> custom.sasl.callback.handler.class=example.CustomCallbackHandler
> 
> 
> 
> This is still a big change because it affects the currently fixed
> enumeration of security protocol definitions, but one that is perhaps
> more
> flexible than defining every new SASL mechanism as a new security
> protocol.
> 
> Thoughts?
> 
> 
> On Tue, Feb 2, 2016 at 12:20 PM, Rajini Sivaram <
> rajinisiva...@googlemail.com> wrote:
> 
> > As Ismael has said, we do not have a requirement to support multiple
> > protocols in a broker. But I agree with Jun's observation that some
> > companies might want to support a different authentication mechanism for
> > internal users or partners. For instance, we do use two different
> > authentication mechanisms, it just so happens that we are able to use
> > certificate-based authentication for internal users, and hence don't
> > require multiple SASL mechanisms in a broker.
> >
> > As Tao has pointed out, mechanism negotiation is a common usage pattern.
> > Many existing protocols that support SASL do already use this pattern. AMQP
> > (
> > http://docs.oasis-open.org/amqp/core/v1.0/os/amqp-core-security-v1.0-os.html#type-sasl-mechanisms),
> > which, as a messaging protocol maybe closer to Kafka in use cases than
> > Zookeeper, is an example. Other examples where the client negotiates or
> > sends SASL mechanism to server include ACAP that is used as an example in
> > the SASL RFCs, POP3, LDAP, SMTP etc. This is not to say that Kafka
> > shouldn't use a different type of mechanism selection that fits better with
> > the existing Kafka design. Just that negotiation is a common pattern and
> > since we typically turn on javax.net.debug to debug TLS negotiation issues,
> > having to use Kafka logging to debug SASL negotiation issues is not that
> > dissimilar.
> >
> >
> > On Tue, Feb 2, 2016 at 6:12 AM, tao xiao  wrote:
> >
> >> I am the author of KIP-44. I hope my use case will add some values to this
> >> discussion. The reason I raised KIP44 is that I want to be able to
> >> implement a custom security protocol that can fulfill the need of my
> >> company. As pointed out by Ismael KIP-43 now supports a pluggable way to
> >> inject custom security provider to SASL I think it is enough to cover the
> >> use case I have and address the concerns raised in KIP-44.
> >>
> >> For multiple security protocols support simultaneously it is not needed in
> >> my use case and I don't foresee it is needed in the future but as i said
> >> this is my use case only there may be other use cases that need it. But if
> >> we want to support it in the future I prefer to get it right at the first
> >> place given the fact that security protocol is an ENUM and if we stick to
> >> that implementation it is very hard to extend in the future when we decide
> >> multiple security protocols is needed.
> >>
> >> Protocol negotiation is a very common usage pattern in security domain. As
> >> suggested in Java SASL doc
> >>
> >> http://docs.oracle.com/java

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-02-03 Thread Rajini Sivaram
A slightly different approach for supporting different SASL mechanisms
within a broker is to allow the same "*security protocol*" to be used on
different ports with different configuration options. An advantage of this
approach is that it extends the configurability of not just SASL, but any
protocol. For instance, it would enable the use of SSL with mutual client
authentication on one port or different certificate chains on another. And
it avoids the need for SASL mechanism negotiation.

Kafka would have the same "*security protocols" *defined as today, but with
(a single) configurable SASL mechanism. To have different configurations of
a protocol within a broker, users can define new protocol names which are
configured versions of existing protocols, perhaps using just configuration
entries and no additional code.

For example:

A single mechanism broker would be configured as:

listeners=SASL_SSL://:9092
sasl.mechanism=GSSAPI
sasl.kerberos.class.name=kafka
...


And a multi-mechanism broker would be configured as:

listeners=gssapi://:9092,plain://:9093,custom://:9094
gssapi.security.protocol=SASL_SSL
gssapi.sasl.mechanism=GSSAPI
gssapi.sasl.kerberos.class.name=kafka
...
plain.security.protocol=SASL_SSL
plain.sasl.mechanism=PLAIN
..
custom.security.protocol=SASL_PLAINTEXT
custom.sasl.mechanism=CUSTOM
custom.sasl.callback.handler.class=example.CustomCallbackHandler



This is still a big change because it affects the currently fixed
enumeration of security protocol definitions, but one that is perhaps more
flexible than defining every new SASL mechanism as a new security protocol.

Thoughts?


On Tue, Feb 2, 2016 at 12:20 PM, Rajini Sivaram <
rajinisiva...@googlemail.com> wrote:

> As Ismael has said, we do not have a requirement to support multiple
> protocols in a broker. But I agree with Jun's observation that some
> companies might want to support a different authentication mechanism for
> internal users or partners. For instance, we do use two different
> authentication mechanisms, it just so happens that we are able to use
> certificate-based authentication for internal users, and hence don't
> require multiple SASL mechanisms in a broker.
>
> As Tao has pointed out, mechanism negotiation is a common usage pattern.
> Many existing protocols that support SASL do already use this pattern. AMQP
> (
> http://docs.oasis-open.org/amqp/core/v1.0/os/amqp-core-security-v1.0-os.html#type-sasl-mechanisms),
> which, as a messaging protocol maybe closer to Kafka in use cases than
> Zookeeper, is an example. Other examples where the client negotiates or
> sends SASL mechanism to server include ACAP that is used as an example in
> the SASL RFCs, POP3, LDAP, SMTP etc. This is not to say that Kafka
> shouldn't use a different type of mechanism selection that fits better with
> the existing Kafka design. Just that negotiation is a common pattern and
> since we typically turn on javax.net.debug to debug TLS negotiation issues,
> having to use Kafka logging to debug SASL negotiation issues is not that
> dissimilar.
>
>
> On Tue, Feb 2, 2016 at 6:12 AM, tao xiao  wrote:
>
>> I am the author of KIP-44. I hope my use case will add some values to this
>> discussion. The reason I raised KIP44 is that I want to be able to
>> implement a custom security protocol that can fulfill the need of my
>> company. As pointed out by Ismael KIP-43 now supports a pluggable way to
>> inject custom security provider to SASL I think it is enough to cover the
>> use case I have and address the concerns raised in KIP-44.
>>
>> For multiple security protocols support simultaneously it is not needed in
>> my use case and I don't foresee it is needed in the future but as i said
>> this is my use case only there may be other use cases that need it. But if
>> we want to support it in the future I prefer to get it right at the first
>> place given the fact that security protocol is an ENUM and if we stick to
>> that implementation it is very hard to extend in the future when we decide
>> multiple security protocols is needed.
>>
>> Protocol negotiation is a very common usage pattern in security domain. As
>> suggested in Java SASL doc
>>
>> http://docs.oracle.com/javase/7/docs/technotes/guides/security/sasl/sasl-refguide.html
>> client
>> first sends out a packet to server and server responds with a list of
>> mechanisms it supports. This is very similar to SSL/TLS negotiation.
>>
>> On Tue, 2 Feb 2016 at 06:39 Ismael Juma  wrote:
>>
>> > On Mon, Feb 1, 2016 at 7:04 PM, Gwen Shapira  wrote:
>> >
>> > > Looking at "existing solutions", it looks like Zookeeper allows
>> plugging
>> > in
>> > > any SASL mechanism, but the server will only support one mechanism at
>> a
>> > > time.
>> > >
>> >
>> > This was the original proposal from Rajini as that is enough for their
>> > needs.
>> >
>> >
>> > > If this is good enough for our use-case (do we actually need to
>> support
>> > > multiple mechanisms at once?), it will simplify life a lot for us (
>> > >

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-02-02 Thread Rajini Sivaram
As Ismael has said, we do not have a requirement to support multiple
protocols in a broker. But I agree with Jun's observation that some
companies might want to support a different authentication mechanism for
internal users or partners. For instance, we do use two different
authentication mechanisms, it just so happens that we are able to use
certificate-based authentication for internal users, and hence don't
require multiple SASL mechanisms in a broker.

As Tao has pointed out, mechanism negotiation is a common usage pattern.
Many existing protocols that support SASL do already use this pattern. AMQP
(
http://docs.oasis-open.org/amqp/core/v1.0/os/amqp-core-security-v1.0-os.html#type-sasl-mechanisms),
which, as a messaging protocol maybe closer to Kafka in use cases than
Zookeeper, is an example. Other examples where the client negotiates or
sends SASL mechanism to server include ACAP that is used as an example in
the SASL RFCs, POP3, LDAP, SMTP etc. This is not to say that Kafka
shouldn't use a different type of mechanism selection that fits better with
the existing Kafka design. Just that negotiation is a common pattern and
since we typically turn on javax.net.debug to debug TLS negotiation issues,
having to use Kafka logging to debug SASL negotiation issues is not that
dissimilar.


On Tue, Feb 2, 2016 at 6:12 AM, tao xiao  wrote:

> I am the author of KIP-44. I hope my use case will add some values to this
> discussion. The reason I raised KIP44 is that I want to be able to
> implement a custom security protocol that can fulfill the need of my
> company. As pointed out by Ismael KIP-43 now supports a pluggable way to
> inject custom security provider to SASL I think it is enough to cover the
> use case I have and address the concerns raised in KIP-44.
>
> For multiple security protocols support simultaneously it is not needed in
> my use case and I don't foresee it is needed in the future but as i said
> this is my use case only there may be other use cases that need it. But if
> we want to support it in the future I prefer to get it right at the first
> place given the fact that security protocol is an ENUM and if we stick to
> that implementation it is very hard to extend in the future when we decide
> multiple security protocols is needed.
>
> Protocol negotiation is a very common usage pattern in security domain. As
> suggested in Java SASL doc
>
> http://docs.oracle.com/javase/7/docs/technotes/guides/security/sasl/sasl-refguide.html
> client
> first sends out a packet to server and server responds with a list of
> mechanisms it supports. This is very similar to SSL/TLS negotiation.
>
> On Tue, 2 Feb 2016 at 06:39 Ismael Juma  wrote:
>
> > On Mon, Feb 1, 2016 at 7:04 PM, Gwen Shapira  wrote:
> >
> > > Looking at "existing solutions", it looks like Zookeeper allows
> plugging
> > in
> > > any SASL mechanism, but the server will only support one mechanism at a
> > > time.
> > >
> >
> > This was the original proposal from Rajini as that is enough for their
> > needs.
> >
> >
> > > If this is good enough for our use-case (do we actually need to support
> > > multiple mechanisms at once?), it will simplify life a lot for us (
> > >
> https://cwiki.apache.org/confluence/display/ZOOKEEPER/Zookeeper+and+SASL
> > )
> > >
> >
> > The current thinking is that it would be useful to support multiple SASL
> > mechanisms simultaneously. In the KIP meeting, Jun mentioned that
> companies
> > sometimes support additional authentication mechanisms for partners, for
> > example. It does make things more complex, as you say, so we need to be
> > sure the complexity is worth it.
> >
> > Two more points:
> >
> > 1. It has been suggested that custom security protocol support is needed
> by
> > some (KIP-44). Rajini enhanced KIP-43 so that a SASL mechanism with a
> > custom provider can be used for this purpose instead. Given this, it
> seems
> > a bit inconsistent and restrictive not to allow multiple SASL mechanisms
> > simultaneously (we do allow SSL and SASL authentication simultaneously,
> > after all).
> >
> > 2. The other option would be to support a single SASL mechanism
> > simultaneously to start with and then extend this to multiple mechanisms
> > simultaneously later (if and when needed). It seems like it would be
> harder
> > to support the latter in the future if we go down this route, but maybe
> > there are ways around this.
> >
> > Thoughts?
> >
> > Ismael
> >
>



-- 
Regards,

Rajini


Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-02-01 Thread tao xiao
I am the author of KIP-44. I hope my use case will add some values to this
discussion. The reason I raised KIP44 is that I want to be able to
implement a custom security protocol that can fulfill the need of my
company. As pointed out by Ismael KIP-43 now supports a pluggable way to
inject custom security provider to SASL I think it is enough to cover the
use case I have and address the concerns raised in KIP-44.

For multiple security protocols support simultaneously it is not needed in
my use case and I don't foresee it is needed in the future but as i said
this is my use case only there may be other use cases that need it. But if
we want to support it in the future I prefer to get it right at the first
place given the fact that security protocol is an ENUM and if we stick to
that implementation it is very hard to extend in the future when we decide
multiple security protocols is needed.

Protocol negotiation is a very common usage pattern in security domain. As
suggested in Java SASL doc
http://docs.oracle.com/javase/7/docs/technotes/guides/security/sasl/sasl-refguide.html
client
first sends out a packet to server and server responds with a list of
mechanisms it supports. This is very similar to SSL/TLS negotiation.

On Tue, 2 Feb 2016 at 06:39 Ismael Juma  wrote:

> On Mon, Feb 1, 2016 at 7:04 PM, Gwen Shapira  wrote:
>
> > Looking at "existing solutions", it looks like Zookeeper allows plugging
> in
> > any SASL mechanism, but the server will only support one mechanism at a
> > time.
> >
>
> This was the original proposal from Rajini as that is enough for their
> needs.
>
>
> > If this is good enough for our use-case (do we actually need to support
> > multiple mechanisms at once?), it will simplify life a lot for us (
> > https://cwiki.apache.org/confluence/display/ZOOKEEPER/Zookeeper+and+SASL
> )
> >
>
> The current thinking is that it would be useful to support multiple SASL
> mechanisms simultaneously. In the KIP meeting, Jun mentioned that companies
> sometimes support additional authentication mechanisms for partners, for
> example. It does make things more complex, as you say, so we need to be
> sure the complexity is worth it.
>
> Two more points:
>
> 1. It has been suggested that custom security protocol support is needed by
> some (KIP-44). Rajini enhanced KIP-43 so that a SASL mechanism with a
> custom provider can be used for this purpose instead. Given this, it seems
> a bit inconsistent and restrictive not to allow multiple SASL mechanisms
> simultaneously (we do allow SSL and SASL authentication simultaneously,
> after all).
>
> 2. The other option would be to support a single SASL mechanism
> simultaneously to start with and then extend this to multiple mechanisms
> simultaneously later (if and when needed). It seems like it would be harder
> to support the latter in the future if we go down this route, but maybe
> there are ways around this.
>
> Thoughts?
>
> Ismael
>


Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-02-01 Thread Ismael Juma
On Mon, Feb 1, 2016 at 7:04 PM, Gwen Shapira  wrote:

> Looking at "existing solutions", it looks like Zookeeper allows plugging in
> any SASL mechanism, but the server will only support one mechanism at a
> time.
>

This was the original proposal from Rajini as that is enough for their
needs.


> If this is good enough for our use-case (do we actually need to support
> multiple mechanisms at once?), it will simplify life a lot for us (
> https://cwiki.apache.org/confluence/display/ZOOKEEPER/Zookeeper+and+SASL)
>

The current thinking is that it would be useful to support multiple SASL
mechanisms simultaneously. In the KIP meeting, Jun mentioned that companies
sometimes support additional authentication mechanisms for partners, for
example. It does make things more complex, as you say, so we need to be
sure the complexity is worth it.

Two more points:

1. It has been suggested that custom security protocol support is needed by
some (KIP-44). Rajini enhanced KIP-43 so that a SASL mechanism with a
custom provider can be used for this purpose instead. Given this, it seems
a bit inconsistent and restrictive not to allow multiple SASL mechanisms
simultaneously (we do allow SSL and SASL authentication simultaneously,
after all).

2. The other option would be to support a single SASL mechanism
simultaneously to start with and then extend this to multiple mechanisms
simultaneously later (if and when needed). It seems like it would be harder
to support the latter in the future if we go down this route, but maybe
there are ways around this.

Thoughts?

Ismael


Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-02-01 Thread Gwen Shapira
Looking at "existing solutions", it looks like Zookeeper allows plugging in
any SASL mechanism, but the server will only support one mechanism at a
time.
If this is good enough for our use-case (do we actually need to support
multiple mechanisms at once?), it will simplify life a lot for us (
https://cwiki.apache.org/confluence/display/ZOOKEEPER/Zookeeper+and+SASL)

On Mon, Feb 1, 2016 at 8:47 AM, Ismael Juma  wrote:

> Hi Gwen,
>
> A few comments below.
>
> On Mon, Feb 1, 2016 at 4:30 PM, Gwen Shapira  wrote:
>
> > Thanks for clarifying, Ismael and Rajini. And I'm sorry for reopenning a
> > point that was clearly discussed already.
>
>
> Your input is definitely welcome. :) There was an initial discussion in the
> KIP meeting, but it was understood that a wider and more detailed
> discussion was needed before we could agree on the right solution.
>
> 1) Having two different ways to decide on the protocol (port + negotiation)
> > is needlessly confusing for administrators. Its just one more complexity
> to
> > figure out when authentication using SASL is already the most complex
> > administrative task one has in Kafka (judging by the number of questions
> we
> > get).
> >
>
> Yes, this is definitely a concern. It would be good to contrast this with
> how the multiple ports option could look like (given pluggable mechanisms,
> it won't be as simple as the existing config for multiple ports).
>
> 2) Troubleshooting. Especially in customer-support (and mailing-list
> > support) environment. Asking for two configuration files and two netstat
> > results is a completely different story than using tcp dump (which
> requires
> > root privileges), catching the correct negotiation packets and decoding
> > them to figure out what went wrong.
> >
>
> Right. Could we mitigate this somewhat with appropriate logging since we
> control the negotiation process?
>
> Ismael
>
> P.S. It is interesting that Cassandra went in the other direction and made
> it possible to use a single port for both encrypted and non-encrypted
> traffic a few months ago:
>
>
> https://git1-us-west.apache.org/repos/asf?p=cassandra.git;a=commit;h=535c3ac7
>


Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-02-01 Thread Ismael Juma
Hi Gwen,

A few comments below.

On Mon, Feb 1, 2016 at 4:30 PM, Gwen Shapira  wrote:

> Thanks for clarifying, Ismael and Rajini. And I'm sorry for reopenning a
> point that was clearly discussed already.


Your input is definitely welcome. :) There was an initial discussion in the
KIP meeting, but it was understood that a wider and more detailed
discussion was needed before we could agree on the right solution.

1) Having two different ways to decide on the protocol (port + negotiation)
> is needlessly confusing for administrators. Its just one more complexity to
> figure out when authentication using SASL is already the most complex
> administrative task one has in Kafka (judging by the number of questions we
> get).
>

Yes, this is definitely a concern. It would be good to contrast this with
how the multiple ports option could look like (given pluggable mechanisms,
it won't be as simple as the existing config for multiple ports).

2) Troubleshooting. Especially in customer-support (and mailing-list
> support) environment. Asking for two configuration files and two netstat
> results is a completely different story than using tcp dump (which requires
> root privileges), catching the correct negotiation packets and decoding
> them to figure out what went wrong.
>

Right. Could we mitigate this somewhat with appropriate logging since we
control the negotiation process?

Ismael

P.S. It is interesting that Cassandra went in the other direction and made
it possible to use a single port for both encrypted and non-encrypted
traffic a few months ago:

https://git1-us-west.apache.org/repos/asf?p=cassandra.git;a=commit;h=535c3ac7


Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-02-01 Thread Gwen Shapira
Thanks for clarifying, Ismael and Rajini. And I'm sorry for reopenning a
point that was clearly discussed already. I wouldn't be doing this if I
didn't think my concerns are important and that the administrative aspect
may have been ignored (or de-prioritize). In my view, we should prioritize
simplicity for users over simplicity for developers, and I'm afraid we are
doing the reverse here.

I agree that supporting extensible ports is a more significant change.
There are two good reasons (and one weak reason) to put this effort in my
opinion:

1) Having two different ways to decide on the protocol (port + negotiation)
is needlessly confusing for administrators. Its just one more complexity to
figure out when authentication using SASL is already the most complex
administrative task one has in Kafka (judging by the number of questions we
get).

2) Troubleshooting. Especially in customer-support (and mailing-list
support) environment. Asking for two configuration files and two netstat
results is a completely different story than using tcp dump (which requires
root privileges), catching the correct negotiation packets and decoding
them to figure out what went wrong.

3) (weak reason): When we first added the multiple ports, people wanted to
make them more extensible for reasons other than security. For example,
allowing the use of different networks for clients and inter-broker
communication. So it may have other good uses.

Can you point to some examples of how other systems (preferably distributed
systems) support multiple SASL mechanisms?

Gwen

On Mon, Feb 1, 2016 at 1:25 AM, Ismael Juma  wrote:

> Hi Gwen,
>
> On Mon, Feb 1, 2016 at 6:06 AM, Gwen Shapira  wrote:
>
> > Could we support separate SASL mechanisms via separate ports?
> >
>
> This option was also discussed in the KIP and there are some advantages as
> you say. However, there are some drawbacks as well.
>
> This was the main reason we separated the ports in first place, to avoid
> > adding protocol negotiation protocol and allow the server to expect the
> > correct packets when they arrive to the correct listener. It will be nice
> > if new additions could use existing design and code.
> >
>
> Yes, but the current design and code are not actually extensible as we use
> an enum for `SecurityProtocol` and the creation of `ChannelBuilder` is
> hardcoded based on that enum. Mapping different SASL mechanisms to
> different ports is actually a bigger change to how the existing code works
> than the option Rajini favoured in his KIP. This is not to say that we
> should not do it, just clarifying that it's not necessarily an advantage in
> terms of code reuse.
>
> I would also like to point out that we do protocol negotiation in Kafka in
> SSL/TLS today, so it is not totally without precedent. The server and
> client negotiate an appropriate SSL/TLS version depending on options set by
> both the client and server, not unlike what Rajini has proposed. Of course,
> this is a more limited version as the protocols are related and the
> negotiation is handled by the SSL/TLS libraries.
>
> Ismael
>


Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-02-01 Thread Ismael Juma
Hi Gwen,

On Mon, Feb 1, 2016 at 6:06 AM, Gwen Shapira  wrote:

> Could we support separate SASL mechanisms via separate ports?
>

This option was also discussed in the KIP and there are some advantages as
you say. However, there are some drawbacks as well.

This was the main reason we separated the ports in first place, to avoid
> adding protocol negotiation protocol and allow the server to expect the
> correct packets when they arrive to the correct listener. It will be nice
> if new additions could use existing design and code.
>

Yes, but the current design and code are not actually extensible as we use
an enum for `SecurityProtocol` and the creation of `ChannelBuilder` is
hardcoded based on that enum. Mapping different SASL mechanisms to
different ports is actually a bigger change to how the existing code works
than the option Rajini favoured in his KIP. This is not to say that we
should not do it, just clarifying that it's not necessarily an advantage in
terms of code reuse.

I would also like to point out that we do protocol negotiation in Kafka in
SSL/TLS today, so it is not totally without precedent. The server and
client negotiate an appropriate SSL/TLS version depending on options set by
both the client and server, not unlike what Rajini has proposed. Of course,
this is a more limited version as the protocols are related and the
negotiation is handled by the SSL/TLS libraries.

Ismael


Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-02-01 Thread Rajini Sivaram
Gwen,

Yes, we should consider the support for different SASL mechanisms via
separate ports. At the moment, this is included under "*Rejected
alternatives*".  It was slightly lost in the text, so I have updated it. It
will be worth exploring this further.

The KIP considers three requirements:

   1. The addition of a non-Kerberos protocol into Kafka, at the moment
   this is SASL/PLAIN
   2. Enable pluggable SASL mechanisms including custom mechanisms
   3. Support for multiple mechanisms in a broker

1) and 2) work well on the existing SASL ports without any mechanism
exchange.
1) and 3) work well on multiple SASL ports as different security protocols

To combine 1), 2) and 3), we have two options

   - Add mechanism exchange to the wire protocol
   - Enable custom security protocols to be defined, change security
   protocol definitions into a String rather than enum and make ChannelBuilder
   configurable

I feel that mechanism exchange makes it easier for users to implement new
SASL mechanisms. But agree that different ports fit in better with the
existing Kafka endpoint selection design.

Thoughts?


On Mon, Feb 1, 2016 at 6:06 AM, Gwen Shapira  wrote:

> Could we support separate SASL mechanisms via separate ports?
>
> This was the main reason we separated the ports in first place, to avoid
> adding protocol negotiation protocol and allow the server to expect the
> correct packets when they arrive to the correct listener. It will be nice
> if new additions could use existing design and code.
>
> On Sun, Jan 31, 2016 at 6:48 AM, Rajini Sivaram <
> rajinisiva...@googlemail.com> wrote:
>
> > Harsha/Gwen,
> >
> > Thank you both for reviewing the KIP. SASL mechanism negotiation was
> added
> > specifically to support multiple SASL mechanisms within a broker. This
> was
> > brought up at the last KIP meeting and the consensus was that it would be
> > useful to support this. It would become harder to add multiple mechanism
> > support later on after support for other mechanisms are added in this
> KIP.
> >
> > I may have misunderstood Harsha's suggestion. This would use a
> > configuration option to specify one mechanism on the client-side and (one
> > or more?) mechanisms on the server-side. The server still needs to know
> the
> > mechanism that the client is using if it supports multiple mechanisms.
> And
> > this requires a message from the client to the server to send the
> mechanism
> > to the server. Since 0.9.0.0 servers expect to see GSSAPI tokens as soon
> as
> > the connection is established, the additional exchange is required to
> > ensure interoperability with 0.9.0.0. Since an exchange is necessary
> > anyway, sending the server mechanisms to the client seemed to make sense.
> > The current GSSAPI->GSSAPI flow remains exactly as it is now and is not
> > impacted by this KIP. Please let me know if I have missed something.
> >
> > The PR for Kafka-3149 (https://github.com/apache/kafka/pull/812)
> contains
> > the changes described in this KIP. It doesn't have tests for the new
> > features, just the code changes to help with the review.
> >
> >
> >
> > On Sun, Jan 31, 2016 at 12:11 AM, Gwen Shapira 
> wrote:
> >
> > > I want to second Harsha's approach. Configuration is usually much
> easier
> > to
> > > configure and troubleshoot than protocol-negotiation-protocols, which
> > we've
> > > been doing our best to generally avoid in Kafka due to operational
> > > complexity.
> > >
> > > Gwen
> > >
> > > On Sat, Jan 30, 2016 at 3:32 AM, Harsha  wrote:
> > >
> > > > Rajini,
> > > >   Whats the need for server returning acceptable mechs as
> > > >   network package. Why not drive it through the JAAS file
> > > >   itself. I don't see handshake any different than what it is
> > > >   now and only login will change and it can be configured
> based
> > > >   on the JAAS file.
> > > >
> > > > -Harsha
> > > >
> > > > On Fri, Jan 29, 2016, at 02:34 AM, Rajini Sivaram wrote:
> > > > > Ismael,
> > > > >
> > > > > The first packet from the client is deliberately empty to
> distinguish
> > > > > between non-negotiating GSSAPI packet and a negotiation packet. If
> > this
> > > > > packet contained mechanisms, then the code in the broker to
> > distinguish
> > > > > this from a GSSAPI packet would be a bit messy. I was thinking that
> > the
> > > > > client needs to wait for server response anyway before it can
> select
> > > > > a mechanism and start the actual SASL auth process. And once the
> > > > > client gets the server response, it would send the selected
> mechanism
> > > > > followed immediately by the first packet of the SASL auth. So
> perhaps
> > > the
> > > > > overhead is not that bad. Did you have a different flow in mind?
> > > > >
> > > > > On Fri, Jan 29, 2016 at 10:12 AM, Ismael Juma 
> > > wrote:
> > > > >
> > > > > > Thanks Rajini. One question: would it make sense for the client
> to
> > > > > > optimistically suggest its preferred SASL mechanism (or

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-01-31 Thread Gwen Shapira
Could we support separate SASL mechanisms via separate ports?

This was the main reason we separated the ports in first place, to avoid
adding protocol negotiation protocol and allow the server to expect the
correct packets when they arrive to the correct listener. It will be nice
if new additions could use existing design and code.

On Sun, Jan 31, 2016 at 6:48 AM, Rajini Sivaram <
rajinisiva...@googlemail.com> wrote:

> Harsha/Gwen,
>
> Thank you both for reviewing the KIP. SASL mechanism negotiation was added
> specifically to support multiple SASL mechanisms within a broker. This was
> brought up at the last KIP meeting and the consensus was that it would be
> useful to support this. It would become harder to add multiple mechanism
> support later on after support for other mechanisms are added in this KIP.
>
> I may have misunderstood Harsha's suggestion. This would use a
> configuration option to specify one mechanism on the client-side and (one
> or more?) mechanisms on the server-side. The server still needs to know the
> mechanism that the client is using if it supports multiple mechanisms. And
> this requires a message from the client to the server to send the mechanism
> to the server. Since 0.9.0.0 servers expect to see GSSAPI tokens as soon as
> the connection is established, the additional exchange is required to
> ensure interoperability with 0.9.0.0. Since an exchange is necessary
> anyway, sending the server mechanisms to the client seemed to make sense.
> The current GSSAPI->GSSAPI flow remains exactly as it is now and is not
> impacted by this KIP. Please let me know if I have missed something.
>
> The PR for Kafka-3149 (https://github.com/apache/kafka/pull/812) contains
> the changes described in this KIP. It doesn't have tests for the new
> features, just the code changes to help with the review.
>
>
>
> On Sun, Jan 31, 2016 at 12:11 AM, Gwen Shapira  wrote:
>
> > I want to second Harsha's approach. Configuration is usually much easier
> to
> > configure and troubleshoot than protocol-negotiation-protocols, which
> we've
> > been doing our best to generally avoid in Kafka due to operational
> > complexity.
> >
> > Gwen
> >
> > On Sat, Jan 30, 2016 at 3:32 AM, Harsha  wrote:
> >
> > > Rajini,
> > >   Whats the need for server returning acceptable mechs as
> > >   network package. Why not drive it through the JAAS file
> > >   itself. I don't see handshake any different than what it is
> > >   now and only login will change and it can be configured based
> > >   on the JAAS file.
> > >
> > > -Harsha
> > >
> > > On Fri, Jan 29, 2016, at 02:34 AM, Rajini Sivaram wrote:
> > > > Ismael,
> > > >
> > > > The first packet from the client is deliberately empty to distinguish
> > > > between non-negotiating GSSAPI packet and a negotiation packet. If
> this
> > > > packet contained mechanisms, then the code in the broker to
> distinguish
> > > > this from a GSSAPI packet would be a bit messy. I was thinking that
> the
> > > > client needs to wait for server response anyway before it can select
> > > > a mechanism and start the actual SASL auth process. And once the
> > > > client gets the server response, it would send the selected mechanism
> > > > followed immediately by the first packet of the SASL auth. So perhaps
> > the
> > > > overhead is not that bad. Did you have a different flow in mind?
> > > >
> > > > On Fri, Jan 29, 2016 at 10:12 AM, Ismael Juma 
> > wrote:
> > > >
> > > > > Thanks Rajini. One question: would it make sense for the client to
> > > > > optimistically suggest its preferred SASL mechanism (or maybe
> > > mechanisms)
> > > > > to avoid a roundtrip?
> > > > >
> > > > > Ismael
> > > > >
> > > > > On Fri, Jan 29, 2016 at 10:04 AM, Rajini Sivaram <
> > > > > rajinisiva...@googlemail.com> wrote:
> > > > >
> > > > > > Following on from the KIP meeting on Tuesday, I have updated the
> > KIP
> > > > > with a
> > > > > > flow for negotiation of mechanisms to support multiple SASL
> > > mechanisms
> > > > > > within a broker. I have also added a configurable Login interface
> > to
> > > > > > support custom mechanisms which require ticket refresh -
> requested
> > > by Tao
> > > > > > Xiao.
> > > > > >
> > > > > > I will work on updating the PR in KAFKA-3149 over the next few
> days
> > > since
> > > > > > it will be useful for review.
> > > > > >
> > > > > > All comments and suggestions are welcome.
> > > > > >
> > > > > >
> > > > > > On Thu, Jan 28, 2016 at 2:35 PM, tao xiao 
> > > wrote:
> > > > > >
> > > > > > > Sounds like a good approach to add provider in login module.
> > Would
> > > love
> > > > > > to
> > > > > > > see updates in the PR to reflect the changes in Login and
> > > > > > > AuthCallbackHandler.
> > > > > > >
> > > > > > > On Thu, 28 Jan 2016 at 19:31 Rajini Sivaram <
> > > > > > rajinisiva...@googlemail.com>
> > > > > > > wrote:
> > > > > > >
> > > > > > > > Tao,
> > > > > > > >
> > > > > > > > We currently add the securi

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-01-31 Thread Rajini Sivaram
Harsha/Gwen,

Thank you both for reviewing the KIP. SASL mechanism negotiation was added
specifically to support multiple SASL mechanisms within a broker. This was
brought up at the last KIP meeting and the consensus was that it would be
useful to support this. It would become harder to add multiple mechanism
support later on after support for other mechanisms are added in this KIP.

I may have misunderstood Harsha's suggestion. This would use a
configuration option to specify one mechanism on the client-side and (one
or more?) mechanisms on the server-side. The server still needs to know the
mechanism that the client is using if it supports multiple mechanisms. And
this requires a message from the client to the server to send the mechanism
to the server. Since 0.9.0.0 servers expect to see GSSAPI tokens as soon as
the connection is established, the additional exchange is required to
ensure interoperability with 0.9.0.0. Since an exchange is necessary
anyway, sending the server mechanisms to the client seemed to make sense.
The current GSSAPI->GSSAPI flow remains exactly as it is now and is not
impacted by this KIP. Please let me know if I have missed something.

The PR for Kafka-3149 (https://github.com/apache/kafka/pull/812) contains
the changes described in this KIP. It doesn't have tests for the new
features, just the code changes to help with the review.



On Sun, Jan 31, 2016 at 12:11 AM, Gwen Shapira  wrote:

> I want to second Harsha's approach. Configuration is usually much easier to
> configure and troubleshoot than protocol-negotiation-protocols, which we've
> been doing our best to generally avoid in Kafka due to operational
> complexity.
>
> Gwen
>
> On Sat, Jan 30, 2016 at 3:32 AM, Harsha  wrote:
>
> > Rajini,
> >   Whats the need for server returning acceptable mechs as
> >   network package. Why not drive it through the JAAS file
> >   itself. I don't see handshake any different than what it is
> >   now and only login will change and it can be configured based
> >   on the JAAS file.
> >
> > -Harsha
> >
> > On Fri, Jan 29, 2016, at 02:34 AM, Rajini Sivaram wrote:
> > > Ismael,
> > >
> > > The first packet from the client is deliberately empty to distinguish
> > > between non-negotiating GSSAPI packet and a negotiation packet. If this
> > > packet contained mechanisms, then the code in the broker to distinguish
> > > this from a GSSAPI packet would be a bit messy. I was thinking that the
> > > client needs to wait for server response anyway before it can select
> > > a mechanism and start the actual SASL auth process. And once the
> > > client gets the server response, it would send the selected mechanism
> > > followed immediately by the first packet of the SASL auth. So perhaps
> the
> > > overhead is not that bad. Did you have a different flow in mind?
> > >
> > > On Fri, Jan 29, 2016 at 10:12 AM, Ismael Juma 
> wrote:
> > >
> > > > Thanks Rajini. One question: would it make sense for the client to
> > > > optimistically suggest its preferred SASL mechanism (or maybe
> > mechanisms)
> > > > to avoid a roundtrip?
> > > >
> > > > Ismael
> > > >
> > > > On Fri, Jan 29, 2016 at 10:04 AM, Rajini Sivaram <
> > > > rajinisiva...@googlemail.com> wrote:
> > > >
> > > > > Following on from the KIP meeting on Tuesday, I have updated the
> KIP
> > > > with a
> > > > > flow for negotiation of mechanisms to support multiple SASL
> > mechanisms
> > > > > within a broker. I have also added a configurable Login interface
> to
> > > > > support custom mechanisms which require ticket refresh - requested
> > by Tao
> > > > > Xiao.
> > > > >
> > > > > I will work on updating the PR in KAFKA-3149 over the next few days
> > since
> > > > > it will be useful for review.
> > > > >
> > > > > All comments and suggestions are welcome.
> > > > >
> > > > >
> > > > > On Thu, Jan 28, 2016 at 2:35 PM, tao xiao 
> > wrote:
> > > > >
> > > > > > Sounds like a good approach to add provider in login module.
> Would
> > love
> > > > > to
> > > > > > see updates in the PR to reflect the changes in Login and
> > > > > > AuthCallbackHandler.
> > > > > >
> > > > > > On Thu, 28 Jan 2016 at 19:31 Rajini Sivaram <
> > > > > rajinisiva...@googlemail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Tao,
> > > > > > >
> > > > > > > We currently add the security provider in a static initializer
> > in our
> > > > > > login
> > > > > > > module. This ensures that the security provider is always
> > installed
> > > > > > before
> > > > > > > Kafka creates SaslServer/SaslClient. As you say, it is also
> > possible
> > > > to
> > > > > > > insert code into your application to add security provider
> before
> > > > Kafka
> > > > > > > clients are created. Since you can also configure the JDK to
> add
> > new
> > > > > > > security providers, I am not sure if there is value in adding
> > more
> > > > > > > configuration in Kafka to add security providers.
> > > > > > >
> > > > > > > On Thu, Jan 28, 2

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-01-30 Thread Gwen Shapira
I want to second Harsha's approach. Configuration is usually much easier to
configure and troubleshoot than protocol-negotiation-protocols, which we've
been doing our best to generally avoid in Kafka due to operational
complexity.

Gwen

On Sat, Jan 30, 2016 at 3:32 AM, Harsha  wrote:

> Rajini,
>   Whats the need for server returning acceptable mechs as
>   network package. Why not drive it through the JAAS file
>   itself. I don't see handshake any different than what it is
>   now and only login will change and it can be configured based
>   on the JAAS file.
>
> -Harsha
>
> On Fri, Jan 29, 2016, at 02:34 AM, Rajini Sivaram wrote:
> > Ismael,
> >
> > The first packet from the client is deliberately empty to distinguish
> > between non-negotiating GSSAPI packet and a negotiation packet. If this
> > packet contained mechanisms, then the code in the broker to distinguish
> > this from a GSSAPI packet would be a bit messy. I was thinking that the
> > client needs to wait for server response anyway before it can select
> > a mechanism and start the actual SASL auth process. And once the
> > client gets the server response, it would send the selected mechanism
> > followed immediately by the first packet of the SASL auth. So perhaps the
> > overhead is not that bad. Did you have a different flow in mind?
> >
> > On Fri, Jan 29, 2016 at 10:12 AM, Ismael Juma  wrote:
> >
> > > Thanks Rajini. One question: would it make sense for the client to
> > > optimistically suggest its preferred SASL mechanism (or maybe
> mechanisms)
> > > to avoid a roundtrip?
> > >
> > > Ismael
> > >
> > > On Fri, Jan 29, 2016 at 10:04 AM, Rajini Sivaram <
> > > rajinisiva...@googlemail.com> wrote:
> > >
> > > > Following on from the KIP meeting on Tuesday, I have updated the KIP
> > > with a
> > > > flow for negotiation of mechanisms to support multiple SASL
> mechanisms
> > > > within a broker. I have also added a configurable Login interface to
> > > > support custom mechanisms which require ticket refresh - requested
> by Tao
> > > > Xiao.
> > > >
> > > > I will work on updating the PR in KAFKA-3149 over the next few days
> since
> > > > it will be useful for review.
> > > >
> > > > All comments and suggestions are welcome.
> > > >
> > > >
> > > > On Thu, Jan 28, 2016 at 2:35 PM, tao xiao 
> wrote:
> > > >
> > > > > Sounds like a good approach to add provider in login module. Would
> love
> > > > to
> > > > > see updates in the PR to reflect the changes in Login and
> > > > > AuthCallbackHandler.
> > > > >
> > > > > On Thu, 28 Jan 2016 at 19:31 Rajini Sivaram <
> > > > rajinisiva...@googlemail.com>
> > > > > wrote:
> > > > >
> > > > > > Tao,
> > > > > >
> > > > > > We currently add the security provider in a static initializer
> in our
> > > > > login
> > > > > > module. This ensures that the security provider is always
> installed
> > > > > before
> > > > > > Kafka creates SaslServer/SaslClient. As you say, it is also
> possible
> > > to
> > > > > > insert code into your application to add security provider before
> > > Kafka
> > > > > > clients are created. Since you can also configure the JDK to add
> new
> > > > > > security providers, I am not sure if there is value in adding
> more
> > > > > > configuration in Kafka to add security providers.
> > > > > >
> > > > > > On Thu, Jan 28, 2016 at 2:25 AM, tao xiao 
> > > > wrote:
> > > > > >
> > > > > > > The callback works for me as long as it has access to Subject
> and
> > > > > mechs.
> > > > > > > The other thing is how we can inject the customized security
> > > provider
> > > > > via
> > > > > > > Security.addProvider()? If I want to implement my own SASL
> mech I
> > > > need
> > > > > to
> > > > > > > call the addProvider() before SASL.create so that my own
> > > > implementation
> > > > > > of
> > > > > > > SASLClient/Sever can be returned. Any thoughts on this? we can
> > > either
> > > > > let
> > > > > > > users inject the provider in their logic code before creating a
> > > > > > > producer/consumer or Kafka does it for users
> > > > > > >
> > > > > > > On Thu, 28 Jan 2016 at 03:36 Rajini Sivaram <
> > > > > > rajinisiva...@googlemail.com>
> > > > > > > wrote:
> > > > > > >
> > > > > > > > Hi Tao,
> > > > > > > >
> > > > > > > > *javax.security.auth.callback.**CallbackHandler *is the
> standard
> > > > way
> > > > > in
> > > > > > > > which SASL clients and server obtain additional mechanism
> > > specific
> > > > > > > > input. *AuthCallbackHandler
> > > > > > > > *simply extends this interface to propagate configuration
> > > > > properties. I
> > > > > > > was
> > > > > > > > going to provide SASL mechanism and Subject to the callback
> > > > handlers
> > > > > as
> > > > > > > > well since the default handlers use these.
> > > > > > > >
> > > > > > > > Your SaslServer/SaslClient implementation can obtain the
> Subject
> > > > > using
> > > > > > > > *Subject.getSubject(**AccessController.getContext(). *But it
> will
> > > > be

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-01-30 Thread Harsha
Rajini,
  Whats the need for server returning acceptable mechs as
  network package. Why not drive it through the JAAS file
  itself. I don't see handshake any different than what it is
  now and only login will change and it can be configured based
  on the JAAS file.

-Harsha

On Fri, Jan 29, 2016, at 02:34 AM, Rajini Sivaram wrote:
> Ismael,
> 
> The first packet from the client is deliberately empty to distinguish
> between non-negotiating GSSAPI packet and a negotiation packet. If this
> packet contained mechanisms, then the code in the broker to distinguish
> this from a GSSAPI packet would be a bit messy. I was thinking that the
> client needs to wait for server response anyway before it can select
> a mechanism and start the actual SASL auth process. And once the
> client gets the server response, it would send the selected mechanism
> followed immediately by the first packet of the SASL auth. So perhaps the
> overhead is not that bad. Did you have a different flow in mind?
> 
> On Fri, Jan 29, 2016 at 10:12 AM, Ismael Juma  wrote:
> 
> > Thanks Rajini. One question: would it make sense for the client to
> > optimistically suggest its preferred SASL mechanism (or maybe mechanisms)
> > to avoid a roundtrip?
> >
> > Ismael
> >
> > On Fri, Jan 29, 2016 at 10:04 AM, Rajini Sivaram <
> > rajinisiva...@googlemail.com> wrote:
> >
> > > Following on from the KIP meeting on Tuesday, I have updated the KIP
> > with a
> > > flow for negotiation of mechanisms to support multiple SASL mechanisms
> > > within a broker. I have also added a configurable Login interface to
> > > support custom mechanisms which require ticket refresh - requested by Tao
> > > Xiao.
> > >
> > > I will work on updating the PR in KAFKA-3149 over the next few days since
> > > it will be useful for review.
> > >
> > > All comments and suggestions are welcome.
> > >
> > >
> > > On Thu, Jan 28, 2016 at 2:35 PM, tao xiao  wrote:
> > >
> > > > Sounds like a good approach to add provider in login module. Would love
> > > to
> > > > see updates in the PR to reflect the changes in Login and
> > > > AuthCallbackHandler.
> > > >
> > > > On Thu, 28 Jan 2016 at 19:31 Rajini Sivaram <
> > > rajinisiva...@googlemail.com>
> > > > wrote:
> > > >
> > > > > Tao,
> > > > >
> > > > > We currently add the security provider in a static initializer in our
> > > > login
> > > > > module. This ensures that the security provider is always installed
> > > > before
> > > > > Kafka creates SaslServer/SaslClient. As you say, it is also possible
> > to
> > > > > insert code into your application to add security provider before
> > Kafka
> > > > > clients are created. Since you can also configure the JDK to add new
> > > > > security providers, I am not sure if there is value in adding more
> > > > > configuration in Kafka to add security providers.
> > > > >
> > > > > On Thu, Jan 28, 2016 at 2:25 AM, tao xiao 
> > > wrote:
> > > > >
> > > > > > The callback works for me as long as it has access to Subject and
> > > > mechs.
> > > > > > The other thing is how we can inject the customized security
> > provider
> > > > via
> > > > > > Security.addProvider()? If I want to implement my own SASL mech I
> > > need
> > > > to
> > > > > > call the addProvider() before SASL.create so that my own
> > > implementation
> > > > > of
> > > > > > SASLClient/Sever can be returned. Any thoughts on this? we can
> > either
> > > > let
> > > > > > users inject the provider in their logic code before creating a
> > > > > > producer/consumer or Kafka does it for users
> > > > > >
> > > > > > On Thu, 28 Jan 2016 at 03:36 Rajini Sivaram <
> > > > > rajinisiva...@googlemail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Hi Tao,
> > > > > > >
> > > > > > > *javax.security.auth.callback.**CallbackHandler *is the standard
> > > way
> > > > in
> > > > > > > which SASL clients and server obtain additional mechanism
> > specific
> > > > > > > input. *AuthCallbackHandler
> > > > > > > *simply extends this interface to propagate configuration
> > > > properties. I
> > > > > > was
> > > > > > > going to provide SASL mechanism and Subject to the callback
> > > handlers
> > > > as
> > > > > > > well since the default handlers use these.
> > > > > > >
> > > > > > > Your SaslServer/SaslClient implementation can obtain the Subject
> > > > using
> > > > > > > *Subject.getSubject(**AccessController.getContext(). *But it will
> > > be
> > > > > good
> > > > > > > to know if callback handlers would work for you - apart from
> > > standard
> > > > > > > callbacks like PasswordCallback, you can define your own
> > callbacks
> > > > too
> > > > > if
> > > > > > > you require.
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > On Wed, Jan 27, 2016 at 3:59 PM, tao xiao 
> > > > > wrote:
> > > > > > >
> > > > > > > > Thanks Rajini. The other thing in my mind is that we should
> > find
> > > a
> > > > > way
> > > > > > to
> > > > > > > > expose subject to SASL so that ot

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-01-29 Thread Rajini Sivaram
Ismael,

The first packet from the client is deliberately empty to distinguish
between non-negotiating GSSAPI packet and a negotiation packet. If this
packet contained mechanisms, then the code in the broker to distinguish
this from a GSSAPI packet would be a bit messy. I was thinking that the
client needs to wait for server response anyway before it can select
a mechanism and start the actual SASL auth process. And once the
client gets the server response, it would send the selected mechanism
followed immediately by the first packet of the SASL auth. So perhaps the
overhead is not that bad. Did you have a different flow in mind?

On Fri, Jan 29, 2016 at 10:12 AM, Ismael Juma  wrote:

> Thanks Rajini. One question: would it make sense for the client to
> optimistically suggest its preferred SASL mechanism (or maybe mechanisms)
> to avoid a roundtrip?
>
> Ismael
>
> On Fri, Jan 29, 2016 at 10:04 AM, Rajini Sivaram <
> rajinisiva...@googlemail.com> wrote:
>
> > Following on from the KIP meeting on Tuesday, I have updated the KIP
> with a
> > flow for negotiation of mechanisms to support multiple SASL mechanisms
> > within a broker. I have also added a configurable Login interface to
> > support custom mechanisms which require ticket refresh - requested by Tao
> > Xiao.
> >
> > I will work on updating the PR in KAFKA-3149 over the next few days since
> > it will be useful for review.
> >
> > All comments and suggestions are welcome.
> >
> >
> > On Thu, Jan 28, 2016 at 2:35 PM, tao xiao  wrote:
> >
> > > Sounds like a good approach to add provider in login module. Would love
> > to
> > > see updates in the PR to reflect the changes in Login and
> > > AuthCallbackHandler.
> > >
> > > On Thu, 28 Jan 2016 at 19:31 Rajini Sivaram <
> > rajinisiva...@googlemail.com>
> > > wrote:
> > >
> > > > Tao,
> > > >
> > > > We currently add the security provider in a static initializer in our
> > > login
> > > > module. This ensures that the security provider is always installed
> > > before
> > > > Kafka creates SaslServer/SaslClient. As you say, it is also possible
> to
> > > > insert code into your application to add security provider before
> Kafka
> > > > clients are created. Since you can also configure the JDK to add new
> > > > security providers, I am not sure if there is value in adding more
> > > > configuration in Kafka to add security providers.
> > > >
> > > > On Thu, Jan 28, 2016 at 2:25 AM, tao xiao 
> > wrote:
> > > >
> > > > > The callback works for me as long as it has access to Subject and
> > > mechs.
> > > > > The other thing is how we can inject the customized security
> provider
> > > via
> > > > > Security.addProvider()? If I want to implement my own SASL mech I
> > need
> > > to
> > > > > call the addProvider() before SASL.create so that my own
> > implementation
> > > > of
> > > > > SASLClient/Sever can be returned. Any thoughts on this? we can
> either
> > > let
> > > > > users inject the provider in their logic code before creating a
> > > > > producer/consumer or Kafka does it for users
> > > > >
> > > > > On Thu, 28 Jan 2016 at 03:36 Rajini Sivaram <
> > > > rajinisiva...@googlemail.com>
> > > > > wrote:
> > > > >
> > > > > > Hi Tao,
> > > > > >
> > > > > > *javax.security.auth.callback.**CallbackHandler *is the standard
> > way
> > > in
> > > > > > which SASL clients and server obtain additional mechanism
> specific
> > > > > > input. *AuthCallbackHandler
> > > > > > *simply extends this interface to propagate configuration
> > > properties. I
> > > > > was
> > > > > > going to provide SASL mechanism and Subject to the callback
> > handlers
> > > as
> > > > > > well since the default handlers use these.
> > > > > >
> > > > > > Your SaslServer/SaslClient implementation can obtain the Subject
> > > using
> > > > > > *Subject.getSubject(**AccessController.getContext(). *But it will
> > be
> > > > good
> > > > > > to know if callback handlers would work for you - apart from
> > standard
> > > > > > callbacks like PasswordCallback, you can define your own
> callbacks
> > > too
> > > > if
> > > > > > you require.
> > > > > >
> > > > > >
> > > > > >
> > > > > > On Wed, Jan 27, 2016 at 3:59 PM, tao xiao 
> > > > wrote:
> > > > > >
> > > > > > > Thanks Rajini. The other thing in my mind is that we should
> find
> > a
> > > > way
> > > > > to
> > > > > > > expose subject to SASL so that other mechanisms are able to use
> > the
> > > > > > > principal and credentials stored in subject to do
> authentication.
> > > > > > >
> > > > > > > I am thinking to have below interface that can be extended by
> > users
> > > > to
> > > > > > > build the SASL client/server instead of having an AuthCallback.
> > > With
> > > > > this
> > > > > > > interface users are able to add their own security provider
> > before
> > > > > > > client/server is returned by SASL. Any thoughts?
> > > > > > >
> > > > > > > Interface SaslClientBuilder {
> > > > > > >
> > > > > > > SaslClient build(mechs, subject, host, otherparams)
>

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-01-29 Thread Ismael Juma
Thanks Rajini. One question: would it make sense for the client to
optimistically suggest its preferred SASL mechanism (or maybe mechanisms)
to avoid a roundtrip?

Ismael

On Fri, Jan 29, 2016 at 10:04 AM, Rajini Sivaram <
rajinisiva...@googlemail.com> wrote:

> Following on from the KIP meeting on Tuesday, I have updated the KIP with a
> flow for negotiation of mechanisms to support multiple SASL mechanisms
> within a broker. I have also added a configurable Login interface to
> support custom mechanisms which require ticket refresh - requested by Tao
> Xiao.
>
> I will work on updating the PR in KAFKA-3149 over the next few days since
> it will be useful for review.
>
> All comments and suggestions are welcome.
>
>
> On Thu, Jan 28, 2016 at 2:35 PM, tao xiao  wrote:
>
> > Sounds like a good approach to add provider in login module. Would love
> to
> > see updates in the PR to reflect the changes in Login and
> > AuthCallbackHandler.
> >
> > On Thu, 28 Jan 2016 at 19:31 Rajini Sivaram <
> rajinisiva...@googlemail.com>
> > wrote:
> >
> > > Tao,
> > >
> > > We currently add the security provider in a static initializer in our
> > login
> > > module. This ensures that the security provider is always installed
> > before
> > > Kafka creates SaslServer/SaslClient. As you say, it is also possible to
> > > insert code into your application to add security provider before Kafka
> > > clients are created. Since you can also configure the JDK to add new
> > > security providers, I am not sure if there is value in adding more
> > > configuration in Kafka to add security providers.
> > >
> > > On Thu, Jan 28, 2016 at 2:25 AM, tao xiao 
> wrote:
> > >
> > > > The callback works for me as long as it has access to Subject and
> > mechs.
> > > > The other thing is how we can inject the customized security provider
> > via
> > > > Security.addProvider()? If I want to implement my own SASL mech I
> need
> > to
> > > > call the addProvider() before SASL.create so that my own
> implementation
> > > of
> > > > SASLClient/Sever can be returned. Any thoughts on this? we can either
> > let
> > > > users inject the provider in their logic code before creating a
> > > > producer/consumer or Kafka does it for users
> > > >
> > > > On Thu, 28 Jan 2016 at 03:36 Rajini Sivaram <
> > > rajinisiva...@googlemail.com>
> > > > wrote:
> > > >
> > > > > Hi Tao,
> > > > >
> > > > > *javax.security.auth.callback.**CallbackHandler *is the standard
> way
> > in
> > > > > which SASL clients and server obtain additional mechanism specific
> > > > > input. *AuthCallbackHandler
> > > > > *simply extends this interface to propagate configuration
> > properties. I
> > > > was
> > > > > going to provide SASL mechanism and Subject to the callback
> handlers
> > as
> > > > > well since the default handlers use these.
> > > > >
> > > > > Your SaslServer/SaslClient implementation can obtain the Subject
> > using
> > > > > *Subject.getSubject(**AccessController.getContext(). *But it will
> be
> > > good
> > > > > to know if callback handlers would work for you - apart from
> standard
> > > > > callbacks like PasswordCallback, you can define your own callbacks
> > too
> > > if
> > > > > you require.
> > > > >
> > > > >
> > > > >
> > > > > On Wed, Jan 27, 2016 at 3:59 PM, tao xiao 
> > > wrote:
> > > > >
> > > > > > Thanks Rajini. The other thing in my mind is that we should find
> a
> > > way
> > > > to
> > > > > > expose subject to SASL so that other mechanisms are able to use
> the
> > > > > > principal and credentials stored in subject to do authentication.
> > > > > >
> > > > > > I am thinking to have below interface that can be extended by
> users
> > > to
> > > > > > build the SASL client/server instead of having an AuthCallback.
> > With
> > > > this
> > > > > > interface users are able to add their own security provider
> before
> > > > > > client/server is returned by SASL. Any thoughts?
> > > > > >
> > > > > > Interface SaslClientBuilder {
> > > > > >
> > > > > > SaslClient build(mechs, subject, host, otherparams)
> > > > > > }
> > > > > >
> > > > > > Interface SaslServerBuilder {
> > > > > > SaslServer build(mechs, subject, host, otherparams)
> > > > > > }
> > > > > >
> > > > > > On Wed, 27 Jan 2016 at 18:54 Rajini Sivaram <
> > > > > rajinisiva...@googlemail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Tao,
> > > > > > >
> > > > > > > Thank you for the explanation. I couldn't find a standard Java
> > > > > interface
> > > > > > > that would be suitable, so will define one based on your
> > > requirement
> > > > > and
> > > > > > > update the KIP.
> > > > > > >
> > > > > > > Regards,
> > > > > > >
> > > > > > > Rajini
> > > > > > >
> > > > > > > On Wed, Jan 27, 2016 at 2:12 AM, tao xiao <
> xiaotao...@gmail.com>
> > > > > wrote:
> > > > > > >
> > > > > > > > Hi Rajini,
> > > > > > > >
> > > > > > > > One requirement I have is to refresh the login token every X
> > > hours.
> > > > > > Like
> > > > > > > > what the Kerberos login does 

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-01-29 Thread tao xiao
Sorry, just ignore previous email. I saw the newly defined interface of the
callback in the KIP which has considered this matter.

On Fri, 29 Jan 2016 at 18:08 tao xiao  wrote:

> Hi Rajini,
>
> Do you consider exposing Subject to AuthCallback as well? It is useful for
> users building their own SASL mechanism so that we have control  where to
> put logon data in subject and how to manipulate in SASL callback
>
>
> On Fri, 29 Jan 2016 at 18:04 Rajini Sivaram 
> wrote:
>
>> Following on from the KIP meeting on Tuesday, I have updated the KIP with
>> a
>> flow for negotiation of mechanisms to support multiple SASL mechanisms
>> within a broker. I have also added a configurable Login interface to
>> support custom mechanisms which require ticket refresh - requested by Tao
>> Xiao.
>>
>> I will work on updating the PR in KAFKA-3149 over the next few days since
>> it will be useful for review.
>>
>> All comments and suggestions are welcome.
>>
>>
>> On Thu, Jan 28, 2016 at 2:35 PM, tao xiao  wrote:
>>
>> > Sounds like a good approach to add provider in login module. Would love
>> to
>> > see updates in the PR to reflect the changes in Login and
>> > AuthCallbackHandler.
>> >
>> > On Thu, 28 Jan 2016 at 19:31 Rajini Sivaram <
>> rajinisiva...@googlemail.com>
>> > wrote:
>> >
>> > > Tao,
>> > >
>> > > We currently add the security provider in a static initializer in our
>> > login
>> > > module. This ensures that the security provider is always installed
>> > before
>> > > Kafka creates SaslServer/SaslClient. As you say, it is also possible
>> to
>> > > insert code into your application to add security provider before
>> Kafka
>> > > clients are created. Since you can also configure the JDK to add new
>> > > security providers, I am not sure if there is value in adding more
>> > > configuration in Kafka to add security providers.
>> > >
>> > > On Thu, Jan 28, 2016 at 2:25 AM, tao xiao 
>> wrote:
>> > >
>> > > > The callback works for me as long as it has access to Subject and
>> > mechs.
>> > > > The other thing is how we can inject the customized security
>> provider
>> > via
>> > > > Security.addProvider()? If I want to implement my own SASL mech I
>> need
>> > to
>> > > > call the addProvider() before SASL.create so that my own
>> implementation
>> > > of
>> > > > SASLClient/Sever can be returned. Any thoughts on this? we can
>> either
>> > let
>> > > > users inject the provider in their logic code before creating a
>> > > > producer/consumer or Kafka does it for users
>> > > >
>> > > > On Thu, 28 Jan 2016 at 03:36 Rajini Sivaram <
>> > > rajinisiva...@googlemail.com>
>> > > > wrote:
>> > > >
>> > > > > Hi Tao,
>> > > > >
>> > > > > *javax.security.auth.callback.**CallbackHandler *is the standard
>> way
>> > in
>> > > > > which SASL clients and server obtain additional mechanism specific
>> > > > > input. *AuthCallbackHandler
>> > > > > *simply extends this interface to propagate configuration
>> > properties. I
>> > > > was
>> > > > > going to provide SASL mechanism and Subject to the callback
>> handlers
>> > as
>> > > > > well since the default handlers use these.
>> > > > >
>> > > > > Your SaslServer/SaslClient implementation can obtain the Subject
>> > using
>> > > > > *Subject.getSubject(**AccessController.getContext(). *But it will
>> be
>> > > good
>> > > > > to know if callback handlers would work for you - apart from
>> standard
>> > > > > callbacks like PasswordCallback, you can define your own callbacks
>> > too
>> > > if
>> > > > > you require.
>> > > > >
>> > > > >
>> > > > >
>> > > > > On Wed, Jan 27, 2016 at 3:59 PM, tao xiao 
>> > > wrote:
>> > > > >
>> > > > > > Thanks Rajini. The other thing in my mind is that we should
>> find a
>> > > way
>> > > > to
>> > > > > > expose subject to SASL so that other mechanisms are able to use
>> the
>> > > > > > principal and credentials stored in subject to do
>> authentication.
>> > > > > >
>> > > > > > I am thinking to have below interface that can be extended by
>> users
>> > > to
>> > > > > > build the SASL client/server instead of having an AuthCallback.
>> > With
>> > > > this
>> > > > > > interface users are able to add their own security provider
>> before
>> > > > > > client/server is returned by SASL. Any thoughts?
>> > > > > >
>> > > > > > Interface SaslClientBuilder {
>> > > > > >
>> > > > > > SaslClient build(mechs, subject, host, otherparams)
>> > > > > > }
>> > > > > >
>> > > > > > Interface SaslServerBuilder {
>> > > > > > SaslServer build(mechs, subject, host, otherparams)
>> > > > > > }
>> > > > > >
>> > > > > > On Wed, 27 Jan 2016 at 18:54 Rajini Sivaram <
>> > > > > rajinisiva...@googlemail.com>
>> > > > > > wrote:
>> > > > > >
>> > > > > > > Tao,
>> > > > > > >
>> > > > > > > Thank you for the explanation. I couldn't find a standard Java
>> > > > > interface
>> > > > > > > that would be suitable, so will define one based on your
>> > > requirement
>> > > > > and
>> > > > > > > update the KIP.
>> > > > > > >
>> > > > > > > Rega

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-01-29 Thread tao xiao
Hi Rajini,

Do you consider exposing Subject to AuthCallback as well? It is useful for
users building their own SASL mechanism so that we have control  where to
put logon data in subject and how to manipulate in SASL callback


On Fri, 29 Jan 2016 at 18:04 Rajini Sivaram 
wrote:

> Following on from the KIP meeting on Tuesday, I have updated the KIP with a
> flow for negotiation of mechanisms to support multiple SASL mechanisms
> within a broker. I have also added a configurable Login interface to
> support custom mechanisms which require ticket refresh - requested by Tao
> Xiao.
>
> I will work on updating the PR in KAFKA-3149 over the next few days since
> it will be useful for review.
>
> All comments and suggestions are welcome.
>
>
> On Thu, Jan 28, 2016 at 2:35 PM, tao xiao  wrote:
>
> > Sounds like a good approach to add provider in login module. Would love
> to
> > see updates in the PR to reflect the changes in Login and
> > AuthCallbackHandler.
> >
> > On Thu, 28 Jan 2016 at 19:31 Rajini Sivaram <
> rajinisiva...@googlemail.com>
> > wrote:
> >
> > > Tao,
> > >
> > > We currently add the security provider in a static initializer in our
> > login
> > > module. This ensures that the security provider is always installed
> > before
> > > Kafka creates SaslServer/SaslClient. As you say, it is also possible to
> > > insert code into your application to add security provider before Kafka
> > > clients are created. Since you can also configure the JDK to add new
> > > security providers, I am not sure if there is value in adding more
> > > configuration in Kafka to add security providers.
> > >
> > > On Thu, Jan 28, 2016 at 2:25 AM, tao xiao 
> wrote:
> > >
> > > > The callback works for me as long as it has access to Subject and
> > mechs.
> > > > The other thing is how we can inject the customized security provider
> > via
> > > > Security.addProvider()? If I want to implement my own SASL mech I
> need
> > to
> > > > call the addProvider() before SASL.create so that my own
> implementation
> > > of
> > > > SASLClient/Sever can be returned. Any thoughts on this? we can either
> > let
> > > > users inject the provider in their logic code before creating a
> > > > producer/consumer or Kafka does it for users
> > > >
> > > > On Thu, 28 Jan 2016 at 03:36 Rajini Sivaram <
> > > rajinisiva...@googlemail.com>
> > > > wrote:
> > > >
> > > > > Hi Tao,
> > > > >
> > > > > *javax.security.auth.callback.**CallbackHandler *is the standard
> way
> > in
> > > > > which SASL clients and server obtain additional mechanism specific
> > > > > input. *AuthCallbackHandler
> > > > > *simply extends this interface to propagate configuration
> > properties. I
> > > > was
> > > > > going to provide SASL mechanism and Subject to the callback
> handlers
> > as
> > > > > well since the default handlers use these.
> > > > >
> > > > > Your SaslServer/SaslClient implementation can obtain the Subject
> > using
> > > > > *Subject.getSubject(**AccessController.getContext(). *But it will
> be
> > > good
> > > > > to know if callback handlers would work for you - apart from
> standard
> > > > > callbacks like PasswordCallback, you can define your own callbacks
> > too
> > > if
> > > > > you require.
> > > > >
> > > > >
> > > > >
> > > > > On Wed, Jan 27, 2016 at 3:59 PM, tao xiao 
> > > wrote:
> > > > >
> > > > > > Thanks Rajini. The other thing in my mind is that we should find
> a
> > > way
> > > > to
> > > > > > expose subject to SASL so that other mechanisms are able to use
> the
> > > > > > principal and credentials stored in subject to do authentication.
> > > > > >
> > > > > > I am thinking to have below interface that can be extended by
> users
> > > to
> > > > > > build the SASL client/server instead of having an AuthCallback.
> > With
> > > > this
> > > > > > interface users are able to add their own security provider
> before
> > > > > > client/server is returned by SASL. Any thoughts?
> > > > > >
> > > > > > Interface SaslClientBuilder {
> > > > > >
> > > > > > SaslClient build(mechs, subject, host, otherparams)
> > > > > > }
> > > > > >
> > > > > > Interface SaslServerBuilder {
> > > > > > SaslServer build(mechs, subject, host, otherparams)
> > > > > > }
> > > > > >
> > > > > > On Wed, 27 Jan 2016 at 18:54 Rajini Sivaram <
> > > > > rajinisiva...@googlemail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Tao,
> > > > > > >
> > > > > > > Thank you for the explanation. I couldn't find a standard Java
> > > > > interface
> > > > > > > that would be suitable, so will define one based on your
> > > requirement
> > > > > and
> > > > > > > update the KIP.
> > > > > > >
> > > > > > > Regards,
> > > > > > >
> > > > > > > Rajini
> > > > > > >
> > > > > > > On Wed, Jan 27, 2016 at 2:12 AM, tao xiao <
> xiaotao...@gmail.com>
> > > > > wrote:
> > > > > > >
> > > > > > > > Hi Rajini,
> > > > > > > >
> > > > > > > > One requirement I have is to refresh the login token every X
> > > hours.
> > > > > > Like
> > > > > > > > what the 

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-01-29 Thread Rajini Sivaram
Following on from the KIP meeting on Tuesday, I have updated the KIP with a
flow for negotiation of mechanisms to support multiple SASL mechanisms
within a broker. I have also added a configurable Login interface to
support custom mechanisms which require ticket refresh - requested by Tao
Xiao.

I will work on updating the PR in KAFKA-3149 over the next few days since
it will be useful for review.

All comments and suggestions are welcome.


On Thu, Jan 28, 2016 at 2:35 PM, tao xiao  wrote:

> Sounds like a good approach to add provider in login module. Would love to
> see updates in the PR to reflect the changes in Login and
> AuthCallbackHandler.
>
> On Thu, 28 Jan 2016 at 19:31 Rajini Sivaram 
> wrote:
>
> > Tao,
> >
> > We currently add the security provider in a static initializer in our
> login
> > module. This ensures that the security provider is always installed
> before
> > Kafka creates SaslServer/SaslClient. As you say, it is also possible to
> > insert code into your application to add security provider before Kafka
> > clients are created. Since you can also configure the JDK to add new
> > security providers, I am not sure if there is value in adding more
> > configuration in Kafka to add security providers.
> >
> > On Thu, Jan 28, 2016 at 2:25 AM, tao xiao  wrote:
> >
> > > The callback works for me as long as it has access to Subject and
> mechs.
> > > The other thing is how we can inject the customized security provider
> via
> > > Security.addProvider()? If I want to implement my own SASL mech I need
> to
> > > call the addProvider() before SASL.create so that my own implementation
> > of
> > > SASLClient/Sever can be returned. Any thoughts on this? we can either
> let
> > > users inject the provider in their logic code before creating a
> > > producer/consumer or Kafka does it for users
> > >
> > > On Thu, 28 Jan 2016 at 03:36 Rajini Sivaram <
> > rajinisiva...@googlemail.com>
> > > wrote:
> > >
> > > > Hi Tao,
> > > >
> > > > *javax.security.auth.callback.**CallbackHandler *is the standard way
> in
> > > > which SASL clients and server obtain additional mechanism specific
> > > > input. *AuthCallbackHandler
> > > > *simply extends this interface to propagate configuration
> properties. I
> > > was
> > > > going to provide SASL mechanism and Subject to the callback handlers
> as
> > > > well since the default handlers use these.
> > > >
> > > > Your SaslServer/SaslClient implementation can obtain the Subject
> using
> > > > *Subject.getSubject(**AccessController.getContext(). *But it will be
> > good
> > > > to know if callback handlers would work for you - apart from standard
> > > > callbacks like PasswordCallback, you can define your own callbacks
> too
> > if
> > > > you require.
> > > >
> > > >
> > > >
> > > > On Wed, Jan 27, 2016 at 3:59 PM, tao xiao 
> > wrote:
> > > >
> > > > > Thanks Rajini. The other thing in my mind is that we should find a
> > way
> > > to
> > > > > expose subject to SASL so that other mechanisms are able to use the
> > > > > principal and credentials stored in subject to do authentication.
> > > > >
> > > > > I am thinking to have below interface that can be extended by users
> > to
> > > > > build the SASL client/server instead of having an AuthCallback.
> With
> > > this
> > > > > interface users are able to add their own security provider before
> > > > > client/server is returned by SASL. Any thoughts?
> > > > >
> > > > > Interface SaslClientBuilder {
> > > > >
> > > > > SaslClient build(mechs, subject, host, otherparams)
> > > > > }
> > > > >
> > > > > Interface SaslServerBuilder {
> > > > > SaslServer build(mechs, subject, host, otherparams)
> > > > > }
> > > > >
> > > > > On Wed, 27 Jan 2016 at 18:54 Rajini Sivaram <
> > > > rajinisiva...@googlemail.com>
> > > > > wrote:
> > > > >
> > > > > > Tao,
> > > > > >
> > > > > > Thank you for the explanation. I couldn't find a standard Java
> > > > interface
> > > > > > that would be suitable, so will define one based on your
> > requirement
> > > > and
> > > > > > update the KIP.
> > > > > >
> > > > > > Regards,
> > > > > >
> > > > > > Rajini
> > > > > >
> > > > > > On Wed, Jan 27, 2016 at 2:12 AM, tao xiao 
> > > > wrote:
> > > > > >
> > > > > > > Hi Rajini,
> > > > > > >
> > > > > > > One requirement I have is to refresh the login token every X
> > hours.
> > > > > Like
> > > > > > > what the Kerberos login does I need to have a background thread
> > > that
> > > > > > > refreshes the token periodically.
> > > > > > >
> > > > > > > I understand most of the login logic would be simple but it is
> > good
> > > > > that
> > > > > > we
> > > > > > > can expose the logic login to users and let them decide what
> they
> > > > want
> > > > > to
> > > > > > > do. And we can have a fallback login component that is used if
> > > users
> > > > > dont
> > > > > > > specify it.
> > > > > > >
> > > > > > > On Tue, 26 Jan 2016 at 20:07 Rajini Sivaram <
> > > > > > rajinisiva...@googlemail.com>
> > > > > > > wrot

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-01-28 Thread tao xiao
Sounds like a good approach to add provider in login module. Would love to
see updates in the PR to reflect the changes in Login and
AuthCallbackHandler.

On Thu, 28 Jan 2016 at 19:31 Rajini Sivaram 
wrote:

> Tao,
>
> We currently add the security provider in a static initializer in our login
> module. This ensures that the security provider is always installed before
> Kafka creates SaslServer/SaslClient. As you say, it is also possible to
> insert code into your application to add security provider before Kafka
> clients are created. Since you can also configure the JDK to add new
> security providers, I am not sure if there is value in adding more
> configuration in Kafka to add security providers.
>
> On Thu, Jan 28, 2016 at 2:25 AM, tao xiao  wrote:
>
> > The callback works for me as long as it has access to Subject and mechs.
> > The other thing is how we can inject the customized security provider via
> > Security.addProvider()? If I want to implement my own SASL mech I need to
> > call the addProvider() before SASL.create so that my own implementation
> of
> > SASLClient/Sever can be returned. Any thoughts on this? we can either let
> > users inject the provider in their logic code before creating a
> > producer/consumer or Kafka does it for users
> >
> > On Thu, 28 Jan 2016 at 03:36 Rajini Sivaram <
> rajinisiva...@googlemail.com>
> > wrote:
> >
> > > Hi Tao,
> > >
> > > *javax.security.auth.callback.**CallbackHandler *is the standard way in
> > > which SASL clients and server obtain additional mechanism specific
> > > input. *AuthCallbackHandler
> > > *simply extends this interface to propagate configuration properties. I
> > was
> > > going to provide SASL mechanism and Subject to the callback handlers as
> > > well since the default handlers use these.
> > >
> > > Your SaslServer/SaslClient implementation can obtain the Subject using
> > > *Subject.getSubject(**AccessController.getContext(). *But it will be
> good
> > > to know if callback handlers would work for you - apart from standard
> > > callbacks like PasswordCallback, you can define your own callbacks too
> if
> > > you require.
> > >
> > >
> > >
> > > On Wed, Jan 27, 2016 at 3:59 PM, tao xiao 
> wrote:
> > >
> > > > Thanks Rajini. The other thing in my mind is that we should find a
> way
> > to
> > > > expose subject to SASL so that other mechanisms are able to use the
> > > > principal and credentials stored in subject to do authentication.
> > > >
> > > > I am thinking to have below interface that can be extended by users
> to
> > > > build the SASL client/server instead of having an AuthCallback. With
> > this
> > > > interface users are able to add their own security provider before
> > > > client/server is returned by SASL. Any thoughts?
> > > >
> > > > Interface SaslClientBuilder {
> > > >
> > > > SaslClient build(mechs, subject, host, otherparams)
> > > > }
> > > >
> > > > Interface SaslServerBuilder {
> > > > SaslServer build(mechs, subject, host, otherparams)
> > > > }
> > > >
> > > > On Wed, 27 Jan 2016 at 18:54 Rajini Sivaram <
> > > rajinisiva...@googlemail.com>
> > > > wrote:
> > > >
> > > > > Tao,
> > > > >
> > > > > Thank you for the explanation. I couldn't find a standard Java
> > > interface
> > > > > that would be suitable, so will define one based on your
> requirement
> > > and
> > > > > update the KIP.
> > > > >
> > > > > Regards,
> > > > >
> > > > > Rajini
> > > > >
> > > > > On Wed, Jan 27, 2016 at 2:12 AM, tao xiao 
> > > wrote:
> > > > >
> > > > > > Hi Rajini,
> > > > > >
> > > > > > One requirement I have is to refresh the login token every X
> hours.
> > > > Like
> > > > > > what the Kerberos login does I need to have a background thread
> > that
> > > > > > refreshes the token periodically.
> > > > > >
> > > > > > I understand most of the login logic would be simple but it is
> good
> > > > that
> > > > > we
> > > > > > can expose the logic login to users and let them decide what they
> > > want
> > > > to
> > > > > > do. And we can have a fallback login component that is used if
> > users
> > > > dont
> > > > > > specify it.
> > > > > >
> > > > > > On Tue, 26 Jan 2016 at 20:07 Rajini Sivaram <
> > > > > rajinisiva...@googlemail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Hi Tao,
> > > > > > >
> > > > > > > Thank you for the review. The changes I had in mind are in the
> PR
> > > > > > > https://github.com/apache/kafka/pull/812. Login for
> non-Kerberos
> > > > > > protocols
> > > > > > > contains very little logic. I was expecting that combined with
> a
> > > > custom
> > > > > > > login module specified in JAAS configuration, this would give
> > > > > sufficient
> > > > > > > flexibility. Is there a specific usecase you have in mind where
> > you
> > > > > need
> > > > > > to
> > > > > > > customize the Login code?
> > > > > > >
> > > > > > > Regards,
> > > > > > >
> > > > > > > Rajini
> > > > > > >
> > > > > > > On Tue, Jan 26, 2016 at 11:15 AM, tao xiao <
> xiaotao...@gmail.com
> > >
> > 

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-01-28 Thread Rajini Sivaram
Tao,

We currently add the security provider in a static initializer in our login
module. This ensures that the security provider is always installed before
Kafka creates SaslServer/SaslClient. As you say, it is also possible to
insert code into your application to add security provider before Kafka
clients are created. Since you can also configure the JDK to add new
security providers, I am not sure if there is value in adding more
configuration in Kafka to add security providers.

On Thu, Jan 28, 2016 at 2:25 AM, tao xiao  wrote:

> The callback works for me as long as it has access to Subject and mechs.
> The other thing is how we can inject the customized security provider via
> Security.addProvider()? If I want to implement my own SASL mech I need to
> call the addProvider() before SASL.create so that my own implementation of
> SASLClient/Sever can be returned. Any thoughts on this? we can either let
> users inject the provider in their logic code before creating a
> producer/consumer or Kafka does it for users
>
> On Thu, 28 Jan 2016 at 03:36 Rajini Sivaram 
> wrote:
>
> > Hi Tao,
> >
> > *javax.security.auth.callback.**CallbackHandler *is the standard way in
> > which SASL clients and server obtain additional mechanism specific
> > input. *AuthCallbackHandler
> > *simply extends this interface to propagate configuration properties. I
> was
> > going to provide SASL mechanism and Subject to the callback handlers as
> > well since the default handlers use these.
> >
> > Your SaslServer/SaslClient implementation can obtain the Subject using
> > *Subject.getSubject(**AccessController.getContext(). *But it will be good
> > to know if callback handlers would work for you - apart from standard
> > callbacks like PasswordCallback, you can define your own callbacks too if
> > you require.
> >
> >
> >
> > On Wed, Jan 27, 2016 at 3:59 PM, tao xiao  wrote:
> >
> > > Thanks Rajini. The other thing in my mind is that we should find a way
> to
> > > expose subject to SASL so that other mechanisms are able to use the
> > > principal and credentials stored in subject to do authentication.
> > >
> > > I am thinking to have below interface that can be extended by users to
> > > build the SASL client/server instead of having an AuthCallback. With
> this
> > > interface users are able to add their own security provider before
> > > client/server is returned by SASL. Any thoughts?
> > >
> > > Interface SaslClientBuilder {
> > >
> > > SaslClient build(mechs, subject, host, otherparams)
> > > }
> > >
> > > Interface SaslServerBuilder {
> > > SaslServer build(mechs, subject, host, otherparams)
> > > }
> > >
> > > On Wed, 27 Jan 2016 at 18:54 Rajini Sivaram <
> > rajinisiva...@googlemail.com>
> > > wrote:
> > >
> > > > Tao,
> > > >
> > > > Thank you for the explanation. I couldn't find a standard Java
> > interface
> > > > that would be suitable, so will define one based on your requirement
> > and
> > > > update the KIP.
> > > >
> > > > Regards,
> > > >
> > > > Rajini
> > > >
> > > > On Wed, Jan 27, 2016 at 2:12 AM, tao xiao 
> > wrote:
> > > >
> > > > > Hi Rajini,
> > > > >
> > > > > One requirement I have is to refresh the login token every X hours.
> > > Like
> > > > > what the Kerberos login does I need to have a background thread
> that
> > > > > refreshes the token periodically.
> > > > >
> > > > > I understand most of the login logic would be simple but it is good
> > > that
> > > > we
> > > > > can expose the logic login to users and let them decide what they
> > want
> > > to
> > > > > do. And we can have a fallback login component that is used if
> users
> > > dont
> > > > > specify it.
> > > > >
> > > > > On Tue, 26 Jan 2016 at 20:07 Rajini Sivaram <
> > > > rajinisiva...@googlemail.com>
> > > > > wrote:
> > > > >
> > > > > > Hi Tao,
> > > > > >
> > > > > > Thank you for the review. The changes I had in mind are in the PR
> > > > > > https://github.com/apache/kafka/pull/812. Login for non-Kerberos
> > > > > protocols
> > > > > > contains very little logic. I was expecting that combined with a
> > > custom
> > > > > > login module specified in JAAS configuration, this would give
> > > > sufficient
> > > > > > flexibility. Is there a specific usecase you have in mind where
> you
> > > > need
> > > > > to
> > > > > > customize the Login code?
> > > > > >
> > > > > > Regards,
> > > > > >
> > > > > > Rajini
> > > > > >
> > > > > > On Tue, Jan 26, 2016 at 11:15 AM, tao xiao  >
> > > > wrote:
> > > > > >
> > > > > > > Hi Rajini,
> > > > > > >
> > > > > > > I think it makes sense to change LoginManager or Login to an
> > > > interface
> > > > > > > which users can extend to provide their own logic of login
> > > otherwise
> > > > it
> > > > > > is
> > > > > > > hard for users to implement a custom SASL mechanism but have no
> > > > control
> > > > > > > over login
> > > > > > >
> > > > > > > On Tue, 26 Jan 2016 at 18:45 Ismael Juma 
> > > wrote:
> > > > > > >
> > > > > > > > Hi Rajini,
> > > > > > > >
> > > > > >

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-01-27 Thread tao xiao
The callback works for me as long as it has access to Subject and mechs.
The other thing is how we can inject the customized security provider via
Security.addProvider()? If I want to implement my own SASL mech I need to
call the addProvider() before SASL.create so that my own implementation of
SASLClient/Sever can be returned. Any thoughts on this? we can either let
users inject the provider in their logic code before creating a
producer/consumer or Kafka does it for users

On Thu, 28 Jan 2016 at 03:36 Rajini Sivaram 
wrote:

> Hi Tao,
>
> *javax.security.auth.callback.**CallbackHandler *is the standard way in
> which SASL clients and server obtain additional mechanism specific
> input. *AuthCallbackHandler
> *simply extends this interface to propagate configuration properties. I was
> going to provide SASL mechanism and Subject to the callback handlers as
> well since the default handlers use these.
>
> Your SaslServer/SaslClient implementation can obtain the Subject using
> *Subject.getSubject(**AccessController.getContext(). *But it will be good
> to know if callback handlers would work for you - apart from standard
> callbacks like PasswordCallback, you can define your own callbacks too if
> you require.
>
>
>
> On Wed, Jan 27, 2016 at 3:59 PM, tao xiao  wrote:
>
> > Thanks Rajini. The other thing in my mind is that we should find a way to
> > expose subject to SASL so that other mechanisms are able to use the
> > principal and credentials stored in subject to do authentication.
> >
> > I am thinking to have below interface that can be extended by users to
> > build the SASL client/server instead of having an AuthCallback. With this
> > interface users are able to add their own security provider before
> > client/server is returned by SASL. Any thoughts?
> >
> > Interface SaslClientBuilder {
> >
> > SaslClient build(mechs, subject, host, otherparams)
> > }
> >
> > Interface SaslServerBuilder {
> > SaslServer build(mechs, subject, host, otherparams)
> > }
> >
> > On Wed, 27 Jan 2016 at 18:54 Rajini Sivaram <
> rajinisiva...@googlemail.com>
> > wrote:
> >
> > > Tao,
> > >
> > > Thank you for the explanation. I couldn't find a standard Java
> interface
> > > that would be suitable, so will define one based on your requirement
> and
> > > update the KIP.
> > >
> > > Regards,
> > >
> > > Rajini
> > >
> > > On Wed, Jan 27, 2016 at 2:12 AM, tao xiao 
> wrote:
> > >
> > > > Hi Rajini,
> > > >
> > > > One requirement I have is to refresh the login token every X hours.
> > Like
> > > > what the Kerberos login does I need to have a background thread that
> > > > refreshes the token periodically.
> > > >
> > > > I understand most of the login logic would be simple but it is good
> > that
> > > we
> > > > can expose the logic login to users and let them decide what they
> want
> > to
> > > > do. And we can have a fallback login component that is used if users
> > dont
> > > > specify it.
> > > >
> > > > On Tue, 26 Jan 2016 at 20:07 Rajini Sivaram <
> > > rajinisiva...@googlemail.com>
> > > > wrote:
> > > >
> > > > > Hi Tao,
> > > > >
> > > > > Thank you for the review. The changes I had in mind are in the PR
> > > > > https://github.com/apache/kafka/pull/812. Login for non-Kerberos
> > > > protocols
> > > > > contains very little logic. I was expecting that combined with a
> > custom
> > > > > login module specified in JAAS configuration, this would give
> > > sufficient
> > > > > flexibility. Is there a specific usecase you have in mind where you
> > > need
> > > > to
> > > > > customize the Login code?
> > > > >
> > > > > Regards,
> > > > >
> > > > > Rajini
> > > > >
> > > > > On Tue, Jan 26, 2016 at 11:15 AM, tao xiao 
> > > wrote:
> > > > >
> > > > > > Hi Rajini,
> > > > > >
> > > > > > I think it makes sense to change LoginManager or Login to an
> > > interface
> > > > > > which users can extend to provide their own logic of login
> > otherwise
> > > it
> > > > > is
> > > > > > hard for users to implement a custom SASL mechanism but have no
> > > control
> > > > > > over login
> > > > > >
> > > > > > On Tue, 26 Jan 2016 at 18:45 Ismael Juma 
> > wrote:
> > > > > >
> > > > > > > Hi Rajini,
> > > > > > >
> > > > > > > Thanks for the KIP. As stated in the KIP, it does not address
> > > > "Support
> > > > > > for
> > > > > > > multiple SASL mechanisms within a broker". Maybe we should also
> > > > mention
> > > > > > > this in the "Rejected Alternatives" section with the
> reasoning. I
> > > > think
> > > > > > > it's particularly relevant to understand if it's not being
> > proposed
> > > > > > because
> > > > > > > we don't think it's useful or due to the additional
> > implementation
> > > > > > > complexity (it's probably a combination). If we think this
> could
> > be
> > > > > > useful
> > > > > > > in the future, it would also be worth thinking about how it is
> > > > affected
> > > > > > if
> > > > > > > we do KIP-43 first (ie will it be easier, harder, etc.)
> > > > > > >
> > > > > > > Thanks,
> 

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-01-27 Thread Rajini Sivaram
Hi Tao,

*javax.security.auth.callback.**CallbackHandler *is the standard way in
which SASL clients and server obtain additional mechanism specific
input. *AuthCallbackHandler
*simply extends this interface to propagate configuration properties. I was
going to provide SASL mechanism and Subject to the callback handlers as
well since the default handlers use these.

Your SaslServer/SaslClient implementation can obtain the Subject using
*Subject.getSubject(**AccessController.getContext(). *But it will be good
to know if callback handlers would work for you - apart from standard
callbacks like PasswordCallback, you can define your own callbacks too if
you require.



On Wed, Jan 27, 2016 at 3:59 PM, tao xiao  wrote:

> Thanks Rajini. The other thing in my mind is that we should find a way to
> expose subject to SASL so that other mechanisms are able to use the
> principal and credentials stored in subject to do authentication.
>
> I am thinking to have below interface that can be extended by users to
> build the SASL client/server instead of having an AuthCallback. With this
> interface users are able to add their own security provider before
> client/server is returned by SASL. Any thoughts?
>
> Interface SaslClientBuilder {
>
> SaslClient build(mechs, subject, host, otherparams)
> }
>
> Interface SaslServerBuilder {
> SaslServer build(mechs, subject, host, otherparams)
> }
>
> On Wed, 27 Jan 2016 at 18:54 Rajini Sivaram 
> wrote:
>
> > Tao,
> >
> > Thank you for the explanation. I couldn't find a standard Java interface
> > that would be suitable, so will define one based on your requirement and
> > update the KIP.
> >
> > Regards,
> >
> > Rajini
> >
> > On Wed, Jan 27, 2016 at 2:12 AM, tao xiao  wrote:
> >
> > > Hi Rajini,
> > >
> > > One requirement I have is to refresh the login token every X hours.
> Like
> > > what the Kerberos login does I need to have a background thread that
> > > refreshes the token periodically.
> > >
> > > I understand most of the login logic would be simple but it is good
> that
> > we
> > > can expose the logic login to users and let them decide what they want
> to
> > > do. And we can have a fallback login component that is used if users
> dont
> > > specify it.
> > >
> > > On Tue, 26 Jan 2016 at 20:07 Rajini Sivaram <
> > rajinisiva...@googlemail.com>
> > > wrote:
> > >
> > > > Hi Tao,
> > > >
> > > > Thank you for the review. The changes I had in mind are in the PR
> > > > https://github.com/apache/kafka/pull/812. Login for non-Kerberos
> > > protocols
> > > > contains very little logic. I was expecting that combined with a
> custom
> > > > login module specified in JAAS configuration, this would give
> > sufficient
> > > > flexibility. Is there a specific usecase you have in mind where you
> > need
> > > to
> > > > customize the Login code?
> > > >
> > > > Regards,
> > > >
> > > > Rajini
> > > >
> > > > On Tue, Jan 26, 2016 at 11:15 AM, tao xiao 
> > wrote:
> > > >
> > > > > Hi Rajini,
> > > > >
> > > > > I think it makes sense to change LoginManager or Login to an
> > interface
> > > > > which users can extend to provide their own logic of login
> otherwise
> > it
> > > > is
> > > > > hard for users to implement a custom SASL mechanism but have no
> > control
> > > > > over login
> > > > >
> > > > > On Tue, 26 Jan 2016 at 18:45 Ismael Juma 
> wrote:
> > > > >
> > > > > > Hi Rajini,
> > > > > >
> > > > > > Thanks for the KIP. As stated in the KIP, it does not address
> > > "Support
> > > > > for
> > > > > > multiple SASL mechanisms within a broker". Maybe we should also
> > > mention
> > > > > > this in the "Rejected Alternatives" section with the reasoning. I
> > > think
> > > > > > it's particularly relevant to understand if it's not being
> proposed
> > > > > because
> > > > > > we don't think it's useful or due to the additional
> implementation
> > > > > > complexity (it's probably a combination). If we think this could
> be
> > > > > useful
> > > > > > in the future, it would also be worth thinking about how it is
> > > affected
> > > > > if
> > > > > > we do KIP-43 first (ie will it be easier, harder, etc.)
> > > > > >
> > > > > > Thanks,
> > > > > > Ismael
> > > > > >
> > > > > > On Mon, Jan 25, 2016 at 9:55 PM, Rajini Sivaram <
> > > > > > rajinisiva...@googlemail.com> wrote:
> > > > > >
> > > > > > > I have just created KIP-43 to extend the SASL implementation in
> > > Kafka
> > > > > to
> > > > > > > support new SASL mechanisms.
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-43%3A+Kafka+SASL+enhancements
> > > > > > >
> > > > > > >
> > > > > > > Comments and suggestions are appreciated.
> > > > > > >
> > > > > > >
> > > > > > > Thank you...
> > > > > > >
> > > > > > > Regards,
> > > > > > >
> > > > > > > Rajini
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>



-- 
Regards,

Rajini


Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-01-27 Thread tao xiao
Thanks Rajini. The other thing in my mind is that we should find a way to
expose subject to SASL so that other mechanisms are able to use the
principal and credentials stored in subject to do authentication.

I am thinking to have below interface that can be extended by users to
build the SASL client/server instead of having an AuthCallback. With this
interface users are able to add their own security provider before
client/server is returned by SASL. Any thoughts?

Interface SaslClientBuilder {

SaslClient build(mechs, subject, host, otherparams)
}

Interface SaslServerBuilder {
SaslServer build(mechs, subject, host, otherparams)
}

On Wed, 27 Jan 2016 at 18:54 Rajini Sivaram 
wrote:

> Tao,
>
> Thank you for the explanation. I couldn't find a standard Java interface
> that would be suitable, so will define one based on your requirement and
> update the KIP.
>
> Regards,
>
> Rajini
>
> On Wed, Jan 27, 2016 at 2:12 AM, tao xiao  wrote:
>
> > Hi Rajini,
> >
> > One requirement I have is to refresh the login token every X hours. Like
> > what the Kerberos login does I need to have a background thread that
> > refreshes the token periodically.
> >
> > I understand most of the login logic would be simple but it is good that
> we
> > can expose the logic login to users and let them decide what they want to
> > do. And we can have a fallback login component that is used if users dont
> > specify it.
> >
> > On Tue, 26 Jan 2016 at 20:07 Rajini Sivaram <
> rajinisiva...@googlemail.com>
> > wrote:
> >
> > > Hi Tao,
> > >
> > > Thank you for the review. The changes I had in mind are in the PR
> > > https://github.com/apache/kafka/pull/812. Login for non-Kerberos
> > protocols
> > > contains very little logic. I was expecting that combined with a custom
> > > login module specified in JAAS configuration, this would give
> sufficient
> > > flexibility. Is there a specific usecase you have in mind where you
> need
> > to
> > > customize the Login code?
> > >
> > > Regards,
> > >
> > > Rajini
> > >
> > > On Tue, Jan 26, 2016 at 11:15 AM, tao xiao 
> wrote:
> > >
> > > > Hi Rajini,
> > > >
> > > > I think it makes sense to change LoginManager or Login to an
> interface
> > > > which users can extend to provide their own logic of login otherwise
> it
> > > is
> > > > hard for users to implement a custom SASL mechanism but have no
> control
> > > > over login
> > > >
> > > > On Tue, 26 Jan 2016 at 18:45 Ismael Juma  wrote:
> > > >
> > > > > Hi Rajini,
> > > > >
> > > > > Thanks for the KIP. As stated in the KIP, it does not address
> > "Support
> > > > for
> > > > > multiple SASL mechanisms within a broker". Maybe we should also
> > mention
> > > > > this in the "Rejected Alternatives" section with the reasoning. I
> > think
> > > > > it's particularly relevant to understand if it's not being proposed
> > > > because
> > > > > we don't think it's useful or due to the additional implementation
> > > > > complexity (it's probably a combination). If we think this could be
> > > > useful
> > > > > in the future, it would also be worth thinking about how it is
> > affected
> > > > if
> > > > > we do KIP-43 first (ie will it be easier, harder, etc.)
> > > > >
> > > > > Thanks,
> > > > > Ismael
> > > > >
> > > > > On Mon, Jan 25, 2016 at 9:55 PM, Rajini Sivaram <
> > > > > rajinisiva...@googlemail.com> wrote:
> > > > >
> > > > > > I have just created KIP-43 to extend the SASL implementation in
> > Kafka
> > > > to
> > > > > > support new SASL mechanisms.
> > > > > >
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-43%3A+Kafka+SASL+enhancements
> > > > > >
> > > > > >
> > > > > > Comments and suggestions are appreciated.
> > > > > >
> > > > > >
> > > > > > Thank you...
> > > > > >
> > > > > > Regards,
> > > > > >
> > > > > > Rajini
> > > > > >
> > > > >
> > > >
> > >
> >
>


Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-01-27 Thread Rajini Sivaram
Tao,

Thank you for the explanation. I couldn't find a standard Java interface
that would be suitable, so will define one based on your requirement and
update the KIP.

Regards,

Rajini

On Wed, Jan 27, 2016 at 2:12 AM, tao xiao  wrote:

> Hi Rajini,
>
> One requirement I have is to refresh the login token every X hours. Like
> what the Kerberos login does I need to have a background thread that
> refreshes the token periodically.
>
> I understand most of the login logic would be simple but it is good that we
> can expose the logic login to users and let them decide what they want to
> do. And we can have a fallback login component that is used if users dont
> specify it.
>
> On Tue, 26 Jan 2016 at 20:07 Rajini Sivaram 
> wrote:
>
> > Hi Tao,
> >
> > Thank you for the review. The changes I had in mind are in the PR
> > https://github.com/apache/kafka/pull/812. Login for non-Kerberos
> protocols
> > contains very little logic. I was expecting that combined with a custom
> > login module specified in JAAS configuration, this would give sufficient
> > flexibility. Is there a specific usecase you have in mind where you need
> to
> > customize the Login code?
> >
> > Regards,
> >
> > Rajini
> >
> > On Tue, Jan 26, 2016 at 11:15 AM, tao xiao  wrote:
> >
> > > Hi Rajini,
> > >
> > > I think it makes sense to change LoginManager or Login to an interface
> > > which users can extend to provide their own logic of login otherwise it
> > is
> > > hard for users to implement a custom SASL mechanism but have no control
> > > over login
> > >
> > > On Tue, 26 Jan 2016 at 18:45 Ismael Juma  wrote:
> > >
> > > > Hi Rajini,
> > > >
> > > > Thanks for the KIP. As stated in the KIP, it does not address
> "Support
> > > for
> > > > multiple SASL mechanisms within a broker". Maybe we should also
> mention
> > > > this in the "Rejected Alternatives" section with the reasoning. I
> think
> > > > it's particularly relevant to understand if it's not being proposed
> > > because
> > > > we don't think it's useful or due to the additional implementation
> > > > complexity (it's probably a combination). If we think this could be
> > > useful
> > > > in the future, it would also be worth thinking about how it is
> affected
> > > if
> > > > we do KIP-43 first (ie will it be easier, harder, etc.)
> > > >
> > > > Thanks,
> > > > Ismael
> > > >
> > > > On Mon, Jan 25, 2016 at 9:55 PM, Rajini Sivaram <
> > > > rajinisiva...@googlemail.com> wrote:
> > > >
> > > > > I have just created KIP-43 to extend the SASL implementation in
> Kafka
> > > to
> > > > > support new SASL mechanisms.
> > > > >
> > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-43%3A+Kafka+SASL+enhancements
> > > > >
> > > > >
> > > > > Comments and suggestions are appreciated.
> > > > >
> > > > >
> > > > > Thank you...
> > > > >
> > > > > Regards,
> > > > >
> > > > > Rajini
> > > > >
> > > >
> > >
> >
>


Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-01-26 Thread tao xiao
Hi Rajini,

One requirement I have is to refresh the login token every X hours. Like
what the Kerberos login does I need to have a background thread that
refreshes the token periodically.

I understand most of the login logic would be simple but it is good that we
can expose the logic login to users and let them decide what they want to
do. And we can have a fallback login component that is used if users dont
specify it.

On Tue, 26 Jan 2016 at 20:07 Rajini Sivaram 
wrote:

> Hi Tao,
>
> Thank you for the review. The changes I had in mind are in the PR
> https://github.com/apache/kafka/pull/812. Login for non-Kerberos protocols
> contains very little logic. I was expecting that combined with a custom
> login module specified in JAAS configuration, this would give sufficient
> flexibility. Is there a specific usecase you have in mind where you need to
> customize the Login code?
>
> Regards,
>
> Rajini
>
> On Tue, Jan 26, 2016 at 11:15 AM, tao xiao  wrote:
>
> > Hi Rajini,
> >
> > I think it makes sense to change LoginManager or Login to an interface
> > which users can extend to provide their own logic of login otherwise it
> is
> > hard for users to implement a custom SASL mechanism but have no control
> > over login
> >
> > On Tue, 26 Jan 2016 at 18:45 Ismael Juma  wrote:
> >
> > > Hi Rajini,
> > >
> > > Thanks for the KIP. As stated in the KIP, it does not address "Support
> > for
> > > multiple SASL mechanisms within a broker". Maybe we should also mention
> > > this in the "Rejected Alternatives" section with the reasoning. I think
> > > it's particularly relevant to understand if it's not being proposed
> > because
> > > we don't think it's useful or due to the additional implementation
> > > complexity (it's probably a combination). If we think this could be
> > useful
> > > in the future, it would also be worth thinking about how it is affected
> > if
> > > we do KIP-43 first (ie will it be easier, harder, etc.)
> > >
> > > Thanks,
> > > Ismael
> > >
> > > On Mon, Jan 25, 2016 at 9:55 PM, Rajini Sivaram <
> > > rajinisiva...@googlemail.com> wrote:
> > >
> > > > I have just created KIP-43 to extend the SASL implementation in Kafka
> > to
> > > > support new SASL mechanisms.
> > > >
> > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-43%3A+Kafka+SASL+enhancements
> > > >
> > > >
> > > > Comments and suggestions are appreciated.
> > > >
> > > >
> > > > Thank you...
> > > >
> > > > Regards,
> > > >
> > > > Rajini
> > > >
> > >
> >
>


Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-01-26 Thread Rajini Sivaram
Ismael,

I have written up a section on supporting multiple mechanisms within a
Kafka broker. At the moment, it is under "Rejected Alternatives", even
though having thought about it, we should possibly include it in this KIP,
unless we are sure it is not going to come up as a requirement. We don't
actually need this feature, but it will be useful to know what others think.

Regards,

Rajini

On Tue, Jan 26, 2016 at 12:00 PM, Rajini Sivaram <
rajinisiva...@googlemail.com> wrote:

> Ismael,
>
> Thank you for your review. The main reason I didn't address the support
> for multiple mechanisms within a broker is because it requires changes to
> the wire protocol to propagate mechanisms. But I do agree that we need to
> understand whether it would be even harder to support this in the future.
> Will give it some thought and write it up in the KIP.
>
> Regards,
>
> Rajini
>
> On Tue, Jan 26, 2016 at 10:44 AM, Ismael Juma  wrote:
>
>> Hi Rajini,
>>
>> Thanks for the KIP. As stated in the KIP, it does not address "Support for
>> multiple SASL mechanisms within a broker". Maybe we should also mention
>> this in the "Rejected Alternatives" section with the reasoning. I think
>> it's particularly relevant to understand if it's not being proposed
>> because
>> we don't think it's useful or due to the additional implementation
>> complexity (it's probably a combination). If we think this could be useful
>> in the future, it would also be worth thinking about how it is affected if
>> we do KIP-43 first (ie will it be easier, harder, etc.)
>>
>> Thanks,
>> Ismael
>>
>> On Mon, Jan 25, 2016 at 9:55 PM, Rajini Sivaram <
>> rajinisiva...@googlemail.com> wrote:
>>
>> > I have just created KIP-43 to extend the SASL implementation in Kafka to
>> > support new SASL mechanisms.
>> >
>> >
>> >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-43%3A+Kafka+SASL+enhancements
>> >
>> >
>> > Comments and suggestions are appreciated.
>> >
>> >
>> > Thank you...
>> >
>> > Regards,
>> >
>> > Rajini
>> >
>>
>
>
>


Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-01-26 Thread Rajini Sivaram
Hi Tao,

Thank you for the review. The changes I had in mind are in the PR
https://github.com/apache/kafka/pull/812. Login for non-Kerberos protocols
contains very little logic. I was expecting that combined with a custom
login module specified in JAAS configuration, this would give sufficient
flexibility. Is there a specific usecase you have in mind where you need to
customize the Login code?

Regards,

Rajini

On Tue, Jan 26, 2016 at 11:15 AM, tao xiao  wrote:

> Hi Rajini,
>
> I think it makes sense to change LoginManager or Login to an interface
> which users can extend to provide their own logic of login otherwise it is
> hard for users to implement a custom SASL mechanism but have no control
> over login
>
> On Tue, 26 Jan 2016 at 18:45 Ismael Juma  wrote:
>
> > Hi Rajini,
> >
> > Thanks for the KIP. As stated in the KIP, it does not address "Support
> for
> > multiple SASL mechanisms within a broker". Maybe we should also mention
> > this in the "Rejected Alternatives" section with the reasoning. I think
> > it's particularly relevant to understand if it's not being proposed
> because
> > we don't think it's useful or due to the additional implementation
> > complexity (it's probably a combination). If we think this could be
> useful
> > in the future, it would also be worth thinking about how it is affected
> if
> > we do KIP-43 first (ie will it be easier, harder, etc.)
> >
> > Thanks,
> > Ismael
> >
> > On Mon, Jan 25, 2016 at 9:55 PM, Rajini Sivaram <
> > rajinisiva...@googlemail.com> wrote:
> >
> > > I have just created KIP-43 to extend the SASL implementation in Kafka
> to
> > > support new SASL mechanisms.
> > >
> > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-43%3A+Kafka+SASL+enhancements
> > >
> > >
> > > Comments and suggestions are appreciated.
> > >
> > >
> > > Thank you...
> > >
> > > Regards,
> > >
> > > Rajini
> > >
> >
>


Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-01-26 Thread Rajini Sivaram
Ismael,

Thank you for your review. The main reason I didn't address the support for
multiple mechanisms within a broker is because it requires changes to the
wire protocol to propagate mechanisms. But I do agree that we need to
understand whether it would be even harder to support this in the future.
Will give it some thought and write it up in the KIP.

Regards,

Rajini

On Tue, Jan 26, 2016 at 10:44 AM, Ismael Juma  wrote:

> Hi Rajini,
>
> Thanks for the KIP. As stated in the KIP, it does not address "Support for
> multiple SASL mechanisms within a broker". Maybe we should also mention
> this in the "Rejected Alternatives" section with the reasoning. I think
> it's particularly relevant to understand if it's not being proposed because
> we don't think it's useful or due to the additional implementation
> complexity (it's probably a combination). If we think this could be useful
> in the future, it would also be worth thinking about how it is affected if
> we do KIP-43 first (ie will it be easier, harder, etc.)
>
> Thanks,
> Ismael
>
> On Mon, Jan 25, 2016 at 9:55 PM, Rajini Sivaram <
> rajinisiva...@googlemail.com> wrote:
>
> > I have just created KIP-43 to extend the SASL implementation in Kafka to
> > support new SASL mechanisms.
> >
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-43%3A+Kafka+SASL+enhancements
> >
> >
> > Comments and suggestions are appreciated.
> >
> >
> > Thank you...
> >
> > Regards,
> >
> > Rajini
> >
>


Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-01-26 Thread tao xiao
Hi Rajini,

I think it makes sense to change LoginManager or Login to an interface
which users can extend to provide their own logic of login otherwise it is
hard for users to implement a custom SASL mechanism but have no control
over login

On Tue, 26 Jan 2016 at 18:45 Ismael Juma  wrote:

> Hi Rajini,
>
> Thanks for the KIP. As stated in the KIP, it does not address "Support for
> multiple SASL mechanisms within a broker". Maybe we should also mention
> this in the "Rejected Alternatives" section with the reasoning. I think
> it's particularly relevant to understand if it's not being proposed because
> we don't think it's useful or due to the additional implementation
> complexity (it's probably a combination). If we think this could be useful
> in the future, it would also be worth thinking about how it is affected if
> we do KIP-43 first (ie will it be easier, harder, etc.)
>
> Thanks,
> Ismael
>
> On Mon, Jan 25, 2016 at 9:55 PM, Rajini Sivaram <
> rajinisiva...@googlemail.com> wrote:
>
> > I have just created KIP-43 to extend the SASL implementation in Kafka to
> > support new SASL mechanisms.
> >
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-43%3A+Kafka+SASL+enhancements
> >
> >
> > Comments and suggestions are appreciated.
> >
> >
> > Thank you...
> >
> > Regards,
> >
> > Rajini
> >
>


Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-01-26 Thread Ismael Juma
Hi Rajini,

Thanks for the KIP. As stated in the KIP, it does not address "Support for
multiple SASL mechanisms within a broker". Maybe we should also mention
this in the "Rejected Alternatives" section with the reasoning. I think
it's particularly relevant to understand if it's not being proposed because
we don't think it's useful or due to the additional implementation
complexity (it's probably a combination). If we think this could be useful
in the future, it would also be worth thinking about how it is affected if
we do KIP-43 first (ie will it be easier, harder, etc.)

Thanks,
Ismael

On Mon, Jan 25, 2016 at 9:55 PM, Rajini Sivaram <
rajinisiva...@googlemail.com> wrote:

> I have just created KIP-43 to extend the SASL implementation in Kafka to
> support new SASL mechanisms.
>
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-43%3A+Kafka+SASL+enhancements
>
>
> Comments and suggestions are appreciated.
>
>
> Thank you...
>
> Regards,
>
> Rajini
>


[DISCUSS] KIP-43: Kafka SASL enhancements

2016-01-25 Thread Rajini Sivaram
I have just created KIP-43 to extend the SASL implementation in Kafka to
support new SASL mechanisms.

https://cwiki.apache.org/confluence/display/KAFKA/KIP-43%3A+Kafka+SASL+enhancements


Comments and suggestions are appreciated.


Thank you...

Regards,

Rajini