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

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

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

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

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

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 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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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 (

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

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 {

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

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

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

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

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

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

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

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

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

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

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

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

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

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 < >

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 >

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

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

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

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

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

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 >

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

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

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

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

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

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

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

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 (

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

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:

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

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

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

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

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

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

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

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

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 >

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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