You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by Stanislav Kozlovski <st...@confluent.io> on 2018/09/03 16:58:20 UTC

Re: [DISCUSS] KIP 368: Allow SASL Connections to Periodically Re-Authenticate

Hi Ron,

I really like this KIP, it is very needed.
I am still looking through it but unfortunately I am not too familiar with
the networking code to evaluate your solution.

I'd like to ask what happens if re-authentication consistently fails. Would
we retry endlessly? Since the functionality for brokers to close
connections is outside the scope of this KIP, what effect would
success/failure in re-authentication have? I think it's worth noting in the
KIP

I also think the rejected alternative of initiating a new connection should
stay rejected. I am not aware of anything currently limiting the client to
connect to the same broker, but I think it would be best if we kept Kafka's
options open (e.g addition of a load balancer in the future) and not
introduce additional client-broker statefulness.

Thanks,
Stanislav

On Tue, Aug 28, 2018 at 5:28 PM Ron Dagostino <rn...@gmail.com> wrote:

> Hi everyone. I created KIP 368: Allow SASL Connections to Periodically
> Re-Authenticate
> <
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-368%3A+Allow+SASL+Connections+to+Periodically+Re-Authenticate
> >
> (
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-368%3A+Allow+SASL+Connections+to+Periodically+Re-Authenticate
> ).
> The motivation for this KIP is as follows:
>
> The adoption of KIP-255: OAuth Authentication via SASL/OAUTHBEARER
> <https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=75968876>
> in
> release 2.0.0 creates the possibility of using information in the bearer
> token to make authorization decisions.  Unfortunately, however, Kafka
> connections are long-lived, so there is no ability to change the bearer
> token associated with a particular connection.  Allowing SASL connections
> to periodically re-authenticate would resolve this.  In addition to this
> motivation there are two others that are security-related.  First, to
> eliminate access to Kafka for connected clients, the current requirement is
> to remove all authorizations (i.e. remove all ACLs).  This is necessary
> because of the long-lived nature of the connections.  It is operationally
> simpler to shut off access at the point of authentication, and with the
> release of KIP-86: Configurable SASL Callback Handlers
> <
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-86%3A+Configurable+SASL+callback+handlers
> >
> it
> is going to become more and more likely that installations will
> authenticate users against external directories (e.g. via LDAP).  The
> ability to stop Kafka access by simply disabling an account in an LDAP
> directory (for example) is desirable.  The second motivating factor for
> re-authentication related to security is that the use of short-lived tokens
> is a common OAuth security recommendation, but issuing a short-lived token
> to a Kafka client (or a broker when OAUTHBEARER is the inter-broker
> protocol) currently has no benefit because once a client is connected to a
> broker the client is never challenged again and the connection may remain
> intact beyond the token expiration time (and may remain intact indefinitely
> under perfect circumstances).  This KIP proposes adding the ability for
> clients (and brokers when OAUTHBEARER is the inter-broker protocol) to
> re-authenticate their connections to brokers and have the new bearer token
> appear on their session rather than the old one.
>
> The description of this KIP is actually quite straightforward from a
> functionality perspective; from an implementation perspective, though, the
> KIP is not so straightforward, so it includes a pull request with a
> proposed implementation.  See https://github.com/apache/kafka/pull/5582.
>
> Ron
>


-- 
Best,
Stanislav

Re: [DISCUSS] KIP 368: Allow SASL Connections to Periodically Re-Authenticate

Posted by Ron Dagostino <rn...@gmail.com>.
<<< It works for authentication because at that point we know that nobody
<<<else is using the socket.  But thereafter I would imagine all bets are
off.*
<<<This is where I have a different opinion.
lol.  I'm going to have to think about this a bit.  My gut is telling me
no, but you haven't steered in the wrong direction yet, so let me mull it
over and see if I can come around to this type of solution.  Thanks for the
alternate point of view.  In the meantime, if anyone else has an opinion on
this, please do contribute your thoughts.

Ron

On Tue, Sep 4, 2018 at 5:12 PM Rajini Sivaram <ra...@gmail.com>
wrote:

> Hi Ron,
>
> Thanks for the response.
>
> Retries: If we reuse authentication code path for re-authentication, we
> might want to treat failures in the same way without retries, but for now,
> lets leave it in.
>
> Implementation: I think we are on the same page.
>
> *I believe Streams and Connect are taken care of because they are built
> on top of standard producers and consumers.*
> Yes, just noticed that the PR does already include changes for all the
> relevant classes.
>
> *First, I do want to make sure we are on the same page with what the
> current changes are with respect to imports.*
> Yes, agree that in terms of imports we are ok even with the one change, but
> we should still try and avoid SASL-specific code outside SASL classes if we
> can.
>
> *The nonce-based proposal (described in the "Rejected Alternatives"
> section) was a way of reusing the authentication code for re-authentication
> -- simply authenticate a new socket to the same server and then transfer
> the principal to the original connection.*
> This may help to keep the changes within the networking/security layer, but
> as you mentioned in the KIP, we don't really want to create another
> connection. So the approach in the PR seems better.
>
>  *I just don't know if it is possible to **make re-authentication work the
> way authentication works.  I imagine that trying to inject stuff directly
> into the socket when that socket is simultaneously being used for something
> else -- it feels like a really bad idea that isn't viable.*
> This is exactly what I was proposing. I can't say for sure it will work
> since I haven't tried it out. And it may not be easy to handle
> re-authentication in this way. But all the code will be contained within a
> few classes. It will be much easier to test since all the tests can be in
> `clients` and we already have the framework for testing various
> authentication scenarios.
>
> * It works for authentication because at that point we know that nobody
> else is using the socket.  But thereafter I would imagine all bets are
> off.*
> This is where I have a different opinion. Rather than treat
> re-authentication as additional data flowing on a socket with arbitrary
> data flowing on it, we should look at it in terms of Kafka requests and
> responses between Kafka clients and brokers. On the broker-side, this
> should actually be very straightforward. When we see a SaslHandshake
> request, we just move the channel back to authenticating state giving the
> request to a new SaslServerAuthenticator and thereafter
> SaslServerAuthenticator would complete authentication and move the channel
> back to ready state when authentication completes. On the client-side, it
> would be a bit more involved. Clients may have multiple requests in-flight.
> So if we move a channel back to authenticating state when responses are
> pending, we may have to cache some responses or propagate responses from
> pending requests to the client layer. But it doesn't feel impossible.
>
>
> On Tue, Sep 4, 2018 at 4:54 PM, Ron Dagostino <rn...@gmail.com> wrote:
>
> > Hi Rajini.  I'm glad you agree this is a good security addition.
> Regarding
> > your questions/comments:
> >
> > 1. Retries.
> >
> > <<<What is the state of a client connection in between re-authentication
> > retries?
> > <<<Can it continue to process requests?
> > Yes, the existing connection remains intact and continues to accept
> > requests and process responses.  In fact, the existing connection is
> never
> > closed/replaced at any time.  The 3 APIs that constitute the
> authentication
> > "dance" (ApiVersionsRequest/Response, SaslHandshakeRequest/Response, and
> > SaslAuthenticateRequest/Response) traverse the existing socket
> connection,
> > and the only impact of re-authentication (aside from sending these three
> > API calls) is that the KafkaChannel#principal() method on the server side
> > ends up returning a different value once re-authentication succeeds
> > (because the KafkaChannel instance ends up with a new instance of
> > SaslServerAuthenticator, which is where the call gets delegated).
> >
> > <<<Also, can you describe scenarios where retries are useful?
> > I have added this sentence to the KIP: "A retry is helpful when
> > re-authentication fails on the server side due to some temporary outage
> > (for example, the re-authentication will fail through no fault of the
> > client if the token endpoint is unavailable and the broker has not yet
> > cached the public key required to validate the token signature)."
> >
> > 2. Metrics:
> >
> > <<<I think we should add separate re-authentication success/failure
> > metrics.
> > Yes, I stated there are none only to be transparent about that fact more
> > than anything, and I am not surprised to get this feedback.  I will add
> > them to the KIP.
> >
> > 3. Implementation
> >
> > <<<Hmm...
> > lol.  Yeah, I'm not surprised to get this feedback, either.  I came at
> this
> > totally fresh (i.e. no knowledge of this part of the code base), and I
> > avoided writing the integration tests (which I think might take a bit of
> > time) because I wanted confirmation on the approach first.  So this
> > feedback is excellent.  Let me try to unpack/address it, and let's see
> > where we end up.
> >
> > <<<NetworkClient shouldn't have to deal with authentication
> > Actually, the only change I made to NetworkClient is to add the below
> > method because there is currently no way to submit a request with a
> > callback and have that request use the default timeout value of the
> client
> > -- currently if you specify a callback you also must explicitly specify a
> > timeout.  I considered adding this to the KafkaClient interface but
> decided
> > this was quicker in terms of getting the KIP and PR out there for
> > discussion.
> >
> > public ClientRequest newClientRequest(String nodeId,
> >
> >                                       AbstractRequest.Builder<?>
> > requestBuilder,
> >
> >                                       long createdTimeMs,
> >
> >                                       boolean expectResponse,
> >
> >                                       RequestCompletionHandler callback)
> {
> >
> >     return newClientRequest(nodeId, requestBuilder, createdTimeMs,
> > expectResponse, defaultRequestTimeoutMs, callback);
> >
> > }
> >
> >
> > <<<changes to every use (AdminClient/KafkaConsumer/KafkaProducer/Broker)
> > to
> >
> > <<<deal with security protocols and SASL mechanisms will be hard to
> > maintain.
> >
> > <<<And what about Connect and Streams - will they need changing as well?
> >
> >
> > I believe Streams and Connect are taken care of because they are built on
> > top of standard producers and consumers.  I nevertheless do agree with
> the
> > comment that this might be hard to maintain.  If there is ever a need
> for a
> > new use of NetworkClient -- whether as part of new inter-broker
> > communication functionality or as part of a client that does not leverage
> > the standard producer/consumer -- then that new use will have to address
> > the issue of how to inject the requests related to re-authentication.  I
> am
> > not happy with this, and I only arrived at it after rejecting the
> > "one-size-fits-all" approach that I mentioned in the "Rejected
> > Alternatives" section.  I think you are suggesting a third alternative,
> > though.  I'll address that in a moment.
> >
> > First, I do want to make sure we are on the same page with what the
> current
> > changes are with respect to imports.  I realize I may be repeating stuff
> > you already know having looked at the PR, and I apologize if this is the
> > case.  I did have to make one change to checkstyle/import-control.xml to
> > allow org.apache.kafka.common.network.KafkaChannel to import the below
> > classes -- aside from that, though, every other import I added was
> already
> > "allowed".
> >
> > // added to KafkaChannel
> >
> > import org.apache.kafka.common.requests.RequestHeader;
> >
> > import org.apache.kafka.common.requests.SaslAuthenticateRequest;
> >
> > import org.apache.kafka.common.requests.SaslAuthenticateResponse;
> >
> > import org.apache.kafka.common.requests.SaslHandshakeRequest;
> >
> > import org.apache.kafka.common.requests.SaslHandshakeResponse;
> >
> > The above needed to be imported because of the addition of these two
> > methods to KafkaChannel:
> >
> >     /**
> >      * Respond to a SASL re-authentication initial handshake. This occurs
> > on the
> >      * SASL Server side of the re-authentication dance (i.e. on the
> > broker).
> >      *
> >      * @param requestHeader
> >      *            the request header
> >      * @param saslHandshakeRequest
> >      *            the initial handshake request to process
> >      * @return the response to return to the client
> >      */
> >     public SaslHandshakeResponse
> > respondToReauthenticationSaslHandshakeRequest(RequestHeader
> requestHeader,
> >             SaslHandshakeRequest saslHandshakeRequest)
> >
> >     /**
> >      * Respond to a SASL re-authentication token exchange. This occurs on
> > the SASL
> >      * Server side of the re-authentication dance (i.e. on the broker).
> >      *
> >      * @param requestHeader
> >      *            the request header
> >      * @param saslAuthenticateRequest
> >      *            the token exchange request to process
> >      * @return the response to return to the client
> >      */
> >     public SaslAuthenticateResponse
> > respondToReauthenticationSaslAuthenticateRequest(RequestHeader
> > requestHeader,
> >             SaslAuthenticateRequest saslAuthenticateRequest)
> >
> > And then of course I route the requests to these methods via the
> following
> > code in kafka.server.KafkaApis:
> >
> >   def handleSaslHandshakeRequest(request: RequestChannel.Request) {
> >
> >     requestChannel.kafkaChannel(request) match {
> >
> >       case Some(kafkaChannel) =>
> >
> >         sendResponseMaybeThrottle(request, _ =>
> >
> kafkaChannel.respondToReauthenticationSaslHandshakeRequest(request.header,
> > request.body[SaslHandshakeRequest]))
> >
> >       case None =>
> >
> >         sendResponseMaybeThrottle(request, _ => new
> > SaslHandshakeResponse(Errors.ILLEGAL_SASL_STATE, Collections.emptySet()))
> >
> >     }
> >
> >   }
> >
> >
> >   def handleSaslAuthenticateRequest(request: RequestChannel.Request) {
> >
> >     requestChannel.kafkaChannel(request) match {
> >
> >       case Some(kafkaChannel) =>
> >
> >         sendResponseMaybeThrottle(request, _ =>
> > kafkaChannel.respondToReauthenticationSaslAuthenticateRequest(request.
> > header,
> > request.body[SaslAuthenticateRequest]))
> >
> >       case None =>
> >
> >         sendResponseMaybeThrottle(request, _ => new
> > SaslAuthenticateResponse(Errors.ILLEGAL_SASL_STATE,
> >
> >             "SaslAuthenticate request received but could not find channel
> > (should not happen)"))
> >
> >     }
> >
> >   }
> >
> > To address what I think you are proposing -- the third alternative.  Are
> > referring to the fact that SaslClientAuthenticator and
> > SaslServerAuthenticator actually talk directly over the socket, and are
> you
> > suggesting that we somehow handle re-authentication in the same way?
> >
> >     // buffers used in `authenticate`
> >
> >     private NetworkReceive netInBuffer;
> >
> >     private Send netOutBuffer;
> >
> > I did not like the fact that authentication and re-authentication ended
> up
> > working differently -- authentication via direct socket sends and
> > re-authentication via handling API requests in the normal
> > kafka.server.KafkaApis API control flow.  The nonce-based proposal
> > (described in the "Rejected Alternatives" section) was a way of reusing
> the
> > authentication code for re-authentication -- simply authenticate a new
> > socket to the same server and then transfer the principal to the original
> > connection.  So I am definitely in favor of having authentication and
> > re-authentication work the same.  I just don't know if it is possible to
> > make re-authentication work the way authentication works.  I imagine that
> > trying to inject stuff directly into the socket when that socket is
> > simultaneously being used for something else -- it feels like a really
> bad
> > idea that isn't viable.  It works for authentication because at that
> point
> > we know that nobody else is using the socket.  But thereafter I would
> > imagine all bets are off.
> >
> > Am I understanding what you were suggesting?  Was the above description
> > helpful?  I think my instincts are similar to yours -- I'm not a a fan of
> > making several changes to make this work in several places, and it feels
> a
> > bit wrong to have authentication and re-authentication work differently
> --
> > I just don't see a way around it at this point.  Do you?
> >
> > Ron
> >
> > On Tue, Sep 4, 2018 at 7:40 AM Rajini Sivaram <ra...@gmail.com>
> > wrote:
> >
> > > Hi Ron,
> > >
> > > Thanks for the KIP. This is going to be a really useful feature to
> > tighten
> > > security.
> > >
> > > I have a few comments/questions:
> > >
> > >    1. Retries: What is the state of a client connection in between
> > >    re-authentication retries? Can it continue to process requests?
> Also,
> > > can
> > >    you describe scenarios where retries are useful?
> > >    2. Metrics: I think we should add separate re-authentication
> > >    success/failure metrics.
> > >    3. Implementation: Hmm... The PR looks quite a lot different from
> > what I
> > >    hoped we would do. In particular, I am concerned about network and
> > > security
> > >    layer code escaping into the client/broker implementation layers.
> > >
> > > "*We leave it to the owners of the **NetworkClient instances to define
> > how
> > > to inject such requests by providing an implementation of the interface
> > to
> > > the SaslChannelBuilder, which in turn provides it to the *
> > > *SaslClientAuthenticator*"
> > >
> > > NetworkClient shouldn't have to deal with authentication and changes to
> > > every use (AdminClient/KafkaConsumer/KafkaProducer/Broker) to deal with
> > > security protocols and SASL mechanisms will be hard to maintain. And
> what
> > > about Connect and Streams - will they need changing as well?
> > >
> > > Have we considered implementing re-authentication within the
> > > network/security layer, like we do authentication? I think we should
> try
> > > and find a way to move channels back to
> authentication/re-authentication
> > > phase so that the rest of the codebase doesn't have special code to
> > handle
> > > SASL. I haven't looked into how this can be done, but it doesn't feel
> > > impossible. Thoughts?
> > >
> > >
> > >
> > > On Mon, Sep 3, 2018 at 10:54 PM, Ron Dagostino <rn...@gmail.com>
> > wrote:
> > >
> > > > I just realized there was one place in the KIP that stated that
> retries
> > > > could occur indefinitely (when a client attempts to change identity,
> > > which
> > > > we arbitrarily disallow).  This was a mistake, a holdover from a
> prior
> > > > draft of the KIP.  This is now fixed.  Retries are never allowed
> > > > indefinitely.
> > > >
> > > > <<<if a connection originally authenticates as User:user1, an attempt
> > to
> > > > re-authenticate as anything else (e.g. User:user2) will fail.
> > > > <<<Retry is allowed indefinitely in this case.
> > > > >>>if a connection originally authenticates as User:user1, an attempt
> > to
> > > > re-authenticate as anything else (e.g. User:user2) will fail.
> > > > >>>Retry is allowed in this case (still subject to the expiration of
> > the
> > > > original credential as described above)
> > > >
> > > > Ron
> > > >
> > > >
> > > > On Mon, Sep 3, 2018 at 5:35 PM Ron Dagostino <rn...@gmail.com>
> > wrote:
> > > >
> > > > > Thanks for the feedback, Stanislav.
> > > > >
> > > > > <<<I am not too familiar with the networking code to evaluate your
> > > > > solution.
> > > > > Yeah, I wasn't familiar with it when I started, either, and I am
> > hoping
> > > > > people who are intimately familiar with it will take a close look.
> > > Some
> > > > of
> > > > > that code seems to be very central to the core of Kafka, and
> > injecting
> > > > > re-authentication attempts into the flow of replica fetching,
> sending
> > > > > transaction markers, and producing or consuming messages is not
> > > > something I
> > > > > am convinced is acceptable under all circumstances.  To be clear,
> > > > though, I
> > > > > am not saying it is problematic, either -- I just don't have enough
> > > > > experience or familiarity to know.  I really do want additional
> eyes
> > on
> > > > > this if possible.
> > > > >
> > > > > Regarding your question about retries, here's the part of the KIP
> > that
> > > > > deals with those:
> > > > >
> > > > > If a re-authentication attempt should fail then the connection will
> > be
> > > > >> told to retry after some delay depending on how many retries have
> > been
> > > > >> attempted so far: after some small amount of time for the first
> > retry
> > > > (e.g.
> > > > >> 1 minute), double that for the second retry, and 4 times the
> initial
> > > > delay
> > > > >> for every retry thereafter.  Retry attempts generally occur at
> least
> > > > until
> > > > >> the current credential expires (but not indefinitely – and of
> course
> > > > they
> > > > >> won't continue if one of them actually succeeds).  There are
> certain
> > > > errors
> > > > >> that result in retries not being attempted (i.e. if some internal
> > > state
> > > > >> doesn't make sense, which generally should not happen).
> > > > >
> > > > >
> > > > > <<<Would we retry endlessly?
> > > > > No.  There may be at most one retry after the client token expires.
> > So
> > > > if
> > > > > a token expires after an hour, and several retry attempts fail
> > > including
> > > > > one at minute 59, then one last attempt will be made at the
> 63-minute
> > > > mark.
> > > > >
> > > > > <<<Since the functionality for brokers to close connections is
> > outside
> > > > the
> > > > > scope of this KIP, what effect would
> > > > > <<<success/failure in re-authentication have
> > > > > Practically speaking, unless authorization is based on the contents
> > of
> > > > the
> > > > > token rather than ACLs, the ultimate success or failure of
> > > > > re-authentication has no effect.  The intent is definitely to
> follow
> > > this
> > > > > KIP with another one to add the ability for brokers to close
> > > connections
> > > > > that use expired credentials, and then at that point the client
> would
> > > > have
> > > > > to successfully re-authenticate to avoid the connection being
> > forcibly
> > > > > closed.
> > > > >
> > > > > Ron
> > > > >
> > > > >
> > > > > On Mon, Sep 3, 2018 at 12:58 PM Stanislav Kozlovski <
> > > > > stanislav@confluent.io> wrote:
> > > > >
> > > > >> Hi Ron,
> > > > >>
> > > > >> I really like this KIP, it is very needed.
> > > > >> I am still looking through it but unfortunately I am not too
> > familiar
> > > > with
> > > > >> the networking code to evaluate your solution.
> > > > >>
> > > > >> I'd like to ask what happens if re-authentication consistently
> > fails.
> > > > >> Would
> > > > >> we retry endlessly? Since the functionality for brokers to close
> > > > >> connections is outside the scope of this KIP, what effect would
> > > > >> success/failure in re-authentication have? I think it's worth
> noting
> > > in
> > > > >> the
> > > > >> KIP
> > > > >>
> > > > >> I also think the rejected alternative of initiating a new
> connection
> > > > >> should
> > > > >> stay rejected. I am not aware of anything currently limiting the
> > > client
> > > > to
> > > > >> connect to the same broker, but I think it would be best if we
> kept
> > > > >> Kafka's
> > > > >> options open (e.g addition of a load balancer in the future) and
> not
> > > > >> introduce additional client-broker statefulness.
> > > > >>
> > > > >> Thanks,
> > > > >> Stanislav
> > > > >>
> > > > >> On Tue, Aug 28, 2018 at 5:28 PM Ron Dagostino <rn...@gmail.com>
> > > > wrote:
> > > > >>
> > > > >> > Hi everyone. I created KIP 368: Allow SASL Connections to
> > > Periodically
> > > > >> > Re-Authenticate
> > > > >> > <
> > > > >> >
> > > > >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-368%
> > > > 3A+Allow+SASL+Connections+to+Periodically+Re-Authenticate
> > > > >> > >
> > > > >> > (
> > > > >> >
> > > > >> >
> > > > >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-368%
> > > > 3A+Allow+SASL+Connections+to+Periodically+Re-Authenticate
> > > > >> > ).
> > > > >> > The motivation for this KIP is as follows:
> > > > >> >
> > > > >> > The adoption of KIP-255: OAuth Authentication via
> SASL/OAUTHBEARER
> > > > >> > <
> > > > >> https://cwiki.apache.org/confluence/pages/viewpage.action?
> > > > pageId=75968876
> > > > >> >
> > > > >> > in
> > > > >> > release 2.0.0 creates the possibility of using information in
> the
> > > > bearer
> > > > >> > token to make authorization decisions.  Unfortunately, however,
> > > Kafka
> > > > >> > connections are long-lived, so there is no ability to change the
> > > > bearer
> > > > >> > token associated with a particular connection.  Allowing SASL
> > > > >> connections
> > > > >> > to periodically re-authenticate would resolve this.  In addition
> > to
> > > > this
> > > > >> > motivation there are two others that are security-related.
> First,
> > > to
> > > > >> > eliminate access to Kafka for connected clients, the current
> > > > >> requirement is
> > > > >> > to remove all authorizations (i.e. remove all ACLs).  This is
> > > > necessary
> > > > >> > because of the long-lived nature of the connections.  It is
> > > > >> operationally
> > > > >> > simpler to shut off access at the point of authentication, and
> > with
> > > > the
> > > > >> > release of KIP-86: Configurable SASL Callback Handlers
> > > > >> > <
> > > > >> >
> > > > >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-86%3A+
> > > > Configurable+SASL+callback+handlers
> > > > >> > >
> > > > >> > it
> > > > >> > is going to become more and more likely that installations will
> > > > >> > authenticate users against external directories (e.g. via LDAP).
> > > The
> > > > >> > ability to stop Kafka access by simply disabling an account in
> an
> > > LDAP
> > > > >> > directory (for example) is desirable.  The second motivating
> > factor
> > > > for
> > > > >> > re-authentication related to security is that the use of
> > short-lived
> > > > >> tokens
> > > > >> > is a common OAuth security recommendation, but issuing a
> > short-lived
> > > > >> token
> > > > >> > to a Kafka client (or a broker when OAUTHBEARER is the
> > inter-broker
> > > > >> > protocol) currently has no benefit because once a client is
> > > connected
> > > > >> to a
> > > > >> > broker the client is never challenged again and the connection
> may
> > > > >> remain
> > > > >> > intact beyond the token expiration time (and may remain intact
> > > > >> indefinitely
> > > > >> > under perfect circumstances).  This KIP proposes adding the
> > ability
> > > > for
> > > > >> > clients (and brokers when OAUTHBEARER is the inter-broker
> > protocol)
> > > to
> > > > >> > re-authenticate their connections to brokers and have the new
> > bearer
> > > > >> token
> > > > >> > appear on their session rather than the old one.
> > > > >> >
> > > > >> > The description of this KIP is actually quite straightforward
> > from a
> > > > >> > functionality perspective; from an implementation perspective,
> > > though,
> > > > >> the
> > > > >> > KIP is not so straightforward, so it includes a pull request
> with
> > a
> > > > >> > proposed implementation.  See https://github.com/apache/kafk
> > > > a/pull/5582
> > > > >> .
> > > > >> >
> > > > >> > Ron
> > > > >> >
> > > > >>
> > > > >>
> > > > >> --
> > > > >> Best,
> > > > >> Stanislav
> > > > >>
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP 368: Allow SASL Connections to Periodically Re-Authenticate

Posted by Rajini Sivaram <ra...@gmail.com>.
Hi Ron,

Thanks for the response.

Retries: If we reuse authentication code path for re-authentication, we
might want to treat failures in the same way without retries, but for now,
lets leave it in.

Implementation: I think we are on the same page.

*I believe Streams and Connect are taken care of because they are built
on top of standard producers and consumers.*
Yes, just noticed that the PR does already include changes for all the
relevant classes.

*First, I do want to make sure we are on the same page with what the
current changes are with respect to imports.*
Yes, agree that in terms of imports we are ok even with the one change, but
we should still try and avoid SASL-specific code outside SASL classes if we
can.

*The nonce-based proposal (described in the "Rejected Alternatives"
section) was a way of reusing the authentication code for re-authentication
-- simply authenticate a new socket to the same server and then transfer
the principal to the original connection.*
This may help to keep the changes within the networking/security layer, but
as you mentioned in the KIP, we don't really want to create another
connection. So the approach in the PR seems better.

 *I just don't know if it is possible to **make re-authentication work the
way authentication works.  I imagine that trying to inject stuff directly
into the socket when that socket is simultaneously being used for something
else -- it feels like a really bad idea that isn't viable.*
This is exactly what I was proposing. I can't say for sure it will work
since I haven't tried it out. And it may not be easy to handle
re-authentication in this way. But all the code will be contained within a
few classes. It will be much easier to test since all the tests can be in
`clients` and we already have the framework for testing various
authentication scenarios.

* It works for authentication because at that point we know that nobody
else is using the socket.  But thereafter I would imagine all bets are off.*
This is where I have a different opinion. Rather than treat
re-authentication as additional data flowing on a socket with arbitrary
data flowing on it, we should look at it in terms of Kafka requests and
responses between Kafka clients and brokers. On the broker-side, this
should actually be very straightforward. When we see a SaslHandshake
request, we just move the channel back to authenticating state giving the
request to a new SaslServerAuthenticator and thereafter
SaslServerAuthenticator would complete authentication and move the channel
back to ready state when authentication completes. On the client-side, it
would be a bit more involved. Clients may have multiple requests in-flight.
So if we move a channel back to authenticating state when responses are
pending, we may have to cache some responses or propagate responses from
pending requests to the client layer. But it doesn't feel impossible.


On Tue, Sep 4, 2018 at 4:54 PM, Ron Dagostino <rn...@gmail.com> wrote:

> Hi Rajini.  I'm glad you agree this is a good security addition.  Regarding
> your questions/comments:
>
> 1. Retries.
>
> <<<What is the state of a client connection in between re-authentication
> retries?
> <<<Can it continue to process requests?
> Yes, the existing connection remains intact and continues to accept
> requests and process responses.  In fact, the existing connection is never
> closed/replaced at any time.  The 3 APIs that constitute the authentication
> "dance" (ApiVersionsRequest/Response, SaslHandshakeRequest/Response, and
> SaslAuthenticateRequest/Response) traverse the existing socket connection,
> and the only impact of re-authentication (aside from sending these three
> API calls) is that the KafkaChannel#principal() method on the server side
> ends up returning a different value once re-authentication succeeds
> (because the KafkaChannel instance ends up with a new instance of
> SaslServerAuthenticator, which is where the call gets delegated).
>
> <<<Also, can you describe scenarios where retries are useful?
> I have added this sentence to the KIP: "A retry is helpful when
> re-authentication fails on the server side due to some temporary outage
> (for example, the re-authentication will fail through no fault of the
> client if the token endpoint is unavailable and the broker has not yet
> cached the public key required to validate the token signature)."
>
> 2. Metrics:
>
> <<<I think we should add separate re-authentication success/failure
> metrics.
> Yes, I stated there are none only to be transparent about that fact more
> than anything, and I am not surprised to get this feedback.  I will add
> them to the KIP.
>
> 3. Implementation
>
> <<<Hmm...
> lol.  Yeah, I'm not surprised to get this feedback, either.  I came at this
> totally fresh (i.e. no knowledge of this part of the code base), and I
> avoided writing the integration tests (which I think might take a bit of
> time) because I wanted confirmation on the approach first.  So this
> feedback is excellent.  Let me try to unpack/address it, and let's see
> where we end up.
>
> <<<NetworkClient shouldn't have to deal with authentication
> Actually, the only change I made to NetworkClient is to add the below
> method because there is currently no way to submit a request with a
> callback and have that request use the default timeout value of the client
> -- currently if you specify a callback you also must explicitly specify a
> timeout.  I considered adding this to the KafkaClient interface but decided
> this was quicker in terms of getting the KIP and PR out there for
> discussion.
>
> public ClientRequest newClientRequest(String nodeId,
>
>                                       AbstractRequest.Builder<?>
> requestBuilder,
>
>                                       long createdTimeMs,
>
>                                       boolean expectResponse,
>
>                                       RequestCompletionHandler callback) {
>
>     return newClientRequest(nodeId, requestBuilder, createdTimeMs,
> expectResponse, defaultRequestTimeoutMs, callback);
>
> }
>
>
> <<<changes to every use (AdminClient/KafkaConsumer/KafkaProducer/Broker)
> to
>
> <<<deal with security protocols and SASL mechanisms will be hard to
> maintain.
>
> <<<And what about Connect and Streams - will they need changing as well?
>
>
> I believe Streams and Connect are taken care of because they are built on
> top of standard producers and consumers.  I nevertheless do agree with the
> comment that this might be hard to maintain.  If there is ever a need for a
> new use of NetworkClient -- whether as part of new inter-broker
> communication functionality or as part of a client that does not leverage
> the standard producer/consumer -- then that new use will have to address
> the issue of how to inject the requests related to re-authentication.  I am
> not happy with this, and I only arrived at it after rejecting the
> "one-size-fits-all" approach that I mentioned in the "Rejected
> Alternatives" section.  I think you are suggesting a third alternative,
> though.  I'll address that in a moment.
>
> First, I do want to make sure we are on the same page with what the current
> changes are with respect to imports.  I realize I may be repeating stuff
> you already know having looked at the PR, and I apologize if this is the
> case.  I did have to make one change to checkstyle/import-control.xml to
> allow org.apache.kafka.common.network.KafkaChannel to import the below
> classes -- aside from that, though, every other import I added was already
> "allowed".
>
> // added to KafkaChannel
>
> import org.apache.kafka.common.requests.RequestHeader;
>
> import org.apache.kafka.common.requests.SaslAuthenticateRequest;
>
> import org.apache.kafka.common.requests.SaslAuthenticateResponse;
>
> import org.apache.kafka.common.requests.SaslHandshakeRequest;
>
> import org.apache.kafka.common.requests.SaslHandshakeResponse;
>
> The above needed to be imported because of the addition of these two
> methods to KafkaChannel:
>
>     /**
>      * Respond to a SASL re-authentication initial handshake. This occurs
> on the
>      * SASL Server side of the re-authentication dance (i.e. on the
> broker).
>      *
>      * @param requestHeader
>      *            the request header
>      * @param saslHandshakeRequest
>      *            the initial handshake request to process
>      * @return the response to return to the client
>      */
>     public SaslHandshakeResponse
> respondToReauthenticationSaslHandshakeRequest(RequestHeader requestHeader,
>             SaslHandshakeRequest saslHandshakeRequest)
>
>     /**
>      * Respond to a SASL re-authentication token exchange. This occurs on
> the SASL
>      * Server side of the re-authentication dance (i.e. on the broker).
>      *
>      * @param requestHeader
>      *            the request header
>      * @param saslAuthenticateRequest
>      *            the token exchange request to process
>      * @return the response to return to the client
>      */
>     public SaslAuthenticateResponse
> respondToReauthenticationSaslAuthenticateRequest(RequestHeader
> requestHeader,
>             SaslAuthenticateRequest saslAuthenticateRequest)
>
> And then of course I route the requests to these methods via the following
> code in kafka.server.KafkaApis:
>
>   def handleSaslHandshakeRequest(request: RequestChannel.Request) {
>
>     requestChannel.kafkaChannel(request) match {
>
>       case Some(kafkaChannel) =>
>
>         sendResponseMaybeThrottle(request, _ =>
> kafkaChannel.respondToReauthenticationSaslHandshakeRequest(request.header,
> request.body[SaslHandshakeRequest]))
>
>       case None =>
>
>         sendResponseMaybeThrottle(request, _ => new
> SaslHandshakeResponse(Errors.ILLEGAL_SASL_STATE, Collections.emptySet()))
>
>     }
>
>   }
>
>
>   def handleSaslAuthenticateRequest(request: RequestChannel.Request) {
>
>     requestChannel.kafkaChannel(request) match {
>
>       case Some(kafkaChannel) =>
>
>         sendResponseMaybeThrottle(request, _ =>
> kafkaChannel.respondToReauthenticationSaslAuthenticateRequest(request.
> header,
> request.body[SaslAuthenticateRequest]))
>
>       case None =>
>
>         sendResponseMaybeThrottle(request, _ => new
> SaslAuthenticateResponse(Errors.ILLEGAL_SASL_STATE,
>
>             "SaslAuthenticate request received but could not find channel
> (should not happen)"))
>
>     }
>
>   }
>
> To address what I think you are proposing -- the third alternative.  Are
> referring to the fact that SaslClientAuthenticator and
> SaslServerAuthenticator actually talk directly over the socket, and are you
> suggesting that we somehow handle re-authentication in the same way?
>
>     // buffers used in `authenticate`
>
>     private NetworkReceive netInBuffer;
>
>     private Send netOutBuffer;
>
> I did not like the fact that authentication and re-authentication ended up
> working differently -- authentication via direct socket sends and
> re-authentication via handling API requests in the normal
> kafka.server.KafkaApis API control flow.  The nonce-based proposal
> (described in the "Rejected Alternatives" section) was a way of reusing the
> authentication code for re-authentication -- simply authenticate a new
> socket to the same server and then transfer the principal to the original
> connection.  So I am definitely in favor of having authentication and
> re-authentication work the same.  I just don't know if it is possible to
> make re-authentication work the way authentication works.  I imagine that
> trying to inject stuff directly into the socket when that socket is
> simultaneously being used for something else -- it feels like a really bad
> idea that isn't viable.  It works for authentication because at that point
> we know that nobody else is using the socket.  But thereafter I would
> imagine all bets are off.
>
> Am I understanding what you were suggesting?  Was the above description
> helpful?  I think my instincts are similar to yours -- I'm not a a fan of
> making several changes to make this work in several places, and it feels a
> bit wrong to have authentication and re-authentication work differently --
> I just don't see a way around it at this point.  Do you?
>
> Ron
>
> On Tue, Sep 4, 2018 at 7:40 AM Rajini Sivaram <ra...@gmail.com>
> wrote:
>
> > Hi Ron,
> >
> > Thanks for the KIP. This is going to be a really useful feature to
> tighten
> > security.
> >
> > I have a few comments/questions:
> >
> >    1. Retries: What is the state of a client connection in between
> >    re-authentication retries? Can it continue to process requests? Also,
> > can
> >    you describe scenarios where retries are useful?
> >    2. Metrics: I think we should add separate re-authentication
> >    success/failure metrics.
> >    3. Implementation: Hmm... The PR looks quite a lot different from
> what I
> >    hoped we would do. In particular, I am concerned about network and
> > security
> >    layer code escaping into the client/broker implementation layers.
> >
> > "*We leave it to the owners of the **NetworkClient instances to define
> how
> > to inject such requests by providing an implementation of the interface
> to
> > the SaslChannelBuilder, which in turn provides it to the *
> > *SaslClientAuthenticator*"
> >
> > NetworkClient shouldn't have to deal with authentication and changes to
> > every use (AdminClient/KafkaConsumer/KafkaProducer/Broker) to deal with
> > security protocols and SASL mechanisms will be hard to maintain. And what
> > about Connect and Streams - will they need changing as well?
> >
> > Have we considered implementing re-authentication within the
> > network/security layer, like we do authentication? I think we should try
> > and find a way to move channels back to authentication/re-authentication
> > phase so that the rest of the codebase doesn't have special code to
> handle
> > SASL. I haven't looked into how this can be done, but it doesn't feel
> > impossible. Thoughts?
> >
> >
> >
> > On Mon, Sep 3, 2018 at 10:54 PM, Ron Dagostino <rn...@gmail.com>
> wrote:
> >
> > > I just realized there was one place in the KIP that stated that retries
> > > could occur indefinitely (when a client attempts to change identity,
> > which
> > > we arbitrarily disallow).  This was a mistake, a holdover from a prior
> > > draft of the KIP.  This is now fixed.  Retries are never allowed
> > > indefinitely.
> > >
> > > <<<if a connection originally authenticates as User:user1, an attempt
> to
> > > re-authenticate as anything else (e.g. User:user2) will fail.
> > > <<<Retry is allowed indefinitely in this case.
> > > >>>if a connection originally authenticates as User:user1, an attempt
> to
> > > re-authenticate as anything else (e.g. User:user2) will fail.
> > > >>>Retry is allowed in this case (still subject to the expiration of
> the
> > > original credential as described above)
> > >
> > > Ron
> > >
> > >
> > > On Mon, Sep 3, 2018 at 5:35 PM Ron Dagostino <rn...@gmail.com>
> wrote:
> > >
> > > > Thanks for the feedback, Stanislav.
> > > >
> > > > <<<I am not too familiar with the networking code to evaluate your
> > > > solution.
> > > > Yeah, I wasn't familiar with it when I started, either, and I am
> hoping
> > > > people who are intimately familiar with it will take a close look.
> > Some
> > > of
> > > > that code seems to be very central to the core of Kafka, and
> injecting
> > > > re-authentication attempts into the flow of replica fetching, sending
> > > > transaction markers, and producing or consuming messages is not
> > > something I
> > > > am convinced is acceptable under all circumstances.  To be clear,
> > > though, I
> > > > am not saying it is problematic, either -- I just don't have enough
> > > > experience or familiarity to know.  I really do want additional eyes
> on
> > > > this if possible.
> > > >
> > > > Regarding your question about retries, here's the part of the KIP
> that
> > > > deals with those:
> > > >
> > > > If a re-authentication attempt should fail then the connection will
> be
> > > >> told to retry after some delay depending on how many retries have
> been
> > > >> attempted so far: after some small amount of time for the first
> retry
> > > (e.g.
> > > >> 1 minute), double that for the second retry, and 4 times the initial
> > > delay
> > > >> for every retry thereafter.  Retry attempts generally occur at least
> > > until
> > > >> the current credential expires (but not indefinitely – and of course
> > > they
> > > >> won't continue if one of them actually succeeds).  There are certain
> > > errors
> > > >> that result in retries not being attempted (i.e. if some internal
> > state
> > > >> doesn't make sense, which generally should not happen).
> > > >
> > > >
> > > > <<<Would we retry endlessly?
> > > > No.  There may be at most one retry after the client token expires.
> So
> > > if
> > > > a token expires after an hour, and several retry attempts fail
> > including
> > > > one at minute 59, then one last attempt will be made at the 63-minute
> > > mark.
> > > >
> > > > <<<Since the functionality for brokers to close connections is
> outside
> > > the
> > > > scope of this KIP, what effect would
> > > > <<<success/failure in re-authentication have
> > > > Practically speaking, unless authorization is based on the contents
> of
> > > the
> > > > token rather than ACLs, the ultimate success or failure of
> > > > re-authentication has no effect.  The intent is definitely to follow
> > this
> > > > KIP with another one to add the ability for brokers to close
> > connections
> > > > that use expired credentials, and then at that point the client would
> > > have
> > > > to successfully re-authenticate to avoid the connection being
> forcibly
> > > > closed.
> > > >
> > > > Ron
> > > >
> > > >
> > > > On Mon, Sep 3, 2018 at 12:58 PM Stanislav Kozlovski <
> > > > stanislav@confluent.io> wrote:
> > > >
> > > >> Hi Ron,
> > > >>
> > > >> I really like this KIP, it is very needed.
> > > >> I am still looking through it but unfortunately I am not too
> familiar
> > > with
> > > >> the networking code to evaluate your solution.
> > > >>
> > > >> I'd like to ask what happens if re-authentication consistently
> fails.
> > > >> Would
> > > >> we retry endlessly? Since the functionality for brokers to close
> > > >> connections is outside the scope of this KIP, what effect would
> > > >> success/failure in re-authentication have? I think it's worth noting
> > in
> > > >> the
> > > >> KIP
> > > >>
> > > >> I also think the rejected alternative of initiating a new connection
> > > >> should
> > > >> stay rejected. I am not aware of anything currently limiting the
> > client
> > > to
> > > >> connect to the same broker, but I think it would be best if we kept
> > > >> Kafka's
> > > >> options open (e.g addition of a load balancer in the future) and not
> > > >> introduce additional client-broker statefulness.
> > > >>
> > > >> Thanks,
> > > >> Stanislav
> > > >>
> > > >> On Tue, Aug 28, 2018 at 5:28 PM Ron Dagostino <rn...@gmail.com>
> > > wrote:
> > > >>
> > > >> > Hi everyone. I created KIP 368: Allow SASL Connections to
> > Periodically
> > > >> > Re-Authenticate
> > > >> > <
> > > >> >
> > > >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-368%
> > > 3A+Allow+SASL+Connections+to+Periodically+Re-Authenticate
> > > >> > >
> > > >> > (
> > > >> >
> > > >> >
> > > >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-368%
> > > 3A+Allow+SASL+Connections+to+Periodically+Re-Authenticate
> > > >> > ).
> > > >> > The motivation for this KIP is as follows:
> > > >> >
> > > >> > The adoption of KIP-255: OAuth Authentication via SASL/OAUTHBEARER
> > > >> > <
> > > >> https://cwiki.apache.org/confluence/pages/viewpage.action?
> > > pageId=75968876
> > > >> >
> > > >> > in
> > > >> > release 2.0.0 creates the possibility of using information in the
> > > bearer
> > > >> > token to make authorization decisions.  Unfortunately, however,
> > Kafka
> > > >> > connections are long-lived, so there is no ability to change the
> > > bearer
> > > >> > token associated with a particular connection.  Allowing SASL
> > > >> connections
> > > >> > to periodically re-authenticate would resolve this.  In addition
> to
> > > this
> > > >> > motivation there are two others that are security-related.  First,
> > to
> > > >> > eliminate access to Kafka for connected clients, the current
> > > >> requirement is
> > > >> > to remove all authorizations (i.e. remove all ACLs).  This is
> > > necessary
> > > >> > because of the long-lived nature of the connections.  It is
> > > >> operationally
> > > >> > simpler to shut off access at the point of authentication, and
> with
> > > the
> > > >> > release of KIP-86: Configurable SASL Callback Handlers
> > > >> > <
> > > >> >
> > > >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-86%3A+
> > > Configurable+SASL+callback+handlers
> > > >> > >
> > > >> > it
> > > >> > is going to become more and more likely that installations will
> > > >> > authenticate users against external directories (e.g. via LDAP).
> > The
> > > >> > ability to stop Kafka access by simply disabling an account in an
> > LDAP
> > > >> > directory (for example) is desirable.  The second motivating
> factor
> > > for
> > > >> > re-authentication related to security is that the use of
> short-lived
> > > >> tokens
> > > >> > is a common OAuth security recommendation, but issuing a
> short-lived
> > > >> token
> > > >> > to a Kafka client (or a broker when OAUTHBEARER is the
> inter-broker
> > > >> > protocol) currently has no benefit because once a client is
> > connected
> > > >> to a
> > > >> > broker the client is never challenged again and the connection may
> > > >> remain
> > > >> > intact beyond the token expiration time (and may remain intact
> > > >> indefinitely
> > > >> > under perfect circumstances).  This KIP proposes adding the
> ability
> > > for
> > > >> > clients (and brokers when OAUTHBEARER is the inter-broker
> protocol)
> > to
> > > >> > re-authenticate their connections to brokers and have the new
> bearer
> > > >> token
> > > >> > appear on their session rather than the old one.
> > > >> >
> > > >> > The description of this KIP is actually quite straightforward
> from a
> > > >> > functionality perspective; from an implementation perspective,
> > though,
> > > >> the
> > > >> > KIP is not so straightforward, so it includes a pull request with
> a
> > > >> > proposed implementation.  See https://github.com/apache/kafk
> > > a/pull/5582
> > > >> .
> > > >> >
> > > >> > Ron
> > > >> >
> > > >>
> > > >>
> > > >> --
> > > >> Best,
> > > >> Stanislav
> > > >>
> > > >
> > >
> >
>

Re: [DISCUSS] KIP 368: Allow SASL Connections to Periodically Re-Authenticate

Posted by Ron Dagostino <rn...@gmail.com>.
Hi Rajini.  I'm glad you agree this is a good security addition.  Regarding
your questions/comments:

1. Retries.

<<<What is the state of a client connection in between re-authentication
retries?
<<<Can it continue to process requests?
Yes, the existing connection remains intact and continues to accept
requests and process responses.  In fact, the existing connection is never
closed/replaced at any time.  The 3 APIs that constitute the authentication
"dance" (ApiVersionsRequest/Response, SaslHandshakeRequest/Response, and
SaslAuthenticateRequest/Response) traverse the existing socket connection,
and the only impact of re-authentication (aside from sending these three
API calls) is that the KafkaChannel#principal() method on the server side
ends up returning a different value once re-authentication succeeds
(because the KafkaChannel instance ends up with a new instance of
SaslServerAuthenticator, which is where the call gets delegated).

<<<Also, can you describe scenarios where retries are useful?
I have added this sentence to the KIP: "A retry is helpful when
re-authentication fails on the server side due to some temporary outage
(for example, the re-authentication will fail through no fault of the
client if the token endpoint is unavailable and the broker has not yet
cached the public key required to validate the token signature)."

2. Metrics:

<<<I think we should add separate re-authentication success/failure metrics.
Yes, I stated there are none only to be transparent about that fact more
than anything, and I am not surprised to get this feedback.  I will add
them to the KIP.

3. Implementation

<<<Hmm...
lol.  Yeah, I'm not surprised to get this feedback, either.  I came at this
totally fresh (i.e. no knowledge of this part of the code base), and I
avoided writing the integration tests (which I think might take a bit of
time) because I wanted confirmation on the approach first.  So this
feedback is excellent.  Let me try to unpack/address it, and let's see
where we end up.

<<<NetworkClient shouldn't have to deal with authentication
Actually, the only change I made to NetworkClient is to add the below
method because there is currently no way to submit a request with a
callback and have that request use the default timeout value of the client
-- currently if you specify a callback you also must explicitly specify a
timeout.  I considered adding this to the KafkaClient interface but decided
this was quicker in terms of getting the KIP and PR out there for
discussion.

public ClientRequest newClientRequest(String nodeId,

                                      AbstractRequest.Builder<?>
requestBuilder,

                                      long createdTimeMs,

                                      boolean expectResponse,

                                      RequestCompletionHandler callback) {

    return newClientRequest(nodeId, requestBuilder, createdTimeMs,
expectResponse, defaultRequestTimeoutMs, callback);

}


<<<changes to every use (AdminClient/KafkaConsumer/KafkaProducer/Broker) to

<<<deal with security protocols and SASL mechanisms will be hard to
maintain.

<<<And what about Connect and Streams - will they need changing as well?


I believe Streams and Connect are taken care of because they are built on
top of standard producers and consumers.  I nevertheless do agree with the
comment that this might be hard to maintain.  If there is ever a need for a
new use of NetworkClient -- whether as part of new inter-broker
communication functionality or as part of a client that does not leverage
the standard producer/consumer -- then that new use will have to address
the issue of how to inject the requests related to re-authentication.  I am
not happy with this, and I only arrived at it after rejecting the
"one-size-fits-all" approach that I mentioned in the "Rejected
Alternatives" section.  I think you are suggesting a third alternative,
though.  I'll address that in a moment.

First, I do want to make sure we are on the same page with what the current
changes are with respect to imports.  I realize I may be repeating stuff
you already know having looked at the PR, and I apologize if this is the
case.  I did have to make one change to checkstyle/import-control.xml to
allow org.apache.kafka.common.network.KafkaChannel to import the below
classes -- aside from that, though, every other import I added was already
"allowed".

// added to KafkaChannel

import org.apache.kafka.common.requests.RequestHeader;

import org.apache.kafka.common.requests.SaslAuthenticateRequest;

import org.apache.kafka.common.requests.SaslAuthenticateResponse;

import org.apache.kafka.common.requests.SaslHandshakeRequest;

import org.apache.kafka.common.requests.SaslHandshakeResponse;

The above needed to be imported because of the addition of these two
methods to KafkaChannel:

    /**
     * Respond to a SASL re-authentication initial handshake. This occurs
on the
     * SASL Server side of the re-authentication dance (i.e. on the broker).
     *
     * @param requestHeader
     *            the request header
     * @param saslHandshakeRequest
     *            the initial handshake request to process
     * @return the response to return to the client
     */
    public SaslHandshakeResponse
respondToReauthenticationSaslHandshakeRequest(RequestHeader requestHeader,
            SaslHandshakeRequest saslHandshakeRequest)

    /**
     * Respond to a SASL re-authentication token exchange. This occurs on
the SASL
     * Server side of the re-authentication dance (i.e. on the broker).
     *
     * @param requestHeader
     *            the request header
     * @param saslAuthenticateRequest
     *            the token exchange request to process
     * @return the response to return to the client
     */
    public SaslAuthenticateResponse
respondToReauthenticationSaslAuthenticateRequest(RequestHeader
requestHeader,
            SaslAuthenticateRequest saslAuthenticateRequest)

And then of course I route the requests to these methods via the following
code in kafka.server.KafkaApis:

  def handleSaslHandshakeRequest(request: RequestChannel.Request) {

    requestChannel.kafkaChannel(request) match {

      case Some(kafkaChannel) =>

        sendResponseMaybeThrottle(request, _ =>
kafkaChannel.respondToReauthenticationSaslHandshakeRequest(request.header,
request.body[SaslHandshakeRequest]))

      case None =>

        sendResponseMaybeThrottle(request, _ => new
SaslHandshakeResponse(Errors.ILLEGAL_SASL_STATE, Collections.emptySet()))

    }

  }


  def handleSaslAuthenticateRequest(request: RequestChannel.Request) {

    requestChannel.kafkaChannel(request) match {

      case Some(kafkaChannel) =>

        sendResponseMaybeThrottle(request, _ =>
kafkaChannel.respondToReauthenticationSaslAuthenticateRequest(request.header,
request.body[SaslAuthenticateRequest]))

      case None =>

        sendResponseMaybeThrottle(request, _ => new
SaslAuthenticateResponse(Errors.ILLEGAL_SASL_STATE,

            "SaslAuthenticate request received but could not find channel
(should not happen)"))

    }

  }

To address what I think you are proposing -- the third alternative.  Are
referring to the fact that SaslClientAuthenticator and
SaslServerAuthenticator actually talk directly over the socket, and are you
suggesting that we somehow handle re-authentication in the same way?

    // buffers used in `authenticate`

    private NetworkReceive netInBuffer;

    private Send netOutBuffer;

I did not like the fact that authentication and re-authentication ended up
working differently -- authentication via direct socket sends and
re-authentication via handling API requests in the normal
kafka.server.KafkaApis API control flow.  The nonce-based proposal
(described in the "Rejected Alternatives" section) was a way of reusing the
authentication code for re-authentication -- simply authenticate a new
socket to the same server and then transfer the principal to the original
connection.  So I am definitely in favor of having authentication and
re-authentication work the same.  I just don't know if it is possible to
make re-authentication work the way authentication works.  I imagine that
trying to inject stuff directly into the socket when that socket is
simultaneously being used for something else -- it feels like a really bad
idea that isn't viable.  It works for authentication because at that point
we know that nobody else is using the socket.  But thereafter I would
imagine all bets are off.

Am I understanding what you were suggesting?  Was the above description
helpful?  I think my instincts are similar to yours -- I'm not a a fan of
making several changes to make this work in several places, and it feels a
bit wrong to have authentication and re-authentication work differently --
I just don't see a way around it at this point.  Do you?

Ron

On Tue, Sep 4, 2018 at 7:40 AM Rajini Sivaram <ra...@gmail.com>
wrote:

> Hi Ron,
>
> Thanks for the KIP. This is going to be a really useful feature to tighten
> security.
>
> I have a few comments/questions:
>
>    1. Retries: What is the state of a client connection in between
>    re-authentication retries? Can it continue to process requests? Also,
> can
>    you describe scenarios where retries are useful?
>    2. Metrics: I think we should add separate re-authentication
>    success/failure metrics.
>    3. Implementation: Hmm... The PR looks quite a lot different from what I
>    hoped we would do. In particular, I am concerned about network and
> security
>    layer code escaping into the client/broker implementation layers.
>
> "*We leave it to the owners of the **NetworkClient instances to define how
> to inject such requests by providing an implementation of the interface to
> the SaslChannelBuilder, which in turn provides it to the *
> *SaslClientAuthenticator*"
>
> NetworkClient shouldn't have to deal with authentication and changes to
> every use (AdminClient/KafkaConsumer/KafkaProducer/Broker) to deal with
> security protocols and SASL mechanisms will be hard to maintain. And what
> about Connect and Streams - will they need changing as well?
>
> Have we considered implementing re-authentication within the
> network/security layer, like we do authentication? I think we should try
> and find a way to move channels back to authentication/re-authentication
> phase so that the rest of the codebase doesn't have special code to handle
> SASL. I haven't looked into how this can be done, but it doesn't feel
> impossible. Thoughts?
>
>
>
> On Mon, Sep 3, 2018 at 10:54 PM, Ron Dagostino <rn...@gmail.com> wrote:
>
> > I just realized there was one place in the KIP that stated that retries
> > could occur indefinitely (when a client attempts to change identity,
> which
> > we arbitrarily disallow).  This was a mistake, a holdover from a prior
> > draft of the KIP.  This is now fixed.  Retries are never allowed
> > indefinitely.
> >
> > <<<if a connection originally authenticates as User:user1, an attempt to
> > re-authenticate as anything else (e.g. User:user2) will fail.
> > <<<Retry is allowed indefinitely in this case.
> > >>>if a connection originally authenticates as User:user1, an attempt to
> > re-authenticate as anything else (e.g. User:user2) will fail.
> > >>>Retry is allowed in this case (still subject to the expiration of the
> > original credential as described above)
> >
> > Ron
> >
> >
> > On Mon, Sep 3, 2018 at 5:35 PM Ron Dagostino <rn...@gmail.com> wrote:
> >
> > > Thanks for the feedback, Stanislav.
> > >
> > > <<<I am not too familiar with the networking code to evaluate your
> > > solution.
> > > Yeah, I wasn't familiar with it when I started, either, and I am hoping
> > > people who are intimately familiar with it will take a close look.
> Some
> > of
> > > that code seems to be very central to the core of Kafka, and injecting
> > > re-authentication attempts into the flow of replica fetching, sending
> > > transaction markers, and producing or consuming messages is not
> > something I
> > > am convinced is acceptable under all circumstances.  To be clear,
> > though, I
> > > am not saying it is problematic, either -- I just don't have enough
> > > experience or familiarity to know.  I really do want additional eyes on
> > > this if possible.
> > >
> > > Regarding your question about retries, here's the part of the KIP that
> > > deals with those:
> > >
> > > If a re-authentication attempt should fail then the connection will be
> > >> told to retry after some delay depending on how many retries have been
> > >> attempted so far: after some small amount of time for the first retry
> > (e.g.
> > >> 1 minute), double that for the second retry, and 4 times the initial
> > delay
> > >> for every retry thereafter.  Retry attempts generally occur at least
> > until
> > >> the current credential expires (but not indefinitely – and of course
> > they
> > >> won't continue if one of them actually succeeds).  There are certain
> > errors
> > >> that result in retries not being attempted (i.e. if some internal
> state
> > >> doesn't make sense, which generally should not happen).
> > >
> > >
> > > <<<Would we retry endlessly?
> > > No.  There may be at most one retry after the client token expires.  So
> > if
> > > a token expires after an hour, and several retry attempts fail
> including
> > > one at minute 59, then one last attempt will be made at the 63-minute
> > mark.
> > >
> > > <<<Since the functionality for brokers to close connections is outside
> > the
> > > scope of this KIP, what effect would
> > > <<<success/failure in re-authentication have
> > > Practically speaking, unless authorization is based on the contents of
> > the
> > > token rather than ACLs, the ultimate success or failure of
> > > re-authentication has no effect.  The intent is definitely to follow
> this
> > > KIP with another one to add the ability for brokers to close
> connections
> > > that use expired credentials, and then at that point the client would
> > have
> > > to successfully re-authenticate to avoid the connection being forcibly
> > > closed.
> > >
> > > Ron
> > >
> > >
> > > On Mon, Sep 3, 2018 at 12:58 PM Stanislav Kozlovski <
> > > stanislav@confluent.io> wrote:
> > >
> > >> Hi Ron,
> > >>
> > >> I really like this KIP, it is very needed.
> > >> I am still looking through it but unfortunately I am not too familiar
> > with
> > >> the networking code to evaluate your solution.
> > >>
> > >> I'd like to ask what happens if re-authentication consistently fails.
> > >> Would
> > >> we retry endlessly? Since the functionality for brokers to close
> > >> connections is outside the scope of this KIP, what effect would
> > >> success/failure in re-authentication have? I think it's worth noting
> in
> > >> the
> > >> KIP
> > >>
> > >> I also think the rejected alternative of initiating a new connection
> > >> should
> > >> stay rejected. I am not aware of anything currently limiting the
> client
> > to
> > >> connect to the same broker, but I think it would be best if we kept
> > >> Kafka's
> > >> options open (e.g addition of a load balancer in the future) and not
> > >> introduce additional client-broker statefulness.
> > >>
> > >> Thanks,
> > >> Stanislav
> > >>
> > >> On Tue, Aug 28, 2018 at 5:28 PM Ron Dagostino <rn...@gmail.com>
> > wrote:
> > >>
> > >> > Hi everyone. I created KIP 368: Allow SASL Connections to
> Periodically
> > >> > Re-Authenticate
> > >> > <
> > >> >
> > >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-368%
> > 3A+Allow+SASL+Connections+to+Periodically+Re-Authenticate
> > >> > >
> > >> > (
> > >> >
> > >> >
> > >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-368%
> > 3A+Allow+SASL+Connections+to+Periodically+Re-Authenticate
> > >> > ).
> > >> > The motivation for this KIP is as follows:
> > >> >
> > >> > The adoption of KIP-255: OAuth Authentication via SASL/OAUTHBEARER
> > >> > <
> > >> https://cwiki.apache.org/confluence/pages/viewpage.action?
> > pageId=75968876
> > >> >
> > >> > in
> > >> > release 2.0.0 creates the possibility of using information in the
> > bearer
> > >> > token to make authorization decisions.  Unfortunately, however,
> Kafka
> > >> > connections are long-lived, so there is no ability to change the
> > bearer
> > >> > token associated with a particular connection.  Allowing SASL
> > >> connections
> > >> > to periodically re-authenticate would resolve this.  In addition to
> > this
> > >> > motivation there are two others that are security-related.  First,
> to
> > >> > eliminate access to Kafka for connected clients, the current
> > >> requirement is
> > >> > to remove all authorizations (i.e. remove all ACLs).  This is
> > necessary
> > >> > because of the long-lived nature of the connections.  It is
> > >> operationally
> > >> > simpler to shut off access at the point of authentication, and with
> > the
> > >> > release of KIP-86: Configurable SASL Callback Handlers
> > >> > <
> > >> >
> > >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-86%3A+
> > Configurable+SASL+callback+handlers
> > >> > >
> > >> > it
> > >> > is going to become more and more likely that installations will
> > >> > authenticate users against external directories (e.g. via LDAP).
> The
> > >> > ability to stop Kafka access by simply disabling an account in an
> LDAP
> > >> > directory (for example) is desirable.  The second motivating factor
> > for
> > >> > re-authentication related to security is that the use of short-lived
> > >> tokens
> > >> > is a common OAuth security recommendation, but issuing a short-lived
> > >> token
> > >> > to a Kafka client (or a broker when OAUTHBEARER is the inter-broker
> > >> > protocol) currently has no benefit because once a client is
> connected
> > >> to a
> > >> > broker the client is never challenged again and the connection may
> > >> remain
> > >> > intact beyond the token expiration time (and may remain intact
> > >> indefinitely
> > >> > under perfect circumstances).  This KIP proposes adding the ability
> > for
> > >> > clients (and brokers when OAUTHBEARER is the inter-broker protocol)
> to
> > >> > re-authenticate their connections to brokers and have the new bearer
> > >> token
> > >> > appear on their session rather than the old one.
> > >> >
> > >> > The description of this KIP is actually quite straightforward from a
> > >> > functionality perspective; from an implementation perspective,
> though,
> > >> the
> > >> > KIP is not so straightforward, so it includes a pull request with a
> > >> > proposed implementation.  See https://github.com/apache/kafk
> > a/pull/5582
> > >> .
> > >> >
> > >> > Ron
> > >> >
> > >>
> > >>
> > >> --
> > >> Best,
> > >> Stanislav
> > >>
> > >
> >
>

Re: [DISCUSS] KIP 368: Allow SASL Connections to Periodically Re-Authenticate

Posted by Rajini Sivaram <ra...@gmail.com>.
Hi Ron,

Thanks for the KIP. This is going to be a really useful feature to tighten
security.

I have a few comments/questions:

   1. Retries: What is the state of a client connection in between
   re-authentication retries? Can it continue to process requests? Also, can
   you describe scenarios where retries are useful?
   2. Metrics: I think we should add separate re-authentication
   success/failure metrics.
   3. Implementation: Hmm... The PR looks quite a lot different from what I
   hoped we would do. In particular, I am concerned about network and security
   layer code escaping into the client/broker implementation layers.

"*We leave it to the owners of the **NetworkClient instances to define how
to inject such requests by providing an implementation of the interface to
the SaslChannelBuilder, which in turn provides it to the *
*SaslClientAuthenticator*"

NetworkClient shouldn't have to deal with authentication and changes to
every use (AdminClient/KafkaConsumer/KafkaProducer/Broker) to deal with
security protocols and SASL mechanisms will be hard to maintain. And what
about Connect and Streams - will they need changing as well?

Have we considered implementing re-authentication within the
network/security layer, like we do authentication? I think we should try
and find a way to move channels back to authentication/re-authentication
phase so that the rest of the codebase doesn't have special code to handle
SASL. I haven't looked into how this can be done, but it doesn't feel
impossible. Thoughts?



On Mon, Sep 3, 2018 at 10:54 PM, Ron Dagostino <rn...@gmail.com> wrote:

> I just realized there was one place in the KIP that stated that retries
> could occur indefinitely (when a client attempts to change identity, which
> we arbitrarily disallow).  This was a mistake, a holdover from a prior
> draft of the KIP.  This is now fixed.  Retries are never allowed
> indefinitely.
>
> <<<if a connection originally authenticates as User:user1, an attempt to
> re-authenticate as anything else (e.g. User:user2) will fail.
> <<<Retry is allowed indefinitely in this case.
> >>>if a connection originally authenticates as User:user1, an attempt to
> re-authenticate as anything else (e.g. User:user2) will fail.
> >>>Retry is allowed in this case (still subject to the expiration of the
> original credential as described above)
>
> Ron
>
>
> On Mon, Sep 3, 2018 at 5:35 PM Ron Dagostino <rn...@gmail.com> wrote:
>
> > Thanks for the feedback, Stanislav.
> >
> > <<<I am not too familiar with the networking code to evaluate your
> > solution.
> > Yeah, I wasn't familiar with it when I started, either, and I am hoping
> > people who are intimately familiar with it will take a close look.  Some
> of
> > that code seems to be very central to the core of Kafka, and injecting
> > re-authentication attempts into the flow of replica fetching, sending
> > transaction markers, and producing or consuming messages is not
> something I
> > am convinced is acceptable under all circumstances.  To be clear,
> though, I
> > am not saying it is problematic, either -- I just don't have enough
> > experience or familiarity to know.  I really do want additional eyes on
> > this if possible.
> >
> > Regarding your question about retries, here's the part of the KIP that
> > deals with those:
> >
> > If a re-authentication attempt should fail then the connection will be
> >> told to retry after some delay depending on how many retries have been
> >> attempted so far: after some small amount of time for the first retry
> (e.g.
> >> 1 minute), double that for the second retry, and 4 times the initial
> delay
> >> for every retry thereafter.  Retry attempts generally occur at least
> until
> >> the current credential expires (but not indefinitely – and of course
> they
> >> won't continue if one of them actually succeeds).  There are certain
> errors
> >> that result in retries not being attempted (i.e. if some internal state
> >> doesn't make sense, which generally should not happen).
> >
> >
> > <<<Would we retry endlessly?
> > No.  There may be at most one retry after the client token expires.  So
> if
> > a token expires after an hour, and several retry attempts fail including
> > one at minute 59, then one last attempt will be made at the 63-minute
> mark.
> >
> > <<<Since the functionality for brokers to close connections is outside
> the
> > scope of this KIP, what effect would
> > <<<success/failure in re-authentication have
> > Practically speaking, unless authorization is based on the contents of
> the
> > token rather than ACLs, the ultimate success or failure of
> > re-authentication has no effect.  The intent is definitely to follow this
> > KIP with another one to add the ability for brokers to close connections
> > that use expired credentials, and then at that point the client would
> have
> > to successfully re-authenticate to avoid the connection being forcibly
> > closed.
> >
> > Ron
> >
> >
> > On Mon, Sep 3, 2018 at 12:58 PM Stanislav Kozlovski <
> > stanislav@confluent.io> wrote:
> >
> >> Hi Ron,
> >>
> >> I really like this KIP, it is very needed.
> >> I am still looking through it but unfortunately I am not too familiar
> with
> >> the networking code to evaluate your solution.
> >>
> >> I'd like to ask what happens if re-authentication consistently fails.
> >> Would
> >> we retry endlessly? Since the functionality for brokers to close
> >> connections is outside the scope of this KIP, what effect would
> >> success/failure in re-authentication have? I think it's worth noting in
> >> the
> >> KIP
> >>
> >> I also think the rejected alternative of initiating a new connection
> >> should
> >> stay rejected. I am not aware of anything currently limiting the client
> to
> >> connect to the same broker, but I think it would be best if we kept
> >> Kafka's
> >> options open (e.g addition of a load balancer in the future) and not
> >> introduce additional client-broker statefulness.
> >>
> >> Thanks,
> >> Stanislav
> >>
> >> On Tue, Aug 28, 2018 at 5:28 PM Ron Dagostino <rn...@gmail.com>
> wrote:
> >>
> >> > Hi everyone. I created KIP 368: Allow SASL Connections to Periodically
> >> > Re-Authenticate
> >> > <
> >> >
> >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-368%
> 3A+Allow+SASL+Connections+to+Periodically+Re-Authenticate
> >> > >
> >> > (
> >> >
> >> >
> >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-368%
> 3A+Allow+SASL+Connections+to+Periodically+Re-Authenticate
> >> > ).
> >> > The motivation for this KIP is as follows:
> >> >
> >> > The adoption of KIP-255: OAuth Authentication via SASL/OAUTHBEARER
> >> > <
> >> https://cwiki.apache.org/confluence/pages/viewpage.action?
> pageId=75968876
> >> >
> >> > in
> >> > release 2.0.0 creates the possibility of using information in the
> bearer
> >> > token to make authorization decisions.  Unfortunately, however, Kafka
> >> > connections are long-lived, so there is no ability to change the
> bearer
> >> > token associated with a particular connection.  Allowing SASL
> >> connections
> >> > to periodically re-authenticate would resolve this.  In addition to
> this
> >> > motivation there are two others that are security-related.  First, to
> >> > eliminate access to Kafka for connected clients, the current
> >> requirement is
> >> > to remove all authorizations (i.e. remove all ACLs).  This is
> necessary
> >> > because of the long-lived nature of the connections.  It is
> >> operationally
> >> > simpler to shut off access at the point of authentication, and with
> the
> >> > release of KIP-86: Configurable SASL Callback Handlers
> >> > <
> >> >
> >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-86%3A+
> Configurable+SASL+callback+handlers
> >> > >
> >> > it
> >> > is going to become more and more likely that installations will
> >> > authenticate users against external directories (e.g. via LDAP).  The
> >> > ability to stop Kafka access by simply disabling an account in an LDAP
> >> > directory (for example) is desirable.  The second motivating factor
> for
> >> > re-authentication related to security is that the use of short-lived
> >> tokens
> >> > is a common OAuth security recommendation, but issuing a short-lived
> >> token
> >> > to a Kafka client (or a broker when OAUTHBEARER is the inter-broker
> >> > protocol) currently has no benefit because once a client is connected
> >> to a
> >> > broker the client is never challenged again and the connection may
> >> remain
> >> > intact beyond the token expiration time (and may remain intact
> >> indefinitely
> >> > under perfect circumstances).  This KIP proposes adding the ability
> for
> >> > clients (and brokers when OAUTHBEARER is the inter-broker protocol) to
> >> > re-authenticate their connections to brokers and have the new bearer
> >> token
> >> > appear on their session rather than the old one.
> >> >
> >> > The description of this KIP is actually quite straightforward from a
> >> > functionality perspective; from an implementation perspective, though,
> >> the
> >> > KIP is not so straightforward, so it includes a pull request with a
> >> > proposed implementation.  See https://github.com/apache/kafk
> a/pull/5582
> >> .
> >> >
> >> > Ron
> >> >
> >>
> >>
> >> --
> >> Best,
> >> Stanislav
> >>
> >
>

Re: [DISCUSS] KIP 368: Allow SASL Connections to Periodically Re-Authenticate

Posted by Ron Dagostino <rn...@gmail.com>.
I just realized there was one place in the KIP that stated that retries
could occur indefinitely (when a client attempts to change identity, which
we arbitrarily disallow).  This was a mistake, a holdover from a prior
draft of the KIP.  This is now fixed.  Retries are never allowed
indefinitely.

<<<if a connection originally authenticates as User:user1, an attempt to
re-authenticate as anything else (e.g. User:user2) will fail.
<<<Retry is allowed indefinitely in this case.
>>>if a connection originally authenticates as User:user1, an attempt to
re-authenticate as anything else (e.g. User:user2) will fail.
>>>Retry is allowed in this case (still subject to the expiration of the
original credential as described above)

Ron


On Mon, Sep 3, 2018 at 5:35 PM Ron Dagostino <rn...@gmail.com> wrote:

> Thanks for the feedback, Stanislav.
>
> <<<I am not too familiar with the networking code to evaluate your
> solution.
> Yeah, I wasn't familiar with it when I started, either, and I am hoping
> people who are intimately familiar with it will take a close look.  Some of
> that code seems to be very central to the core of Kafka, and injecting
> re-authentication attempts into the flow of replica fetching, sending
> transaction markers, and producing or consuming messages is not something I
> am convinced is acceptable under all circumstances.  To be clear, though, I
> am not saying it is problematic, either -- I just don't have enough
> experience or familiarity to know.  I really do want additional eyes on
> this if possible.
>
> Regarding your question about retries, here's the part of the KIP that
> deals with those:
>
> If a re-authentication attempt should fail then the connection will be
>> told to retry after some delay depending on how many retries have been
>> attempted so far: after some small amount of time for the first retry (e.g.
>> 1 minute), double that for the second retry, and 4 times the initial delay
>> for every retry thereafter.  Retry attempts generally occur at least until
>> the current credential expires (but not indefinitely – and of course they
>> won't continue if one of them actually succeeds).  There are certain errors
>> that result in retries not being attempted (i.e. if some internal state
>> doesn't make sense, which generally should not happen).
>
>
> <<<Would we retry endlessly?
> No.  There may be at most one retry after the client token expires.  So if
> a token expires after an hour, and several retry attempts fail including
> one at minute 59, then one last attempt will be made at the 63-minute mark.
>
> <<<Since the functionality for brokers to close connections is outside the
> scope of this KIP, what effect would
> <<<success/failure in re-authentication have
> Practically speaking, unless authorization is based on the contents of the
> token rather than ACLs, the ultimate success or failure of
> re-authentication has no effect.  The intent is definitely to follow this
> KIP with another one to add the ability for brokers to close connections
> that use expired credentials, and then at that point the client would have
> to successfully re-authenticate to avoid the connection being forcibly
> closed.
>
> Ron
>
>
> On Mon, Sep 3, 2018 at 12:58 PM Stanislav Kozlovski <
> stanislav@confluent.io> wrote:
>
>> Hi Ron,
>>
>> I really like this KIP, it is very needed.
>> I am still looking through it but unfortunately I am not too familiar with
>> the networking code to evaluate your solution.
>>
>> I'd like to ask what happens if re-authentication consistently fails.
>> Would
>> we retry endlessly? Since the functionality for brokers to close
>> connections is outside the scope of this KIP, what effect would
>> success/failure in re-authentication have? I think it's worth noting in
>> the
>> KIP
>>
>> I also think the rejected alternative of initiating a new connection
>> should
>> stay rejected. I am not aware of anything currently limiting the client to
>> connect to the same broker, but I think it would be best if we kept
>> Kafka's
>> options open (e.g addition of a load balancer in the future) and not
>> introduce additional client-broker statefulness.
>>
>> Thanks,
>> Stanislav
>>
>> On Tue, Aug 28, 2018 at 5:28 PM Ron Dagostino <rn...@gmail.com> wrote:
>>
>> > Hi everyone. I created KIP 368: Allow SASL Connections to Periodically
>> > Re-Authenticate
>> > <
>> >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-368%3A+Allow+SASL+Connections+to+Periodically+Re-Authenticate
>> > >
>> > (
>> >
>> >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-368%3A+Allow+SASL+Connections+to+Periodically+Re-Authenticate
>> > ).
>> > The motivation for this KIP is as follows:
>> >
>> > The adoption of KIP-255: OAuth Authentication via SASL/OAUTHBEARER
>> > <
>> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=75968876
>> >
>> > in
>> > release 2.0.0 creates the possibility of using information in the bearer
>> > token to make authorization decisions.  Unfortunately, however, Kafka
>> > connections are long-lived, so there is no ability to change the bearer
>> > token associated with a particular connection.  Allowing SASL
>> connections
>> > to periodically re-authenticate would resolve this.  In addition to this
>> > motivation there are two others that are security-related.  First, to
>> > eliminate access to Kafka for connected clients, the current
>> requirement is
>> > to remove all authorizations (i.e. remove all ACLs).  This is necessary
>> > because of the long-lived nature of the connections.  It is
>> operationally
>> > simpler to shut off access at the point of authentication, and with the
>> > release of KIP-86: Configurable SASL Callback Handlers
>> > <
>> >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-86%3A+Configurable+SASL+callback+handlers
>> > >
>> > it
>> > is going to become more and more likely that installations will
>> > authenticate users against external directories (e.g. via LDAP).  The
>> > ability to stop Kafka access by simply disabling an account in an LDAP
>> > directory (for example) is desirable.  The second motivating factor for
>> > re-authentication related to security is that the use of short-lived
>> tokens
>> > is a common OAuth security recommendation, but issuing a short-lived
>> token
>> > to a Kafka client (or a broker when OAUTHBEARER is the inter-broker
>> > protocol) currently has no benefit because once a client is connected
>> to a
>> > broker the client is never challenged again and the connection may
>> remain
>> > intact beyond the token expiration time (and may remain intact
>> indefinitely
>> > under perfect circumstances).  This KIP proposes adding the ability for
>> > clients (and brokers when OAUTHBEARER is the inter-broker protocol) to
>> > re-authenticate their connections to brokers and have the new bearer
>> token
>> > appear on their session rather than the old one.
>> >
>> > The description of this KIP is actually quite straightforward from a
>> > functionality perspective; from an implementation perspective, though,
>> the
>> > KIP is not so straightforward, so it includes a pull request with a
>> > proposed implementation.  See https://github.com/apache/kafka/pull/5582
>> .
>> >
>> > Ron
>> >
>>
>>
>> --
>> Best,
>> Stanislav
>>
>

Re: [DISCUSS] KIP 368: Allow SASL Connections to Periodically Re-Authenticate

Posted by Ron Dagostino <rn...@gmail.com>.
Thanks for the feedback, Stanislav.

<<<I am not too familiar with the networking code to evaluate your solution.
Yeah, I wasn't familiar with it when I started, either, and I am hoping
people who are intimately familiar with it will take a close look.  Some of
that code seems to be very central to the core of Kafka, and injecting
re-authentication attempts into the flow of replica fetching, sending
transaction markers, and producing or consuming messages is not something I
am convinced is acceptable under all circumstances.  To be clear, though, I
am not saying it is problematic, either -- I just don't have enough
experience or familiarity to know.  I really do want additional eyes on
this if possible.

Regarding your question about retries, here's the part of the KIP that
deals with those:

If a re-authentication attempt should fail then the connection will be told
> to retry after some delay depending on how many retries have been attempted
> so far: after some small amount of time for the first retry (e.g. 1
> minute), double that for the second retry, and 4 times the initial delay
> for every retry thereafter.  Retry attempts generally occur at least until
> the current credential expires (but not indefinitely – and of course they
> won't continue if one of them actually succeeds).  There are certain errors
> that result in retries not being attempted (i.e. if some internal state
> doesn't make sense, which generally should not happen).


<<<Would we retry endlessly?
No.  There may be at most one retry after the client token expires.  So if
a token expires after an hour, and several retry attempts fail including
one at minute 59, then one last attempt will be made at the 63-minute mark.

<<<Since the functionality for brokers to close connections is outside the
scope of this KIP, what effect would
<<<success/failure in re-authentication have
Practically speaking, unless authorization is based on the contents of the
token rather than ACLs, the ultimate success or failure of
re-authentication has no effect.  The intent is definitely to follow this
KIP with another one to add the ability for brokers to close connections
that use expired credentials, and then at that point the client would have
to successfully re-authenticate to avoid the connection being forcibly
closed.

Ron


On Mon, Sep 3, 2018 at 12:58 PM Stanislav Kozlovski <st...@confluent.io>
wrote:

> Hi Ron,
>
> I really like this KIP, it is very needed.
> I am still looking through it but unfortunately I am not too familiar with
> the networking code to evaluate your solution.
>
> I'd like to ask what happens if re-authentication consistently fails. Would
> we retry endlessly? Since the functionality for brokers to close
> connections is outside the scope of this KIP, what effect would
> success/failure in re-authentication have? I think it's worth noting in the
> KIP
>
> I also think the rejected alternative of initiating a new connection should
> stay rejected. I am not aware of anything currently limiting the client to
> connect to the same broker, but I think it would be best if we kept Kafka's
> options open (e.g addition of a load balancer in the future) and not
> introduce additional client-broker statefulness.
>
> Thanks,
> Stanislav
>
> On Tue, Aug 28, 2018 at 5:28 PM Ron Dagostino <rn...@gmail.com> wrote:
>
> > Hi everyone. I created KIP 368: Allow SASL Connections to Periodically
> > Re-Authenticate
> > <
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-368%3A+Allow+SASL+Connections+to+Periodically+Re-Authenticate
> > >
> > (
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-368%3A+Allow+SASL+Connections+to+Periodically+Re-Authenticate
> > ).
> > The motivation for this KIP is as follows:
> >
> > The adoption of KIP-255: OAuth Authentication via SASL/OAUTHBEARER
> > <
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=75968876>
> > in
> > release 2.0.0 creates the possibility of using information in the bearer
> > token to make authorization decisions.  Unfortunately, however, Kafka
> > connections are long-lived, so there is no ability to change the bearer
> > token associated with a particular connection.  Allowing SASL connections
> > to periodically re-authenticate would resolve this.  In addition to this
> > motivation there are two others that are security-related.  First, to
> > eliminate access to Kafka for connected clients, the current requirement
> is
> > to remove all authorizations (i.e. remove all ACLs).  This is necessary
> > because of the long-lived nature of the connections.  It is operationally
> > simpler to shut off access at the point of authentication, and with the
> > release of KIP-86: Configurable SASL Callback Handlers
> > <
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-86%3A+Configurable+SASL+callback+handlers
> > >
> > it
> > is going to become more and more likely that installations will
> > authenticate users against external directories (e.g. via LDAP).  The
> > ability to stop Kafka access by simply disabling an account in an LDAP
> > directory (for example) is desirable.  The second motivating factor for
> > re-authentication related to security is that the use of short-lived
> tokens
> > is a common OAuth security recommendation, but issuing a short-lived
> token
> > to a Kafka client (or a broker when OAUTHBEARER is the inter-broker
> > protocol) currently has no benefit because once a client is connected to
> a
> > broker the client is never challenged again and the connection may remain
> > intact beyond the token expiration time (and may remain intact
> indefinitely
> > under perfect circumstances).  This KIP proposes adding the ability for
> > clients (and brokers when OAUTHBEARER is the inter-broker protocol) to
> > re-authenticate their connections to brokers and have the new bearer
> token
> > appear on their session rather than the old one.
> >
> > The description of this KIP is actually quite straightforward from a
> > functionality perspective; from an implementation perspective, though,
> the
> > KIP is not so straightforward, so it includes a pull request with a
> > proposed implementation.  See https://github.com/apache/kafka/pull/5582.
> >
> > Ron
> >
>
>
> --
> Best,
> Stanislav
>