Hi, Rajini,

One more thing. It seems that we should bump up the version of
SaslHandshakeRequest? This way, the client can figure out which SASL
mechanisms the broker is capable of supporting through ApiVersionRequest.
We discussed this briefly as part of KIP-43.

Thanks,

Jun



On Tue, Nov 1, 2016 at 7:41 AM, Rajini Sivaram <rajinisiva...@googlemail.com
> wrote:

> If there are no more comments, I will start vote on this KIP later this
> week. In the meantime, please feel free to post any feedback or
> suggestions. Initial implementation is here:
> https://github.com/apache/kafka/pull/2086.
>
> Thank you,
>
> Rajini
>
> On Thu, Oct 27, 2016 at 11:18 AM, Rajini Sivaram <
> rajinisiva...@googlemail.com> wrote:
>
> > Jun,
> >
> > 4) Agree, it does make the implementation simpler. Updated KIP.
> > 5) Thank you, that looks neater. Updated KIP.
> >
> > On Wed, Oct 26, 2016 at 6:59 PM, Jun Rao <j...@confluent.io> wrote:
> >
> >> Hi, Rajini,
> >>
> >> Thanks for the reply.
> >>
> >> 4. Implementation wise, it seems to me that it's simpler to read from
> the
> >> cache than reading directly from ZK since the config manager already
> >> propagates all config changes through ZK. Also, it's probably a good
> idea
> >> to limit the places in the code base that directly accesses ZK.
> >>
> >> 5. Yes, it seems that it makes sense to add the new SCRAM configurations
> >> to
> >> the existing /config/users/<encoded-user>. I am not sure how one would
> >> remove the SCRAM configurations in the example though since the
> properties
> >> specified in add-config is not the ones actually being stored. An
> >> alternative is to doing sth like the following. It may still feel a bit
> >> weird and I am wondering if there is a clearer approach.
> >>
> >> bin/kafka-configs.sh --zookeeper localhost:2181 --alter --add-config
> >> 'scram_sha-256=[password=alice-secret,iterations=4096],scram_sha-1=
> >> [password=alice-secret,iterations=4096]' --entity-type users
> >> --entity-name
> >> alice
> >>
> >> bin/kafka-configs.sh --zookeeper localhost:2181 --alter --delete-config
> >> 'scram_sha-256,scram_sha-1' --entity-type users --entity-name alice
> >>
> >> Thanks,
> >>
> >> Jun
> >>
> >>
> >> On Wed, Oct 26, 2016 at 4:35 AM, Rajini Sivaram <
> >> rajinisiva...@googlemail.com> wrote:
> >>
> >> > Hi Jun,
> >> >
> >> > Thank you for reviewing the KIP. Answers below:
> >> >
> >> >
> >> >    1. Yes, agree, Updated KIP.
> >> >    2. User specifies a password and iteration count. kaka-configs.sh
> >> >    generates a random salt and then generates StoredKey and ServerKey
> >> for
> >> > that
> >> >    password using the same message formatter implementation used for
> >> SCRAM
> >> >    authentication. I have added some more detail to the KIP (
> >> >    https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >> > 84%3A+Support+SASL+SCRAM+mechanisms#KIP-84:
> SupportSASLSCRAMmechanisms-
> >> > Tools).
> >> >    Does that answer the question?
> >> >    3. I started off thinking just one (SCRAM-SHA-256) and then thought
> >> >    another one is required to make sure that the implementation can
> cope
> >> > with
> >> >    multiple SCRAM mechanisms. But actually you are right, we can
> support
> >> > all.
> >> >    I haven't added the old md2/md5 mechanisms that aren't very secure,
> >> but
> >> > I
> >> >    have included all the SHA algorithms.
> >> >    4. Since credentials are only required when a connection is made,
> it
> >> >    feels like we can just read the latest value from ZK rather than
> >> cache
> >> > all
> >> >    users and keep them updated. Having said that, we can always add
> >> caching
> >> >    later if we find that the overhead of reading from ZK every time is
> >> too
> >> >    expensive. Since caching doesn't change any externals, this can be
> >> done
> >> > in
> >> >    a JIRA later - would that be ok?
> >> >    5. Thanks, updated. I have changed the property names to include
> >> >    mechanism. To avoid four separate properties per mechanism in ZK, I
> >> have
> >> >    changed the format to use a single property (lower-case mechanism
> >> name)
> >> >    with four values concatenated in a format similar to SCRAM
> messages.
> >> >
> >> > Do you think storing SCRAM credentials in /config/users/<encoded-user>
> >> > along with existing quota properties is okay? Or should they be under
> a
> >> > different path (eg. /config/credentials/<encoded-user>)? Or should
> >> they be
> >> > on a completely different path like ACLs with a separate tool instead
> of
> >> > reusing kaka-configs.sh?
> >> >
> >> > Thank you,
> >> >
> >> > Rajini
> >> >
> >> > On Tue, Oct 25, 2016 at 11:55 PM, Jun Rao <j...@confluent.io> wrote:
> >> >
> >> > > Hi, Rajini,
> >> > >
> >> > > Thanks for the proposal. Looks good overall and seems quite useful
> >> (e.g.
> >> > > for supporting delegation tokens). A few comments/questions below.
> >> > >
> >> > > 1. For the ZK data format change, should we use the same convention
> >> as in
> >> > > KIP-55 to use encoded user name (i.e.,
> /config/users/<encoded-user1>)
> >> ?
> >> > >
> >> > > 2. For tooling, could you describe how user typically generates
> >> > > scam_server_key and scram_stored_key to be used by kafka-config.sh?
> >> > >
> >> > > 3. Is there a particular reason to only support sha1 and sha128?
> >> Should
> >> > we
> >> > > support more hashes listed below in the future?
> >> > > http://www.iana.org/assignments/hash-function-
> >> > > text-names/hash-function-text-names.xhtml
> >> > >
> >> > > 4. Is there a reason not to cache user credentials in the broker?
> The
> >> > > dynamic config mechanism already supports loading configs into
> >> broker's
> >> > > cache. Checking credentials from broker's cache is more efficient
> than
> >> > > reading from ZK each time.
> >> > >
> >> > > 5. Typo "scram_iteration-4096" (= instead of -).
> >> > >
> >> > > Thanks,
> >> > >
> >> > > Jun
> >> > >
> >> > >
> >> > >
> >> > > On Tue, Oct 4, 2016 at 6:43 AM, Rajini Sivaram <
> >> > > rajinisiva...@googlemail.com
> >> > > > wrote:
> >> > >
> >> > > > Hi all,
> >> > > >
> >> > > > I have just created KIP-84 to add SCRAM-SHA-1 and SCRAM-SHA-256
> SASL
> >> > > > mechanisms to Kafka:
> >> > > >
> >> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >> > > > 84%3A+Support+SASL+SCRAM+mechanisms
> >> > > >
> >> > > >
> >> > > > Comments and suggestions are welcome.
> >> > > >
> >> > > > Thank you...
> >> > > >
> >> > > > Regards,
> >> > > >
> >> > > > Rajini
> >> > > >
> >> > >
> >> >
> >> >
> >> >
> >> > --
> >> > Regards,
> >> >
> >> > Rajini
> >> >
> >>
> >
> >
> >
> > --
> > Regards,
> >
> > Rajini
> >
>
>
>
> --
> Regards,
>
> Rajini
>

Reply via email to